From e286902e41bb4c5a0d0777022732028ae9e27f02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 15 Aug 2022 19:33:54 +0000 Subject: [PATCH 001/526] Reimplement read-in-order optimisation on top of query plan. --- ...reuseStorageOrderingForWindowFunctions.cpp | 236 ++++++++++++++++++ 1 file changed, 236 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 4d3c268ab7e..a07bb21026e 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -25,6 +26,241 @@ namespace DB::QueryPlanOptimizations { +ReadFromMergeTree * findReadingStep(QueryPlan::Node * node) +{ + IQueryPlanStep * step = node->step.get(); + if (auto * reading = typeid_cast(step)) + return reading; + + if (node->children.size() != 1) + return nullptr; + + if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) + return findReadingStep(node->children.front()); + + return nullptr; +} + +void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression, NameSet & filter_columns) +{ + /// Expression may replace a column. Check it. + /// TODO: there may possibly be a chain of aliases, we may check it and update filter names + for (auto & output : expression->getOutputs()) + if (filter_columns.contains(output->result_name) && output->type != ActionsDAG::ActionType::INPUT) + filter_columns.erase(output->result_name); + + if (dag) + dag = ActionsDAG::merge(std::move(*dag), std::move(*expression->clone())); + else + dag = expression->clone(); +} + +void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, NameSet & filter_columns) +{ + IQueryPlanStep * step = node->step.get(); + if (auto * reading = typeid_cast(step)) + return; + + if (node->children.size() != 1) + return; + + buildSortingDAG(node->children.front(), dag, filter_columns); + + if (auto * expression = typeid_cast(step)) + appendExpression(dag, expression->getExpression(), filter_columns); + + if (auto * filter = typeid_cast(step)) + { + appendExpression(dag, filter->getExpression(), filter_columns); + filter_columns.insert(filter->getFilterColumnName()); + } + + if (auto * array_join = typeid_cast(step)) + { + const auto & array_joined_columns = array_join->arrayJoin()->columns; + + /// Can't use filter whihc happened before ARRAY JOIN. + /// If there were a condition on const array, after ARRAY JOIN it can be broken. + /// Example : 'SELECT y FROM (select y from table where x = 1) ARRAY JOIN [1, 2] as x ORDER BY x' + /// where 'table' has it's own column 'x' in sorting key 'ORDER BY x'. + for (const auto & column : array_joined_columns) + filter_columns.erase(column); + + /// Remove array joined columns from outputs. + /// Types are changed after ARRAY JOIN, and we can't use this columns anyway. + ActionsDAG::NodeRawConstPtrs outputs; + outputs.reserve(dag->getOutputs().size()); + + for (const auto & output : dag->getOutputs()) + { + if (!array_joined_columns.contains(output->result_name)) + outputs.push_back(output); + } + } +} + +size_t calculateFixedPrefixSize(const ActionsDAG & dag, NameSet & filter_columns, const Names & sorting_key_columns) +{ + NameSet fiexd_columns; + std::vector stack; + for (const auto * output : dag.getOutputs()) + { + if (auto it = filter_columns.find(output->result_name); it != filter_columns.end()) + { + filter_columns.erase(it); + stack.clear(); + stack.push_back(output); + while (!stack.empty()) + { + const auto * node = stack.back(); + stack.pop_back(); + if (node->type == ActionsDAG::ActionType::FUNCTION) + { + const auto & name = node->function_base->getName(); + if (name == "and") + { + for (const auto * arg : node->children) + stack.push_back(arg); + } + else if (name == "equals") + { + fiexd_columns.insert(node->result_name); + } + } + } + } + } + + size_t prefix_size = 0; + for (const auto & name : sorting_key_columns) + { + if (!fiexd_columns.contains(name)) + break; + + ++prefix_size; + } + + return prefix_size; +} + +/// Optimize in case of exact match with order key element +/// or in some simple cases when order key element is wrapped into monotonic function. +/// Returns on of {-1, 0, 1} - direction of the match. 0 means - doesn't match. +int matchSortDescriptionAndKey( + const ActionsDAGPtr & dag, + const SortColumnDescription & sort_column, + const std::string & sorting_key_column) +{ + /// If required order depend on collation, it cannot be matched with primary key order. + /// Because primary keys cannot have collations. + if (sort_column.collator) + return 0; + + int current_direction = sort_column.direction; + /// For the path: order by (sort_column, ...) + bool exact_match = sort_column.column_name == sorting_key_column; + if (exact_match && !dag) + return current_direction; + + const auto * node = dag->tryFindInOutputs(sort_column.column_name); + /// It is possible when e.g. sort by array joined column. + if (!node) + return 0; + + if (exact_match && node->type == ActionsDAG::ActionType::INPUT) + return current_direction; + + while (node->children.size() == 1) + { + if (node->type == ActionsDAG::ActionType::FUNCTION) + { + const auto & func = *node->function_base; + if (!func.hasInformationAboutMonotonicity()) + return 0; + + auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {}); + if (!monotonicity.is_monotonic) + return 0; + + if (!monotonicity.is_positive) + current_direction *= -1; + + } + else if (node->type != ActionsDAG::ActionType::ALIAS) + return 0; + + node = node->children.front(); + if (node->result_name == sorting_key_column) + return current_direction; + } + + return 0; +} + +SortDescription buildPrefixSortDescription( + size_t fixed_prefix_size, + const ActionsDAGPtr & dag, + const SortDescription & description, + const Names & sorting_key_columns) +{ + size_t descr_prefix_size = std::min(description.size(), sorting_key_columns.size() - fixed_prefix_size); + + SortDescription order_key_prefix_descr; + order_key_prefix_descr.reserve(description.size()); + + for (size_t i = 0; i < fixed_prefix_size; ++i) + order_key_prefix_descr.push_back(description[i]); + + int read_direction = description.at(0).direction; + + for (size_t i = 0; i < descr_prefix_size; ++i) + { + int current_direction = matchSortDescriptionAndKey( + dag, description[i], sorting_key_columns[i + fixed_prefix_size]); + + if (!current_direction || (i > 0 && current_direction != read_direction)) + break; + + if (i == 0) + read_direction = current_direction; + + order_key_prefix_descr.push_back(description[i]); + } + + return order_key_prefix_descr; +} + +void optimizeReadInOrder(QueryPlan::Node & node) +{ + if (node.children.size() != 1) + return; + + auto * sorting = typeid_cast(node.step.get()); + if (!sorting) + return; + + ReadFromMergeTree * reading = findReadingStep(node.children.front()); + if (!reading) + return; + + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); + if (sorting_key.column_names.empty()) + return; + + ActionsDAGPtr dag; + NameSet filter_columns; + buildSortingDAG(node.children.front(), dag, filter_columns); + + const auto & description = sorting->getSortDescription(); + const auto & sorting_key_columns = sorting_key.column_names; + + size_t fixed_prefix_size = 0; + if (dag) + calculateFixedPrefixSize(*dag, filter_columns, sorting_key_columns); + + auto prefix_description = buildPrefixSortDescription(fixed_prefix_size, dag, description, sorting_key_columns); +} + size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) { /// Find the following sequence of steps, add InputOrderInfo and apply prefix sort description to From dc03a83ad7812353c6c12dc52df4e989e46d5df2 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 19 Aug 2022 17:56:02 +0000 Subject: [PATCH 002/526] Reimplement read-in-order optimisation on top of query plan. --- src/Interpreters/ActionsDAG.cpp | 16 +- src/Interpreters/ActionsDAG.h | 5 + ...reuseStorageOrderingForWindowFunctions.cpp | 328 +++++++++++++----- 3 files changed, 254 insertions(+), 95 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e935f2cc349..6a993059a03 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1168,6 +1168,17 @@ ActionsDAGPtr ActionsDAG::makeAddingColumnActions(ColumnWithTypeAndName column) ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) { + first.mergeInplace(std::move(second)); + + /// Drop unused inputs and, probably, some actions. + first.removeUnusedActions(); + + return std::make_shared(std::move(first)); +} + +void ActionsDAG::mergeInplace(ActionsDAG && second) +{ + auto & first = *this; /// first: x (1), x (2), y ==> x (2), z, x (3) /// second: x (1), x (2), x (3) ==> x (3), x (2), x (1) /// merge: x (1), x (2), x (3), y =(first)=> x (2), z, x (4), x (3) =(second)=> x (3), x (4), x (2), z @@ -1255,11 +1266,6 @@ ActionsDAGPtr ActionsDAG::merge(ActionsDAG && first, ActionsDAG && second) first.nodes.splice(first.nodes.end(), std::move(second.nodes)); first.projected_output = second.projected_output; - - /// Drop unused inputs and, probably, some actions. - first.removeUnusedActions(); - - return std::make_shared(std::move(first)); } ActionsDAG::SplitResult ActionsDAG::split(std::unordered_set split_nodes) const diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 51191d27f97..40b15d47bf8 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -271,6 +271,11 @@ public: /// Otherwise, any two actions may be combined. static ActionsDAGPtr merge(ActionsDAG && first, ActionsDAG && second); + /// The result is similar to merge(*this, second); + /// Invariant : no nodes are removed from the first (this) DAG. + /// So that pointers to nodes are kept valid. + void mergeInplace(ActionsDAG && second); + using SplitResult = std::pair; /// Split ActionsDAG into two DAGs, where first part contains all nodes from split_nodes and their children. diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index a07bb21026e..39825cdaffb 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB::QueryPlanOptimizations @@ -41,21 +42,60 @@ ReadFromMergeTree * findReadingStep(QueryPlan::Node * node) return nullptr; } -void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression, NameSet & filter_columns) -{ - /// Expression may replace a column. Check it. - /// TODO: there may possibly be a chain of aliases, we may check it and update filter names - for (auto & output : expression->getOutputs()) - if (filter_columns.contains(output->result_name) && output->type != ActionsDAG::ActionType::INPUT) - filter_columns.erase(output->result_name); +/// FixedColumns are columns which values become constants after filtering. +/// In a query "SELECT x, y, z FROM table WHERE x = 1 AND y = 'a' ORDER BY x, y, z" +/// Fixed columns are 'x' and 'y'. +using FixedColumns = std::unordered_set; +/// Right now we find only simple cases like 'and(..., and(..., and(column = value, ...), ...' +void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fiexd_columns) +{ + std::stack stack; + stack.push(&filter_expression); + + while (!stack.empty()) + { + const auto * node = stack.top(); + stack.pop(); + if (node->type == ActionsDAG::ActionType::FUNCTION) + { + const auto & name = node->function_base->getName(); + if (name == "and") + { + for (const auto * arg : node->children) + stack.push(arg); + } + else if (name == "equals") + { + const ActionsDAG::Node * maybe_fixed_column = nullptr; + bool is_singe = true; + for (const auto & child : node->children) + { + if (!child->column) + { + if (maybe_fixed_column) + maybe_fixed_column = child; + else + is_singe = false; + } + } + + if (maybe_fixed_column && is_singe) + fiexd_columns.insert(maybe_fixed_column); + } + } + } +} + +void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) +{ if (dag) - dag = ActionsDAG::merge(std::move(*dag), std::move(*expression->clone())); + dag->mergeInplace(std::move(*expression->clone())); else dag = expression->clone(); } -void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, NameSet & filter_columns) +void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, FixedColumns & fixed_columns) { IQueryPlanStep * step = node->step.get(); if (auto * reading = typeid_cast(step)) @@ -64,28 +104,22 @@ void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, NameSet & filt if (node->children.size() != 1) return; - buildSortingDAG(node->children.front(), dag, filter_columns); + buildSortingDAG(node->children.front(), dag, fixed_columns); if (auto * expression = typeid_cast(step)) - appendExpression(dag, expression->getExpression(), filter_columns); + appendExpression(dag, expression->getExpression()); if (auto * filter = typeid_cast(step)) { - appendExpression(dag, filter->getExpression(), filter_columns); - filter_columns.insert(filter->getFilterColumnName()); + appendExpression(dag, filter->getExpression()); + if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) + appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); } if (auto * array_join = typeid_cast(step)) { const auto & array_joined_columns = array_join->arrayJoin()->columns; - /// Can't use filter whihc happened before ARRAY JOIN. - /// If there were a condition on const array, after ARRAY JOIN it can be broken. - /// Example : 'SELECT y FROM (select y from table where x = 1) ARRAY JOIN [1, 2] as x ORDER BY x' - /// where 'table' has it's own column 'x' in sorting key 'ORDER BY x'. - for (const auto & column : array_joined_columns) - filter_columns.erase(column); - /// Remove array joined columns from outputs. /// Types are changed after ARRAY JOIN, and we can't use this columns anyway. ActionsDAG::NodeRawConstPtrs outputs; @@ -99,56 +133,192 @@ void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, NameSet & filt } } -size_t calculateFixedPrefixSize(const ActionsDAG & dag, NameSet & filter_columns, const Names & sorting_key_columns) +/// Here we try to find inner DAG inside outer DAG. +/// Build a map: inner.nodes -> outer.nodes. +// using NodesMap = std::unordered_map; +int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * outer) { - NameSet fiexd_columns; - std::vector stack; - for (const auto * output : dag.getOutputs()) + using Parents = std::set; + std::unordered_map inner_parents; + std::unordered_map inner_inputs; + { - if (auto it = filter_columns.find(output->result_name); it != filter_columns.end()) + std::stack stack; + stack.push(inner); + inner_parents.emplace(inner, Parents()); + while (!stack.empty()) { - filter_columns.erase(it); - stack.clear(); - stack.push_back(output); - while (!stack.empty()) + const auto * node = stack.top(); + stack.pop(); + + if (node->type == ActionsDAG::ActionType::INPUT) + inner_inputs.emplace(node->result_name, node); + + for (const auto * child : node->children) { - const auto * node = stack.back(); - stack.pop_back(); - if (node->type == ActionsDAG::ActionType::FUNCTION) - { - const auto & name = node->function_base->getName(); - if (name == "and") - { - for (const auto * arg : node->children) - stack.push_back(arg); - } - else if (name == "equals") - { - fiexd_columns.insert(node->result_name); - } - } + auto [it, inserted] = inner_parents.emplace(child, Parents()); + it->second.emplace(node); + + if (inserted) + stack.push(child); } } } - size_t prefix_size = 0; - for (const auto & name : sorting_key_columns) - { - if (!fiexd_columns.contains(name)) - break; + std::unordered_map outer_to_inner; + std::unordered_map direction; - ++prefix_size; + { + struct Frame + { + const ActionsDAG::Node * node; + ActionsDAG::NodeRawConstPtrs mapped_children; + int direction = 1; + }; + + std::stack stack; + stack.push(Frame{outer, {}}); + while (!stack.empty()) + { + auto & frame = stack.top(); + frame.mapped_children.reserve(frame.node->children.size()); + + while (frame.mapped_children.size() < frame.node->children.size()) + { + const auto * child = frame.node->children[frame.mapped_children.size()]; + auto it = outer_to_inner.find(child); + if (it == outer_to_inner.end()) + { + stack.push(Frame{child, {}}); + break; + } + frame.mapped_children.push_back(it->second); + } + + if (frame.mapped_children.size() < frame.node->children.size()) + continue; + + if (frame.node->type == ActionsDAG::ActionType::INPUT) + { + const ActionsDAG::Node * mapped = nullptr; + if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) + mapped = it->second; + + outer_to_inner.emplace(frame.node, mapped); + } + else if (frame.node->type == ActionsDAG::ActionType::ALIAS) + { + outer_to_inner.emplace(frame.node, frame.mapped_children.at(0)); + } + else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) + { + bool found_all_children = true; + size_t num_found_inner_roots = 0; + for (const auto * child : frame.mapped_children) + { + if (!child) + found_all_children = false; + else if (child == inner) + ++num_found_inner_roots; + } + + bool found_monotonic_wrapper = false; + if (num_found_inner_roots == 1) + { + if (frame.node->function_base->hasInformationAboutMonotonicity()) + { + size_t num_const_args = 0; + const ActionsDAG::Node * monotonic_child = nullptr; + for (const auto * child : frame.node->children) + { + if (child->column) + ++num_const_args; + else + monotonic_child = child; + } + + if (monotonic_child && num_const_args + 1 == frame.node->children.size()) + { + auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); + if (info.is_always_monotonic) + { + found_monotonic_wrapper = true; + outer_to_inner[frame.node] = inner; + + int cur_direction = info.is_positive ? 1 : -1; + auto it = direction.find(monotonic_child); + if (it != direction.end()) + cur_direction *= it->second; + + direction[frame.node] = cur_direction; + } + } + } + } + + if (!found_monotonic_wrapper && found_all_children && !frame.mapped_children.empty()) + { + Parents container; + Parents * intersection = &inner_parents[frame.mapped_children[0]]; + + if (frame.mapped_children.size() > 1) + { + std::vector other_parents; + other_parents.reserve(frame.mapped_children.size()); + for (size_t i = 1; i < frame.mapped_children.size(); ++i) + other_parents.push_back(&inner_parents[frame.mapped_children[i]]); + + for (const auto * parent : *intersection) + { + bool is_common = true; + for (const auto * set : other_parents) + { + if (!set->contains(parent)) + { + is_common = false; + break; + } + } + + if (is_common) + container.insert(parent); + } + + intersection = &container; + } + + if (!intersection->empty()) + { + auto func_name = frame.node->function_base->getName(); + for (const auto * parent : *intersection) + if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) + outer_to_inner[frame.node] = parent; + } + } + } + + stack.pop(); + } } - return prefix_size; + if (outer_to_inner[outer] != inner) + return 0; + + int res = 1; + if (auto it = direction.find(outer); it != direction.end()) + res = it->second; + + return res; } + /// Optimize in case of exact match with order key element /// or in some simple cases when order key element is wrapped into monotonic function. /// Returns on of {-1, 0, 1} - direction of the match. 0 means - doesn't match. int matchSortDescriptionAndKey( const ActionsDAGPtr & dag, const SortColumnDescription & sort_column, + const ActionsDAG & sort_column_dag, const std::string & sorting_key_column) { /// If required order depend on collation, it cannot be matched with primary key order. @@ -156,51 +326,28 @@ int matchSortDescriptionAndKey( if (sort_column.collator) return 0; - int current_direction = sort_column.direction; /// For the path: order by (sort_column, ...) - bool exact_match = sort_column.column_name == sorting_key_column; - if (exact_match && !dag) - return current_direction; + if (!dag && sort_column.column_name == sorting_key_column) + return sort_column.direction; - const auto * node = dag->tryFindInOutputs(sort_column.column_name); + const auto * outer = dag->tryFindInOutputs(sort_column.column_name); /// It is possible when e.g. sort by array joined column. - if (!node) + if (!outer) return 0; - if (exact_match && node->type == ActionsDAG::ActionType::INPUT) - return current_direction; + const auto * inner = sort_column_dag.tryFindInOutputs(sorting_key_column); + /// This should not happen. + if (!inner) + return 0; - while (node->children.size() == 1) - { - if (node->type == ActionsDAG::ActionType::FUNCTION) - { - const auto & func = *node->function_base; - if (!func.hasInformationAboutMonotonicity()) - return 0; - - auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {}); - if (!monotonicity.is_monotonic) - return 0; - - if (!monotonicity.is_positive) - current_direction *= -1; - - } - else if (node->type != ActionsDAG::ActionType::ALIAS) - return 0; - - node = node->children.front(); - if (node->result_name == sorting_key_column) - return current_direction; - } - - return 0; + return isMonotonicSubtree(inner, outer) * sort_column.direction; } SortDescription buildPrefixSortDescription( size_t fixed_prefix_size, const ActionsDAGPtr & dag, const SortDescription & description, + const ActionsDAG & sorting_key_dag, const Names & sorting_key_columns) { size_t descr_prefix_size = std::min(description.size(), sorting_key_columns.size() - fixed_prefix_size); @@ -216,7 +363,7 @@ SortDescription buildPrefixSortDescription( for (size_t i = 0; i < descr_prefix_size; ++i) { int current_direction = matchSortDescriptionAndKey( - dag, description[i], sorting_key_columns[i + fixed_prefix_size]); + dag, description[i], sorting_key_dag, sorting_key_columns[i + fixed_prefix_size]); if (!current_direction || (i > 0 && current_direction != read_direction)) break; @@ -248,17 +395,18 @@ void optimizeReadInOrder(QueryPlan::Node & node) return; ActionsDAGPtr dag; - NameSet filter_columns; - buildSortingDAG(node.children.front(), dag, filter_columns); + FixedColumns fixed_columns; + buildSortingDAG(node.children.front(), dag, fixed_columns); const auto & description = sorting->getSortDescription(); const auto & sorting_key_columns = sorting_key.column_names; size_t fixed_prefix_size = 0; - if (dag) - calculateFixedPrefixSize(*dag, filter_columns, sorting_key_columns); - auto prefix_description = buildPrefixSortDescription(fixed_prefix_size, dag, description, sorting_key_columns); + auto prefix_description = buildPrefixSortDescription( + fixed_prefix_size, + dag, description, + sorting_key.expression->getActionsDAG(), sorting_key_columns); } size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) From f0fd85a24f3a6e4595cdbba053dc4764feaad762 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 22 Aug 2022 18:13:13 +0000 Subject: [PATCH 003/526] Reimplement read-in-order optimisation on top of query plan. --- .../QueryPlan/Optimizations/Optimizations.h | 3 +- .../optimizePrimaryKeyCondition.cpp | 2 + ...reuseStorageOrderingForWindowFunctions.cpp | 194 +++++++++++++----- src/Processors/QueryPlan/SortingStep.h | 1 + 4 files changed, 148 insertions(+), 52 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 904f30e84b0..1b4ffb8340e 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -12,6 +12,7 @@ namespace QueryPlanOptimizations /// This is the main function which optimizes the whole QueryPlan tree. void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizeReadInOrder(QueryPlan::Node & node); void optimizePrimaryKeyCondition(QueryPlan::Node & root); /// Optimization is a function applied to QueryPlan::Node. @@ -63,7 +64,7 @@ inline const auto & getOptimizations() {tryMergeExpressions, "mergeExpressions", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, - {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan} + // {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan} }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 7d682c408e5..ebaf99dbd3f 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -33,6 +33,8 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) continue; } + optimizeReadInOrder(*frame.node); + auto add_filter = [&](auto & storage) { for (auto iter=stack.rbegin() + 1; iter!=stack.rend(); ++iter) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 39825cdaffb..154a6d12b1a 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -133,6 +133,66 @@ void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, FixedColumns & } } +void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns) +{ + struct Frame + { + const ActionsDAG::Node * node; + size_t next_child = 0; + }; + + std::stack stack; + std::unordered_set visited; + for (const auto & node : dag->getNodes()) + { + if (visited.contains(&node)) + continue; + + stack.push({&node}); + visited.insert(&node); + while (!stack.empty()) + { + auto & frame = stack.top(); + for (; frame.next_child < frame.node->children.size(); ++frame.next_child) + if (!visited.contains(frame.node->children[frame.next_child])) + break; + + if (frame.next_child < frame.node->children.size()) + { + const auto * child = frame.node->children[frame.next_child]; + visited.insert(child); + stack.push({child}); + ++frame.next_child; + } + else + { + /// Ignore constants here, will check them separately + if (!frame.node->column) + { + if (frame.node->type == ActionsDAG::ActionType::ALIAS) + { + if (fixed_columns.contains(frame.node->children.at(0))) + fixed_columns.insert(frame.node); + } + else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) + { + if (frame.node->function_base->isDeterministicInScopeOfQuery()) + { + bool all_args_fixed_or_const = true; + for (const auto * child : frame.node->children) + if (!child->column || !fixed_columns.contains(child)) + all_args_fixed_or_const = false; + + if (all_args_fixed_or_const) + fixed_columns.insert(frame.node); + } + } + } + } + } + } +} + /// Here we try to find inner DAG inside outer DAG. /// Build a map: inner.nodes -> outer.nodes. // using NodesMap = std::unordered_map; @@ -311,66 +371,82 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * return res; } - -/// Optimize in case of exact match with order key element -/// or in some simple cases when order key element is wrapped into monotonic function. -/// Returns on of {-1, 0, 1} - direction of the match. 0 means - doesn't match. -int matchSortDescriptionAndKey( - const ActionsDAGPtr & dag, - const SortColumnDescription & sort_column, - const ActionsDAG & sort_column_dag, - const std::string & sorting_key_column) -{ - /// If required order depend on collation, it cannot be matched with primary key order. - /// Because primary keys cannot have collations. - if (sort_column.collator) - return 0; - - /// For the path: order by (sort_column, ...) - if (!dag && sort_column.column_name == sorting_key_column) - return sort_column.direction; - - const auto * outer = dag->tryFindInOutputs(sort_column.column_name); - /// It is possible when e.g. sort by array joined column. - if (!outer) - return 0; - - const auto * inner = sort_column_dag.tryFindInOutputs(sorting_key_column); - /// This should not happen. - if (!inner) - return 0; - - return isMonotonicSubtree(inner, outer) * sort_column.direction; -} - SortDescription buildPrefixSortDescription( - size_t fixed_prefix_size, + const FixedColumns & fixed_columns, const ActionsDAGPtr & dag, const SortDescription & description, const ActionsDAG & sorting_key_dag, - const Names & sorting_key_columns) + const Names & sorting_key_columns, + int & read_direction) { - size_t descr_prefix_size = std::min(description.size(), sorting_key_columns.size() - fixed_prefix_size); - SortDescription order_key_prefix_descr; order_key_prefix_descr.reserve(description.size()); - for (size_t i = 0; i < fixed_prefix_size; ++i) - order_key_prefix_descr.push_back(description[i]); + /// This is a result direction we will read from MergeTree + /// 1 - in order, + /// -1 - in reverse order, + /// 0 - usual read, don't apply optimization + /// + /// So far, 0 means any direction is possible. It is ok for constant prefix. + read_direction = 0; - int read_direction = description.at(0).direction; - - for (size_t i = 0; i < descr_prefix_size; ++i) + for (size_t i = 0, next_sort_key = 0; i < description.size() && next_sort_key < sorting_key_columns.size(); ++i) { - int current_direction = matchSortDescriptionAndKey( - dag, description[i], sorting_key_dag, sorting_key_columns[i + fixed_prefix_size]); + const auto & sort_column = description[i]; + const auto & sorting_key_column = sorting_key_columns[next_sort_key]; - if (!current_direction || (i > 0 && current_direction != read_direction)) + /// If required order depend on collation, it cannot be matched with primary key order. + /// Because primary keys cannot have collations. + if (sort_column.collator) + return order_key_prefix_descr; + + /// Direction for current sort key. + int current_direction = 0; + + if (!dag) + { + if (sort_column.column_name != sorting_key_column) + return order_key_prefix_descr; + + current_direction = sort_column.direction; + ++next_sort_key; + } + else + { + const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(sort_column.column_name); + /// It is possible when e.g. sort by array joined column. + if (!sort_node) + return order_key_prefix_descr; + + const ActionsDAG::Node * sort_column_node = sorting_key_dag.tryFindInOutputs(sorting_key_column); + /// This should not happen. + if (!sort_column_node) + return order_key_prefix_descr; + + bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); + + /// We try to find the match even if column is fixed. In this case, potentially more keys will match. + /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' + /// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1' + /// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order. + current_direction = isMonotonicSubtree(sort_column_node, sort_node) * sort_column.direction; + + if (current_direction == 0 || !is_fixed_column) + return order_key_prefix_descr; + + if (current_direction) + ++next_sort_key; + + if (is_fixed_column) + current_direction = 0; + } + + /// read_direction == 0 means we can choose any global direction. + /// current_direction == 0 means current key if fixed and any direction is possible for it. + if (current_direction && read_direction && current_direction != read_direction) break; - if (i == 0) - read_direction = current_direction; - + read_direction = current_direction; order_key_prefix_descr.push_back(description[i]); } @@ -401,12 +477,28 @@ void optimizeReadInOrder(QueryPlan::Node & node) const auto & description = sorting->getSortDescription(); const auto & sorting_key_columns = sorting_key.column_names; - size_t fixed_prefix_size = 0; - + int read_direction = 0; auto prefix_description = buildPrefixSortDescription( - fixed_prefix_size, + fixed_columns, dag, description, - sorting_key.expression->getActionsDAG(), sorting_key_columns); + sorting_key.expression->getActionsDAG(), sorting_key_columns, + read_direction); + + /// It is possible that prefix_description is not empty, but read_direction is 0. + /// It means that some prefix of sorting key matched, but it was constant. + /// In this case, read-in-order is useless. + if (read_direction == 0 || prefix_description.empty()) + return; + + auto limit = sorting->getLimit(); + + auto order_info = std::make_shared( + SortDescription{}, + std::move(prefix_description), + read_direction, limit); + + reading->setQueryInfoInputOrderInfo(order_info); + sorting->convertToFinishSorting(order_info->order_key_prefix_descr); } size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index ce78bb863bf..a872edd48b6 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -46,6 +46,7 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeActions(FormatSettings & settings) const override; + UInt64 getLimit() const { return limit; } /// Add limit or change it to lower value. void updateLimit(size_t limit_); From 7f46b6a48b0b987e12f393558ee29359b82b9c02 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 8 Aug 2022 14:10:50 +0000 Subject: [PATCH 004/526] Add Hudi engine template --- src/Storages/StorageHudi.cpp | 35 +++++++++++++++++++++++++++++++ src/Storages/StorageHudi.h | 21 +++++++++++++++++++ src/Storages/registerStorages.cpp | 3 +++ 3 files changed, 59 insertions(+) create mode 100644 src/Storages/StorageHudi.cpp create mode 100644 src/Storages/StorageHudi.h diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp new file mode 100644 index 00000000000..9c7e5485f3c --- /dev/null +++ b/src/Storages/StorageHudi.cpp @@ -0,0 +1,35 @@ +#include +#include + +namespace DB { + +namespace ErrorCodes { + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +StorageHudi::StorageHudi( + const StorageID & table_id_, + ColumnsDescription /*columns_description_*/, + ConstraintsDescription /*constraints_description_*/, + const String & /*comment*/ +) : IStorage(table_id_) {} + + + +void registerStorageHudi(StorageFactory & factory) +{ + factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) + { + if (!args.engine_args.empty()) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Engine {} doesn't support any arguments ({} given)", + args.engine_name, + args.engine_args.size()); + + return std::make_shared(args.table_id, args.columns, args.constraints, args.comment); + }); +} + +} + diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h new file mode 100644 index 00000000000..e4fb52c6d39 --- /dev/null +++ b/src/Storages/StorageHudi.h @@ -0,0 +1,21 @@ +#pragma once + +#include "config_core.h" + +#include + +namespace DB +{ + +class StorageHudi final : public IStorage { +public: + StorageHudi( + const StorageID & table_id_, + ColumnsDescription columns_description_, + ConstraintsDescription constraints_description_, + const String & comment); + + String getName() const override { return "Hudi"; } +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 575b3de7ae2..b5561243e56 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -88,6 +88,7 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif +void registerStorageHudi(StorageFactory & factory); void registerStorages() { @@ -171,6 +172,8 @@ void registerStorages() #if USE_SQLITE registerStorageSQLite(factory); #endif + + registerStorageHudi(factory); } } From c5ae7126517d7326ad02981a97c9989be571dd99 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 22 Aug 2022 09:37:20 +0000 Subject: [PATCH 005/526] Add list files on s3 --- src/Storages/StorageHudi.cpp | 136 ++++++++++++++++++++++++++++++++--- src/Storages/StorageHudi.h | 24 ++++++- 2 files changed, 150 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 9c7e5485f3c..8c920808e5a 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,18 +1,122 @@ + +#include + #include #include +#include + +#include +#include +#include +#include +#include + namespace DB { namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; } StorageHudi::StorageHudi( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key, const StorageID & table_id_, ColumnsDescription /*columns_description_*/, ConstraintsDescription /*constraints_description_*/, - const String & /*comment*/ -) : IStorage(table_id_) {} + const String & /*comment*/, + ContextPtr context_ +) : IStorage(table_id_) + , s3_configuration({uri_, access_key_, secret_access_key, {}, {}, {}}) + , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) +{ + context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); + updateS3Configuration(context_, s3_configuration); + + const auto & client = s3_configuration.client; + { + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + const auto key = "hudi"; + bool is_finished{false}; + const auto bucket{s3_configuration.uri.bucket}; + + request.SetBucket(bucket); + //request.SetPrefix(key); + + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(key), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto& filename = obj.GetKey(); + LOG_DEBUG(log, "Found file: {}", filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + } + +} + +void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) +{ + auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); + + bool need_update_configuration = settings != S3Settings{}; + if (need_update_configuration) + { + if (upd.rw_settings != settings.rw_settings) + upd.rw_settings = settings.rw_settings; + } + + upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + + if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) + return; + + Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); + HeaderCollection headers; + if (upd.access_key_id.empty()) + { + credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); + headers = settings.auth_settings.headers; + } + + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( + settings.auth_settings.region, + ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + + client_configuration.endpointOverride = upd.uri.endpoint; + client_configuration.maxConnections = upd.rw_settings.max_connections; + + upd.client = S3::ClientFactory::instance().create( + client_configuration, + upd.uri.is_virtual_hosted_style, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + settings.auth_settings.server_side_encryption_customer_key_base64, + std::move(headers), + settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + + upd.auth_settings = std::move(settings.auth_settings); +} @@ -20,14 +124,28 @@ void registerStorageHudi(StorageFactory & factory) { factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) { - if (!args.engine_args.empty()) - throw Exception( - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Engine {} doesn't support any arguments ({} given)", - args.engine_name, - args.engine_args.size()); + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - return std::make_shared(args.table_id, args.columns, args.constraints, args.comment); + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + + S3::URI s3_uri(Poco::URI(configuration.url)); + + return std::make_shared( + s3_uri, + configuration.auth_settings.access_key_id, + configuration.auth_settings.secret_access_key, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext()); }); } diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index e4fb52c6d39..d64214d6c4b 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -3,6 +3,16 @@ #include "config_core.h" #include +#include + +namespace Poco { + class Logger; +} + +namespace Aws::S3 +{ + class S3Client; +} namespace DB { @@ -10,12 +20,24 @@ namespace DB class StorageHudi final : public IStorage { public: StorageHudi( + const S3::URI& uri_, + const String& access_key_, + const String& secret_access_key_, const StorageID & table_id_, ColumnsDescription columns_description_, ConstraintsDescription constraints_description_, - const String & comment); + const String & comment, + ContextPtr context_ + ); String getName() const override { return "Hudi"; } + +private: + static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); + +private: + StorageS3::S3Configuration s3_configuration; + Poco::Logger * log; }; } From 842c636354e2abba87cd6d3b89db4718365ab780 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 24 Aug 2022 15:07:37 +0000 Subject: [PATCH 006/526] Undone hudi --- src/Storages/StorageHudi.cpp | 188 +++++++++++++++++++++++++++-------- src/Storages/StorageHudi.h | 19 +++- 2 files changed, 166 insertions(+), 41 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 8c920808e5a..afad299241c 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -22,57 +22,75 @@ namespace ErrorCodes { StorageHudi::StorageHudi( const S3::URI & uri_, const String & access_key_, - const String & secret_access_key, + const String & secret_access_key_, const StorageID & table_id_, - ColumnsDescription /*columns_description_*/, - ConstraintsDescription /*constraints_description_*/, - const String & /*comment*/, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_ ) : IStorage(table_id_) - , s3_configuration({uri_, access_key_, secret_access_key, {}, {}, {}}) + , s3_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) + , query("") + , engine(nullptr) { context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); + StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, s3_configuration); - - const auto & client = s3_configuration.client; + + if (columns_.empty()) { - Aws::S3::Model::ListObjectsV2Request request; - Aws::S3::Model::ListObjectsV2Outcome outcome; - - const auto key = "hudi"; - bool is_finished{false}; - const auto bucket{s3_configuration.uri.bucket}; - - request.SetBucket(bucket); - //request.SetPrefix(key); - - while (!is_finished) - { - outcome = client->ListObjectsV2(request); - if (!outcome.IsSuccess()) - throw Exception( - ErrorCodes::S3_ERROR, - "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", - quoteString(bucket), - quoteString(key), - backQuote(outcome.GetError().GetExceptionName()), - quoteString(outcome.GetError().GetMessage())); - - const auto & result_batch = outcome.GetResult().GetContents(); - for (const auto & obj : result_batch) - { - const auto& filename = obj.GetKey(); - LOG_DEBUG(log, "Found file: {}", filename); - } - - request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - is_finished = !outcome.GetResult().GetIsTruncated(); - } + auto columns = StorageS3::getTableStructureFromData( + "Parquet", + uri_, + access_key_, + secret_access_key_, + "", + false, + std::nullopt, + context_ + ); + storage_metadata.setColumns(columns); + } + else + { + storage_metadata.setColumns(columns_); } + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); } +Pipe StorageHudi::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) +{ + auto keys = getKeysFromS3(); + + auto new_query = generateQueryFromKeys(std::forward(keys)); + // construct new engine + if (new_query != query) { + query = new_query; + auto new_query_uri = s3_configuration.uri.toString() + "/" + query; + engine = std::make_shared( + S3::URI(Poco::URI(new_query_uri)), + + ); + } + + return engine->read(column_names, storage_snapshot, + query_info, context, processed_stage, + max_block_size, num_streams) + + } + void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); @@ -118,6 +136,98 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati upd.auth_settings = std::move(settings.auth_settings); } +std::vector StorageHudi::getKeysFromS3() { + std::vector keys; + + const auto & client = s3_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{s3_configuration.uri.bucket}; + const std::string key = ""; + + request.SetBucket(bucket); + + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(key), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto& filename = obj.GetKey(); + keys.push_back(filename); + LOG_DEBUG(log, "Found file: {}", filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) { + // filter only .parquet files + std::erase_if(keys, [](const std::string& s) { + if (s.size() >= 8) { + return s.substr(s.size() - 8) != ".parquet"; + } + return true; + }); + + // for each partition path take only latest parquet file + + std::unordered_map> latest_parquets; + + for (const auto& key : keys) { + + auto slash = key.find_last_of("/"); + std::string path; + if (slash == std::string::npos) { + path = ""; + } else { + path = key.substr(0, slash); + } + + uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_")+1)); + + auto it = latest_parquets.find(path); + + if (it != latest_parquets.end()) { + if (it->second.second < timestamp) { + it->second = {key, timestamp}; + } + } else { + latest_parquets[path] = {key, timestamp}; + } + } + + std::vector filtered_keys; + std::transform(latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto& kv){return kv.second.first;}); + + std::string new_query; + + for (auto&& key : filtered_keys) { + if (!new_query.empty()) { + new_query += ","; + } + new_query += key; + } + new_query = "{" + new_query + "}"; + + return new_query; +} void registerStorageHudi(StorageFactory & factory) diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index d64214d6c4b..b02de7bf130 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -24,20 +24,35 @@ public: const String& access_key_, const String& secret_access_key_, const StorageID & table_id_, - ColumnsDescription columns_description_, - ConstraintsDescription constraints_description_, + const ColumnsDescription & columns_, + const ConstraintsDescription & constraints_, const String & comment, ContextPtr context_ ); String getName() const override { return "Hudi"; } + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + private: static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); +private: + std::vector getKeysFromS3(); + std::string generateQueryFromKeys(std::vector&& keys); + private: StorageS3::S3Configuration s3_configuration; Poco::Logger * log; + std::string query; + std::shared_ptr engine; }; } From 66c9305668f9ce515a86be04cd01cf376c80da73 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 25 Aug 2022 08:48:49 +0000 Subject: [PATCH 007/526] Add Hudi engine --- src/Storages/StorageHudi.cpp | 84 +++++++++++++++++++++--------------- src/Storages/StorageHudi.h | 7 ++- 2 files changed, 53 insertions(+), 38 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index afad299241c..21086a7e085 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,16 +1,16 @@ - +#include +#include #include -#include +#include #include #include #include #include #include -#include -#include +#include namespace DB { @@ -29,38 +29,55 @@ StorageHudi::StorageHudi( const String & comment, ContextPtr context_ ) : IStorage(table_id_) - , s3_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) - , query("") - , engine(nullptr) { - context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, base_configuration); - updateS3Configuration(context_, s3_configuration); + auto keys = getKeysFromS3(); + + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + LOG_DEBUG(log, "New uri: {}", new_uri); + + auto s3_uri = S3::URI(Poco::URI(new_uri)); +// StorageS3::S3Configuration s3_configuration{s3_uri, access_key_, secret_access_key_, {}, {}, {}}; + if (columns_.empty()) { auto columns = StorageS3::getTableStructureFromData( - "Parquet", - uri_, + String("Parquet"), + s3_uri, access_key_, secret_access_key_, "", false, - std::nullopt, + {}, context_ - ); + ); storage_metadata.setColumns(columns); } - else - { + else storage_metadata.setColumns(columns_); - } storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + s3_uri, + access_key_, + secret_access_key_, + table_id_, + String("Parquet"), // format name + base_configuration.rw_settings, + columns_, + constraints_, + comment, + context_, + std::nullopt + ); } Pipe StorageHudi::read( @@ -72,24 +89,18 @@ Pipe StorageHudi::read( size_t max_block_size, unsigned num_streams) { - auto keys = getKeysFromS3(); + updateS3Configuration(context, base_configuration); - auto new_query = generateQueryFromKeys(std::forward(keys)); - // construct new engine - if (new_query != query) { - query = new_query; - auto new_query_uri = s3_configuration.uri.toString() + "/" + query; - engine = std::make_shared( - S3::URI(Poco::URI(new_query_uri)), - - ); - } + //auto keys = getKeysFromS3(); - return engine->read(column_names, storage_snapshot, + //auto new_uri = base_configuration.uri.uri.toString() + "/" + generateQueryFromKeys(std::forward(keys)); + //s3_configuration.uri = S3::URI(Poco::URI(new_uri)); + + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, - max_block_size, num_streams) + max_block_size, num_streams); - } +} void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { @@ -139,13 +150,13 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati std::vector StorageHudi::getKeysFromS3() { std::vector keys; - const auto & client = s3_configuration.client; + const auto & client = base_configuration.client; Aws::S3::Model::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; bool is_finished{false}; - const auto bucket{s3_configuration.uri.bucket}; + const auto bucket{base_configuration.uri.bucket}; const std::string key = ""; request.SetBucket(bucket); @@ -167,7 +178,7 @@ std::vector StorageHudi::getKeysFromS3() { { const auto& filename = obj.GetKey(); keys.push_back(filename); - LOG_DEBUG(log, "Found file: {}", filename); + //LOG_DEBUG(log, "Found file: {}", filename); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); @@ -256,7 +267,12 @@ void registerStorageHudi(StorageFactory & factory) args.constraints, args.comment, args.getContext()); - }); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } } diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index b02de7bf130..61e2af1340d 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -17,7 +17,7 @@ namespace Aws::S3 namespace DB { -class StorageHudi final : public IStorage { +class StorageHudi : public IStorage { public: StorageHudi( const S3::URI& uri_, @@ -49,10 +49,9 @@ private: std::string generateQueryFromKeys(std::vector&& keys); private: - StorageS3::S3Configuration s3_configuration; + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; Poco::Logger * log; - std::string query; - std::shared_ptr engine; }; } From e82c4800f282bb750cbd3fbec066d264c63b7d4a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 26 Aug 2022 18:17:32 +0000 Subject: [PATCH 008/526] Better --- src/Storages/StorageHudi.cpp | 134 ++++++++++++++++++----------------- src/Storages/StorageHudi.h | 23 +++--- 2 files changed, 80 insertions(+), 77 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 21086a7e085..efba2d3f85f 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -12,9 +12,11 @@ #include -namespace DB { +namespace DB +{ -namespace ErrorCodes { +namespace ErrorCodes +{ extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; } @@ -24,11 +26,11 @@ StorageHudi::StorageHudi( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, - const ColumnsDescription & columns_, + ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_ -) : IStorage(table_id_) + ContextPtr context_) + : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) { @@ -38,25 +40,16 @@ StorageHudi::StorageHudi( auto keys = getKeysFromS3(); auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); - + LOG_DEBUG(log, "New uri: {}", new_uri); auto s3_uri = S3::URI(Poco::URI(new_uri)); -// StorageS3::S3Configuration s3_configuration{s3_uri, access_key_, secret_access_key_, {}, {}, {}}; if (columns_.empty()) { - auto columns = StorageS3::getTableStructureFromData( - String("Parquet"), - s3_uri, - access_key_, - secret_access_key_, - "", - false, - {}, - context_ - ); - storage_metadata.setColumns(columns); + columns_ + = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); @@ -76,30 +69,21 @@ StorageHudi::StorageHudi( constraints_, comment, context_, - std::nullopt - ); + std::nullopt); } Pipe StorageHudi::read( - const Names & column_names, - const StorageSnapshotPtr & storage_snapshot, - SelectQueryInfo & query_info, - ContextPtr context, - QueryProcessingStage::Enum processed_stage, - size_t max_block_size, - unsigned num_streams) + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) { updateS3Configuration(context, base_configuration); - //auto keys = getKeysFromS3(); - - //auto new_uri = base_configuration.uri.uri.toString() + "/" + generateQueryFromKeys(std::forward(keys)); - //s3_configuration.uri = S3::URI(Poco::URI(new_uri)); - - return s3engine->read(column_names, storage_snapshot, - query_info, context, processed_stage, - max_block_size, num_streams); - + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) @@ -128,7 +112,8 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( settings.auth_settings.region, - ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getRemoteHostFilter(), + ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); client_configuration.endpointOverride = upd.uri.endpoint; @@ -147,11 +132,12 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati upd.auth_settings = std::move(settings.auth_settings); } -std::vector StorageHudi::getKeysFromS3() { +std::vector StorageHudi::getKeysFromS3() +{ std::vector keys; const auto & client = base_configuration.client; - + Aws::S3::Model::ListObjectsV2Request request; Aws::S3::Model::ListObjectsV2Outcome outcome; @@ -176,7 +162,7 @@ std::vector StorageHudi::getKeysFromS3() { const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto& filename = obj.GetKey(); + const auto & filename = obj.GetKey(); keys.push_back(filename); //LOG_DEBUG(log, "Found file: {}", filename); } @@ -188,49 +174,64 @@ std::vector StorageHudi::getKeysFromS3() { return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) { +std::string StorageHudi::generateQueryFromKeys(std::vector && keys) +{ // filter only .parquet files - std::erase_if(keys, [](const std::string& s) { - if (s.size() >= 8) { - return s.substr(s.size() - 8) != ".parquet"; - } - return true; - }); + std::erase_if( + keys, + [](const std::string & s) + { + if (s.size() >= 8) + { + return s.substr(s.size() - 8) != ".parquet"; + } + return true; + }); // for each partition path take only latest parquet file std::unordered_map> latest_parquets; - - for (const auto& key : keys) { + for (const auto & key : keys) + { auto slash = key.find_last_of("/"); std::string path; - if (slash == std::string::npos) { + if (slash == std::string::npos) + { path = ""; - } else { + } + else + { path = key.substr(0, slash); } - uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_")+1)); + uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); - - if (it != latest_parquets.end()) { - if (it->second.second < timestamp) { + + if (it != latest_parquets.end()) + { + if (it->second.second < timestamp) + { it->second = {key, timestamp}; } - } else { + } + else + { latest_parquets[path] = {key, timestamp}; } } std::vector filtered_keys; - std::transform(latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto& kv){return kv.second.first;}); + std::transform( + latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); std::string new_query; - - for (auto&& key : filtered_keys) { - if (!new_query.empty()) { + + for (auto && key : filtered_keys) + { + if (!new_query.empty()) + { new_query += ","; } new_query += key; @@ -243,7 +244,9 @@ std::string StorageHudi::generateQueryFromKeys(std::vector&& keys) void registerStorageHudi(StorageFactory & factory) { - factory.registerStorage("Hudi", [](const StorageFactory::Arguments & args) + factory.registerStorage( + "Hudi", + [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; if (engine_args.empty()) @@ -269,11 +272,10 @@ void registerStorageHudi(StorageFactory & factory) args.getContext()); }, { - .supports_settings = true, - .supports_schema_inference = true, - .source_access_type = AccessType::S3, - }); + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); } } - diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 61e2af1340d..dd5cc18495e 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -5,30 +5,31 @@ #include #include -namespace Poco { - class Logger; +namespace Poco +{ +class Logger; } namespace Aws::S3 { - class S3Client; +class S3Client; } namespace DB { -class StorageHudi : public IStorage { +class StorageHudi : public IStorage +{ public: StorageHudi( - const S3::URI& uri_, - const String& access_key_, - const String& secret_access_key_, + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, const StorageID & table_id_, - const ColumnsDescription & columns_, + ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_ - ); + ContextPtr context_); String getName() const override { return "Hudi"; } @@ -46,7 +47,7 @@ private: private: std::vector getKeysFromS3(); - std::string generateQueryFromKeys(std::vector&& keys); + std::string generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; From 3e87fd0ee46236a159bab37412199ea927402db3 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 09:14:05 +0000 Subject: [PATCH 009/526] Add test template --- .../integration/test_storage_hudi/__init__.py | 0 tests/integration/test_storage_hudi/test.py | 107 ++++++++++++ .../.hoodie/.20220830083647456.commit.crc | Bin 0 -> 60 bytes .../.20220830083647456.commit.requested.crc | Bin 0 -> 8 bytes .../.hoodie/.20220830083647456.inflight.crc | Bin 0 -> 44 bytes .../test_table/.hoodie/.hoodie.properties.crc | Bin 0 -> 16 bytes .../.hoodie/20220830083647456.commit | 165 ++++++++++++++++++ .../20220830083647456.commit.requested | 0 .../.hoodie/20220830083647456.inflight | 161 +++++++++++++++++ .../test_table/.hoodie/hoodie.properties | 21 +++ .../.hoodie/.00000000000000.deltacommit.crc | Bin 0 -> 72 bytes .../.00000000000000.deltacommit.inflight.crc | Bin 0 -> 32 bytes .../.00000000000000.deltacommit.requested.crc | Bin 0 -> 8 bytes .../.20220830083647456.deltacommit.crc | Bin 0 -> 76 bytes ...20220830083647456.deltacommit.inflight.crc | Bin 0 -> 32 bytes ...0220830083647456.deltacommit.requested.crc | Bin 0 -> 8 bytes .../metadata/.hoodie/.hoodie.properties.crc | Bin 0 -> 16 bytes .../.hoodie/00000000000000.deltacommit | 97 ++++++++++ .../00000000000000.deltacommit.inflight | 116 ++++++++++++ .../00000000000000.deltacommit.requested | 0 .../.hoodie/20220830083647456.deltacommit | 97 ++++++++++ .../20220830083647456.deltacommit.inflight | 116 ++++++++++++ .../20220830083647456.deltacommit.requested | 0 .../metadata/.hoodie/hoodie.properties | 14 ++ ....files-0000_00000000000000.log.1_0-0-0.crc | Bin 0 -> 12 bytes ...iles-0000_00000000000000.log.1_0-52-57.crc | Bin 0 -> 96 bytes ...iles-0000_00000000000000.log.2_0-83-93.crc | Bin 0 -> 96 bytes .../files/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes .../.files-0000_00000000000000.log.1_0-0-0 | Bin 0 -> 124 bytes .../.files-0000_00000000000000.log.1_0-52-57 | Bin 0 -> 10928 bytes .../.files-0000_00000000000000.log.2_0-83-93 | Bin 0 -> 11180 bytes .../metadata/files/.hoodie_partition_metadata | 4 + .../sao_paulo/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...ad-0_0-73-83_20220830083647456.parquet.crc | Bin 0 -> 3432 bytes .../sao_paulo/.hoodie_partition_metadata | 4 + ...aebbad-0_0-73-83_20220830083647456.parquet | Bin 0 -> 437831 bytes .../..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...55-0_1-73-84_20220830083647456.parquet.crc | Bin 0 -> 3432 bytes .../san_francisco/.hoodie_partition_metadata | 4 + ...268e55-0_1-73-84_20220830083647456.parquet | Bin 0 -> 438186 bytes .../chennai/..hoodie_partition_metadata.crc | Bin 0 -> 12 bytes ...e1-0_2-73-85_20220830083647456.parquet.crc | Bin 0 -> 3428 bytes .../india/chennai/.hoodie_partition_metadata | 4 + ...e5d6e1-0_2-73-85_20220830083647456.parquet | Bin 0 -> 437623 bytes 44 files changed, 910 insertions(+) create mode 100644 tests/integration/test_storage_hudi/__init__.py create mode 100644 tests/integration/test_storage_hudi/test.py create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-52-57 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 create mode 100644 tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata create mode 100644 tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet diff --git a/tests/integration/test_storage_hudi/__init__.py b/tests/integration/test_storage_hudi/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py new file mode 100644 index 00000000000..716ec15524e --- /dev/null +++ b/tests/integration/test_storage_hudi/test.py @@ -0,0 +1,107 @@ +import logging +import os + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(f"{SCRIPT_DIR}/test_table"): + for name in files: + started_cluster.minio_client.fput_object(bucket, os.path.join(SCRIPT_DIR, address, name), os.path.join(address, name)) + + for obj in list( + minio.list_objects( + bucket, + recursive=True, + ) + ): + logging.info(obj.name) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "dummy", + with_minio=True + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["dummy"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/hudi', 'minio', 'minio123')""" + + run_query(instance, create_query) + +def test_select_query(): + pass \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.crc new file mode 100644 index 0000000000000000000000000000000000000000..4bba97b9515ddcd9dd09f226e1cd81e89f1e9026 GIT binary patch literal 60 zcmV-C0K@-da$^7h00IEYSJ`4^`+RY$Y`hZ?T{ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.commit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.20220830083647456.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..21984c840bc3f9329c55d1d5824515745f9d0468 GIT binary patch literal 44 zcmYc;N@ieSU}Cskw&P5j)zKi6c-hD;M%E!O)z;;Tsn#4f+2CBxmvH@r-orPu0BZCS AO#lD@ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/.hoodie.properties.crc new file mode 100644 index 0000000000000000000000000000000000000000..f67f76b7101c0cf7b1aba9818ac05738d10a419c GIT binary patch literal 16 XcmYc;N@ieSU}6a3SsyCGs3rmc8^!|c literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit new file mode 100644 index 00000000000..f8d6c248f49 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit @@ -0,0 +1,165 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"test_table_record\",\"namespace\":\"hoodie.test_table\",\"fields\":[{\"name\":\"begin_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"begin_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"driver\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"end_lat\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"end_lon\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"fare\",\"type\":[\"null\",\"double\"],\"default\":null},{\"name\":\"partitionpath\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"rider\",\"type\":[\"null\",\"string\"],\"default\":null},{\"name\":\"ts\",\"type\":[\"null\",\"long\"],\"default\":null},{\"name\":\"uuid\",\"type\":[\"null\",\"string\"],\"default\":null}]}" + }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0", + "path" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 3, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 437831, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/brazil/sao_paulo", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437831, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "34b1b177-f0af-467b-9214-473ead268e55-0", + "path" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 5, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 438186, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "americas/united_states/san_francisco", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 438186, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0", + "path" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "prevCommit" : "null", + "numWrites" : 2, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 437623, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "asia/india/chennai", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 437623, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 563, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "92aa634e-d83f-4057-a385-ea3b22e5d6e1-0" : "asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet", + "34b1b177-f0af-467b-9214-473ead268e55-0" : "americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet", + "8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0" : "americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.commit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight new file mode 100644 index 00000000000..f5ef5c92c2b --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/20220830083647456.inflight @@ -0,0 +1,161 @@ +{ + "partitionToWriteStats" : { + "americas/brazil/sao_paulo" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "americas/united_states/san_francisco" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "asia/india/chennai" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 3, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 5, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 2, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", "asia/india/chennai" ], + "fileIdAndRelativePaths" : { + "" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties new file mode 100644 index 00000000000..9ae364baf33 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/hoodie.properties @@ -0,0 +1,21 @@ +#Updated at 2022-08-30T08:36:49.089844Z +#Tue Aug 30 08:36:49 UTC 2022 +hoodie.table.type=COPY_ON_WRITE +hoodie.table.metadata.partitions=files +hoodie.table.precombine.field=ts +hoodie.table.partition.fields=partitionpath +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=2702201862 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.timeline.timezone=LOCAL +hoodie.table.recordkey.fields=uuid +hoodie.table.name=test_table +hoodie.partition.metafile.use.base.format=false +hoodie.datasource.write.hive_style_partitioning=false +hoodie.populate.meta.fields=true +hoodie.table.keygenerator.class=org.apache.hudi.keygen.SimpleKeyGenerator +hoodie.table.base.file.format=PARQUET +hoodie.database.name= +hoodie.datasource.write.partitionpath.urlencode=false +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.crc new file mode 100644 index 0000000000000000000000000000000000000000..352b882ec5a697e6d4d0754866a1cdec0be1d3be GIT binary patch literal 72 zcmV-O0Jr~Ra$^7h00ID+B#3JXBN@*inEmG*kvdVpwWAbT>N@Qjh0Y0p<^OR6NMYYe e)p}4$Xm-%78APNu2!RHTD1CAabbD4@y#KTXRUT6S literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..b6b8f7fc1a3a439cc0c8cf1a3c06491fcf54223f GIT binary patch literal 32 ncmYc;N@ieSU}8AynOb;@FLqIaqtjZc*wA7%;rSPOY^;g^r9}$) literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.00000000000000.deltacommit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.crc new file mode 100644 index 0000000000000000000000000000000000000000..54abc5e9b72e87753d458af67e967cccaa242c8e GIT binary patch literal 76 zcmV-S0JHyNa$^7h00IEQeq^6A>X*!Te*(sYWI5Hv06BTKu=gO04+eJ-1Ey1Rqt#G) ismNQFuvqj*Kj+@fF0)E(wf0GSx`G_6G3KKp3jiwT&ms^2 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.inflight.crc new file mode 100644 index 0000000000000000000000000000000000000000..ec7cb5faf561c70caf37bd118b025f9e36947c07 GIT binary patch literal 32 ocmYc;N@ieSU}8AynOf-f;LElIN2j$?v7yDY|IOy^v9T%w0J~ufc>n+a literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.20220830083647456.deltacommit.requested.crc new file mode 100644 index 0000000000000000000000000000000000000000..3b7b044936a890cd8d651d349a752d819d71d22c GIT binary patch literal 8 PcmYc;N@ieSU}69O2$TUk literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/.hoodie.properties.crc new file mode 100644 index 0000000000000000000000000000000000000000..509ae4501ee2922036326030efe3e5c74d1db059 GIT binary patch literal 16 XcmYc;N@ieSU}EsgOFNnOa!LyTBu)jD literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit new file mode 100644 index 00000000000..6d22af6dd2e --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.1_0-52-57", + "prevCommit" : "00000000000000", + "numWrites" : 1, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 10928, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 10928, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 1, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 67, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.1_0-52-57" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight new file mode 100644 index 00000000000..bb2542e0186 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 1, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/00000000000000.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit new file mode 100644 index 00000000000..210201f7135 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit @@ -0,0 +1,97 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ] + }, + "compacted" : false, + "extraMetadata" : { + "schema" : "{\"type\":\"record\",\"name\":\"HoodieMetadataRecord\",\"namespace\":\"org.apache.hudi.avro.model\",\"doc\":\"A record saved within the Metadata Table\",\"fields\":[{\"name\":\"key\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"type\",\"type\":\"int\",\"doc\":\"Type of the metadata record\"},{\"name\":\"filesystemMetadata\",\"type\":[\"null\",{\"type\":\"map\",\"values\":{\"type\":\"record\",\"name\":\"HoodieMetadataFileInfo\",\"fields\":[{\"name\":\"size\",\"type\":\"long\",\"doc\":\"Size of the file\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"True if this file has been deleted\"}]},\"avro.java.string\":\"String\"}],\"doc\":\"Contains information about partitions and files within the dataset\"},{\"name\":\"BloomFilterMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataBloomFilter\",\"doc\":\"Data file bloom filter details\",\"fields\":[{\"name\":\"type\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Bloom filter type code\"},{\"name\":\"timestamp\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"},\"doc\":\"Instant timestamp when this metadata was created/updated\"},{\"name\":\"bloomFilter\",\"type\":\"bytes\",\"doc\":\"Bloom filter binary byte array\"},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Bloom filter entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of bloom filters for all data files in the user table\",\"default\":null},{\"name\":\"ColumnStatsMetadata\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"HoodieMetadataColumnStats\",\"doc\":\"Data file column statistics\",\"fields\":[{\"name\":\"fileName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"File name for which this column statistics applies\",\"default\":null},{\"name\":\"columnName\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}],\"doc\":\"Column name for which this column statistics applies\",\"default\":null},{\"name\":\"minValue\",\"type\":[\"null\",{\"type\":\"record\",\"name\":\"BooleanWrapper\",\"doc\":\"A record wrapping boolean type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"boolean\"}]},{\"type\":\"record\",\"name\":\"IntWrapper\",\"doc\":\"A record wrapping int type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"LongWrapper\",\"doc\":\"A record wrapping long type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]},{\"type\":\"record\",\"name\":\"FloatWrapper\",\"doc\":\"A record wrapping float type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"float\"}]},{\"type\":\"record\",\"name\":\"DoubleWrapper\",\"doc\":\"A record wrapping double type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"double\"}]},{\"type\":\"record\",\"name\":\"BytesWrapper\",\"doc\":\"A record wrapping bytes type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"bytes\"}]},{\"type\":\"record\",\"name\":\"StringWrapper\",\"doc\":\"A record wrapping string type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"DateWrapper\",\"doc\":\"A record wrapping Date logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"int\"}]},{\"type\":\"record\",\"name\":\"DecimalWrapper\",\"doc\":\"A record wrapping Decimal logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"bytes\",\"logicalType\":\"decimal\",\"precision\":30,\"scale\":15}}]},{\"type\":\"record\",\"name\":\"TimeMicrosWrapper\",\"doc\":\"A record wrapping Time-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":{\"type\":\"long\",\"logicalType\":\"time-micros\"}}]},{\"type\":\"record\",\"name\":\"TimestampMicrosWrapper\",\"doc\":\"A record wrapping Timestamp-micros logical type to be able to be used it w/in Avro's Union\",\"fields\":[{\"name\":\"value\",\"type\":\"long\"}]}],\"doc\":\"Minimum value in the range. Based on user data table schema, we can convert this to appropriate type\",\"default\":null},{\"name\":\"maxValue\",\"type\":[\"null\",\"BooleanWrapper\",\"IntWrapper\",\"LongWrapper\",\"FloatWrapper\",\"DoubleWrapper\",\"BytesWrapper\",\"StringWrapper\",\"DateWrapper\",\"DecimalWrapper\",\"TimeMicrosWrapper\",\"TimestampMicrosWrapper\"],\"doc\":\"Maximum value in the range. Based on user data table schema, we can convert it to appropriate type\",\"default\":null},{\"name\":\"valueCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of values\",\"default\":null},{\"name\":\"nullCount\",\"type\":[\"null\",\"long\"],\"doc\":\"Total count of null values\",\"default\":null},{\"name\":\"totalSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total storage size on disk\",\"default\":null},{\"name\":\"totalUncompressedSize\",\"type\":[\"null\",\"long\"],\"doc\":\"Total uncompressed storage size on disk\",\"default\":null},{\"name\":\"isDeleted\",\"type\":\"boolean\",\"doc\":\"Column range entry valid/deleted flag\"}]}],\"doc\":\"Metadata Index of column statistics for all data files in the user table\",\"default\":null}]}" + }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "files-0000", + "path" : "files/.files-0000_00000000000000.log.2_0-83-93", + "prevCommit" : "00000000000000", + "numWrites" : 4, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 11180, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : "files", + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 11180, + "minEventTime" : null, + "maxEventTime" : null, + "logVersion" : 2, + "logOffset" : 0, + "baseFile" : "", + "logFiles" : [ ".files-0000_00000000000000.log.1_0-52-57", ".files-0000_00000000000000.log.1_0-0-0", ".files-0000_00000000000000.log.2_0-83-93" ], + "recordsStats" : { + "val" : null, + "present" : false + }, + "columnStats" : { + "val" : null, + "present" : false + } + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 39, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "files-0000" : "files/.files-0000_00000000000000.log.2_0-83-93" + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight new file mode 100644 index 00000000000..ea1b6a10c13 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.inflight @@ -0,0 +1,116 @@ +{ + "partitionToWriteStats" : { + "files" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ] + }, + "compacted" : false, + "extraMetadata" : { }, + "operationType" : "UPSERT_PREPPED", + "writeStats" : [ { + "fileId" : "", + "path" : null, + "prevCommit" : "null", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 0, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + }, { + "fileId" : "files-0000", + "path" : null, + "prevCommit" : "00000000000000", + "numWrites" : 0, + "numDeletes" : 0, + "numUpdateWrites" : 4, + "numInserts" : 0, + "totalWriteBytes" : 0, + "totalWriteErrors" : 0, + "tempPath" : null, + "partitionPath" : null, + "totalLogRecords" : 0, + "totalLogFilesCompacted" : 0, + "totalLogSizeCompacted" : 0, + "totalUpdatedRecordsCompacted" : 0, + "totalLogBlocks" : 0, + "totalCorruptLogBlock" : 0, + "totalRollbackBlocks" : 0, + "fileSizeInBytes" : 0, + "minEventTime" : null, + "maxEventTime" : null + } ], + "totalRecordsDeleted" : 0, + "totalLogRecordsCompacted" : 0, + "totalLogFilesCompacted" : 0, + "totalCompactedRecordsUpdated" : 0, + "totalLogFilesSize" : 0, + "totalScanTime" : 0, + "totalCreateTime" : 0, + "totalUpsertTime" : 0, + "minAndMaxEventTime" : { + "Optional.empty" : { + "val" : null, + "present" : false + } + }, + "writePartitionPaths" : [ "files" ], + "fileIdAndRelativePaths" : { + "" : null, + "files-0000" : null + } +} \ No newline at end of file diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/20220830083647456.deltacommit.requested new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties new file mode 100644 index 00000000000..845df718f6d --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/.hoodie/hoodie.properties @@ -0,0 +1,14 @@ +#Properties saved on 2022-08-30T08:36:47.657528Z +#Tue Aug 30 08:36:47 UTC 2022 +hoodie.compaction.payload.class=org.apache.hudi.metadata.HoodieMetadataPayload +hoodie.table.type=MERGE_ON_READ +hoodie.archivelog.folder=archived +hoodie.timeline.layout.version=1 +hoodie.table.checksum=1983687495 +hoodie.datasource.write.drop.partition.columns=false +hoodie.table.recordkey.fields=key +hoodie.table.name=test_table_metadata +hoodie.populate.meta.fields=false +hoodie.table.keygenerator.class=org.apache.hudi.metadata.HoodieTableMetadataKeyGenerator +hoodie.table.base.file.format=HFILE +hoodie.table.version=5 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-0-0.crc new file mode 100644 index 0000000000000000000000000000000000000000..e016a7f52627a5b0d3862ad6a1f662fc3ad8d94a GIT binary patch literal 12 TcmYc;N@ieSU}9ifcZC}O5mf@t literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.1_0-52-57.crc new file mode 100644 index 0000000000000000000000000000000000000000..c1136be0c0e257f2aa3aeed6a5c32dd7e097379c GIT binary patch literal 96 zcmYc;N@ieSU}De?RQxLt_>_I4%7)XcTl}8oT=)}k%-!ySA(_>TZG7lbpTeoRdX$3CKW0JTK069!9Qvd(} literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..files-0000_00000000000000.log.2_0-83-93.crc new file mode 100644 index 0000000000000000000000000000000000000000..0aaf9e50256c454e3ecb14ec9a5c5fae645fa37d GIT binary patch literal 96 zcmYc;N@ieSU}ES={`d8LX<`T0sRhSg8n$*$xud%G6-VxaBWxdUt-gH5P1~*4QmFm& gwYUbcDec;eKAvHn%Cvog%j`|7N-J<-`{&K~0Q~|n`~Uy| literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..1b5364eed343b390fb97133b3c42ae8d9e1c6af5 GIT binary patch literal 12 TcmYc;N@ieSU}9K*=PV-t6l(*N literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.1_0-0-0 new file mode 100644 index 0000000000000000000000000000000000000000..9bf687c1a4b5fe36c79ae9759b132a53717475c4 GIT binary patch literal 124 zcmY%P2zBvPW`KY^AjOD;L85#HC;%h`1QtNd1jGi64ADOMMd^Bp1&PTSsd^cuDVci7 j`MJ6Id3w3|DXBSnE~z=GC8PcPz0)n2wJ|VGo!@#RMsVc`%MB zneYAtA;HizP6+rcN1fy<3Fd^vkc5+xke~`+MnZ_ekj0SN zE<_b)BmLVY$r9i<1DTXQ}6zuVKq85Tk&SDUW38fOFMKlJ0@ichf zhH;gGS%;!Y441(X+7+3lQ7NKQ(69$HB#GCTJqlfe6EDP|dXJ*jX#5K?7mbiKMq>mn zibiPp3o$9h6sU;x?0-X`{swiz(Kvtyv+iGo#2`+ZpFH3YwR4j6oWROW3j)S)nsFGb z6(JBJL`Ye6C^fp%$OqOlz-)zzr)X5El~6B+bl8+gG`JaqC{!p=mhR{NweAZR5a3cq z8of11VF_I#gcSsko)&)yxTVFX&BSA@C+}8o_W1!oWUACBldaLZS#24e7*1K?p8k7ozmf zC3D&-l-38&R}h2}utKBcz?ApTt)3~}#(ltw>J3LR3o#$TN?_Aq3knJh>+OYZBGS2a zFiO-qiv^-xiZKU?LEzHgH4|PUW${(_#`u zfIys8(ZB^jD0-4-c~Lt+2L_Icr-7)Sxt}NuN62`H837?AiNxz08-x_Qk%Qtium#Q# zCUMpX5t1qp84T2@6P#8TkN}sUam*>u_a)FXK|+W^0Ws>*Xik#{R21+vt;qp>MQ(3G zp;F=@G(uC_m2^aCGl9-K{VPz7Nmb?d+dav-y?7$=!Ae{bVW@dhGR;Vk|`XSzDC{jlL#(DoghBw zFeDIYN!a3#tqIu*5Td}d1SP_QVnI2M;GiVOV^9)=d!{7Z`Yu#gm=45pp!DlIz`Ry4#?qU^*dtt1C zTIla_%DU2Qk0Grd`z#hN<_9cw1m72d*W6Cc1)fvB#*7J{y3v$*2bVZ9 zG6TG>O2=+U52~s9$l0IgSjDTVspW1-m*sbkG2)GmEG;vJjb1MLcHCSr!7*!}y})Nh za8Qtt`8prGxIOI)_3bCWXS`rUPDe-C-n|_uWw~2xw$}W0@_I+y2ah_(mJZ{jq4hE@ zF!Cc6R;XupBKS#rMQU-+>#{psdy{j)E8fai5U@nB%ohMMzQy5K`Sbu}GNNl@`HD9Q z+j(OxE|}#kSyEnM2}O1C-e*&hf6VGEx&@7Yz`v}WC^h^)?s2ksZ#Z%uCzox0Op9(uVL7qxkO zDYEf8SikDtf^5&);&Y;tc=6>s-#G>ccNG6mTd z@Q6tt>|L^+oJ8Cu&EJu`$CYdn7kZr&A1zwjcmm#Bmwv%)f^EA)ds^YejJ(X4Npf=C zNsFMb8dHB6_37px5-X*N>$$f_G}0#~_*nv=guqWxM%+;(>3$$r(P?nAKF&U#R|`_D03avoH#PrchiRW4nehPWlQ+3 zCwBWcjGj2&?o8X2#$UqnzRAB(1PrrZ#7U+u}nYp>ai~WA@UT?wT;on`pC^CynKG!ZqB#m_{f7xGqE$Nhsc~iW(qF_O?)5D-rtnGJF2!|^vg$f+>zz&BEu?N zuBJG+q`aK!V3yS~qugY+gKOmJSFWEHIQY!nG^1MSbRu_8tWPQ4YPo&2S;7vp_1vm_ z?vLSl$AyT|5zokorUf6rX}xh@-b$~~_4fHz=6{}ZT+hb}UE9OXjtLYUT9xNHbHD!- zx5+7W4uL;rJkF`F9PXC7rE=VHRJ6^pSeE{q;=8sA=jQ9yk6YUwtbR-NbT22aKxz3| zx#fgpmnw-Oy=X_Br`c`C$L9P|rVWyWVb!boS4A^O`%lLe@>(V9#cTM7hkt2*XPXex5XC^6AjmBSMV?=m06!x+$sn-w2K z*m8ShVi1Nk6ynF@D$s4knU&(~HDmn#^7Fch-|aMU_`!9Xj}?d0d^;ph7|cX|*e<}d z)o5(wL@so3=?tUU9CxGFjQlxoa8}Ok`G*olfWH#htmvfvJIbG_9G5Ql2Dug?5pXM# z`J2md85l4yU|_(&fPn!60|o{R3>X+NFkoQ7z<_}P0|Wmb41i~%U5`aMj`{xqt$WqU literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.files-0000_00000000000000.log.2_0-83-93 new file mode 100644 index 0000000000000000000000000000000000000000..13a41e55cb2603f59eaf78103707eed6f56d0e61 GIT binary patch literal 11180 zcmeHMdpuNWA0HB;RW?Zp$56SBYZ7(~lO+r?x?wqH4o+syndY2vNp^cn%XV2xS1YZq z>+Rm!WV=zQRVZb*(2X`JZJV}Ky1dUhGls}zd-HzYKg{PdA7jq*{GRXg{e7R`n6O#z2bqG?7Vo}V{_7$n6kw{n~Aem0okR*$q>=DR94F4zwrF#sEK@vWS zIT0)-Q4+BDz=8DI@dtuO?f5B>ok61n8xbHRLLD&Kfs4R#0Rm(4 z>0Ge{flzAuC;?IPp=g+ZA`k?Fz=@*gCWR*TO0)bBxZ@ZJqZk39U>{;3Oo<7CBX9`` z$;CxLFvh1BB08T->a-IG*%`c>0LO)Z6%rBmO?mI!b}}X3I2)`e-Eai85cLtP1U3z} zARxf7?p}CDL~^(GM~UsuqJgw8MVW(mAaHfwH5G0M5*AVllDh1J7V^S?5sZZ7=AgJ} z;8NPla+SvcPxHhG3&cw&5P6laxlz@1QLk)z59tkF<6`cQ6nH&EQS+0 z8|#M@-ysLVNMH+`A(U?=6CxBTfTNf|jdH=Y>jLt^_((i;3Of4|=ouG7us{G&>XKJM=t~cmyswzqP+LZ zk2;+kDowU4qZ@|)NKsB6@GJsR!I`EhE*j-UOZCwW4Fros0#xeu9+I4N&PPNiCG4Z& z6QbB`DjxUDU046PNdx5^G1xpJmOD*eTEtNgKn(!NA}T~5XA%do6U00~t@I-h8XrPQ zD9#Ef&lQv^lL#mX1A!4(>>gq0oFT`^Y!B8{1nyC2UKrUoNRaUc0w9%0dj;nMQdQsR zsDw5U@aRO@D?Sea4)=q;NNU(XAfyfVP9YnY06FzdBA*^N5Fn+Ndq?R;<%PaM(&=L$ zFiP`MufCR+g?(d`M$rKPDL=yfiPODq1*DFCD1sUV3GgVC2MY#ZJ9HV{J4}R!3SmLt zl(VG+2N0~Bd#O+?m@;0t(ms?g1pq>?2qXfOpaQg$oi%9m0C%A4w3(dL>rT0#Z1Y8V zV!R(`fEsHlqzC^mHZ1EF-rX-%7s@a&``#H{kp7eH(Gv!KkHCqb69SHr7d5^ph6*J@ zs&J(G8fnunhOsDw1-XF^gAak0gf9N*nvkvlAp+bb2w@8-4wU0C21;@)77>GRPqjoK zi-SBb#zkV3I!$y*4yxZC${-;e|Nb(A*{z&-CvE$@lP0?)rLGB}OQ`J{*4~W`vXj+u z1mCAr>bv*&xPw(V{-d!9YN3C|DeX#kTtfHk(w3;b@Rj;s4wH+MAkgusp;RwGPo#U5 z9=t_e)O?6xYF^*gNeT_soy+UFUSSg?E{3BJh-&pIk@8U@2F&b9X|(kq3|vxyfJFdd z_kSfxr&*xq_(#k3qm7eQ?)0I9KCq4J-hzBkD;Jt1y@}NRo9)VVb@TBy1DBL`;DU0} zTiFX-RZa(E6#NZVs!GQ)R$p}H46zxh9RAY^$LUU)+??_CPZRONvC(@U8=bDIU;cDg z&TZw?^Zpk_Y86GJ3u{bF{h5!w8zM_vrc6xUt7`Z!i+v?0^V1wT=uCIkZ)-PhP15Fb zE{E7IPUW@Cb-;=nui$DLPb>Bts1|(oEOx4|%~jjmTHboU#yUql%YE*2q;#z)L;U0J zQ)yFq%#@Jde=(ZVXz;>lzI}P^@*<7gg&M2k^x1#wLF$Ub%?w;T8m$>WYuT%uvE05Y z;0$UxAus*@u0LPpt-Smo#l`Vn^W3*FIf2HkkO%Yk!QKni-B1<1n4zlWM#{T<)*W73 zv)Q;JOC!w&lP5&=- zaokJ0Nq{sgX6#)r_S{*1eq39?nT`3 ze}?wOfO)!!fseMF&^`WON6w1pQLhYcB&2K(tzB4E>{X-GGIHvTYxt?nFJfaIrXAT| z@%Gi}7w2BwN_l2=L-M5l-w#9ctrEA)u}aIXUhs2Vmfx1f>vt=^%)kEdX;|oy6JJ#h zxxBq~!8Mh}Z`}0{Eqe0Od3+12Cic&$inA3@UZfALRQXzx`Xoos<%D2L8>6m?xoFsn z;)I*?L+lfiLjKOE(G1GU7HJmd8`v6}9C-aW?X?T>=BwMs_uP{Bq`g_R@X`II)Phf@ zz$bkaTd$6`FTHuwd#va;BvYAt`FzWd;{ptP`QqT+e~mx9Xp{1B&e?DGp0y3v)KRWd z$(VTPDEpF)`nlu>>FV3uJ;eN~-OEyok`1E*eyA&b(3Z7y%s7o~tAN>k&-ADf{_tyayk2(1sbK|Ura#!;y1}JNDp6j>FrllbgPFKbx0*V!_eR+N{mtP4lLvo?p(&t4f)})>!RSAELL} z!kck#MW$JKeTdtL#b!rKNTp@GaXWq1o6L!MJzo1>YEDS5SIO5_m(q4Tu4{QzUn zwZV&!lrneCv)<~QZX3b9aNTc~ha;!4?7}3&Ab#N{$kib()^5*mV$GRAQ~N*MG&R05 zwwYn0>6D&3arsfL9HzB;Zp+DHRg+!k?*y+|XTzTH6Q4M@hZnrmZ2Yt{F1jfyjw{lv zhjW&xubIDj`B2S=JBk-ynw?;p<+s=*1UYc1&cwIj(q--YstnCcEfHUF^tH$W{})QO z4=1g1TkBU9Fx~fXBUf2H;QUj&YM%o0*JDMl8K$o4ZW+m(xOMf4I-|xN3?K3MmhmX& zSP!i~9L9tMlshxSWpF1a>gRCpvwmWyWUr_k4 z1}Em%ndarMt6SVJZ``w@iKAw^G-YTClsD|M@tv=tpFUplNzV3wVqwki$z>JS zjGT23o_M^n;ArWh<4JGk&)za^0^>_F-5D`9MGCikY{%9<+&ke4XX!6x`mWImw=;cy zuwnX&UJ*0)D`}lxZ>n!4vWQiked}kovl~9~*F)OPTOs`&>+i5`{aABBV?>_&MLQK_ z!l@^oRhFlxuzzy5wlA#Ks^qJ!6yDX*Ip&_8cdptZIG!aqH1A?F6kL+!yK#cSR|y`g=WB^Q*!6b-ORUJ=|8gsggHNSafBaetPq3@sJG5>Plm5SYex{ zs6owD_vYfEty%{amK`{?kK<|ZaxbfOR6&XU^ja$(?^T#%)^%;|*VTlE?vCFoozI2V z*^Sb8#0$99sAo9Fg~N+zZi%$YsozlTwX%MeXo5}o)}~>;DxS?ZKRwG=+@dr7&Pgra zhUi)W!|N~eHQSRLu(JJ(nA+is@_#9hzsj)~U%T?`ovNI?#SDw}HGdpFGFMk$P!u#X zG@~Jb+jRZLHnrRvmb)S^1#ZrCZkxsReWRY4lsK zKkdF*oL$=(o$a+H?9mLy?e9XbaA#2|k7?|ydUwdkD~dYiP=%j~f;GcYX}W?Z!=EwV zw(C!6C=Px^q3Z#b^oKZ4s%G|{?GCC8SPX+l;nZ(@gRemb1{oM+V32`91_l`zWMGhi eK?VjH7-V3Ofk6fa8Th|o0NjgqT#Yi!cl-we?@3z# literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata new file mode 100644 index 00000000000..abeb826f8a4 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/.hoodie/metadata/files/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:48 UTC 2022 +commitTime=00000000000000 +partitionDepth=1 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..901b972aff0c2c1daa875aaf7ffe23d1fd77e34f GIT binary patch literal 12 TcmYc;N@ieSU}9)DlYRvN5nclQ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..eaff8b6a2b1a4363260f136c3a81f30a515c23f3 GIT binary patch literal 3432 zcmYc;N@ieSU}Et4_Q^jnZDTXn?nK7z2iUh?SX!{-Ka;_`y8G6=OaC^Clrt}?{;_xW zu>-E_;srQLpE);}OiL4z|3WPstIk$HtXbZZ=dCc4RZPdGen;n#L~P<9v8=N;X}?x< zUcx1>)jzQpmmEIv-`7$O(Kt_amnw^9iwYLc`+G)$Cj-a~oH@!qj@PI#|Kj#L~OZruZYP zy*7(~Gctd$us5V$)xhVLMQ5Zxqstq59}7}JM}hgW%T97H#3;ivl?=wfkB?#bw4==|f=+9Bv- z`1q50EUsEKyA|P6Lkqd9aZ@Eh=Jqb{EJrp!*z3w;WIi(cp&|cUWIj4O_0~L1bTJU$ zIqdK@WbyXk+7-xrWOnP-JQVjJ%h7}F@X>*v9`=x8N2-WQ_o}l;ucPa+TBg1ST@0Cj zLg_>rh`-O>b|r}J2s59KEKe+(Gw&(EIFYv~b%vRD_tf4uFt*L2V0joD8y!3Cq8~PK zLSl7l-Sg1Z+_;~gg)Ro-i#}*I!zPYPOvfgl3zr;7+*_goXFCp?zHP2cvAPSJT4b@n zs85B+d~ECnGuEXYj@!X)E^P9+#Fl-&1J3`r)!o7IB1u!&Pk gtRcgX?Dn7bagPbO+z}tYD-K<*&gZnGueNqE0Lh$Gn*aa+ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/brazil/sao_paulo/8a9a08bb-8cbc-4ec9-a2d4-8a6cdcaebbad-0_0-73-83_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d929431ccd391e2d88189be66c42046c70a86252 GIT binary patch literal 437831 zcmeI*4Rl;podEDjOw-WqC+!<18ll7zK{_Q((tNPEkd`7dJBiDGIymYWX-W2iaZJeUoI8fqYO( zQZkcY543OIz4w3r_jljCcV8wmb4Kg?Rt9CkJA%`L9S?oxhWRH1r9oLFa>O+ik-t3lAtTNV&CI`D#^BqjNRt2S=&x2KjoD2=-uTfb*0;T z+CLi2EX-W~o;w%*?I#ySn$B6&uzoPwOC`ZmCBe2^?)+rMnBjumE9bvyWyP%Z#~$-S zb>|PBnl<-@^FDpW{~dAUckf?v^Y1&qe(7y{f7>>;FE{ z_L*mHdwa{fj#=9s`^uj;oUruR@=w42riZKVy6(G2Z94SgPd$9^4^R8>f9<{H$B*y& z=`+7Q=YiR~ufFPz(+)7ZNN_@s4t{*|m!3Uj%-I<{KrVRn*n1oQ`Ik3d+VgLpj{f4| ze|cfi(~n;EyBjjQW=B4J>^t9g_+aPRNWZ-1)_Z185NW}8HvIm3|M1)XKWu;E={@a- zX7-&}@$c#LZ#%ndLF5zbAKEq$sVt}pP7a>_!oHs@80Gb3c)*@|%dD*FOA_Z*99|!vi;5__>dNeedF}kwdOM`INj! z-@fOLo90Xq>9C(IKl90Dzxwra&-Pzb_UL6Fy6UQ}vu=F$uMdAB68UEL*5`*1-=L%Y z&B5y6n$KSQr`er)b?TJKVJef+jv^3Q< zrJGU>m9b^1`sVhI`gB9QF5l2;w_0}L%b=GPB$gPeXCkl<%O+Gbf9j(9_=qa~iG3p;F1GkZMAmrli*SMB8lz3_3sc+kq$s{{dEBaEK``Y>v?OpksZM~_EOm8wc zJQB=3sx-E^ZLo4%N2a^GGpyd(ojNLVT2=NOnJr)Pit^2=^LjG*6%Nf+7>wKYvD8KR z#nD{xxryGs&c4n}Px$QY+mv5rQLak5vn$os6F#AHrE|gr4rxzq?CfdlO7um7{KuJ| z!34_(5BdBivr|3Ekp_d0vemQA%hHM7)L_eb!zsNoqgew-VJ>7@*sQc~^I)U0En7O1 zLnYDj%Bsqj7Svo)70j(%RZ&)6IVNXN7A=o{wj}!VlA23uDyqgf=Z>)}8aaCOxp|Cw zk;>)aBC{+g%`T#||3spHw=7r^z3;G~JyP?la5bGxEfS_S=1dOcwx=Yzqa>_*LSy9U zvCrdUpWbW3r$pFm`1}id?OzteqW!U;H&XM*(ve=rI6MaW>@SU8R~pvL_PJl@{Mddy zwevH&JDbr>l|g?=q-ICBNXpI8IEUmwS~r$NpD77z*KD6vHNo{`f}(KZ#y<;A&~A*0 ze0*6DjsE1#L3oa<@vVDx+o8QM5vwiG5Jln^E z;hHsf{0yBk2M?N(=z}F;Y0ZE9SylduA%8pUC~qy1Uor;Xca942esg$mN~C7jypeI` z&dCFvZi;Rz2}{FKEeONs7D2I!+{!2$e)Qh_@R!$Qr)6n& z052{YnU-Mx4u@4>6(K)#nne^OF&_u`tz z53P!gEZ_1rL!2@9k8vRNXS>^XWUx9?vvWyRP&V`=46npPQ*d=jSP)J@ZW$OGUK?ho zVD#SnDfnS_3i_+FQ?M(%^Uqx)2d?e|;RmlzyMigaK0TJrVPAFj`V@Wc&}ey0Mf8V< zEeY3|U9rVOC*Ku`^LsNZHg{}FbthuCS8T2ByXf3h_3_n%Ypv?#)!Efj_;_TcH?+JP zD%yN*q9a>AYC#y5r#n+!$<5WrpS?8~CASXCTMCBt2D?1Fx@SvQSNL#qUvFp6#_DsH zS0^(aVd!vIcn25WaP?ImpRID~^6}ygEFC8$;K)*QQo;?d0|%C-lM-@xDf_yj4KIC1 zM@eS3gzNf&X&6U;sBxy}KpGD&szUcne6y?i1L<;j>a({1 zqXWG9a0l)b4y0k<=FzRXGDFX-acr~q9-~{0m*hE@Mq-OD=$E~d=5B#0ZR%p&{ zrdrpx_M|`1FtFQS1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5;&LK0Y4NXg2T009C72oNAZfB*pk1PBlyK!5-N0;2@{lyek`9j`&a@$(w6R3$)w z009C72oNAJGX>Vn%wZcf0RjXF5FkK+009C7rl&yoA<_O27$vZF)E+ws5FkK+009C7 z2oNAZ;2;RBJ_vp2f&c*m1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009E~E8sQE{gqOW z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0{bDben0A{LZFZYe78j*&6ZpV6o7ztHy6MpNs#~n0t5&UAV6TI3;0>UOg{pQ zp8x>@1PBlyK!5-N0t5&UAV7csftf3?uAqj#rl7KwECB)p2oNAZfB*pk1PDx>fIGyg zvlW>D0RjXF5FkK+009C72oNAZfB=E%Bj9JD(`PPZLVy4P0t5&UAV7cs0RjXF5FkK+ z009D1A>iZ7r$SXs0t5&UAV7csfpG*1>qAz?8AP$}HaW#Qv&Kb$009C72oNAZfB*pk z1tZ|*ykMqEq67#KAV7cs0RjXF5FkK+009C72oNAZfB=C)7x134LZ3nT6Cgl<009E8 zQNVBJyhbq92@oJafB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pkGfrURj2p256Cgl<009C7 z2oNAZfB*pk1jZ5Y%_if(*iC={0RjXF5FpU%MKuQj0RmGe;0!Z$wjvWCK!5-N0t5&U zC>nt^MKcluAwYlt0RjXF5FkK+z$6NIm1Pn!g_?!}>!)E(QX)Wr009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+0D&009C72oNAZfB*pk z1PBlyK!5-N0t5&UATYxP*3R(p8$JO71PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 z2oNAZ;57(%YwByjQk4J!0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C73PZrxo)*SD$&vs80t5&UAV7cs0RjXF5FkK+0D*!L z*jO-`N|XQr0t5&UAV7cs0RjXF5FkK+009C72uz;9n#l_#5&;4P2oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009CAQ^02u9!zb05gARv;54 zO#%c65FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZ zfB*pk1PBlyK!5-N0t5&UIA{WE4_Z$;B0zuu0RjXF5FkK+009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF z5Fl`X0{iu$S?dlkxMl z0t5&UAV7cs0RjXF5FkK+009C72+TAAzp*~kX54rQ5FkK+009C72oNAJ?FD@C>9n6B z2@)VcfB*pk1PBlyK!5-N0t5&UAV7csfte|=W@ZlCs0k1tK!5-N0t5&UAV7cs0RjXF z5FkK+009C72)r=@-evU0xa*Yw0RjXF5FkK+009C72oNYr0ly1fl+$WZ2S>n{1Rfk` zy$~QkfB*pk1PBlyK!5-N0t5&UAV7cs0RodR;AO(eHx-cp0RjXF5FkK+009C72oNAZ zfB*pk1PBlyP!IxMC@P3)k|Y5F1PBlyK%npiRu}#djDP?E0t5&UAV8p)1uR&Ld3KGB z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+0D)H%@bzo2CZwDI0RjYO zrhw05n3?l#)C33+AV7cs0Rq!pz(ReRPmuHo5FkKcItzGL{dArkxe*{hfB*pk1PBnA zLIJ-XHifuC6Cgl<009C72oNAZfIwjhtS_u|r!ZT;f-^a|j;P6T6omi*0t5&UAV7cs z0RjXFyiC9i_scjuAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0RjXF5FkK+009C7idA53u@20*2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5GZ5; zUxHo8vnO`~1PBlyK!5-N0t5(5Hv!+SKHX+SJ_HC5AV7cs0RsCi@Xkmq*p$g6J5z1l zojq;6sg6uSSrv)B``kqDd0SF_@$TNG z=cjr%cV>E)u3WLQZbjYFquaN1b|sfKHKponJDO`->g$^uQyr`7I_f*piN=olhW4gZ jx<1URzOJsLrQw*$mljk-BK!Ye~1JEqzTD?!G-NC?o7ZX2NLJ6O8t#bo?N+Ai>(i!>tzs|XoxH(&35_$ zG9Q`!yF4@&nUBnNRQEY_&QbdevKXc8pOYtvfXvWr*ZThIa2E?9^K5^7NXDgRrJ>yl znB4Z~KkviX*yzKTUdLe*hlz1$#I1+1(dp8+%Vl9=6WM;o!`R0*+cm@3)S~kXs>@(z zo!KJGkZ|-hINhz@W9$tR2hoxaR`#*|huheCf?nbY5L9vrPaz-^w9VN8>1oKvvM^mnS z_z?pyTP(aU&X1@H;DnhQ`ZsVgh#mE|u^T2<6Ib>Q#zv=kPjhdEiQQZ+fZ|`6IE>~` zc>Dy$rZ#=Nw@wt7nT;QmP|`X~-4r*~y)ZU58a=*{)ts6o+mFmAm7VwX>tkZ{wEe8X zs3%fIRN!#~Gh;&Nu^lk>K+!vR-Qnq*PbY9OvOA=>*Ia<{jdv`T3z7<%!eb)|FC%8j zc&fnUanTlOTNZ-kcHF9=xb9Qh>yaydJd6X`u4es7IO_&vbuhNjcPSni8$>^}K7v<& zZ>_RWbzmJUvL0%(Lj^ZUfz0bs*bB~woVKrOi823X%mQ9w)S;_+?EWAXT@1vJ;qsWQ zaAxBgm|WnisqDPcw;mMIy3V*IJJo$b4$DKRKrsA)9yn zbmnXrf1S|=eHa^=HV<74&R27t_Y+-zm>RwXw^1tZ-hrntlMBvMks5t z+N%v?qtjQ{iI~8|n)2(VQ#xC|!uZH^p2Si`WIjIj>-DKWLGrmi7TKySr+;wS2NK`s!MO_n1=?lb literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/americas/united_states/san_francisco/34b1b177-f0af-467b-9214-473ead268e55-0_1-73-84_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..053841a0b91b8966c1f374a245825dc5ac0de548 GIT binary patch literal 438186 zcmeI*3v^s%odEDjnwGTE(%Rl(0#Ry(ShNXwH4oM`qy?7Bv*OC~@T8Mu(gxE6k`zG@ z0a+J|b`cTK!-}Bma(u8XdVmA#E)NA4N(9zLK{&GDE+`dOaVs2H-8)GpnUWNS<~5V~ zaTI3m{T~1C_rLefcPIDWIidLjRY6hk?%=rKnukC8#YOK83WK6Zw`$JIA{p2_}R{_`;-O?!iMGrVa3tfZLJ$N zY`AjchIx^Nci(kKvdyr{rwW6!gD?H&rmG8+bt2=}*>~>cN6tI)$a&Fw=N;D7(c0a5 zdbB6jbHUMfFTU#HB^C4APJ3%v+Zol#Q|9kn%hks$+QH3`*Uf8(wysv(8 z@BTXu`thUZpVo2eBPZXp{(=YJw*CGMosSLN_BY3!U%LL+XKlG5T6FKK%2VcFdhg%; z>bcW@e_L>2`G;=$(%*ih^s*guZlAaB|8D>L-+uJ5fB3-Mt{0y<{h}j|dFaf6Z?!-2 z)B%m>pIb4IjJYT{A!PW^KivJbf-&eP#y4y@YWuO}_aAfIsz(ob@PSq9j{Wo@73)^5 zD}VHmY1i8CZNKKz z-#j4}d@%8s759GPFP6V@)-A7m?AaF!Vz=G8`j|ZoFA|&06jVeW z{!?Mt`kVtw3%4#i>7M(VPJ4M>;a{Khbkh$9Ubv*V^LtH)|KQ-&4}SZBrhOj$&Z{4L z`T3@Y`q~ex7>XzJ#y@uS!S5KD`-p!`tlamxrqgdY_P)W!i-OYN@Zi_i?iko-4C^!D9&yyc_pds-{E^kwweuq9{_9UK zUo;IuXFv4Y_5El3wCU<|pS5^a6opEjT zPny<6F8#ttnPyK1zjsG+!vgvm_P8-Ib`Tc6YU`=r8w$J}*ud(S552q{d zKW5!<_P=t@OH~Uekp2GP@@JpC`1J^XwBh9c`TD0ncVT?vEB9|)e#+;6@Y{k*7M=Qs zr&e6@$20qWa^m8Jk%Nll4y3QMAq z$Y6zFOof)JmdcXYxXWwPDjrx_U$?rdvZ}7UE>YWBUR&GRTHctbY%H&hTWcWE}hInmqM7VkS` zOLu2~qP?ZBKi;3{JESk(-O|w;?{4etYwHP{&7@dbw1?HywpO)Pg<*75#yiSu>*~X3 ztE+0uYwK$g@%HMvhQ#XC<&`Z}VO-@6wJl>*N@5djqCPF0;?zO8Bvza{9;dHP9Ztj5 zsUvRM>ILn+oo6L_%d4wuN@8hm(ux$Pj%{HJsl!?N>eSIFT%9^lrLSIi_Idj}QC@Rc zxca}2-@EYfwv!G^Tb&X9o6phw-p=;as77C>^;`U*8=rdkqJnVCjTLVy`pS;v%e2)~ zutkY&euq;?>1Ry>nAfPkU#gBHU;C zTl(XzUFk1ddJ}Cuz3sumf3>9u^>)E`yGc4Y@IdM?rn9}4yIJth=Yu5KA z&g||fuVL=K9V>meRVXo`iyvQe`kMZPj|~1@%~NeMV6$BbaZwlTDrr%I<)a6yrCT-y~aoqoybieU{;LC-PvhOYk=kCLM+vs6y(EBq5(c25c z0`DFDtdEUM0K&nJ%m7B0rfwWAvXiYVlAT;0PN7qWwBZBn&}LH*eW)N@TXw_3()3wJ`s=WxwE0N-nsNBP ze{jh6!xh1ik+PSP)AnJ#)Db*wST8LKhn1Sf#75>e$zhFMnm(+@lf&Bf&g8IeUi{iF zc9FYzao9y_DikYD&5x2@j9r@E#qS2wGuXwBC8J#oe%7Rha7#hijwNCHskv0FI5oja zwm)`hdiz`7oy^RE$sydiY&0{wet->jG<5vBb6MH-OLOb^b^W`O8^VEu$FFVC(J>Bv z6c3K={OCYI*|uoe*Mrj7=-e>vWe7R$@^QAi+mhYwJSbQbDf@lcThZ`#7kOP29o)@tt^?ZD`P`rNqzzk&K(BZANusWGB$;ue*c|} zVP{zqH2RrXbU|5hbmxA{!dYxrZ0Ydfx*~p7Z%;*E+or_kc_^1~mG z#Fl)tH~fC4w_39O&4j`Tx-009C72oNAZfB*pk1PBlyK!5-N0t5&U zAV7cs0RjXF5FkK+009C72oNAZfIyxItjm*e8YTe(1PBlyFv|tj&hqhPHoWk0t^OfE zfB*pkvq`{1*=AEVjfDUK0t5&UAV7cs0RodC;2~U-fT#%q1PBlyK!5-N0t5);yMU9+ zsow%z$~E=IdLTf6009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C7 zvO>VK-?AbNg2+YzPmjvRtcdA#1su^|m&Zy11PBlyK!5-N0t5&UAV7e?lnVH6W=eTA zPJjRb0t5&UAV7csf!q*qwMlN|Nt6T#5FjwE0=_$)R&JdUAV7cs0RjXF5FkK+z!(8P zJu-&G0s;gG5FkK+009C72oNAZAb$m1nGfT1_%%!K!5-N0t5&UAV7csf!QJ8TY}k<$1EL%dkwNQF~TB1fB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjYaRlsi< zATVnMoPy8V z)EXE80<%oO@%nWVy5@D(Sw&!2z-D4on6Q9ntWB6u0|W>VAV7e?ObTqA$CQ#Md z}~xF_N{9I1PBlykY@sZUp3DXZny*p5FkK+009C72oNAZ zfB=EX5%AZX$#K*Qf!q?vpTAh=R$fF)fB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72xN(X=T2lvB62Z|OmFqHk>z4`#Y}(z0RjZ3T);)XQ%W=^c0t5&UAV7cs0Rq`1;L@UO z%7$175FkK+K$Z%aFl13;BqNKGArt}xa!0`519K-=A|*h8009C72oNAZfB*pk1PEk> zfN$foA`OBdK!5-N0t5&U$YX(Zc|3MQCqRGz0RjXF5FkK+009C72oNAZfIuz_xQQp1 zvnTe+5b!sSNl;uf32n@>CS1=p%Mxlh1PBlyKwuIDoP$jwrltrGAV7cs0RjXF5FkK+ z009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5Xc9CwfQheVYBmn{ha$CT}D7W(`dIAIp5XeFS z7Yt@$QiPN(0#mztcmzYXak-A90D&nK@D=+M z;%a&V0-gjh0X$W+R={?hwW-Paz?|7)K(dJ}Cuz3nX@Nt_cqu(G~xbya0mU3p!iwza&rwzajq zF;UrAURP7u*ic;;uWx8iq}7g}y-V%(){c(0n)>?k%8rJ{^4gBJ)#VNG`0DcdnyS{; z_SV{}`gm<5_Wm>Cy=QJo^p|h$U4B-gx39COdwEqwRdq%6@|CSyI=kAJ*ViX%D%%<= z8*6GB>Jn|MtJ`YYI^uP0HMOnviH@3@>Z+RR>bAz(gG*lBr!*4D{C|;WzBVs%Lh}cz F{vYhxb9w*( literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/..hoodie_partition_metadata.crc new file mode 100644 index 0000000000000000000000000000000000000000..901b972aff0c2c1daa875aaf7ffe23d1fd77e34f GIT binary patch literal 12 TcmYc;N@ieSU}9)DlYRvN5nclQ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..9fae4a530439b64300fa078deaf19eda992646cc GIT binary patch literal 3428 zcmYc;N@ieSU}9LgovlRl$*&o!w7J$i+&5ESa<^O7g0GXplb&tertr)`@_*KkCYS!s zSp_9GrruK^z`?M=oej-IZNbM&{?$(3GN^?ooM@W50 zN|X#fHG3AANrU9gr}Lb{rw&=}yO~EOG9MdTP<+czY~u71o4t5qBDL*$VO%drZ8J$V zZ`zv^;5^-XD1e7ly)gBfQ!8y@Y(lhQO|2?SjlY1mGmMRmUc8AR9Gf`0nA`W2OVGtY z{FJ3riPDcvFSW&_EK2SYGE@rj|2x;g5>)IAQ5V}Y?@baJ(l0gR1I-$~D9L*^61 z{=O%omKb%|)Nov|slp~steB~Wv?D&XKhAtOj!%9t$vx>ln}^LksYhh|&xoZ~Tk|S39;q5B@qb@UJVd_ITCTGFeLzTABT;@Pe vx2w(Ar)TD${v2lK^_B8BVQg%)Ps42ciE@c=u*reM=EWYrk1Y3UV;L&|A4@|J literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata new file mode 100644 index 00000000000..a8c28b02044 --- /dev/null +++ b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/.hoodie_partition_metadata @@ -0,0 +1,4 @@ +#partition metadata +#Tue Aug 30 08:36:50 UTC 2022 +commitTime=20220830083647456 +partitionDepth=3 diff --git a/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet b/tests/integration/test_storage_hudi/test_table/asia/india/chennai/92aa634e-d83f-4057-a385-ea3b22e5d6e1-0_2-73-85_20220830083647456.parquet new file mode 100644 index 0000000000000000000000000000000000000000..c1ba250ed09695e1f95880fc1bf3457620be17b2 GIT binary patch literal 437623 zcmeI*3wV{~eE{$i3?U#Og5UAPMnu#+UI->RNg&Qu0^%d;I#B1V)`fF&P9QTRm;^14 zm#Oukb6Q=&b+vVBr*^Gdt+VQGuCtSFEu#*nPMx$`M%iPlYqfoxb6)m+CppQ{1mqeL z^2-y#cfR-X|G&TYyS%xaa~3zATOAY!CkCGg?s)2Rw@*Gb7!wr7Vn^Ln8teI(mRY*A zs-~)@=E17^SUNeaYub6q*q?v%w@(hj@*oyW3FZdBdS>Hu2bYK46bB_i*s#BKb9(90 zr7bN>OJZeTeD{H<&#=kLpe6Y1OKZL~CTbEJyh(iQ+A~VdIHM%~PbKr(TGH+5-;H-1 z)^Y9G|2QR9JMFdC=l1mU7e&Ft;pZKjAIyax&{weGJCh!tF|oMuu#3L`+|=(Z{q9dT zr=I@3@kihF?yH}=@3xD6e%`(ReCfCE`&>)SygxYT=#!qEe$m!*-#_xoQ{&SfdSUhE z`l4-X&wt~w)9c@j9em?wFT5ZMSscs?76-4r{)3v1e}i{JrtrlqwFsSQLaq{L;-&y)u44yh%@7JbB){iDNgtu=u|Aqt5%a2M_-G zIaeR^>Rs#Zc;t~sXFR>|rZc|RxZ&SVIPT^Xv*R|bdG4IYuKU}?&zF?Uz2W@_F8RTq zC4c_2Z~WImZ$G|%!a2K{T`Z)yAh>_?#$S!w!8|^&?Zw9)-tyCF|CjyEw=+L^?}rbM zi+%FqI~MnnL_Iw6(pPRR-9Zl@|Jk3MRQ>cjfAsD>SCpN6_%Ba;Xk4s&`dO=UJroD! z!Rf(kcfI=LAp?B=Gn$VRHq1J^^55!eYD;2Q{=@TYC-1>n&kr`$?8#e{Z|`^@qMzzvjI^elzySv(Ls0hWgj{{m&CcI~Z!q(sjRl>+&o9{@sJVQTi{f z2QPc^pxF7pb<&c)9?lOQxbCI)9dz(sUBl7q{_2VE{`gN5Cq1$5%WEdbZu;>b{!K5{ zkRG{51oMNNZdv#KxPkK$UQmwTaL%IMX}Np#Th){HU|JRjufOrV8+W97P0Q3P{_@7p zU)Olm>vxTtaYU@_TeH5(D2w)wjndOzM-YEs{2wx3*`(ug%M|?d_@7 zvcw=e3!2Pps7a;jleO8(%-m#4Wo=d6oXS*kZe3+Il}y*vWa~2Z+3L!wrkct*$;!EP zO#?<;me|uS<`kq;`qizs969~H7fvfnlwLXcTlanX*{!D)7H|3Q<*(dx$d;)12WPKH z+&<^0g~byxovoK=J1c9dlVyp52ZcRMeECytFMei2)I---FIp43^_9XNN?$(l!iFz= z<;ziVtYrF`Z~pL`A1Hn*J~p=Xh3a`>@!RW`wodx})8`d*J_z+k`H$~x&Ezxg-!AC0 zbX)E3Jav6hc*G@255MP~m!r*s;(c%=mB+6NTB7r?tUSJ@D7dYtcl*sT!M6PNrm=lh zZZ7THe4{kDAz$Oo@%`Id5ALh7VPbH7zRJc){Z%$i4zA7bUq7X<`r4^|n{Q1GuE}RI ztv8bp;+-jH9<#ioBh#9l-JR{~Zt6~@+X^?EIIESPxQn8dWEp2|(l9V=F} zhSghFWRHuTQyyLDqVmH&RNj@nti7YK!qj|)o_?Dy&8{vij^~S4raHS@yIVWjn^vZ} zmlsw!G+(8qwJqD!p6X9vLYTn#baq*5dsADgI~Ei^?r85xu%zc0&vzJA8JBI(^f%~v z6jhJPid#~h*`Ah@`cnFEMq_gaVZO`au-TaIuAWB4t5&sUdQ0NPW#wh>98z&rc`&i8 zuC%zMY*5aiI9?pTr6~R%MHN?7l$H;2Ssi3oJa+uR%kLodVr8>(%h@r}dbKzh6N|;) zI662izBvfev5GC>?Iubs7N$1nOy+XCvnYOBQCRoX`q=S(uSL>M(Z{lchg zW(JA)x8p%)tm4gZy_}Ca$l;NT`JxrE@vkNo2Gs@_}!-EHkVilXl z_RrCF4#|RP+BCLeb8-1#cZ$LGg;O=ig>JB#gG|<=GlO{iM~R>%R&jH6$vf$1!v5GfB_X)ix zN&nR*=Y3K9hN7_i)c%`qqVJ9zje(#@2AjpFe(juZlefO;LPPQCM2>_Js1nRY2i(7^q-< zP*^eu-r8fLW41gvBUZ6tV*j}Em)+h`U0)QIhNH@_DiWpnrA9QWfqM%_^|fQdQQdoF zG^&l0J{+OYZR4acLVjVA=v%Es5eDupjPQ6AVM{K;rpf&ga<_-v04|JQTU4=Wa@c)- zVUsA$uXdvD2ktHG{`si;yQfD3SU;tIT0VHo$;IipF|40bvFVVZx#IjHH+$2gE6&=f z{i*fb^KvO&6MwQOEUUQx(DFq88mwS59RHyE2RT;O92=czM+Ng^6>o*nihIwq@P68R zm|PR?50UdrxkTUkEa1y4)=~*L&|xX4*cy$vkvyIMM58 zL^qlE9aG}P6{YcKg2TeKVq0Qb@8xxN>hjKx*&3m&UEg9Z zx4zsfUG}X<_o%nN_3OZ1G99bJ#d&TTwxi$MxTAe{8uzS>2evGj{q2P9T|f_P`q3HZ zn(kh{(b9Z(LiSC4w2mLx!AB1_x7^>IhTUBQTeWrco>|+mjlO{x*lK%8UUGFTap)DD zt=-wq*(*9S*|yoy)gnml!009C72oNAZfB*pk1PBlyK!5-N0t5&U7)k-Z%os{-u?P?# zK!5-N0t5&UAV7cs0RjXF5FkK+0D%!B;0>ep%~UPgH{I%$009C72oNAZfB*pk1PBly zK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfWW>M@Tai*T3G)C2oNAZfB*pk1PBly zFp2~gj-s&|DuGcX;M5*PlWnL32oNAZfB*pk1PBlyK!5-N0t5&U7>)vdP(2)HMNR|= z5FkK+009C72oNAZfB=CZ74U{bL&_}}0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5)`QvuJS->1AfCqRGz z0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF44uG|p$nCdAKZhDd+_0RjXF5FkKcL<#t%{)n0`$r2zyfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oNB!hXURhbq@hGCqRGz0RjXF5FkK+009C72oTr@0$wS2AE4@z009C72oNAZ zfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0tEJ( zz=Hi2SzrPL2oNAZfB*pk1PBlyK!5-N0s{pW4%}rA0RjXF5FkK+009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N0t5&UAV7cs z0Rkggz>DCHxJFBW009C72oNAZfB*pk1PBlyK!5-N0tAM; zz{264p!^6BAV6S$3M|^6!Nn#(fB*pk1PB}u0`m`uQ5XaP0t5&UAV7csfgu&})RQ6Q z7K{J^0t5&UAV7cs0RjXF5FkK+009C72oNAZV3Y~?v$jz-;f70q009C72oNAZfB*pk z1PBlyK!5-N0t5&UAV7e?5DIwdwjsn8iU0uu1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+ z009C72oTtzfLD6nA&)`=1PBlyK!5-N0t5&UAh3S~ycFF2aTk>U0RjXF5FkK+009C7 z2oNAZfB*pk1PBlyK!5-N0t5&UAV7cs0RjXF5FkK+009C72oNAZfB*pk1PBlyK!5-N z0t5&UAV7cs0RjXF5FkK+009C72oNAZfB=C#5%9{Yd!nZ;0RjXF5FkK+009CcN?^%| z%2u)j2oNAZfB*pk1PBlyK!5-N0tAMGz_Q_xgd7MEAV7cs0RjXF5Ez*Peoa3z=S|iG z2oNAZfB*pk2fV5FkK+009C72oNAZfB=EvE3j<%rY1821a>Uo^xd(KQUU}B5FkK+009C72<%UR z`TH}t*aQd=AV6R*2>g015iIZM$h2mgR+VUO=TN0bL$&gDywR% z>MLtg>2zg7DwU~h&L-=t)AgBbbF!hJb?Va}Xq~RfRM%(fQk5;~nsnH@rM@zis%x&y zCe!J#ESt(!C1Z)RR;D^HTb1puT+un>@@!{UYe)Nx>eYJ0b={eb!WU{6@SyR*8P GtN$O{ABc|t literal 0 HcmV?d00001 From 2abe284859263d7285903196884bcf1de6debcfd Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 17:38:02 +0000 Subject: [PATCH 010/526] Fix engine bug --- src/Storages/StorageHudi.cpp | 11 ++++++----- src/Storages/StorageHudi.h | 1 + 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index efba2d3f85f..b4eba258d0e 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -33,6 +33,7 @@ StorageHudi::StorageHudi( : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) + , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); @@ -42,7 +43,7 @@ StorageHudi::StorageHudi( auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); - + LOG_DEBUG(log, "Table path: {}", table_path); auto s3_uri = S3::URI(Poco::URI(new_uri)); if (columns_.empty()) @@ -143,9 +144,9 @@ std::vector StorageHudi::getKeysFromS3() bool is_finished{false}; const auto bucket{base_configuration.uri.bucket}; - const std::string key = ""; request.SetBucket(bucket); + request.SetPrefix(table_path); while (!is_finished) { @@ -155,16 +156,16 @@ std::vector StorageHudi::getKeysFromS3() ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", quoteString(bucket), - quoteString(key), + quoteString(table_path), backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage())); const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto & filename = obj.GetKey(); + const auto & filename = obj.GetKey().substr(table_path.size()); // object name without tablepath prefix keys.push_back(filename); - //LOG_DEBUG(log, "Found file: {}", filename); + LOG_DEBUG(log, "Found file: {}", filename); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index dd5cc18495e..47dff1c2a7b 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -53,6 +53,7 @@ private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; + String table_path; }; } From 6b95faf2b3a9bdae644156dcd0764030a9bb570a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 30 Aug 2022 17:38:57 +0000 Subject: [PATCH 011/526] Add createquery test --- .../test_storage_hudi/configs/conf.xml | 11 +++++++++++ tests/integration/test_storage_hudi/test.py | 16 ++++++++++------ 2 files changed, 21 insertions(+), 6 deletions(-) create mode 100644 tests/integration/test_storage_hudi/configs/conf.xml diff --git a/tests/integration/test_storage_hudi/configs/conf.xml b/tests/integration/test_storage_hudi/configs/conf.xml new file mode 100644 index 00000000000..e3e8627d95e --- /dev/null +++ b/tests/integration/test_storage_hudi/configs/conf.xml @@ -0,0 +1,11 @@ + + + + + http://nginx:80/test_{_partition_id} + PUT + TSV + column1 UInt32, column2 UInt32, column3 UInt32 + + + diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 716ec15524e..b2f6048f408 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -1,5 +1,6 @@ import logging import os +import json import helpers.client import pytest @@ -50,17 +51,19 @@ def prepare_s3_bucket(started_cluster): def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket - for address, dirs, files in os.walk(f"{SCRIPT_DIR}/test_table"): + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR):] + for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(SCRIPT_DIR, address, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) for obj in list( - minio.list_objects( + started_cluster.minio_client.list_objects( bucket, recursive=True, ) ): - logging.info(obj.name) + logging.info(obj.object_name) @pytest.fixture(scope="module") def started_cluster(): @@ -68,6 +71,7 @@ def started_cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance( "dummy", + main_configs=["configs/conf.xml"], with_minio=True ) @@ -99,9 +103,9 @@ def test_create_query(started_cluster): instance = started_cluster.instances["dummy"] bucket = started_cluster.minio_bucket - create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/hudi', 'minio', 'minio123')""" + create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" run_query(instance, create_query) def test_select_query(): - pass \ No newline at end of file + pass From 7d8cc20c4600fd5fd3aab76ca84584ed65c07896 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 31 Aug 2022 09:26:53 +0000 Subject: [PATCH 012/526] Add select test --- .../test_storage_hudi/configs/conf.xml | 11 ----- tests/integration/test_storage_hudi/test.py | 41 +++++++++++++------ 2 files changed, 28 insertions(+), 24 deletions(-) delete mode 100644 tests/integration/test_storage_hudi/configs/conf.xml diff --git a/tests/integration/test_storage_hudi/configs/conf.xml b/tests/integration/test_storage_hudi/configs/conf.xml deleted file mode 100644 index e3e8627d95e..00000000000 --- a/tests/integration/test_storage_hudi/configs/conf.xml +++ /dev/null @@ -1,11 +0,0 @@ - - - - - http://nginx:80/test_{_partition_id} - PUT - TSV - column1 UInt32, column2 UInt32, column3 UInt32 - - - diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index b2f6048f408..e2d97f6489f 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -5,6 +5,7 @@ import json import helpers.client import pytest from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -57,21 +58,12 @@ def upload_test_table(started_cluster): for name in files: started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) - for obj in list( - started_cluster.minio_client.list_objects( - bucket, - recursive=True, - ) - ): - logging.info(obj.object_name) - @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "dummy", - main_configs=["configs/conf.xml"], + "main_server", with_minio=True ) @@ -100,12 +92,35 @@ def run_query(instance, query, stdin=None, settings=None): def test_create_query(started_cluster): - instance = started_cluster.instances["dummy"] + instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket create_query = f"""CREATE TABLE hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" run_query(instance, create_query) -def test_select_query(): - pass +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = ['_hoodie_commit_time', '_hoodie_commit_seqno', '_hoodie_record_key', + '_hoodie_partition_path', '_hoodie_file_name', 'begin_lat', + 'begin_lon', 'driver', 'end_lat', 'end_lon', + 'fare', 'partitionpath', 'rider', 'ts', 'uuid'] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM hudi FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert(len(result) > 0) + + # test if all partition paths is presented in result + distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi FORMAT TSV" + result = run_query(instance, distinct_select_query).splitlines() + expected = ['americas/brazil/sao_paulo', 'asia/india/chennai', 'americas/united_states/san_francisco'] + + assert TSV(result) == TSV(expected) \ No newline at end of file From 0e9c3f299c75c16c05cef5b65be04fa539c2b4cd Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 31 Aug 2022 09:47:46 +0000 Subject: [PATCH 013/526] Fix bug in test --- tests/integration/test_storage_hudi/test.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index e2d97f6489f..c9415e28151 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -119,8 +119,8 @@ def test_select_query(started_cluster): assert(len(result) > 0) # test if all partition paths is presented in result - distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi FORMAT TSV" - result = run_query(instance, distinct_select_query).splitlines() - expected = ['americas/brazil/sao_paulo', 'asia/india/chennai', 'americas/united_states/san_francisco'] - + distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + result = run_query(instance, distinct_select_query) + expected = ['americas/brazil/sao_paulo', 'americas/united_states/san_francisco', 'asia/india/chennai'] + assert TSV(result) == TSV(expected) \ No newline at end of file From fc2c8f37b1efa1e4f3cecd6583758272223da28b Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 1 Sep 2022 19:21:53 +0000 Subject: [PATCH 014/526] Add DeltaLake storage template, file downlad from s3 --- src/Storages/StorageDelta.cpp | 294 ++++++++++++++++++++++++++++++ src/Storages/StorageDelta.h | 106 +++++++++++ src/Storages/registerStorages.cpp | 2 + 3 files changed, 402 insertions(+) create mode 100644 src/Storages/StorageDelta.cpp create mode 100644 src/Storages/StorageDelta.h diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp new file mode 100644 index 00000000000..8da5b5ce323 --- /dev/null +++ b/src/Storages/StorageDelta.cpp @@ -0,0 +1,294 @@ +#include +#include +#include + +#include +#include +#include + +#include +#include + +#include +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int S3_ERROR; +} + +void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) { + file_update_time[key] = timestamp; +} + +void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) { + file_update_time.erase(key); +} + +std::vector DeltaLakeMetadata::ListCurrentFiles() && { + std::vector keys; + keys.reserve(file_update_time.size()); + + for (auto && [k, _] : file_update_time) { + keys.push_back(k); + } + + return keys; +} + +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, + const String & table_path_, + Poco::Logger * log_) : + base_configuration(configuration_) + , table_path(table_path_) + , metadata() + , log(log_) + { + Init(); + } + +void JsonMetadataGetter::Init() { + auto keys = getJsonLogFiles(); + char localbuf[100]; + + for (const String & key : keys) { + auto buf = createS3ReadBuffer(key); + + while (!buf->eof()) { + buf->read(localbuf, 100); + + LOG_DEBUG(log, "{}", String(localbuf)); + } + } + +} + +std::vector JsonMetadataGetter::getJsonLogFiles() { + std::vector keys; + + const auto & client = base_configuration.client; + + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + bool is_finished{false}; + const auto bucket{base_configuration.uri.bucket}; + + request.SetBucket(bucket); + request.SetPrefix(table_path + "_delta_log"); + + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(table_path), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + const auto & filename = obj.GetKey(); + + if (filename.substr(filename.size() - 5) == ".json") + keys.push_back(filename); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return keys; +} + +std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { + // size_t object_size = DB::S3::getObjectSize(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, false); + + // TBD: add parallel downloads + return std::make_unique(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, /* max single read retries */ 10, ReadSettings{}); +} + +StorageDelta::StorageDelta( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, + const StorageID & table_id_, + ColumnsDescription /*columns_*/, + const ConstraintsDescription & /*constraints_*/, + const String & /*comment*/, + ContextPtr context_) + : IStorage(table_id_) + , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) + , table_path(uri_.key) +{ + //StorageInMemoryMetadata storage_metadata; + updateS3Configuration(context_, base_configuration); + + Init(); + + // auto keys = getKeysFromS3(); + + // auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + + // LOG_DEBUG(log, "New uri: {}", new_uri); + // LOG_DEBUG(log, "Table path: {}", table_path); + // auto s3_uri = S3::URI(Poco::URI(new_uri)); + + // if (columns_.empty()) + // { + // columns_ + // = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + // storage_metadata.setColumns(columns_); + // } + // else + // storage_metadata.setColumns(columns_); + + // storage_metadata.setConstraints(constraints_); + // storage_metadata.setComment(comment); + // setInMemoryMetadata(storage_metadata); + + // s3engine = std::make_shared( + // s3_uri, + // access_key_, + // secret_access_key_, + // table_id_, + // String("Parquet"), // format name + // base_configuration.rw_settings, + // columns_, + // constraints_, + // comment, + // context_, + // std::nullopt); +} + +void StorageDelta::Init() { + JsonMetadataGetter getter{base_configuration, table_path, log}; +} + +Pipe StorageDelta::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) +{ + updateS3Configuration(context, base_configuration); + + return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); +} + +void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) +{ + auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); + + bool need_update_configuration = settings != S3Settings{}; + if (need_update_configuration) + { + if (upd.rw_settings != settings.rw_settings) + upd.rw_settings = settings.rw_settings; + } + + upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + + if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) + return; + + Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); + HeaderCollection headers; + if (upd.access_key_id.empty()) + { + credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); + headers = settings.auth_settings.headers; + } + + S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( + settings.auth_settings.region, + ctx->getRemoteHostFilter(), + ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + + client_configuration.endpointOverride = upd.uri.endpoint; + client_configuration.maxConnections = upd.rw_settings.max_connections; + + upd.client = S3::ClientFactory::instance().create( + client_configuration, + upd.uri.is_virtual_hosted_style, + credentials.GetAWSAccessKeyId(), + credentials.GetAWSSecretKey(), + settings.auth_settings.server_side_encryption_customer_key_base64, + std::move(headers), + settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), + settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); + + upd.auth_settings = std::move(settings.auth_settings); +} + +String StorageDelta::generateQueryFromKeys(std::vector && keys) +{ + String new_query; + + for (auto && key : keys) + { + if (!new_query.empty()) + { + new_query += ","; + } + new_query += key; + } + new_query = "{" + new_query + "}"; + + return new_query; +} + + +void registerStorageDelta(StorageFactory & factory) +{ + factory.registerStorage( + "DeltaLake", + [](const StorageFactory::Arguments & args) + { + auto & engine_args = args.engine_args; + if (engine_args.empty()) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + + auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + + S3::URI s3_uri(Poco::URI(configuration.url)); + + return std::make_shared( + s3_uri, + configuration.auth_settings.access_key_id, + configuration.auth_settings.secret_access_key, + args.table_id, + args.columns, + args.constraints, + args.comment, + args.getContext()); + }, + { + .supports_settings = true, + .supports_schema_inference = true, + .source_access_type = AccessType::S3, + }); +} + +} diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h new file mode 100644 index 00000000000..2aec4f815f3 --- /dev/null +++ b/src/Storages/StorageDelta.h @@ -0,0 +1,106 @@ +#pragma once + +#include "config_core.h" + +#include +#include + +#include +#include + +namespace Poco +{ +class Logger; +} + +namespace Aws::S3 +{ +class S3Client; +} + +namespace DB +{ + +// class to parse json deltalake metadata and find files needed for query in table +class DeltaLakeMetadata { +public: + DeltaLakeMetadata() = default; + +public: + void add(const String & filename, uint64_t timestamp); + void remove(const String & filename, uint64_t timestamp); + +public: + std::vector ListCurrentFiles() &&; + +private: + std::unordered_map file_update_time; +}; + +// class to get deltalake log json files and read json from them +class JsonMetadataGetter +{ +public: + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, + const String & table_path_, + Poco::Logger * log_ + ); + +private: + void Init(); + + std::vector getJsonLogFiles(); + +private: + std::unique_ptr createS3ReadBuffer(const String & key); + +public: + std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } + +private: + StorageS3::S3Configuration base_configuration; + String table_path; + DeltaLakeMetadata metadata; + Poco::Logger * log; +}; + +class StorageDelta : public IStorage +{ +public: + StorageDelta( + const S3::URI & uri_, + const String & access_key_, + const String & secret_access_key_, + const StorageID & table_id_, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, + ContextPtr context_); + + String getName() const override { return "DeltaLake"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + +private: + void Init(); + static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); + + +private: + String generateQueryFromKeys(std::vector && keys); + +private: + StorageS3::S3Configuration base_configuration; + std::shared_ptr s3engine; + Poco::Logger * log; + String table_path; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index b5561243e56..c0d153a5efa 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -89,6 +89,7 @@ void registerStorageSQLite(StorageFactory & factory); #endif void registerStorageHudi(StorageFactory & factory); +void registerStorageDelta(StorageFactory & factory); void registerStorages() { @@ -174,6 +175,7 @@ void registerStorages() #endif registerStorageHudi(factory); + registerStorageDelta(factory); } } From c68257d711cbcc61150adc9c4a503eea23063a0b Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 06:54:16 +0000 Subject: [PATCH 015/526] Implement StorageDelta --- src/Storages/StorageDelta.cpp | 137 ++++++++++++++++++++++------------ 1 file changed, 90 insertions(+), 47 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 8da5b5ce323..05b418e4208 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -56,15 +56,58 @@ JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuratio void JsonMetadataGetter::Init() { auto keys = getJsonLogFiles(); - char localbuf[100]; - + + // read data from every json log file for (const String & key : keys) { auto buf = createS3ReadBuffer(key); + String json_str; + size_t opening(0), closing(0); + char c; - while (!buf->eof()) { - buf->read(localbuf, 100); + while (buf->read(c)) { + // skip all space characters for JSON to parse correctly + if (isspace(c)) { + continue; + } + + json_str.push_back(c); + + if (c == '{') + opening++; + else if (c == '}') + closing++; + + if (opening == closing) { + + LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); + + JSON json(json_str); + + if (json.has("add")) { + auto path = json["add"]["path"].getString(); + auto timestamp = json["add"]["modificationTime"].getInt(); + + metadata.add(path, timestamp); + + LOG_DEBUG(log, "Path {}", path); + LOG_DEBUG(log, "Timestamp {}", timestamp); + } else if (json.has("remove")) { + auto path = json["remove"]["path"].getString(); + auto timestamp = json["remove"]["modificationTime"].getInt(); + + metadata.remove(path, timestamp); + + LOG_DEBUG(log, "Path {}", path); + LOG_DEBUG(log, "Timestamp {}", timestamp); + } + + // reset + opening = 0; + closing = 0; + json_str.clear(); + + } - LOG_DEBUG(log, "{}", String(localbuf)); } } @@ -124,57 +167,57 @@ StorageDelta::StorageDelta( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, - ColumnsDescription /*columns_*/, - const ConstraintsDescription & /*constraints_*/, - const String & /*comment*/, + ColumnsDescription columns_, + const ConstraintsDescription & constraints_, + const String & comment, ContextPtr context_) : IStorage(table_id_) , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(uri_.key) { - //StorageInMemoryMetadata storage_metadata; + StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - Init(); - - // auto keys = getKeysFromS3(); - - // auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); - - // LOG_DEBUG(log, "New uri: {}", new_uri); - // LOG_DEBUG(log, "Table path: {}", table_path); - // auto s3_uri = S3::URI(Poco::URI(new_uri)); - - // if (columns_.empty()) - // { - // columns_ - // = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); - // storage_metadata.setColumns(columns_); - // } - // else - // storage_metadata.setColumns(columns_); - - // storage_metadata.setConstraints(constraints_); - // storage_metadata.setComment(comment); - // setInMemoryMetadata(storage_metadata); - - // s3engine = std::make_shared( - // s3_uri, - // access_key_, - // secret_access_key_, - // table_id_, - // String("Parquet"), // format name - // base_configuration.rw_settings, - // columns_, - // constraints_, - // comment, - // context_, - // std::nullopt); -} - -void StorageDelta::Init() { JsonMetadataGetter getter{base_configuration, table_path, log}; + + auto keys = getter.getFiles(); + + for (const String & path : keys) { + LOG_DEBUG(log, "{}", path); + } + + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + + LOG_DEBUG(log, "New uri: {}", new_uri); + LOG_DEBUG(log, "Table path: {}", table_path); + auto s3_uri = S3::URI(Poco::URI(new_uri)); + + if (columns_.empty()) + { + columns_ + = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + storage_metadata.setColumns(columns_); + } + else + storage_metadata.setColumns(columns_); + + storage_metadata.setConstraints(constraints_); + storage_metadata.setComment(comment); + setInMemoryMetadata(storage_metadata); + + s3engine = std::make_shared( + s3_uri, + access_key_, + secret_access_key_, + table_id_, + String("Parquet"), // format name + base_configuration.rw_settings, + columns_, + constraints_, + comment, + context_, + std::nullopt); } Pipe StorageDelta::read( From 4cc397da84e6bf3bd6157b7be7ce578cb02aba4c Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 07:06:24 +0000 Subject: [PATCH 016/526] Add tests for deltalake storage --- .../test_storage_delta/__init__.py | 0 tests/integration/test_storage_delta/test.py | 118 ++++++++++++++++++ .../_delta_log/.00000000000000000000.json.crc | Bin 0 -> 32 bytes .../_delta_log/.00000000000000000001.json.crc | Bin 0 -> 40 bytes .../_delta_log/00000000000000000000.json | 9 ++ .../_delta_log/00000000000000000001.json | 13 ++ ...-831e-2ab223e7c176.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b556-cd6ebe7630c9.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b929-a68f54aa1e6b.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8144-d0d9d0ff572c.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...42ca-831e-2ab223e7c176.c000.snappy.parquet | Bin 0 -> 2795 bytes ...4514-b556-cd6ebe7630c9.c000.snappy.parquet | Bin 0 -> 2795 bytes ...42de-b929-a68f54aa1e6b.c000.snappy.parquet | Bin 0 -> 2878 bytes ...493a-8144-d0d9d0ff572c.c000.snappy.parquet | Bin 0 -> 2878 bytes ...-afe4-658c02e1aeb5.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-a37c-1539c1bb57b1.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-b8e2-817f80097b3b.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8135-23184ffdc617.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...41e5-afe4-658c02e1aeb5.c000.snappy.parquet | Bin 0 -> 2966 bytes ...47f5-a37c-1539c1bb57b1.c000.snappy.parquet | Bin 0 -> 2966 bytes ...466b-b8e2-817f80097b3b.c000.snappy.parquet | Bin 0 -> 2878 bytes ...4fe8-8135-23184ffdc617.c000.snappy.parquet | Bin 0 -> 2879 bytes ...-acd2-d2bab8e66748.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-841b-22762fcfc509.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-9527-a3dcd269f99e.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...-8ca0-7594340b2c66.c000.snappy.parquet.crc | Bin 0 -> 32 bytes ...424f-acd2-d2bab8e66748.c000.snappy.parquet | Bin 0 -> 2795 bytes ...41e9-841b-22762fcfc509.c000.snappy.parquet | Bin 0 -> 2794 bytes ...463a-9527-a3dcd269f99e.c000.snappy.parquet | Bin 0 -> 2795 bytes ...45c7-8ca0-7594340b2c66.c000.snappy.parquet | Bin 0 -> 2795 bytes 30 files changed, 140 insertions(+) create mode 100644 tests/integration/test_storage_delta/__init__.py create mode 100644 tests/integration/test_storage_delta/test.py create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json create mode 100644 tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet create mode 100644 tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet diff --git a/tests/integration/test_storage_delta/__init__.py b/tests/integration/test_storage_delta/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py new file mode 100644 index 00000000000..a1cc6345619 --- /dev/null +++ b/tests/integration/test_storage_delta/test.py @@ -0,0 +1,118 @@ +import logging +import os +import json + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +def prepare_s3_bucket(started_cluster): + bucket_read_write_policy = { + "Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*", + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*", + }, + ], + } + + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy( + started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) + ) + +def upload_test_table(started_cluster): + bucket = started_cluster.minio_bucket + + for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): + address_without_prefix = address[len(SCRIPT_DIR):] + + for name in files: + started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + cluster.add_instance( + "main_server", + with_minio=True + ) + + logging.info("Starting cluster...") + cluster.start() + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + + upload_test_table(cluster) + logging.info("Test table uploaded") + + yield cluster + + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + + +def test_create_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + + create_query = f"""CREATE TABLE deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + +def test_select_query(started_cluster): + instance = started_cluster.instances["main_server"] + bucket = started_cluster.minio_bucket + columns = ['begin_lat', + 'begin_lon', 'driver', 'end_lat', 'end_lon', + 'fare', 'rider', 'ts', 'uuid'] + + # create query in case table doesn't exist + create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + + run_query(instance, create_query) + + select_query = "SELECT {} FROM deltalake FORMAT TSV" + + for column_name in columns: + result = run_query(instance, select_query.format(column_name)).splitlines() + assert(len(result) > 0) diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000000.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..327e28983364280c30587f7c0432a548421b39d2 GIT binary patch literal 32 ocmYc;N@ieSU}6Y-KR?}UN#0|x{?hJ)>i?zFf&yZyPN~ZS0KQ@jbpQYW literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc b/tests/integration/test_storage_delta/test_table/_delta_log/.00000000000000000001.json.crc new file mode 100644 index 0000000000000000000000000000000000000000..fd48c97957675d0896f9a5abd1f7ab17ed1036e2 GIT binary patch literal 40 wcmYc;N@ieSU}Bibee&Zz8;eO*Q@6+W^~-ViO9mCsF7{}b3b9qWw(i1403d-6qW}N^ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json new file mode 100644 index 00000000000..45fd233fd48 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000000.json @@ -0,0 +1,9 @@ +{"protocol":{"minReaderVersion":1,"minWriterVersion":2}} +{"metaData":{"id":"6eae6736-e014-439d-8301-070bfa5fc358","format":{"provider":"parquet","options":{}},"schemaString":"{\"type\":\"struct\",\"fields\":[{\"name\":\"begin_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"begin_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"driver\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lat\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"end_lon\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"fare\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}},{\"name\":\"partitionpath\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"rider\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}},{\"name\":\"ts\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"uuid\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}]}","partitionColumns":["partitionpath"],"configuration":{},"createdTime":1661963201495}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661963203028,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203056,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661963202988,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878,"modificationTime":1661963203044,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661963203072,"dataChange":true}} +{"commitInfo":{"timestamp":1661963203129,"operation":"WRITE","operationParameters":{"mode":"ErrorIfExists","partitionBy":"[\"partitionpath\"]"},"isolationLevel":"Serializable","isBlindAppend":true,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json new file mode 100644 index 00000000000..408d5e1ded7 --- /dev/null +++ b/tests/integration/test_storage_delta/test_table/_delta_log/00000000000000000001.json @@ -0,0 +1,13 @@ +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2794,"modificationTime":1661964654586,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878,"modificationTime":1661964654518,"dataChange":true}} +{"add":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet","partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2879,"modificationTime":1661964654558,"dataChange":true}} +{"add":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet","partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795,"modificationTime":1661964654582,"dataChange":true}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2966}} +{"remove":{"path":"partitionpath=americas%252Funited_states%252Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/united_states/san_francisco"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"remove":{"path":"partitionpath=americas%252Fbrazil%252Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"americas/brazil/sao_paulo"},"size":2878}} +{"remove":{"path":"partitionpath=asia%252Findia%252Fchennai/part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet","deletionTimestamp":1661964655238,"dataChange":true,"extendedFileMetadata":true,"partitionValues":{"partitionpath":"asia/india/chennai"},"size":2795}} +{"commitInfo":{"timestamp":1661964655251,"operation":"WRITE","operationParameters":{"mode":"Overwrite","partitionBy":"[\"partitionpath\"]"},"readVersion":0,"isolationLevel":"Serializable","isBlindAppend":false,"operationMetrics":{"numFiles":"6","numOutputRows":"10","numOutputBytes":"17107"},"engineInfo":"Apache-Spark/3.2.2 Delta-Lake/1.1.0"}} diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-7212b9be-df70-42ca-831e-2ab223e7c176.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..0d07fe9805f910e56d8fd1faa94b16fa1e52f59d GIT binary patch literal 32 ocmYc;N@ieSU}8{`ik6si`R}A+<}WX0E{ka&($LSF9RBwn0IB>8uK)l5 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..1b17a91ca75be612199b8cca4cc7bdb81484cea2 GIT binary patch literal 32 ocmYc;N@ieSU}E^@y?B#jm+++B2@)xGM;lKa($LSF9RBwn0J^6QasU7T literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/.part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..f7f1df8479de818fab8d3ab6f8bc07dacc4763f3 GIT binary patch literal 32 qcmV+*0N?*(a$^7h00IE(#)#e7^VS5TQ*7%Xbga<4Zrk`Q1Y?EWUaC*81LFTU|% zbf2)hM;o=buYN$oMsW(id*$*wM})AE(+*n%I4kYit5!Rm%@{7{`&a(FEeXL)`dZ_Y z<IR_e|`J)J)aQv z=t(ETFl5-SRrVzKW&itIvJlG>eC*EsJIfM6K=~qm8UMZf`Q?xhrhnm39s|mJzL+T& zGuiZ5wp2;y%zQCDQOuXpf)cEj&kR|0rNqGr(f^klTNV76a5+bN&38YVu9 zaOAkGhD+9bt$DU?cK5DZO@yQ14?4GX9G#9)9=auDH7k6Bv!dbD=dYS=%h?&Q>&!O# z!6|n;j?3Tc)he!s$XZNWOTw#(Xn6$%;ZnSmuuNoficBY!Dc*D(5j5(xBDX>jq1oZc z2$7qI@CaE8;wMmYc2Jfb}XzRAA653T#n=h6SwJC!DdvqL8fkCS%(5C`x3Ki(XP-6mi!+7cUeZbD%EJu`m&?>TDqwHHY`wp

X1q{ElCQ)@NVfT3sgEK*MlM0C#Ab(me}dsbI(w_ej5R@bbU zT~qzRznJL0zF;LL6P<3mSMDZKiE7QNS2~Hw7Zwvuvq4?nJOQs8U1{}7bqm(bUcGKo zI0^f$RDw|_CKs2MQvarCH6NV98KitaBwTCG51)cNbveO7&tMb$pA*p8{^0WV=YVfN z=<&N9zPH{wNa5(mhVLCt_oXGIo^o3*-Dt<%9*t-84D@JxwA8EBEAc|XGRDdiV@1Q5 s7`Mv#Y}qJR&GE94D;2D&VZi^#Y_?p?ozT|c>-k?l^j-*|8Tgy~54oVi`2YX_ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00000-df1117a8-d568-4514-b556-cd6ebe7630c9.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..a9652efacb0111d1ad3b72e982d3f5cafc7594b4 GIT binary patch literal 2795 zcmbtWU1%It6uvY2H@jVIt+|&OSa4a|C6(PFyR)0khTz|W)KWr_5=vF-?Cxx`xIgLs zC?y0W6a2hlRBPPB6VCNU*fUvN+bM*7+J1p$yD_`#T4+y*c?Xlm!{&?nT5KZDI@cr+;`O7DS zJ$Bm3Fbo;CYn6Qoe%1fswk*W51RuZq;O?@75Kz8^U%`Jbe{n4&gz2{&$`e4D&lfZ0 zVkVm&%a$tXoS84CCyMz}dfc$`#X@ePFjgJc9+fp!VFec;mNkNq<7R~hH-myZ!i%e^ z!a@j9SXqIALD9|132p{OH!CB!85G=MuXLKqu@IoJn!$bs1^0=e>ZA@8=F-N9f~TQ; zNQ;qK53Ni1f@|E9JnNS1!M@LfyFPs0jot8j)|CKWk*FDTK06)lEST+AFLuiFR>Q={ z5ssXc)o{t0t2Hmy&F=nntBG(F{6XioPNK6h%0st=tY(F8a8@*&`utV1Z81WA>IO+3#ZQLK{H2)-d9axXySq(a^e zj=+3NscNe|r<)5VOd;JKNIHCJJGE9*4H$Y>&m#5IKt$(KU5D9qsb_WdcI$P$VRg-l z*)`Q4{fmk2d5^N>poBz0yfczPOlZnhomm<_UP+=t`?ss#~yb_Ud($ z!b#X~r4o!fF}b+3l=?SCtNHL0&LHLcA>mqcZuk`3smlotdj^}}|D1r%_J^0ZKL>pK zVUOSK@V)icVG2h-Hhk}Jx-Tyw^^Dtc=|(&5_Gmn#XP`&pqorQ0UWpe9mN8bI7%Lja t#JE+?XUj&pYL1tUT&ZAH4Fmo^X0zpD?v%C$U(f&gq4z=v&A{K>e*k^vziI#g literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-3fd0374b-5fcf-42de-b929-a68f54aa1e6b.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..fad00b6c557f37bf65d19b8958bc6d1564f89044 GIT binary patch literal 2878 zcmbtWU1%It6uvV%yP4f4@rT@DMi+@ox+bu*Y<4!gn;5%kz#`IzQh)FVTW04cE^cd;#Lj-!*~#Xym;$IU=p*KL#RI$0%W~X zc-^ST$x#Im@5cl9^xtPs_7g;|bpbxyx&ZRol$Sm^e=gL4@+)KU=!=j2mRnohzfik$ zh2HS(<7>aO9o(Cbe)q&jx3+U{zTe!Kkba@=U;lajaxTo=J^mGg)m+Hnee5PU&e=R!8$`IhohYw60}!Euj-X!_ZRWrkR(M z86_*Hv~)sN6KPd8Rl`t{hLTOFN$CMr5_wv9SUR<5B>|z9nHCz%POV*Y$StoVdN$Z~ z;c2}9ftJu~ru6|cEt%I$>jGw{G%k41sfHxFYi~TwGQ>NHcH@V-$)_X^b(2n;H7xM> z5Tp-DVZn4(D=eP257$}ewBd7>%bc@p_#NLZzjJyqfUhwG0(hEF^;YN9%4>(K+N;K_ ziXTKcw3`)!d1IzfI$Tt1ckjz3gnPjrXjN-B+8-u3B&5eE>25%4N8E{xGu4XGiu7A~ zrhyz7*;=(_;_CH+Zd(LS3QMa|K8!=s7>@!_6z58q6*%D~;ZT&HbUV#$KLHIp{pX%S z65&~ZWEfN%eRx1<@^}x5n(YMTl0#qu0qs?hrwvR9+E2X%n}lG~B3OZm(hCwekYH!c zf?s5WWhT19MlUd7Z@+ujK;(G|UJH_9*4$%>B+e3byAKP(S`hC;(RJXYX`^LvG_A8B ziasCFc~eAG`>9vdiGCp14w9@jV36NU;7MA;2L$v1iadRlgrNP@OVA($%@842=3D}+ zd7SGnbAV3YugKF!P0-sPHLyE#Vy|+VDrOb79X2@GODE9SxrI z4%OHoY-_vlG>UEk8)r8$^tz*D%MVKEBoTQ!UWk(RQ?HUOOUW*hr;CA>67oQ-=VPJ& z#0b|IuuZ1Xa90CI>}q;>?{MrsEDCGA_%wrrzj_2N$_uCX0T@*=Q7l(x;_93V10`#7FB-5f)r&=yEGHn|h(#!LWMY0{A@*;G za_Qb7S~Jm&hYJ@ao9-uD3PBmBn|3?8C)AvqqO?EeYr}Ar6H?~;bN#e+N!;S5Z z_mu@CK5e(0JX;yGdvtI#J_20PlL>ZvX%Q literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Fbrazil%2Fsao_paulo/part-00001-d0760f2d-45e8-493a-8144-d0d9d0ff572c.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..1e79eb2d238771c7dbcda8b6dc5308d818038e67 GIT binary patch literal 2878 zcmbtWU1%It6uvt=KEC+P z>dT*P|2Sqai%@SW2*_%s z@S0YMC5I(IydU@BbAMl0>LrM*c>peC9sqeQ$}69&tp?jreyPo$eE!j2b1TaS=W183 z(G$M)_R4Qe8~5fT-#`Art(&PZAZP^F;mPS2-jKW}ukVSFro=h`U`LU-Q#_6PDGWTD7Zeb zG1E_HM&#FBe#Q+3?xEJ3f2&vLA1H2n6snpX#p8$dy-*SCv<^ zX&FC&aBv?Z1oGNcp>(_`*Y5r8>`C|&5ZOJ zd8UCZ7@1ntG_m!1LA5MAD~8172p7UZam0@}D2ja~%%TK?f!Fi zA&GD;Kr#%fjUL>`H#xi?Mf6sJa>*vpfq?d^z|jUK1g*Pnf^9;uZ4fNNMCk?zEJ(1j zX28#T_ytd7k%?ULgxvk^UIUTqCb%s~j2d%~A(B`_)a@S3^D6=TB#Nv9CrukIi>+y$ z0a5h&2=<*KqFQ&|qE7b$!OZ~4S_208-2|?rHM~GTFQCBDTS*97cijXHLeLBnf(6zg zFq+3c{sIf=^!^GQz0?G~b=Qsl1wr37&>Qe<(VCe3Xu!Y8zw3!Ke330SlJp>jkZ-R>Na2&W20!G%?wu0j!p{TCV4d8hn@P#iC4>V-T-J!xTC^Haj;L z{WnCpbng)6OmyPmz=hIO=SQ#xr~|ND4F3$zt(J>TSNg3U?H`U0Lyz_k=Ie!`+Mmg2QbNflMkOhm)|8Q?A}P9@R-{xu eqv?_aKSq;DWi&M;F2h6kUw^33h0r1R<@^s%AmzsZ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..ded74b8da6d88f7d8d20b2ac33a21855f6d673d4 GIT binary patch literal 32 ocmYc;N@ieSU}7i?k3TYPRdRK{e@9!*kEv_aA3vL{@}lK70K+;Cvj6}9 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00000-a8bac363-ee42-47f5-a37c-1539c1bb57b1.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..e0123ca128f2bc4783823c8f5b0c4f1c0af8cf24 GIT binary patch literal 32 ocmYc;N@ieSU}AXu&+L9ypMxH6vLf$lJ>?4Z$Im9KylA-%0I>)RmjD0& literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..065f09eab7a820723d2a81886e88180dbe0e3b5f GIT binary patch literal 32 ocmYc;N@ieSU}A6)6~1(D_9OEblk0^KX)J3xp>Xw_!r`Om0lUc#EC2ui literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/.part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..8688bc7218dab15bb560ebf9a764482f98a9c33d GIT binary patch literal 32 ocmYc;N@ieSU}C8HX3}>qXQo=B?MI`lqMw>IuISI4!18eq0Kb9_ivR!s literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00000-8dcd9986-b57d-41e5-afe4-658c02e1aeb5.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..83d3695feb61efb287689fdfc8bcffe140eed604 GIT binary patch literal 2966 zcmbtWZ)h837=LrUrkB{R)%86u5=ISO)4 zd++l;&+qxY&+mCIy$7ETV1&DHH~x6`_Uaxtw!^C(a|rD#FbJWVy2dM?+&cdEM%>N3 z(fsU}bMg6wKgPJ!=kbc$*81#g&TZTVH#U0O?*A=*{PFe&7izD@zqs`NUp0GINznOkRWmsZpP=9ZmyLQV#^6}?s7t?hC+XL=H2Zw zzPfk)z)I%#__51>K2;c4jL*D%D*4m$8qN5Fi$A|rw@t=ltC4~8!5`w+Z6}f5emPz^ zn`ORva*4W~`)K5AW*fJLW3M)@h*J0L+2>9=u6K8zUO#x?=%t6cf#?HxA3l9$?j5c| zw88HHqPbLJcRb%*|MBK_34eBRdhL`Aq15w-e?);b78l>pkNSg`QX(p33GXW9%;17|VIq zSl*~L&d%74-N`yU&5#g)bK%kY{D6`>d?>HIr1vZMK7^h78Ns3IeW}c$v{Lx@dp3h` zJ=pE#YVAi)xJex1Qm1DUrbGFS949t5spRysXKm@8sUr(UrjRe2*kUo2s91Qr%k7%* za&GK&g*jx0q8KS*hPU%C^BFJKWh%{-ACKA${h7OPiO@bk86keO5jXQI4*W3k4pswX zGX}sQ08p(892GDDsNS^#j1z!yCjpp*(NYf-SfEh3wNm&I8$V_Pt9RArw(8xw1|Ul( zSS`ra0b_^-Hb99y5I@z3d4AD}A3@$tz@*nk%VKcaWPlWHAHnD;LaKV#O6n^D@ZLRy z)N;+80G3Nzt^okrfC5Lmk^ofiS^<^`z)A^Vo;3kV;xW>nXCXT6UxA}dO`=!tTBC1~ z=;QSy{TsDrP0D&Kg+IZM+Q8=hwAO7I0hT3zn4`h6zfCnpn@Bb4aS!s&*xi_|h#0hP zFqyFf6FNx*W4sV1)w@}Yyf zbC_9XRr)-G-(81fi9RIze8AeryAY^lWK4n0Gf#NI879V{6^Qmk`sFi#{Ul0lH zRVVUC(mohNM~ZsEm&+dY_3H&Cp%fHhi5>D3UK!9mot}IlSJVog4o@K*sdl!AqX~g}mvT&hF;o$all^-5KtS!$^3nA~~@?*HY1F zOTZTZjkdI@#Z)@cBFnnu*P{NIBt;{-77l8XmQ*5|6jEh9DM|3BHW<`mp?22=URDms&hSI3|!i1 zKJaZK6nUcg;G1)airL2M)wkSRm<`{&+1LKiZ;98R?s(*S?Rw(#xr=|)>|Rb_7JG3k zzA$!s#o2=y%p$Za;s&H-K0U1G#c)Uh&rP@mpIf-MvWs|T%K>rTazLntLVt7S!_tLK z5wGuF-M5teJ#p;Dt%JtUd}8Lklc}E;S7^i^U-|jnx-B9;y&N675dI-?$8iGb9XArj z*&OrLvkTPh+@-NEnJvs3j=j;iBuc$k&%SWNb*H!Y)aw3yN9P{vg|HvSd+@36uf5Mz z5^V^&Aklm}xwAaqS^e?eRsnx@WqReL1EKVbhkqr{>(~CiO`cGmzZ`gs!Eb+k`IiP; z$j3LHFPE?l!q8K)uE}ah)YNE9j7VWwRO7+8nA9RcNtQK5l4Ina8j0(2G^UDCRnx?X zt|+1!RFh&#*K|1^Pj!V;K}Y|YV5BRiXnIoY3dKWWL<{L+EEq|NU9y@|lByaC1!Ye& z;}JMYAUINPVaCHFR8~`>fSPj0rgmF2@d&mT+ZB%X9XwDf*wwVZpf+19cOHEpTQ z+GTdnBUoqK1&($D?9GN)?%TFiC!uXS-Kn5@c@Afxe8}VFht1O@gZnDNHO7=K)|kgx z);!*=HSf+iP3dGEo@R&(z`1;&zA&WZ4<9OMujqpceh^{zUPf@K`an8+D5Dtve$QnQ zt_Qud+^xOn887ieOzQM((lRK&k?Taqrj)#1wyZ7HnHsX7WQ;;t#TJX{WJSX}Jzmdw zKkO~Hr^|^rXo?vUW_Tz6DxdXpJr>g}`thjUcHdE!F@49YBGjQ%L}-rFMWt0PeZ8CHKeqaR=!9pVoS9E5LRH5NkEq#)L_>*P@cAI@A_GzypOW_}hM#8qlXX2&)V{5SBW!DX z*<+Y)!U8|vfTtXY|HQ>R{T%TiQk`#;TT31+;w3HInq=+i(DM^Aqwc)T9k92s>!-w#{ NekfgsP(S=5{|8dZ|6Tw9 literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-7e34b80c-8fe9-466b-b8e2-817f80097b3b.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0fe5f31b711527987d2a509cce139e6711fdc643 GIT binary patch literal 2878 zcmbtW|7#m%7=Q0_Nv~PkS?PP+Nstn{qs1g7xin2nF>{E_DOkjPLF>vTcd5ZN>GDO9 zQkjL7{sF@Y4vNqpg0%Z#Klo)Zs2#G==?{ujr&MKBM$@9QNbd$mJUU^I7)7>AT6NKOY$U za^-f?JiR^t$2W(Y9OqKcc|Vx@Df#p5dwaXb*OCC}!@YR%{QY~LQOsfvp{|%0khNm= z4ZRqQMifAN1oz-8e|>bRiy(6K0C-cxHPzxcUh_{@eb}b$1hV^!|5ipLUNO{pE!Y_WShdqY&O> z_;Gw;CsG3O(>DzU@W012}O-XGehK@o`|a%H9j1Sr54C|S2G?IwvaU~*kvy#Zuio=pN0cIrup}Lt?7|gWj4zt&;uOvDi97lLs zI6$y!=OCa30yC|d)0-9s%vNn|?{=#kl4!4;{xrSdPc!Z?_qCEwN$hJSohGbL;PE)5 z4@o{j!)MLa0*jBC%XQY;t-7q`JZCMdZtHB#ZSCIj;4Owg0MGLAj?%PRe0`#neodcJ z@uLWPkFa8UN}tT;Cvs}}-_M0S!X04u)T?y_J?|qp#HCHoYqmrEi5w?3E~6Ipx~JR7 zGj-&^$d*fW6IZEZHPa$+l25uUL-%^6K^}RaC{{?A6*%FHFfQ|>cBk3xC!l_-|Lk2z zBHRm*43lc36ZZ(qJbn_%S|dRrZxFmJoRu{opuH;cw1Ejhe&fEl3I&gO4SWI78Imomdd6?f4lauLCDd8!d~a zX`Ka8^!kX_og$(dcb%dxb^*b!?Idee*MkI(q*Yx&Krf)k(_2Xh8h4!p9}~sCM8h3=;O+1Wx9KCB6qnRX`LA#mSI5t-?SF z8M7q>@9k2ykQd!bC>n|)aiHNu>0B-ZW9VE(FNcbSH$zi;S=H3CD&FB{{N=Z%b^nOJ zR4!K1Wq-h*$?7?+@|C|Hxi0KkM{PbD%qUYJ2a##;dCNA44*`CJv|srD{1%`N-MF{ eke*Q#_!^By)5Ea==`y?s|LcdcZ3vCQm-9cOPvge` literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=americas%2Funited_states%2Fsan_francisco/part-00001-cebe56e9-0e6f-4fe8-8135-23184ffdc617.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..b43afdfcd30b48121adaf093c9cb4a089ca3cdfe GIT binary patch literal 2879 zcmbtWZ)h837=Q0_Nv~PgS?PP+Nstn{&|xklcWIiIYUV!3{D(#69~4(E$z2VmNtZv0 zlu8*oO1}sqo8X`bLnL(lun%(zldT&JYCp-)PML~qj?$oGl_C1Y=e@i4?pz|I69Ref zectEydw$RJywBalvY%&O?z?pQ`n6wZ zgr~pRy3;Pg=7-GsQtF3He0FnGzHp8n|NZXPd4C&p^zp{(7kv}^Z$8(qcBjq?39|O-OyLB=WT4u%u0ZSxG?1GSdo!nHJq+4!ZS~M9+ig2u}+K z2u|%h1hhb4rZw|=)53t+sg3L1?UX|j-L==BrWgEa#y#f2PVy;Fg| z4(?^e?yNplC`}f%+P|O6C4_sx9xmpEHqOv0y5;FN^GqE% zFtW9(W#a1fLf*CroD`ByMxc9x(r^IrP!uO5%nF?Fif}l>kGq}bwx588oc?onA&GD= zKr&3Kjor9kSmE(wC}Ok{luHhQ0R*&HMV>Y=A!yz85;O@x(LZcf$fgPHj)9pM-gY4{_*av{$M@GODE9SxrSHr3cL ztFa3oK#?YSOV~{ez3wP!`aubuBqC483sKU#>s9ibsbm+)(^|k%LN17TeN6Np8Q}&4 zw(%4i>S*AI9ZfH94Das2qR{BUD-06;>=L*LFMP`P!>EdiV!1LE(`GanC^2)k#NfSM zEtE^5UyUg-1&RAwPE?N+V=#t})b(1dQhqBot=F`?R@205+-$h^_KY4L3s-BEdaf3Z zhK+(=%vZx>ug-=`+B7jaqXDe;>3q4KE$Z-GsuznIS&o6f9t~6I@Yw9!T=ZUua_Rmd zteNQgL%@a7ROcbs1JniBsRk3^|0p11`u^$7$?m%ERDP}M`WDMONgR1^xW1k7zBY%% zr|g!Krz-*d>A454cnH|v$RfVrn YMahjO_em$=MfhJoROmuz0zRGp0nPZ}-~a#s literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..6756a13f94b6124c35aba9fbc36430f94f6cb589 GIT binary patch literal 32 ocmYc;N@ieSU}7+nO`pM%)grPo=iZb5l`+DHH1zW(hyT3?0Hk^hod5s; literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..c099f0af8e41171ba95d8d9b461fa5a50bcf7024 GIT binary patch literal 32 ocmYc;N@ieSU}E5ZXt}{4&gbbjUR8ekdm`T#+%%E&Sa!ex0I07E<^TWy literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-a3499b25-46da-463a-9527-a3dcd269f99e.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..73d821f134ba262e5b03a94466fc5b4e2ec13a26 GIT binary patch literal 32 ocmYc;N@ieSU}8{w^|hQKl+|V4-?W?3rjjv-H1zW(hyT3?0IAjtI{*Lx literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/.part-00001-cbd68744-0f7d-45c7-8ca0-7594340b2c66.c000.snappy.parquet.crc new file mode 100644 index 0000000000000000000000000000000000000000..48b7b139c7afe8ebc53035ed7573f8b0aac32497 GIT binary patch literal 32 ocmYc;N@ieSU}CW0e|F?5&uf>E&bnik%6)qeY3S!o4*z=(0Jf(MQ~&?~ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-714ed689-3609-424f-acd2-d2bab8e66748.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..4fde2de24ac6e65d2ba8eaec23e06f3e3c4b33a4 GIT binary patch literal 2795 zcmbtWU1%d!6uvV_+9Vq*YVTzRf`qUex{?e{CQX`B{6nOy%d&`+Wp`zFGBc^c{B-l< z52X*TK8OpIMIUM@vSLL9Q9(fvk?u={r3yZ%*wsbQhf+|A5B2Avc<#*Hxpy}a(iQS> z=KOr$Ip>~xZcaY*coriZ#|eDtv)!xVIF{ja2!|0G$;$|#fV433@jK6b`|AUq+s1o) zyTLwgUp-b=-YD)+w=tZ?m*2Yb>Yaky=y4Y<3Yb=_`i$Mmgb+ZU#pm(6pZ>NZ)}ejkLLLUlQn_Mhv&G3w z$+8QXe7-oIDHU`pW1AC&iOE8_m^V%JE?HH=tl@%)WtAZ0npva4%tXN)<;_(U;X`l< zv$g^P6Gby?CzzQinpqpc%tXN)@mr@VToxR{tY;9Q>eMpUy zSzlU{@M%xEFZtR{*%wy=zPKI0r##>Fps!sD;U$Txf#?01SbN@RJ$P~IJiqW#&$dJ=#78nGh z+-bX}V7FVfJPVQ4xVoBxdmU9L;5gw>{FJavWO9t0NQI~Q&~b^NF@F@f7Mcjd4kaT* zHV)xYvO0wCN2%Fff@Z@fm<0kht6ZW4y;_lAixO-(1Zz?p$Gj4}t~v0tpCG4F+W^n6ydVNWU%g1NK`E9a_+gY((rnFX#=HSrNOKmF1}|;9+H5ExUCU`Xq#W(>XkVylu)8jF?T*%JKC9L3j$s)c zL-`@NknB7^Zzrdc?M|y}c9Q92rE1r#c5>>>Lb74hsmb3bVD-GMX183k;k(hT)eI_5 zLA;$#GU(*g!s253R)}Wf_95Iw%Htv7YGZEj6uhm=0S;P&1Mq(;pu7F{>7D0*haa^3 zPMgQpng?kdeQbE_V7yN*BITGja+!K7;mv3wt7TzE6JzCWwPq!XMOz;?C&x>=K3Raj pNjXzDD@MW8^W~yl(RKL$n9G@^{84okUeEvfp*M#SIthPs{{_TK!qNZ$ literal 0 HcmV?d00001 diff --git a/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet b/tests/integration/test_storage_delta/test_table/partitionpath=asia%2Findia%2Fchennai/part-00000-db7e2844-bba1-41e9-841b-22762fcfc509.c000.snappy.parquet new file mode 100644 index 0000000000000000000000000000000000000000..d936d4315277401fb42f95e78be35aedfcdb7ba0 GIT binary patch literal 2794 zcmbtWU1%It6uvY2liBTBQFAXdu*kBsLn^yNc4xDj4MDAlgc?ebAf>Ievoo`CaevbN zQ6%KS_##>=RS2;ZsqsNXi7$dsBv2>{0Smrpv|0*6kf4&M_(L8%cV_O)Z8yTE!akfi zKi_xGx#ym{CyqXs!w4sF3g5YQyO+cwd_}Q@(7}R;5DE#aM>jV zeVtM7eE;pu&>n6m`smco&z~{3tKVI}F74s=&%5Vp54`d8lR&EC$M9cYfB83$_Q;f* zVGJ^C*Q~n|+`hH;XM7LHw_f|@_M(6gK%T+p@JC<#zQxy}eeOaY0?2ZuYUgsL@od?$ zi`hbNS)~?Xf+*zW8}Cx0|abV1Bnv!Yk7hVO0eM&tO!Y*@JjHy=D@4g;4M7QZwD|TOGEe>r0xO_YZW^Rzm8oX@~=!F(L&fo3bZ`IVgd+$8ls(A z7w;wD8nrG00lRd8L<#z}Ji#(0xGYhEWmsyg0%u_S^p`_`&hBg=QS^Q-kG@LLHyrd$ zKxgOR&Y%N7Lr$uj03Td=P6UEI^*qH5O0g8fPa$;;o|mw9)G*Y3`fFiGpRJ5QqUrZ* zdF&sZ^kcL+7a~FW&OLF^-xSD20c?|l?Ulj2Ag_V8)L>|QJ=+N^lchLb6%hF&LPS*} z??i`Tzhz{()t=RiIRmzk<}4%)UfNEr)s!Q;me=w~KGNsWIbYXccb)IqU9H`EMQhky z!!o*t{6ly?-FUmqOUZrlsce7Wo z8&sTtcsrA3(CLZ!g@w$$5Uu9@L%54Hh=+u0&Dp_I@V0IMu-_USfd5kg-R<{J?>q-V z_Gt=%(Kr_2GmIsKo|zC4LIGj@e)sto_O@Sg-R|7j zSdZ-EcJIN(-5ZDQQMYlN#vfn*;_4yJZS1sz76r^otM-oB%H(sJi}}x2cXvaaGox;= z&s|1WcNz7={^x&3_i_96_-yIZ>(`ju#+A!o#`bahnS{{&?cB>in!?BN!&^T*3UF?R zPdXV!Aj4Lz@>qf&{J7o^bG|IW{BN<}-xm-9$n*Fu{MSdAzjYvw0Hj_t3{%sx znOvco%S@PsY^GSq8kv$_F6OfZvt*PD%Cn*(ORV9Xh((1U5bg?84LlnsW9>z~_4b9fanW4R@pA}A zkBV}nY|hsj7wWq8_;<5`a188W=d_NZ({YMJLq^O-#Sd`4s5`XA39$mXxtk`UkgoyVTY0t zA~#3z80m%aizqcSNYHF}1T#RuX4RJ{!Jw8W=u?8eO|T}!am+2j?V1fgL*`Oz!ZbhX zybK)NCEzVQ&x27+$jS(Q4W;&ghqa0wg;&QO5P4U|muMmEBKcaL;KLXY+#R8v+7h26 z;2O0h0s*^pzC;NIwLHNpCFn_%U=@}ctH2%@Fa6a3ptC#cOB8)j%cFlo(fc;~4xqDh zaAweke}l}Xb^zYL@|*|+`|5d$o0MWDf?q|c&0rh{-J^z~_R`-BLi%iF_!3QjP|IWg zVy7RW&FR83!_v3!iH&|mAYB1$)5GnR;k+QPp|;d;XnZ|?#;{CQqWEh8kv~F2OiARE z@EGj3v@ADU^Qyk6!xmEQg`~nu+paYma!6D2Y97hQ20YrA>MHE6OC8fvTg`XX1=G?i zx~0q8!KI}2-lCbDO17<5$FP#=WVL42EA8aexus-7U!W#$pMcf%t~5L4x(VNnPQ9*E zaSGzibdo_Qr_?!C;wJcQ7OZApeCv&|;UqP7oGN-0HTSyrV^CX;P2n{1Pb zBFnz0tk8$8g2=L6)Y5f*a1kE_K@|3>zU+%2r43R_@nILCiXc`WR?nT8J9FEtkgbr1 zGw0{~&N=tob94UW3k8g@ic|Q3JFV}{(bUP@$R#p+s*5n z8}WVI?%lt*d;Rb|>NbkA`2B02Tsh3SjhuDS5`bCjG+wtlxne>0F#q}d&TfQrX4Kt{ znI&{(mr+0HfBa`+AGf>5rz;;`yUN@)FE4$T+{f()5P~Ipy3w)R$cWQt78nMj zXm?#xxYujcJPQ#ORawhODo!XTqDX>6@l(Pg5y=^HE+b6_Ll;N{jryY)XrYNP>`*d7 zX0U{K2w^eI8#Ay^kwob*cYy5_)7k(tc8Fv*WP zF9Qd633v<7bN>(~WF>~5N0~j~VXb0E;n%STME;ctBw7f&NP(6ocqa)2cVe_t+v1}H zT%)!{AYhj+kSM{RmM2)H1Zxr{ScRp=DsTqIPk%K8=^5|bu^uB|>1L*7= z+!=J>UnJ9+9e@w6JSPIdzIvYG2Blbu<5MWJ6;^TBJ8BqeKmDyRq|a7HAkp*(wLJDu zPWo}$oMm`sSo+RAanLUdWLW^)0tpMcf#uC;sBrUl=vUbAUX zaRTD4Y??u*Cl(eLvyVcwTaORnF47<#5^l6+hfl%Vx&gpJYj6PmPX%}nNshf49Wa?wpajUND@c*$`G%I7rlr?xg|Lcc7iXn6!{^tG*9k0SS literal 0 HcmV?d00001 From 616b72d665540e4748fa4b45f3321e61121d4456 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 2 Sep 2022 07:11:47 +0000 Subject: [PATCH 017/526] Fix bug --- src/Storages/StorageDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 05b418e4208..52f955dee0a 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -93,7 +93,7 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Timestamp {}", timestamp); } else if (json.has("remove")) { auto path = json["remove"]["path"].getString(); - auto timestamp = json["remove"]["modificationTime"].getInt(); + auto timestamp = json["remove"]["deletionTimestamp"].getInt(); metadata.remove(path, timestamp); From a77aa6b2b0bfaf80fc8078bc29789c091cc98a8f Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 15:01:34 +0000 Subject: [PATCH 018/526] Remove unneeded comments, format --- src/Storages/StorageDelta.cpp | 83 ++++++++++++++++++++--------------- src/Storages/StorageDelta.h | 12 +++-- 2 files changed, 52 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 52f955dee0a..5fa60206360 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -2,9 +2,9 @@ #include #include -#include #include #include +#include #include #include @@ -24,49 +24,52 @@ namespace ErrorCodes extern const int S3_ERROR; } -void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) { +void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) +{ file_update_time[key] = timestamp; } -void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) { +void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) +{ file_update_time.erase(key); } -std::vector DeltaLakeMetadata::ListCurrentFiles() && { +std::vector DeltaLakeMetadata::ListCurrentFiles() && +{ std::vector keys; keys.reserve(file_update_time.size()); - for (auto && [k, _] : file_update_time) { + for (auto && [k, _] : file_update_time) + { keys.push_back(k); } return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, - const String & table_path_, - Poco::Logger * log_) : - base_configuration(configuration_) - , table_path(table_path_) - , metadata() - , log(log_) - { - Init(); - } +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) + : base_configuration(configuration_), table_path(table_path_), metadata(), log(log_) +{ + Init(); +} -void JsonMetadataGetter::Init() { +void JsonMetadataGetter::Init() +{ auto keys = getJsonLogFiles(); - + // read data from every json log file - for (const String & key : keys) { + for (const String & key : keys) + { auto buf = createS3ReadBuffer(key); String json_str; size_t opening(0), closing(0); char c; - while (buf->read(c)) { + while (buf->read(c)) + { // skip all space characters for JSON to parse correctly - if (isspace(c)) { + if (isspace(c)) + { continue; } @@ -77,13 +80,14 @@ void JsonMetadataGetter::Init() { else if (c == '}') closing++; - if (opening == closing) { - + if (opening == closing) + { LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); JSON json(json_str); - if (json.has("add")) { + if (json.has("add")) + { auto path = json["add"]["path"].getString(); auto timestamp = json["add"]["modificationTime"].getInt(); @@ -91,7 +95,9 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Path {}", path); LOG_DEBUG(log, "Timestamp {}", timestamp); - } else if (json.has("remove")) { + } + else if (json.has("remove")) + { auto path = json["remove"]["path"].getString(); auto timestamp = json["remove"]["deletionTimestamp"].getInt(); @@ -100,20 +106,18 @@ void JsonMetadataGetter::Init() { LOG_DEBUG(log, "Path {}", path); LOG_DEBUG(log, "Timestamp {}", timestamp); } - + // reset opening = 0; closing = 0; json_str.clear(); - } - } } - } -std::vector JsonMetadataGetter::getJsonLogFiles() { +std::vector JsonMetadataGetter::getJsonLogFiles() +{ std::vector keys; const auto & client = base_configuration.client; @@ -143,7 +147,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { for (const auto & obj : result_batch) { const auto & filename = obj.GetKey(); - + if (filename.substr(filename.size() - 5) == ".json") keys.push_back(filename); } @@ -155,11 +159,17 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { return keys; } -std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { - // size_t object_size = DB::S3::getObjectSize(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, false); +std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +{ // TBD: add parallel downloads - return std::make_unique(base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, /* max single read retries */ 10, ReadSettings{}); + return std::make_unique( + base_configuration.client, + base_configuration.uri.bucket, + key, + base_configuration.uri.version_id, + /* max single read retries */ 10, + ReadSettings{}); } StorageDelta::StorageDelta( @@ -178,12 +188,13 @@ StorageDelta::StorageDelta( { StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - + JsonMetadataGetter getter{base_configuration, table_path, log}; auto keys = getter.getFiles(); - for (const String & path : keys) { + for (const String & path : keys) + { LOG_DEBUG(log, "{}", path); } @@ -200,7 +211,7 @@ StorageDelta::StorageDelta( storage_metadata.setColumns(columns_); } else - storage_metadata.setColumns(columns_); + storage_metadata.setColumns(columns_); storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 2aec4f815f3..4e9199a5faa 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -22,7 +22,8 @@ namespace DB { // class to parse json deltalake metadata and find files needed for query in table -class DeltaLakeMetadata { +class DeltaLakeMetadata +{ public: DeltaLakeMetadata() = default; @@ -38,13 +39,10 @@ private: }; // class to get deltalake log json files and read json from them -class JsonMetadataGetter +class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, - const String & table_path_, - Poco::Logger * log_ - ); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); private: void Init(); @@ -91,7 +89,7 @@ public: private: void Init(); static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - + private: String generateQueryFromKeys(std::vector && keys); From f75376b7f8eaafd38a3c4d0ad13b875a73d1b778 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 18:05:33 +0000 Subject: [PATCH 019/526] Apply black formatter to tests --- tests/integration/test_storage_delta/test.py | 38 +++++++++---- tests/integration/test_storage_hudi/test.py | 59 ++++++++++++++------ 2 files changed, 68 insertions(+), 29 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index a1cc6345619..76dab554a57 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -9,6 +9,7 @@ from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): bucket_read_write_policy = { "Version": "2012-10-17", @@ -49,23 +50,26 @@ def prepare_s3_bucket(started_cluster): started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) ) + def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): - address_without_prefix = address[len(SCRIPT_DIR):] + address_without_prefix = address[len(SCRIPT_DIR) :] for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "main_server", - with_minio=True - ) + cluster.add_instance("main_server", with_minio=True) logging.info("Starting cluster...") cluster.start() @@ -77,10 +81,11 @@ def started_cluster(): logging.info("Test table uploaded") yield cluster - + finally: cluster.shutdown() + def run_query(instance, query, stdin=None, settings=None): # type: (ClickHouseInstance, str, object, dict) -> str @@ -99,13 +104,22 @@ def test_create_query(started_cluster): run_query(instance, create_query) + def test_select_query(started_cluster): instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket - columns = ['begin_lat', - 'begin_lon', 'driver', 'end_lat', 'end_lon', - 'fare', 'rider', 'ts', 'uuid'] - + columns = [ + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "rider", + "ts", + "uuid", + ] + # create query in case table doesn't exist create_query = f"""CREATE TABLE IF NOT EXISTS deltalake ENGINE=DeltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" @@ -115,4 +129,4 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() - assert(len(result) > 0) + assert len(result) > 0 diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index c9415e28151..549421afd89 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -9,6 +9,7 @@ from helpers.test_tools import TSV SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + def prepare_s3_bucket(started_cluster): bucket_read_write_policy = { "Version": "2012-10-17", @@ -49,23 +50,26 @@ def prepare_s3_bucket(started_cluster): started_cluster.minio_bucket, json.dumps(bucket_read_write_policy) ) + def upload_test_table(started_cluster): bucket = started_cluster.minio_bucket for address, dirs, files in os.walk(SCRIPT_DIR + "/test_table"): - address_without_prefix = address[len(SCRIPT_DIR):] + address_without_prefix = address[len(SCRIPT_DIR) :] for name in files: - started_cluster.minio_client.fput_object(bucket, os.path.join(address_without_prefix, name), os.path.join(address, name)) + started_cluster.minio_client.fput_object( + bucket, + os.path.join(address_without_prefix, name), + os.path.join(address, name), + ) + @pytest.fixture(scope="module") def started_cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance( - "main_server", - with_minio=True - ) + cluster.add_instance("main_server", with_minio=True) logging.info("Starting cluster...") cluster.start() @@ -77,10 +81,11 @@ def started_cluster(): logging.info("Test table uploaded") yield cluster - + finally: cluster.shutdown() + def run_query(instance, query, stdin=None, settings=None): # type: (ClickHouseInstance, str, object, dict) -> str @@ -99,14 +104,28 @@ def test_create_query(started_cluster): run_query(instance, create_query) + def test_select_query(started_cluster): instance = started_cluster.instances["main_server"] bucket = started_cluster.minio_bucket - columns = ['_hoodie_commit_time', '_hoodie_commit_seqno', '_hoodie_record_key', - '_hoodie_partition_path', '_hoodie_file_name', 'begin_lat', - 'begin_lon', 'driver', 'end_lat', 'end_lon', - 'fare', 'partitionpath', 'rider', 'ts', 'uuid'] - + columns = [ + "_hoodie_commit_time", + "_hoodie_commit_seqno", + "_hoodie_record_key", + "_hoodie_partition_path", + "_hoodie_file_name", + "begin_lat", + "begin_lon", + "driver", + "end_lat", + "end_lon", + "fare", + "partitionpath", + "rider", + "ts", + "uuid", + ] + # create query in case table doesn't exist create_query = f"""CREATE TABLE IF NOT EXISTS hudi ENGINE=Hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" @@ -116,11 +135,17 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() - assert(len(result) > 0) + assert len(result) > 0 - # test if all partition paths is presented in result - distinct_select_query = "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + # test if all partition paths is presented in result + distinct_select_query = ( + "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" + ) result = run_query(instance, distinct_select_query) - expected = ['americas/brazil/sao_paulo', 'americas/united_states/san_francisco', 'asia/india/chennai'] + expected = [ + "americas/brazil/sao_paulo", + "americas/united_states/san_francisco", + "asia/india/chennai", + ] - assert TSV(result) == TSV(expected) \ No newline at end of file + assert TSV(result) == TSV(expected) From 07d12338ca7acfb828737c0744fb01ceeb5a6233 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 6 Sep 2022 18:12:51 +0000 Subject: [PATCH 020/526] Fix ErrorCodes style --- src/Storages/StorageDelta.cpp | 2 +- src/Storages/StorageHudi.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 5fa60206360..518c25c2dff 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -20,8 +20,8 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; + extern const int BAD_ARGUMENTS; } void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index b4eba258d0e..f3e16d2969a 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -17,7 +17,7 @@ namespace DB namespace ErrorCodes { - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } From ba08366af18bd89ea47e15146028b0c8f280f928 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 07:16:32 +0000 Subject: [PATCH 021/526] Fix fasttest build --- src/Storages/StorageDelta.cpp | 7 ++++++- src/Storages/StorageDelta.h | 4 ++++ src/Storages/StorageHudi.cpp | 7 ++++++- src/Storages/StorageHudi.h | 4 ++++ 4 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 518c25c2dff..9f53d0edba8 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,5 +1,8 @@ -#include #include + +#if USE_AWS_S3 + +#include #include #include @@ -346,3 +349,5 @@ void registerStorageDelta(StorageFactory & factory) } } + +#endif diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 4e9199a5faa..cf9cc737526 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -2,6 +2,8 @@ #include "config_core.h" +#if USE_AWS_S3 + #include #include @@ -102,3 +104,5 @@ private: }; } + +#endif diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index f3e16d2969a..6162f2ba4dc 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,5 +1,8 @@ -#include #include + +#if USE_AWS_S3 + +#include #include #include @@ -280,3 +283,5 @@ void registerStorageHudi(StorageFactory & factory) } } + +#endif diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 47dff1c2a7b..5e1aae81182 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -2,6 +2,8 @@ #include "config_core.h" +#if USE_AWS_S3 + #include #include @@ -57,3 +59,5 @@ private: }; } + +#endif From 152da1a034b79798d996b6e412f20f3bb9537d2f Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 08:13:08 +0000 Subject: [PATCH 022/526] Fix build --- src/Storages/registerStorages.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index c0d153a5efa..c64c2e8d2cd 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -34,6 +34,8 @@ void registerStorageMeiliSearch(StorageFactory& factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory); +void registerStorageHudi(StorageFactory & factory); +void registerStorageDelta(StorageFactory & factory); #endif #if USE_HDFS @@ -88,8 +90,6 @@ void registerStorageFileLog(StorageFactory & factory); void registerStorageSQLite(StorageFactory & factory); #endif -void registerStorageHudi(StorageFactory & factory); -void registerStorageDelta(StorageFactory & factory); void registerStorages() { @@ -121,6 +121,8 @@ void registerStorages() #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); + registerStorageHudi(factory); + registerStorageDelta(factory); #endif #if USE_HDFS @@ -174,8 +176,6 @@ void registerStorages() registerStorageSQLite(factory); #endif - registerStorageHudi(factory); - registerStorageDelta(factory); } } From 3f9022bc7f2d326a41382c23fd4709be6bc99c27 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 7 Sep 2022 10:48:11 +0000 Subject: [PATCH 023/526] Fix createClientConfiguration arguments --- src/Storages/StorageDelta.cpp | 3 ++- src/Storages/StorageHudi.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 9f53d0edba8..c2d421c827f 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -276,7 +276,8 @@ void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurat settings.auth_settings.region, ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = upd.rw_settings.max_connections; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 6162f2ba4dc..9b6207338eb 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -118,7 +118,8 @@ void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurati settings.auth_settings.region, ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging); + ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = upd.rw_settings.max_connections; From 0a1734bb69f85ae9ccc009b720567efbcdf56301 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Mon, 19 Sep 2022 15:23:07 +0000 Subject: [PATCH 024/526] Fix clang-tidy errors --- src/Storages/StorageDelta.cpp | 10 +++++----- src/Storages/StorageDelta.h | 2 +- src/Storages/StorageHudi.h | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index c2d421c827f..46db33279ab 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -27,14 +27,14 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } -void DeltaLakeMetadata::add(const String & key, uint64_t timestamp) +void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) { - file_update_time[key] = timestamp; + file_update_time[filename] = timestamp; } -void DeltaLakeMetadata::remove(const String & key, uint64_t /*timestamp */) +void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */) { - file_update_time.erase(key); + file_update_time.erase(filename); } std::vector DeltaLakeMetadata::ListCurrentFiles() && @@ -51,7 +51,7 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && } JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) - : base_configuration(configuration_), table_path(table_path_), metadata(), log(log_) + : base_configuration(configuration_), table_path(table_path_), log(log_) { Init(); } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index cf9cc737526..f4547666c8c 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -94,7 +94,7 @@ private: private: - String generateQueryFromKeys(std::vector && keys); + static String generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 5e1aae81182..ceb12f5f550 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -49,7 +49,7 @@ private: private: std::vector getKeysFromS3(); - std::string generateQueryFromKeys(std::vector && keys); + static std::string generateQueryFromKeys(std::vector && keys); private: StorageS3::S3Configuration base_configuration; From 3fb26aefa39eda4d1f23b5b189699ea264fa8af1 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 10:24:47 +0000 Subject: [PATCH 025/526] Remove unneeded logging, better --- src/Storages/StorageDelta.cpp | 39 ++++++++++------------------------- src/Storages/StorageDelta.h | 12 ++--------- 2 files changed, 13 insertions(+), 38 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 46db33279ab..a79762e29db 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -18,6 +18,9 @@ #include +#include +#include + namespace DB { @@ -25,6 +28,7 @@ namespace ErrorCodes { extern const int S3_ERROR; extern const int BAD_ARGUMENTS; + extern const int INCORRECT_DATA; } void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) @@ -34,7 +38,9 @@ void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) void DeltaLakeMetadata::remove(const String & filename, uint64_t /*timestamp */) { - file_update_time.erase(filename); + bool erase = file_update_time.erase(filename); + if (!erase) + throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid table metadata, tried to remove {} before adding it", filename); } std::vector DeltaLakeMetadata::ListCurrentFiles() && @@ -95,9 +101,6 @@ void JsonMetadataGetter::Init() auto timestamp = json["add"]["modificationTime"].getInt(); metadata.add(path, timestamp); - - LOG_DEBUG(log, "Path {}", path); - LOG_DEBUG(log, "Timestamp {}", timestamp); } else if (json.has("remove")) { @@ -105,9 +108,6 @@ void JsonMetadataGetter::Init() auto timestamp = json["remove"]["deletionTimestamp"].getInt(); metadata.remove(path, timestamp); - - LOG_DEBUG(log, "Path {}", path); - LOG_DEBUG(log, "Timestamp {}", timestamp); } // reset @@ -132,7 +132,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() const auto bucket{base_configuration.uri.bucket}; request.SetBucket(bucket); - request.SetPrefix(table_path + "_delta_log"); + request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) { @@ -151,7 +151,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { const auto & filename = obj.GetKey(); - if (filename.substr(filename.size() - 5) == ".json") + if (std::filesystem::path(filename).extension() == ".json") keys.push_back(filename); } @@ -164,7 +164,6 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { - // TBD: add parallel downloads return std::make_unique( base_configuration.client, @@ -196,11 +195,6 @@ StorageDelta::StorageDelta( auto keys = getter.getFiles(); - for (const String & path : keys) - { - LOG_DEBUG(log, "{}", path); - } - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -225,7 +219,7 @@ StorageDelta::StorageDelta( access_key_, secret_access_key_, table_id_, - String("Parquet"), // format name + "Parquet", // format name base_configuration.rw_settings, columns_, constraints_, @@ -297,18 +291,7 @@ void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configurat String StorageDelta::generateQueryFromKeys(std::vector && keys) { - String new_query; - - for (auto && key : keys) - { - if (!new_query.empty()) - { - new_query += ","; - } - new_query += key; - } - new_query = "{" + new_query + "}"; - + std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f4547666c8c..f7fa4120495 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -29,11 +29,9 @@ class DeltaLakeMetadata public: DeltaLakeMetadata() = default; -public: void add(const String & filename, uint64_t timestamp); void remove(const String & filename, uint64_t timestamp); -public: std::vector ListCurrentFiles() &&; private: @@ -46,18 +44,15 @@ class JsonMetadataGetter public: JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); + std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } + private: void Init(); std::vector getJsonLogFiles(); -private: std::unique_ptr createS3ReadBuffer(const String & key); -public: - std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } - -private: StorageS3::S3Configuration base_configuration; String table_path; DeltaLakeMetadata metadata; @@ -92,11 +87,8 @@ private: void Init(); static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - -private: static String generateQueryFromKeys(std::vector && keys); -private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; From 745331a3f3991440d4a87554fac3c3f8b26add75 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 11:43:20 +0000 Subject: [PATCH 026/526] Add format argument with default value to storage --- src/Storages/StorageDelta.cpp | 33 +++++++++++++++++++-------------- src/Storages/StorageDelta.h | 1 + 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index a79762e29db..9b36b234fc5 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -27,7 +27,7 @@ namespace DB namespace ErrorCodes { extern const int S3_ERROR; - extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int INCORRECT_DATA; } @@ -179,6 +179,7 @@ StorageDelta::StorageDelta( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -204,7 +205,7 @@ StorageDelta::StorageDelta( if (columns_.empty()) { columns_ - = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -219,7 +220,7 @@ StorageDelta::StorageDelta( access_key_, secret_access_key_, table_id_, - "Parquet", // format name + format_name_, base_configuration.rw_settings, columns_, constraints_, @@ -303,23 +304,27 @@ void registerStorageDelta(StorageFactory & factory) [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); + if (engine_args.empty() || engine_args.size() < 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); + + String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); + String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + String format = "Parquet"; + if (engine_args.size() == 4) { + format = checkAndGetLiteralArgument(engine_args[3], "format"); + } - configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - - S3::URI s3_uri(Poco::URI(configuration.url)); + auto s3_uri = S3::URI(Poco::URI(table_url)); return std::make_shared( s3_uri, - configuration.auth_settings.access_key_id, - configuration.auth_settings.secret_access_key, + access_key_id, + secret_access_key, args.table_id, + format, args.columns, args.constraints, args.comment, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f7fa4120495..010fc53546d 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -67,6 +67,7 @@ public: const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, From 8fe4485ee8ddc049cff3775a7b9f6293a4a457a6 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Tue, 20 Sep 2022 14:16:27 +0000 Subject: [PATCH 027/526] Change JSON parsing --- src/Storages/StorageDelta.cpp | 97 ++++++++++++++++++++--------------- src/Storages/StorageDelta.h | 9 +++- 2 files changed, 64 insertions(+), 42 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 9b36b234fc5..ec0f233f9ed 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -70,51 +70,18 @@ void JsonMetadataGetter::Init() for (const String & key : keys) { auto buf = createS3ReadBuffer(key); - String json_str; - size_t opening(0), closing(0); - char c; - while (buf->read(c)) + while (!buf->eof()) { - // skip all space characters for JSON to parse correctly - if (isspace(c)) - { + String json_str = readJSONStringFromBuffer(buf); + + if (json_str.empty()) { continue; } - json_str.push_back(c); + const JSON json(json_str); - if (c == '{') - opening++; - else if (c == '}') - closing++; - - if (opening == closing) - { - LOG_DEBUG(log, "JSON {}, {}", json_str, json_str.size()); - - JSON json(json_str); - - if (json.has("add")) - { - auto path = json["add"]["path"].getString(); - auto timestamp = json["add"]["modificationTime"].getInt(); - - metadata.add(path, timestamp); - } - else if (json.has("remove")) - { - auto path = json["remove"]["path"].getString(); - auto timestamp = json["remove"]["deletionTimestamp"].getInt(); - - metadata.remove(path, timestamp); - } - - // reset - opening = 0; - closing = 0; - json_str.clear(); - } + handleJSON(json); } } } @@ -162,10 +129,10 @@ std::vector JsonMetadataGetter::getJsonLogFiles() return keys; } -std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) { // TBD: add parallel downloads - return std::make_unique( + return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, @@ -174,6 +141,54 @@ std::unique_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } +String JsonMetadataGetter::readJSONStringFromBuffer(std::shared_ptr buf) { + String json_str; + + int32_t opening(0), closing(0); + + do { + char c; + + if (!buf->read(c)) + return json_str; + + // skip all space characters for JSON to parse correctly + if (isspace(c)) + { + continue; + } + + json_str.push_back(c); + + if (c == '{') + opening++; + else if (c == '}') + closing++; + + } while (opening != closing || opening == 0); + + LOG_DEBUG(log, "JSON {}", json_str); + + return json_str; +} + +void JsonMetadataGetter::handleJSON(const JSON & json) { + if (json.has("add")) + { + auto path = json["add"]["path"].getString(); + auto timestamp = json["add"]["modificationTime"].getInt(); + + metadata.add(path, timestamp); + } + else if (json.has("remove")) + { + auto path = json["remove"]["path"].getString(); + auto timestamp = json["remove"]["deletionTimestamp"].getInt(); + + metadata.remove(path, timestamp); + } +} + StorageDelta::StorageDelta( const S3::URI & uri_, const String & access_key_, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 010fc53546d..1b9ee960080 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -51,7 +51,14 @@ private: std::vector getJsonLogFiles(); - std::unique_ptr createS3ReadBuffer(const String & key); + std::shared_ptr createS3ReadBuffer(const String & key); + + /* every metadata file contains many jsons + this function reads one json from buffer + */ + String readJSONStringFromBuffer(std::shared_ptr buf); + + void handleJSON(const JSON & json); StorageS3::S3Configuration base_configuration; String table_path; From 3f7779e874ac27ab772dafdd36a923b32700db3c Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 21 Sep 2022 16:14:51 +0000 Subject: [PATCH 028/526] Rename method, change parsing method --- src/Storages/StorageDelta.cpp | 50 +++++++++-------------------------- src/Storages/StorageDelta.h | 7 +---- 2 files changed, 14 insertions(+), 43 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index ec0f233f9ed..5818e770a37 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -6,6 +6,7 @@ #include #include +#include #include #include @@ -31,7 +32,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; } -void DeltaLakeMetadata::add(const String & filename, uint64_t timestamp) +void DeltaLakeMetadata::setLastModifiedTime(const String & filename, uint64_t timestamp) { file_update_time[filename] = timestamp; } @@ -72,12 +73,18 @@ void JsonMetadataGetter::Init() auto buf = createS3ReadBuffer(key); while (!buf->eof()) - { - String json_str = readJSONStringFromBuffer(buf); + { + // may be some invalid characters before json + char c; + while ( buf->peek(c) && c != '{') buf->ignore(); + if (buf->eof()) + break; + + String json_str; + readJSONObjectPossiblyInvalid(json_str, *buf); - if (json_str.empty()) { + if (json_str.empty()) continue; - } const JSON json(json_str); @@ -141,44 +148,13 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } -String JsonMetadataGetter::readJSONStringFromBuffer(std::shared_ptr buf) { - String json_str; - - int32_t opening(0), closing(0); - - do { - char c; - - if (!buf->read(c)) - return json_str; - - // skip all space characters for JSON to parse correctly - if (isspace(c)) - { - continue; - } - - json_str.push_back(c); - - if (c == '{') - opening++; - else if (c == '}') - closing++; - - } while (opening != closing || opening == 0); - - LOG_DEBUG(log, "JSON {}", json_str); - - return json_str; -} - void JsonMetadataGetter::handleJSON(const JSON & json) { if (json.has("add")) { auto path = json["add"]["path"].getString(); auto timestamp = json["add"]["modificationTime"].getInt(); - metadata.add(path, timestamp); + metadata.setLastModifiedTime(path, timestamp); } else if (json.has("remove")) { diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 1b9ee960080..542d0b7c4c0 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -29,7 +29,7 @@ class DeltaLakeMetadata public: DeltaLakeMetadata() = default; - void add(const String & filename, uint64_t timestamp); + void setLastModifiedTime(const String & filename, uint64_t timestamp); void remove(const String & filename, uint64_t timestamp); std::vector ListCurrentFiles() &&; @@ -52,11 +52,6 @@ private: std::vector getJsonLogFiles(); std::shared_ptr createS3ReadBuffer(const String & key); - - /* every metadata file contains many jsons - this function reads one json from buffer - */ - String readJSONStringFromBuffer(std::shared_ptr buf); void handleJSON(const JSON & json); From 4d1abdc80891a468869007c19120d324c877e499 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 21 Sep 2022 16:54:02 +0000 Subject: [PATCH 029/526] Remove unneeded logging, add comment, better --- src/Storages/StorageDelta.cpp | 6 +++--- src/Storages/StorageDelta.h | 3 +-- src/Storages/StorageHudi.cpp | 7 ++----- 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 5818e770a37..951337f88c3 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -57,8 +57,8 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_) - : base_configuration(configuration_), table_path(table_path_), log(log_) +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_) + : base_configuration(configuration_), table_path(table_path_) { Init(); } @@ -183,7 +183,7 @@ StorageDelta::StorageDelta( StorageInMemoryMetadata storage_metadata; updateS3Configuration(context_, base_configuration); - JsonMetadataGetter getter{base_configuration, table_path, log}; + JsonMetadataGetter getter{base_configuration, table_path}; auto keys = getter.getFiles(); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 542d0b7c4c0..fb6e19b7488 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -42,7 +42,7 @@ private: class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, Poco::Logger * log_); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_); std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } @@ -58,7 +58,6 @@ private: StorageS3::S3Configuration base_configuration; String table_path; DeltaLakeMetadata metadata; - Poco::Logger * log; }; class StorageDelta : public IStorage diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 9b6207338eb..55f7e4b9c10 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -186,11 +186,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) keys, [](const std::string & s) { - if (s.size() >= 8) - { - return s.substr(s.size() - 8) != ".parquet"; - } - return true; + return std::filesystem::path(s).extension() != "parquet"; }); // for each partition path take only latest parquet file @@ -210,6 +206,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) path = key.substr(0, slash); } + // every filename contains metadata splitted by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); From 13e50c771c3b18db30e817b9ba7ae8a6fe1abb6e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 11:03:43 +0000 Subject: [PATCH 030/526] Fix bug, use static method --- src/Storages/StorageDelta.cpp | 52 ++------------------------------ src/Storages/StorageDelta.h | 2 -- src/Storages/StorageHudi.cpp | 56 +++-------------------------------- src/Storages/StorageHudi.h | 4 --- src/Storages/StorageS3.h | 2 ++ 5 files changed, 8 insertions(+), 108 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 951337f88c3..2c7f348ef94 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -181,7 +181,7 @@ StorageDelta::StorageDelta( , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; - updateS3Configuration(context_, base_configuration); + StorageS3::updateS3Configuration(context_, base_configuration); JsonMetadataGetter getter{base_configuration, table_path}; @@ -229,65 +229,17 @@ Pipe StorageDelta::read( size_t max_block_size, unsigned num_streams) { - updateS3Configuration(context, base_configuration); + StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -void StorageDelta::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) -{ - auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - - bool need_update_configuration = settings != S3Settings{}; - if (need_update_configuration) - { - if (upd.rw_settings != settings.rw_settings) - upd.rw_settings = settings.rw_settings; - } - - upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); - - if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) - return; - - Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); - HeaderCollection headers; - if (upd.access_key_id.empty()) - { - credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); - headers = settings.auth_settings.headers; - } - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - settings.auth_settings.region, - ctx->getRemoteHostFilter(), - ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); - - client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = upd.rw_settings.max_connections; - - upd.client = S3::ClientFactory::instance().create( - client_configuration, - upd.uri.is_virtual_hosted_style, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - settings.auth_settings.server_side_encryption_customer_key_base64, - std::move(headers), - settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); - - upd.auth_settings = std::move(settings.auth_settings); -} - String StorageDelta::generateQueryFromKeys(std::vector && keys) { std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } - void registerStorageDelta(StorageFactory & factory) { factory.registerStorage( diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index fb6e19b7488..7b25d2c618e 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -87,8 +87,6 @@ public: private: void Init(); - static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - static String generateQueryFromKeys(std::vector && keys); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 55f7e4b9c10..80555efcafd 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -39,7 +39,7 @@ StorageHudi::StorageHudi( , table_path(uri_.key) { StorageInMemoryMetadata storage_metadata; - updateS3Configuration(context_, base_configuration); + StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); @@ -84,59 +84,11 @@ Pipe StorageHudi::read( QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) -{ - updateS3Configuration(context, base_configuration); - +{ + StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } -void StorageHudi::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) -{ - auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - - bool need_update_configuration = settings != S3Settings{}; - if (need_update_configuration) - { - if (upd.rw_settings != settings.rw_settings) - upd.rw_settings = settings.rw_settings; - } - - upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); - - if (upd.client && (!upd.access_key_id.empty() || settings.auth_settings == upd.auth_settings)) - return; - - Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key); - HeaderCollection headers; - if (upd.access_key_id.empty()) - { - credentials = Aws::Auth::AWSCredentials(settings.auth_settings.access_key_id, settings.auth_settings.secret_access_key); - headers = settings.auth_settings.headers; - } - - S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( - settings.auth_settings.region, - ctx->getRemoteHostFilter(), - ctx->getGlobalContext()->getSettingsRef().s3_max_redirects, - ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); - - client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = upd.rw_settings.max_connections; - - upd.client = S3::ClientFactory::instance().create( - client_configuration, - upd.uri.is_virtual_hosted_style, - credentials.GetAWSAccessKeyId(), - credentials.GetAWSSecretKey(), - settings.auth_settings.server_side_encryption_customer_key_base64, - std::move(headers), - settings.auth_settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)), - settings.auth_settings.use_insecure_imds_request.value_or(ctx->getConfigRef().getBool("s3.use_insecure_imds_request", false))); - - upd.auth_settings = std::move(settings.auth_settings); -} - std::vector StorageHudi::getKeysFromS3() { std::vector keys; @@ -186,7 +138,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) keys, [](const std::string & s) { - return std::filesystem::path(s).extension() != "parquet"; + return std::filesystem::path(s).extension() != ".parquet"; }); // for each partition path take only latest parquet file diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index ceb12f5f550..e0d5cf5a329 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -44,14 +44,10 @@ public: size_t max_block_size, unsigned num_streams) override; -private: - static void updateS3Configuration(ContextPtr, StorageS3::S3Configuration &); - private: std::vector getKeysFromS3(); static std::string generateQueryFromKeys(std::vector && keys); -private: StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; Poco::Logger * log; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 3a02237570d..b6589bf49aa 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -216,6 +216,8 @@ public: private: friend class StorageS3Cluster; friend class TableFunctionS3Cluster; + friend class StorageHudi; + friend class StorageDelta; S3Configuration s3_configuration; std::vector keys; From db7cb4e9b5249c7d31ffbe377d3feffcbc2b0af4 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 11:21:32 +0000 Subject: [PATCH 031/526] apply clang-format --- src/Storages/StorageDelta.cpp | 54 +++++++++++++++++++---------------- src/Storages/StorageDelta.h | 10 +++---- src/Storages/StorageHudi.cpp | 27 +++++++----------- src/Storages/StorageHudi.h | 4 +-- 4 files changed, 47 insertions(+), 48 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 2c7f348ef94..027312385bb 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -2,25 +2,25 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include -#include -#include +# include +# include +# include +# include -#include -#include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include -#include -#include +# include +# include namespace DB { @@ -73,16 +73,17 @@ void JsonMetadataGetter::Init() auto buf = createS3ReadBuffer(key); while (!buf->eof()) - { + { // may be some invalid characters before json char c; - while ( buf->peek(c) && c != '{') buf->ignore(); - if (buf->eof()) + while (buf->peek(c) && c != '{') + buf->ignore(); + if (buf->eof()) break; String json_str; readJSONObjectPossiblyInvalid(json_str, *buf); - + if (json_str.empty()) continue; @@ -148,7 +149,8 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String ReadSettings{}); } -void JsonMetadataGetter::handleJSON(const JSON & json) { +void JsonMetadataGetter::handleJSON(const JSON & json) +{ if (json.has("add")) { auto path = json["add"]["path"].getString(); @@ -195,8 +197,7 @@ StorageDelta::StorageDelta( if (columns_.empty()) { - columns_ - = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -248,15 +249,18 @@ void registerStorageDelta(StorageFactory & factory) { auto & engine_args = args.engine_args; if (engine_args.empty() || engine_args.size() < 3) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); + - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - + String format = "Parquet"; - if (engine_args.size() == 4) { + if (engine_args.size() == 4) + { format = checkAndGetLiteralArgument(engine_args[3], "format"); } diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 7b25d2c618e..c5dc0b2fd07 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -4,11 +4,11 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include +# include +# include namespace Poco { @@ -52,7 +52,7 @@ private: std::vector getJsonLogFiles(); std::shared_ptr createS3ReadBuffer(const String & key); - + void handleJSON(const JSON & json); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 80555efcafd..2da5a1e8b5e 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -2,18 +2,18 @@ #if USE_AWS_S3 -#include -#include +# include +# include -#include -#include -#include +# include +# include +# include -#include -#include -#include +# include +# include +# include -#include +# include namespace DB { @@ -84,7 +84,7 @@ Pipe StorageHudi::read( QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) -{ +{ StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); } @@ -134,12 +134,7 @@ std::vector StorageHudi::getKeysFromS3() std::string StorageHudi::generateQueryFromKeys(std::vector && keys) { // filter only .parquet files - std::erase_if( - keys, - [](const std::string & s) - { - return std::filesystem::path(s).extension() != ".parquet"; - }); + std::erase_if(keys, [](const std::string & s) { return std::filesystem::path(s).extension() != ".parquet"; }); // for each partition path take only latest parquet file diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index e0d5cf5a329..d782106d2b0 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -4,8 +4,8 @@ #if USE_AWS_S3 -#include -#include +# include +# include namespace Poco { From 952d546a7c979f43f7ce8801f049f3774b465877 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 28 Sep 2022 16:19:06 +0200 Subject: [PATCH 032/526] Better logging for docs builder CI=true is not pushed from github actions inside container, so it can use some fancy console output formats which may accidentally suppress the docusaurus output, makind the real error not visible in CI report (see https://github.com/ClickHouse/ClickHouse/pull/41254#issuecomment-1259275622 ) --- docker/docs/builder/run.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/docker/docs/builder/run.sh b/docker/docs/builder/run.sh index a4f678b2f24..87e6218547f 100755 --- a/docker/docs/builder/run.sh +++ b/docker/docs/builder/run.sh @@ -25,6 +25,7 @@ done sed -i '/onBrokenMarkdownLinks:/ s/ignore/error/g' docusaurus.config.js if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then + export CI=true exec yarn build "$@" fi From 0d3670d516f8dbae493784dc50c09c726d21793a Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Wed, 28 Sep 2022 17:30:15 +0000 Subject: [PATCH 033/526] Fix typo --- src/Storages/StorageHudi.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 2da5a1e8b5e..4db372e7e96 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -153,7 +153,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys) path = key.substr(0, slash); } - // every filename contains metadata splitted by "_", timestamp is after last "_" + // every filename contains metadata split by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); auto it = latest_parquets.find(path); From 74f026ba65f53c54f33bb83d175a40d17465e66e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 29 Sep 2022 20:15:27 +0000 Subject: [PATCH 034/526] Change logic error, add format argument to Hudi engine --- src/Storages/StorageHudi.cpp | 49 ++++++++++++++++++++++-------------- src/Storages/StorageHudi.h | 3 ++- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 4db372e7e96..78f3a0dd319 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -20,6 +20,7 @@ namespace DB namespace ErrorCodes { + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } @@ -29,6 +30,7 @@ StorageHudi::StorageHudi( const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -43,7 +45,7 @@ StorageHudi::StorageHudi( auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), format_); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); @@ -51,8 +53,7 @@ StorageHudi::StorageHudi( if (columns_.empty()) { - columns_ - = StorageS3::getTableStructureFromData(String("Parquet"), s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(format_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); storage_metadata.setColumns(columns_); } else @@ -67,7 +68,7 @@ StorageHudi::StorageHudi( access_key_, secret_access_key_, table_id_, - String("Parquet"), // format name + format_, base_configuration.rw_settings, columns_, constraints_, @@ -131,12 +132,15 @@ std::vector StorageHudi::getKeysFromS3() return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector && keys) +std::string StorageHudi::generateQueryFromKeys(std::vector && keys, String format) { - // filter only .parquet files - std::erase_if(keys, [](const std::string & s) { return std::filesystem::path(s).extension() != ".parquet"; }); + // make format lowercase + std::transform(format.begin(), format.end(), format.begin(), [](unsigned char c) { return std::tolower(c); }); - // for each partition path take only latest parquet file + // filter only files with specific format + std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); + + // for each partition path take only latest file std::unordered_map> latest_parquets; @@ -198,23 +202,30 @@ void registerStorageHudi(StorageFactory & factory) [](const StorageFactory::Arguments & args) { auto & engine_args = args.engine_args; - if (engine_args.empty()) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "External data source must have arguments"); - - auto configuration = StorageS3::getConfiguration(engine_args, args.getLocalContext()); - - configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); - configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + if (engine_args.empty() || engine_args.size() < 3) + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - S3::URI s3_uri(Poco::URI(configuration.url)); + String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); + String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + String format = "Parquet"; + if (engine_args.size() == 4) + { + format = checkAndGetLiteralArgument(engine_args[3], "format"); + } + + auto s3_uri = S3::URI(Poco::URI(table_url)); return std::make_shared( s3_uri, - configuration.auth_settings.access_key_id, - configuration.auth_settings.secret_access_key, + access_key_id, + secret_access_key, args.table_id, + format, args.columns, args.constraints, args.comment, diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index d782106d2b0..7b647345441 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -28,6 +28,7 @@ public: const String & access_key_, const String & secret_access_key_, const StorageID & table_id_, + const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, @@ -46,7 +47,7 @@ public: private: std::vector getKeysFromS3(); - static std::string generateQueryFromKeys(std::vector && keys); + static std::string generateQueryFromKeys(std::vector && keys, String format); StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; From 97dd75194a92913ef91bd2bc23d8203694ff171e Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 29 Sep 2022 20:36:54 +0000 Subject: [PATCH 035/526] Remove unused error code --- src/Storages/StorageHudi.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 78f3a0dd319..4aff4ff3a43 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -21,7 +21,6 @@ namespace DB namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int BAD_ARGUMENTS; extern const int S3_ERROR; } From 68abf43767d6f6e998373094147b7901b6222063 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 9 Oct 2022 00:00:14 +0000 Subject: [PATCH 036/526] Better INTERVAL parsing and execution --- src/Functions/FunctionBinaryArithmetic.h | 71 ++++++++++++ src/Functions/vectorFunctions.cpp | 105 ++++++++++++++++++ src/Parsers/ExpressionListParsers.cpp | 81 ++++++++------ .../02457_tuple_of_intervals.reference | 17 +++ .../0_stateless/02457_tuple_of_intervals.sql | 21 ++++ 5 files changed, 262 insertions(+), 33 deletions(-) create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.reference create mode 100644 tests/queries/0_stateless/02457_tuple_of_intervals.sql diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 174e98dd81f..399cffac85e 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -708,6 +708,41 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + + /// Exactly one argument must be Date or DateTime + if (first_is_date_or_datetime == second_is_date_or_datetime) + return {}; + + if (!isTuple(type0) && !isTuple(type1)) + return {}; + + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + /// We construct another function and call it. + if constexpr (!is_plus && !is_minus) + return {}; + + if (isTuple(type0) && second_is_date_or_datetime && is_minus) + throw Exception("Wrong order of arguments for function " + String(name) + ": argument of Tuple type cannot be first", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + std::string function_name; + if (is_plus) + { + function_name = "addTupleOfIntervals"; + } + else if (is_minus) + { + function_name = "subtractTupleOfIntervals"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -906,6 +941,20 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeDateTimeTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + /// Tuple argument must be second. + if (isTuple(arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1121,6 +1170,22 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + /// Tuple argument must be second. + if (isTuple(new_arguments[0].type)) + std::swap(new_arguments[0], new_arguments[1]); + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1553,6 +1618,12 @@ public: return executeDateTimeIntervalPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. + if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 20571f67eff..4e9f2a71f8c 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -415,6 +415,108 @@ public: } }; +template +class FunctionDateOrDateTimeOperationTupleOfIntervals : public ITupleFunction +{ +public: + static constexpr auto name = Impl::name; + + explicit FunctionDateOrDateTimeOperationTupleOfIntervals(ContextPtr context_) : ITupleFunction(context_) {} + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), getName()}; + + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + + if (!cur_tuple) + throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}. Should be a tuple", + arguments[0].type->getName(), getName()}; + + const auto & cur_types = cur_tuple->getElements(); + + Columns cur_elements; + if (arguments[1].column) + cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_types.size(); + if (tuple_size == 0) + return arguments[0].type; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + DataTypePtr res_type = arguments[0].type; + for (size_t i = 0; i < tuple_size; ++i) + { + try + { + ColumnWithTypeAndName left{res_type, {}}; + ColumnWithTypeAndName right{cur_elements.empty() ? nullptr : cur_elements[i], cur_types[i], {}}; + auto plus_elem = plus->build({left, right}); + res_type = plus_elem->getResultType(); + } + catch (DB::Exception & e) + { + e.addMessage("While executing function {} for tuple element {}", getName(), i); + throw; + } + } + + return res_type; + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const auto * cur_tuple = checkAndGetDataType(arguments[1].type.get()); + const auto & cur_types = cur_tuple->getElements(); + auto cur_elements = getTupleElements(*arguments[1].column); + + size_t tuple_size = cur_elements.size(); + if (tuple_size == 0) + return arguments[0].column; + + auto plus = FunctionFactory::instance().get(Impl::func_name, context); + ColumnWithTypeAndName res; + for (size_t i = 0; i < tuple_size; ++i) + { + ColumnWithTypeAndName column{cur_elements[i], cur_types[i], {}}; + auto elem_plus = plus->build(ColumnsWithTypeAndName{i == 0 ? arguments[0] : res, column}); + auto res_type = elem_plus->getResultType(); + res.column = elem_plus->execute({i == 0 ? arguments[0] : res, column}, res_type, input_rows_count); + res.type = res_type; + } + + return res.column; + } +}; + +struct AddTupleOfIntervalsImpl +{ + static constexpr auto name = "addTupleOfIntervals"; + static constexpr auto func_name = "plus"; +}; + +struct SubtractTupleOfIntervalsImpl +{ + static constexpr auto name = "subtractTupleOfIntervals"; + static constexpr auto func_name = "minus"; +}; + +using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + +using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1282,6 +1384,9 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index f7a016a59e4..353f22b03b6 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -1684,6 +1684,16 @@ private: class IntervalLayer : public Layer { public: + bool getResult(ASTPtr & node) override + { + if (elements.size() == 1) + node = elements[0]; + else + node = makeASTFunction("tuple", std::move(elements)); + + return true; + } + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// INTERVAL 1 HOUR or INTERVAL expr HOUR @@ -1693,49 +1703,54 @@ public: if (state == 0) { + state = 1; + auto begin = pos; auto init_expected = expected; ASTPtr string_literal; + String literal; + //// A String literal followed INTERVAL keyword, /// the literal can be a part of an expression or /// include Number and INTERVAL TYPE at the same time - if (ParserStringLiteral{}.parse(pos, string_literal, expected)) + if (ParserStringLiteral{}.parse(pos, string_literal, expected) + && string_literal->as().value.tryGet(literal)) { - String literal; - if (string_literal->as().value.tryGet(literal)) + Tokens tokens(literal.data(), literal.data() + literal.size()); + IParser::Pos token_pos(tokens, 0); + Expected token_expected; + ASTPtr expr; + + if (!ParserNumber{}.parse(token_pos, expr, token_expected)) + return false; + + /// case: INTERVAL '1' HOUR + /// back to begin + if (!token_pos.isValid()) { - Tokens tokens(literal.data(), literal.data() + literal.size()); - IParser::Pos token_pos(tokens, 0); - Expected token_expected; - ASTPtr expr; - - if (!ParserNumber{}.parse(token_pos, expr, token_expected)) - { - return false; - } - else - { - /// case: INTERVAL '1' HOUR - /// back to begin - if (!token_pos.isValid()) - { - pos = begin; - expected = init_expected; - } - else - { - /// case: INTERVAL '1 HOUR' - if (!parseIntervalKind(token_pos, token_expected, interval_kind)) - return false; - - elements = {makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)}; - finished = true; - return true; - } - } + pos = begin; + expected = init_expected; + return true; } + + /// case: INTERVAL '1 HOUR' + if (!parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + + /// case: INTERVAL '1 HOUR 1 SECOND ...' + while (token_pos.isValid()) + { + if (!ParserNumber{}.parse(token_pos, expr, token_expected) || + !parseIntervalKind(token_pos, token_expected, interval_kind)) + return false; + + pushResult(makeASTFunction(interval_kind.toNameOfFunctionToIntervalDataType(), expr)); + } + + finished = true; } - state = 1; return true; } diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference new file mode 100644 index 00000000000..40bbfb35d91 --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -0,0 +1,17 @@ +SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) +- +2022-11-12 +2022-11-12 +2022-11-12 +- +2023-07-11 00:01:59 +2023-07-11 00:01:59 +2023-07-11 00:01:59 +- +2021-07-31 23:00:00 +2021-07-31 23:00:00 +2021-07-31 23:00:00 +- +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 +2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql new file mode 100644 index 00000000000..2c2feaf522a --- /dev/null +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -0,0 +1,21 @@ +EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; +SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); +SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; + +SELECT '-'; +SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; +SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); +SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; +SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); +SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; + +SELECT '-'; +SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; +SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); +SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; From f823b9a177e77ee1a6c25f6779c13f9e031c662b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 10 Oct 2022 13:31:10 +0800 Subject: [PATCH 037/526] add not like to atom map --- src/Storages/MergeTree/KeyCondition.cpp | 21 +++++++++++++++++++ ...58_key_condition_not_like_prefix.reference | 1 + .../002458_key_condition_not_like_prefix.sql | 5 +++++ 3 files changed, 27 insertions(+) create mode 100644 tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference create mode 100644 tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 24b3a4a60b9..ccdd26eb333 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -560,6 +560,27 @@ const KeyCondition::AtomMap KeyCondition::atom_map return true; } }, + { + "notLike", + [] (RPNElement & out, const Field & value) + { + if (value.getType() != Field::Types::String) + return false; + + String prefix = extractFixedPrefixFromLikePattern(value.get()); + if (prefix.empty()) + return false; + + String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix); + + out.function = RPNElement::FUNCTION_NOT_IN_RANGE; + out.range = !right_bound.empty() + ? Range(prefix, true, right_bound, false) + : Range::createLeftBounded(prefix, true); + + return true; + } + }, { "startsWith", [] (RPNElement & out, const Field & value) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference new file mode 100644 index 00000000000..87766d889a3 --- /dev/null +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference @@ -0,0 +1 @@ +200000 diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql new file mode 100644 index 00000000000..211fa5662e7 --- /dev/null +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql @@ -0,0 +1,5 @@ +CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; +INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); +INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); +SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; \ No newline at end of file From af1d306b12756e3c2f5d5de4bb7df0086c95ba77 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:16:12 +0000 Subject: [PATCH 038/526] Add Interval arithmetics --- src/DataTypes/IDataType.h | 1 + src/Functions/FunctionBinaryArithmetic.h | 100 +++++++- src/Functions/FunctionUnaryArithmetic.h | 10 +- src/Functions/vectorFunctions.cpp | 227 +++++++++++++++++- .../02457_tuple_of_intervals.reference | 31 ++- .../0_stateless/02457_tuple_of_intervals.sql | 53 ++-- 6 files changed, 380 insertions(+), 42 deletions(-) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index c93128ced95..45353796f3c 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -408,6 +408,7 @@ inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); } inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); } inline bool isMap(const DataTypePtr & data_type) {return WhichDataType(data_type).isMap(); } +inline bool isInterval(const DataTypePtr & data_type) {return WhichDataType(data_type).isInterval(); } inline bool isNothing(const DataTypePtr & data_type) { return WhichDataType(data_type).isNothing(); } inline bool isUUID(const DataTypePtr & data_type) { return WhichDataType(data_type).isUUID(); } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 399cffac85e..e4919d3e9d7 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -633,7 +634,8 @@ class FunctionBinaryArithmetic : public IFunction DataTypeInt8, DataTypeInt16, DataTypeInt32, DataTypeInt64, DataTypeInt128, DataTypeInt256, DataTypeDecimal32, DataTypeDecimal64, DataTypeDecimal128, DataTypeDecimal256, DataTypeDate, DataTypeDateTime, - DataTypeFixedString, DataTypeString>; + DataTypeFixedString, DataTypeString, + DataTypeInterval>; using Floats = TypeList; @@ -709,10 +711,10 @@ class FunctionBinaryArithmetic : public IFunction } static FunctionOverloadResolverPtr - getFunctionForTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + getFunctionForDateTupleOfIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { - bool first_is_date_or_datetime = isDate(type0) || isDateTime(type0) || isDateTime64(type0); - bool second_is_date_or_datetime = isDate(type1) || isDateTime(type1) || isDateTime64(type1); + bool first_is_date_or_datetime = isDateOrDate32(type0) || isDateTime(type0) || isDateTime64(type0); + bool second_is_date_or_datetime = isDateOrDate32(type1) || isDateTime(type1) || isDateTime64(type1); /// Exactly one argument must be Date or DateTime if (first_is_date_or_datetime == second_is_date_or_datetime) @@ -735,7 +737,7 @@ class FunctionBinaryArithmetic : public IFunction { function_name = "addTupleOfIntervals"; } - else if (is_minus) + else { function_name = "subtractTupleOfIntervals"; } @@ -743,6 +745,47 @@ class FunctionBinaryArithmetic : public IFunction return FunctionFactory::instance().get(function_name, context); } + static FunctionOverloadResolverPtr + getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) + { + /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals + /// and the second argument is the Inteval of a different kind. + /// We construct another function (example: addIntervals) and call it + + if constexpr (!is_plus && !is_minus) + return {}; + + const auto * tuple_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_0 = checkAndGetDataType(type0.get()); + const auto * interval_data_type_1 = checkAndGetDataType(type1.get()); + + if ((!tuple_data_type_0 && !interval_data_type_0) || !interval_data_type_1) + return {}; + + if (interval_data_type_0 && interval_data_type_0->equals(*interval_data_type_1)) + return {}; + + if (tuple_data_type_0) + { + auto & tuple_types = tuple_data_type_0->getElements(); + for (auto & type : tuple_types) + if (!isInterval(type)) + return {}; + } + + std::string function_name; + if (is_plus) + { + function_name = "addInterval"; + } + else + { + function_name = "subtractInterval"; + } + + return FunctionFactory::instance().get(function_name, context); + } + static FunctionOverloadResolverPtr getFunctionForTupleArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { @@ -955,6 +998,16 @@ class FunctionBinaryArithmetic : public IFunction return function->execute(new_arguments, result_type, input_rows_count); } + ColumnPtr executeIntervalTupleOfIntervalsPlusMinus(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, + size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const + { + ColumnsWithTypeAndName new_arguments = arguments; + + auto function = function_builder->build(new_arguments); + + return function->execute(new_arguments, result_type, input_rows_count); + } + ColumnPtr executeTupleNumberOperator(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count, const FunctionOverloadResolverPtr & function_builder) const { @@ -1171,7 +1224,7 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0], arguments[1], context)) { ColumnsWithTypeAndName new_arguments(2); @@ -1186,6 +1239,18 @@ public: return function->getResultType(); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0], arguments[1], context)) + { + ColumnsWithTypeAndName new_arguments(2); + + for (size_t i = 0; i < 2; ++i) + new_arguments[i].type = arguments[i]; + + auto function = function_builder->build(new_arguments); + return function->getResultType(); + } + /// Special case when the function is multiply or divide, one of arguments is Tuple and another is Number. if (auto function_builder = getFunctionForTupleAndNumberArithmetic(arguments[0], arguments[1], context)) { @@ -1237,6 +1302,21 @@ public: type_res = std::make_shared(); return true; } + else if constexpr (std::is_same_v || std::is_same_v) + { + if constexpr (std::is_same_v && + std::is_same_v) + { + if constexpr (is_plus || is_minus) + { + if (left.getKind() == right.getKind()) + { + type_res = std::make_shared(left.getKind()); + return true; + } + } + } + } else { using ResultDataType = typename BinaryOperationTraits::ResultDataType; @@ -1619,11 +1699,17 @@ public: } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Tuple. - if (auto function_builder = getFunctionForTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + if (auto function_builder = getFunctionForDateTupleOfIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) { return executeDateTimeTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); } + /// Special case when the function is plus or minus, one of arguments is Interval/Tuple of Intervals and another is Interval. + if (auto function_builder = getFunctionForMergeIntervalsArithmetic(arguments[0].type, arguments[1].type, context)) + { + return executeIntervalTupleOfIntervalsPlusMinus(arguments, result_type, input_rows_count, function_builder); + } + /// Special case when the function is plus, minus or multiply, both arguments are tuples. if (auto function_builder = getFunctionForTupleArithmetic(arguments[0].type, arguments[1].type, context)) { diff --git a/src/Functions/FunctionUnaryArithmetic.h b/src/Functions/FunctionUnaryArithmetic.h index 445eb45fd9d..f5ddc5cb67c 100644 --- a/src/Functions/FunctionUnaryArithmetic.h +++ b/src/Functions/FunctionUnaryArithmetic.h @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -145,7 +146,8 @@ class FunctionUnaryArithmetic : public IFunction DataTypeDecimal, DataTypeDecimal, DataTypeDecimal, - DataTypeFixedString + DataTypeFixedString, + DataTypeInterval >(type, std::forward(f)); } @@ -211,6 +213,12 @@ public: return false; result = std::make_shared(type.getN()); } + else if constexpr (std::is_same_v) + { + if constexpr (!IsUnaryOperation::negate) + return false; + result = std::make_shared(type.getKind()); + } else { using T0 = typename DataType::FieldType; diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 4e9f2a71f8c..007875a8b81 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -517,6 +518,172 @@ using FunctionAddTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfInterv using FunctionSubtractTupleOfIntervals = FunctionDateOrDateTimeOperationTupleOfIntervals; +template +struct FunctionTupleOperationInterval : public ITupleFunction +{ +public: + static constexpr auto name = is_minus ? "subtractInterval" : "addInterval"; + + explicit FunctionTupleOperationInterval(ContextPtr context_) : ITupleFunction(context_) {} + + static FunctionPtr create(ContextPtr context_) + { + return std::make_shared(context_); + } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (!isTuple(arguments[0]) && !isInterval(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0]->getName(), getName()); + + if (!isInterval(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0]->getName(), getName()); + + DataTypes types; + + const auto * tuple = checkAndGetDataType(arguments[0].get()); + + if (tuple) + { + const auto & cur_types = tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + types.back()->getName(), getName()); + + types = cur_types; + } + else + { + types = {arguments[0]}; + } + + const auto * interval_last = checkAndGetDataType(types.back().get()); + const auto * interval_new = checkAndGetDataType(arguments[1].get()); + + if (!interval_last->equals(*interval_new)) + types.push_back(arguments[1]); + + return std::make_shared(types); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + if (!isInterval(arguments[1].type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of second argument of function {}, must be Interval", + arguments[0].type->getName(), getName()); + + Columns tuple_columns; + + const auto * first_tuple = checkAndGetDataType(arguments[0].type.get()); + const auto * first_interval = checkAndGetDataType(arguments[0].type.get()); + const auto * second_interval = checkAndGetDataType(arguments[1].type.get()); + + bool can_be_merged; + + if (first_interval) + { + can_be_merged = first_interval->equals(*second_interval); + + if (can_be_merged) + tuple_columns.resize(1); + else + tuple_columns.resize(2); + + tuple_columns[0] = arguments[0].column->convertToFullColumnIfConst(); + } + else if (first_tuple) + { + const auto & cur_types = first_tuple->getElements(); + + for (auto & type : cur_types) + if (!isInterval(type)) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of Tuple element of first argument of function {}, must be Interval", + type->getName(), getName()); + + auto cur_elements = getTupleElements(*arguments[0].column); + size_t tuple_size = cur_elements.size(); + + if (tuple_size == 0) + { + can_be_merged = false; + } + else + { + const auto * tuple_last_interval = checkAndGetDataType(cur_types.back().get()); + can_be_merged = tuple_last_interval->equals(*second_interval); + } + + if (can_be_merged) + tuple_columns.resize(tuple_size); + else + tuple_columns.resize(tuple_size + 1); + + for (size_t i = 0; i < tuple_size; ++i) + tuple_columns[i] = cur_elements[i]; + } + else + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of first argument of function {}, must be Tuple or Interval", + arguments[0].type->getName(), getName()); + + + ColumnPtr & last_column = tuple_columns.back(); + + if (can_be_merged) + { + ColumnWithTypeAndName left{last_column, arguments[1].type, {}}; + + if constexpr (is_minus) + { + auto minus = FunctionFactory::instance().get("minus", context); + auto elem_minus = minus->build({left, arguments[1]}); + last_column = elem_minus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + else + { + auto plus = FunctionFactory::instance().get("plus", context); + auto elem_plus = plus->build({left, arguments[1]}); + last_column = elem_plus->execute({left, arguments[1]}, arguments[1].type, input_rows_count) + ->convertToFullColumnIfConst(); + } + } + else + { + if constexpr (is_minus) + { + auto negate = FunctionFactory::instance().get("negate", context); + auto elem_negate = negate->build({arguments[1]}); + last_column = elem_negate->execute({arguments[1]}, arguments[1].type, input_rows_count); + } + else + { + last_column = arguments[1].column; + } + } + + return ColumnTuple::create(tuple_columns); + } +}; + +using FunctionTupleAddInterval = FunctionTupleOperationInterval; + +using FunctionTupleSubtractInterval = FunctionTupleOperationInterval; + + /// this is for convenient usage in LNormalize template class FunctionLNorm : public ITupleFunction {}; @@ -1384,8 +1551,64 @@ REGISTER_FUNCTION(VectorFunctions) factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction( + { + R"( +Consecutively adds a tuple of intervals to a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT addTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Consecutively subtracts a tuple of intervals from a Date or a DateTime. +[example:tuple] +)", + Documentation::Examples{ + {"tuple", "WITH toDate('2018-01-01') AS date SELECT subtractTupleOfIntervals(date, (INTERVAL 1 DAY, INTERVAL 1 YEAR))"}, + }, + Documentation::Categories{"Tuple", "Interval", "Date", "DateTime"} + }); + + factory.registerFunction( + { + R"( +Adds an interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT addInterval(INTERVAL 1 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); + factory.registerFunction( + { + R"( +Adds an negated interval to another interval or tuple of intervals. The returned value is tuple of intervals. +[example:tuple] +[example:interval1] + +If the types of the first interval (or the interval in the tuple) and the second interval are the same they will be merged into one interval. +[example:interval2] +)", + Documentation::Examples{ + {"tuple", "SELECT subtractInterval((INTERVAL 1 DAY, INTERVAL 1 YEAR), INTERVAL 1 MONTH)"}, + {"interval1", "SELECT subtractInterval(INTERVAL 1 DAY, INTERVAL 1 MONTH)"}, + {"interval2", "SELECT subtractInterval(INTERVAL 2 DAY, INTERVAL 1 DAY)"}, + }, + Documentation::Categories{"Tuple", "Interval"} + }); factory.registerFunction(); factory.registerFunction(); diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index 40bbfb35d91..dd190dce891 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,17 +1,16 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) -- -2022-11-12 -2022-11-12 -2022-11-12 -- -2023-07-11 00:01:59 -2023-07-11 00:01:59 -2023-07-11 00:01:59 -- -2021-07-31 23:00:00 -2021-07-31 23:00:00 -2021-07-31 23:00:00 -- -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 -2021-06-10 23:59:59.000 +--- +3 IntervalSecond +(1,2) Tuple(IntervalHour, IntervalSecond) +(1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(2,1) Tuple(IntervalSecond, IntervalHour) +--- +-3 IntervalSecond +(-1,-2) Tuple(IntervalHour, IntervalSecond) +(-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) +(-2,-1) Tuple(IntervalSecond, IntervalHour) +--- +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 2c2feaf522a..d4065ab98f8 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -1,21 +1,42 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH; -SELECT '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH); -SELECT '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR; -SELECT '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR); -SELECT '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '-'; -SELECT '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR; -SELECT '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR); -SELECT '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR'; +SELECT '---'; -SELECT '-'; -SELECT '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND; -SELECT '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND); -SELECT '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND'; +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); + +SELECT '---'; + +WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, + '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, + '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, + '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + +WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, + '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, + '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, + '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; + + +WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, + '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, + '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, + '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file From cf6471c6b9965a82acb16b4a576e5f8a1ecf123d Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 02:48:04 +0000 Subject: [PATCH 039/526] Fix style --- src/Functions/FunctionBinaryArithmetic.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index e4919d3e9d7..c13cc67a1bc 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -749,7 +749,7 @@ class FunctionBinaryArithmetic : public IFunction getFunctionForMergeIntervalsArithmetic(const DataTypePtr & type0, const DataTypePtr & type1, ContextPtr context) { /// Special case when the function is plus or minus, first argument is Interval or Tuple of Intervals - /// and the second argument is the Inteval of a different kind. + /// and the second argument is the Interval of a different kind. /// We construct another function (example: addIntervals) and call it if constexpr (!is_plus && !is_minus) From 8ebe01d9cf1807fcfe782de6efb61142a81e6d5b Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 14 Oct 2022 11:33:56 +0800 Subject: [PATCH 040/526] system.detached_parts add column bytes_on_disk and path --- .../MergeTree/DataPartStorageOnDisk.cpp | 5 +++++ .../MergeTree/DataPartStorageOnDisk.h | 4 ++++ .../System/StorageSystemDetachedParts.cpp | 20 ++++++++++++------- 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e2a2f3f793f..e50cee4b654 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -121,6 +121,11 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & return res; } +UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) +{ + return calculateTotalSizeOnDiskImpl(disk, from); +} + UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const { return calculateTotalSizeOnDiskImpl(volume->getDisk(), fs::path(root_path) / part_dir); diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index adf1b78cdfb..0ad4e8f5239 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -112,6 +112,9 @@ public: void changeRootPath(const std::string & from_root, const std::string & to_root) override; DataPartStorageBuilderPtr getBuilder() const override; + + static UInt64 calculateTotalSizeOnDisk(const DiskPtr &disk, const String &from); + private: VolumePtr volume; std::string root_path; @@ -127,6 +130,7 @@ private: MergeTreeDataPartState state, Poco::Logger * log, bool is_projection) const; + }; class DataPartStorageBuilderOnDisk final : public IDataPartStorageBuilder diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 574ce4f44c2..9220a053e04 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -16,15 +17,17 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i { StorageInMemoryMetadata storage_metadata; storage_metadata.setColumns(ColumnsDescription{{ - {"database", std::make_shared()}, - {"table", std::make_shared()}, - {"partition_id", std::make_shared(std::make_shared())}, - {"name", std::make_shared()}, - {"disk", std::make_shared()}, - {"reason", std::make_shared(std::make_shared())}, + {"database", std::make_shared()}, + {"table", std::make_shared()}, + {"partition_id", std::make_shared(std::make_shared())}, + {"name", std::make_shared()}, + {"bytes_on_disk", std::make_shared()}, + {"disk", std::make_shared()}, + {"path", std::make_shared()}, + {"reason", std::make_shared(std::make_shared())}, {"min_block_number", std::make_shared(std::make_shared())}, {"max_block_number", std::make_shared(std::make_shared())}, - {"level", std::make_shared(std::make_shared())} + {"level", std::make_shared(std::make_shared())} }}); setInMemoryMetadata(storage_metadata); } @@ -50,11 +53,14 @@ Pipe StorageSystemDetachedParts::read( for (const auto & p : parts) { size_t i = 0; + String detached_part_path = fs::path(MergeTreeData::DETACHED_DIR_NAME) / p.dir_name; new_columns[i++]->insert(info.database); new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); + new_columns[i++]->insert(DataPartStorageOnDisk::calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); new_columns[i++]->insert(p.disk->getName()); + new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); new_columns[i++]->insert(p.valid_name ? p.min_block : Field()); new_columns[i++]->insert(p.valid_name ? p.max_block : Field()); From 555d2759202eaa1d08ba3c54dcc99507e188c069 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Fri, 14 Oct 2022 15:18:53 +0200 Subject: [PATCH 041/526] Update src/Storages/MergeTree/DataPartStorageOnDisk.h --- src/Storages/MergeTree/DataPartStorageOnDisk.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 0ad4e8f5239..79988e9baab 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -113,7 +113,7 @@ public: DataPartStorageBuilderPtr getBuilder() const override; - static UInt64 calculateTotalSizeOnDisk(const DiskPtr &disk, const String &from); + static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from); private: VolumePtr volume; From dc99e67d3f577b90e72cad5aa2623b1099bf16b4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 14 Oct 2022 17:45:17 +0300 Subject: [PATCH 042/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 7248728864e..0728d3e1127 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db > >(tail -100000 > server.log) 2>&1 & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db >> server.log 2>&1 & server_pid=$! kill -0 $server_pid From 10335aec136c541416979fee3b59c78a3221202c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Oct 2022 15:09:35 +0000 Subject: [PATCH 043/526] Improve using structure from insertion table in table functions --- src/Core/Settings.h | 2 +- src/Interpreters/Context.cpp | 73 +++++++++++++++++-- src/Interpreters/Context.h | 11 ++- src/Interpreters/JoinedTables.cpp | 11 +-- src/Interpreters/JoinedTables.h | 3 +- src/Interpreters/executeQuery.cpp | 3 +- src/TableFunctions/ITableFunction.h | 6 +- src/TableFunctions/ITableFunctionFileLike.cpp | 5 ++ src/TableFunctions/ITableFunctionFileLike.h | 2 + src/TableFunctions/TableFunctionS3.cpp | 5 ++ src/TableFunctions/TableFunctionS3.h | 2 + ...e_structure_from_insertion_table.reference | 9 +++ ...458_use_structure_from_insertion_table.sql | 39 ++++++++++ 13 files changed, 152 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference create mode 100644 tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5dedc6117aa..e87a2b99c91 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -609,7 +609,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) \ M(Bool, load_marks_asynchronously, false, "Load MergeTree marks asynchronously", 0) \ \ - M(Bool, use_structure_from_insertion_table_in_table_functions, false, "Use structure from insertion table instead of schema inference from data", 0) \ + M(UInt64, use_structure_from_insertion_table_in_table_functions, 2, "Use structure from insertion table instead of schema inference from data. Possible values: 0 - disabled, 1 - enabled, 2 - auto", 0) \ \ M(UInt64, http_max_tries, 10, "Max attempts to read via http.", 0) \ M(UInt64, http_retry_initial_backoff_ms, 100, "Min milliseconds for backoff, when retrying read via http", 0) \ diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 321a46baff6..793135477d0 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -75,6 +75,8 @@ #include #include #include +#include +#include #include #include #include @@ -1184,7 +1186,7 @@ void Context::addQueryFactoriesInfo(QueryLogFactories factory_type, const String } -StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) +StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint) { auto hash = table_expression->getTreeHash(); String key = toString(hash.first) + '_' + toString(hash.second); @@ -1194,15 +1196,61 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression) if (!res) { TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().get(table_expression, shared_from_this()); - if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint()) + if (getSettingsRef().use_structure_from_insertion_table_in_table_functions && table_function_ptr->needStructureHint() && hasInsertionTable()) { - const auto & insertion_table = getInsertionTable(); - if (!insertion_table.empty()) + const auto & structure_hint = DatabaseCatalog::instance().getTable(getInsertionTable(), shared_from_this())->getInMemoryMetadataPtr()->getColumns(); + bool use_columns_from_insert_query = true; + + /// use_structure_from_insertion_table_in_table_functions=2 means `auto` + if (select_query_hint && getSettingsRef().use_structure_from_insertion_table_in_table_functions == 2) { - const auto & structure_hint - = DatabaseCatalog::instance().getTable(insertion_table, shared_from_this())->getInMemoryMetadataPtr()->columns; - table_function_ptr->setStructureHint(structure_hint); + const auto * expression_list = select_query_hint->select()->as(); + Names columns_names; + bool have_asteriks = false; + /// First, check if we have only identifiers, asteriks and literals in select expression, + /// and if no, we cannot use the structure from insertion table. + for (const auto & expression : expression_list->children) + { + if (auto * identifier = expression->as()) + { + columns_names.push_back(identifier->name()); + } + else if (expression->as()) + { + have_asteriks = true; + } + else if (!expression->as()) + { + use_columns_from_insert_query = false; + break; + } + } + + /// Check that all identifiers are column names from insertion table. + for (const auto & column_name : columns_names) + { + if (!structure_hint.has(column_name)) + { + use_columns_from_insert_query = false; + break; + } + } + + /// If we don't have asteriks but only subset of columns, we should use + /// structure from insertion table only in case when table function + /// supports reading subset of columns from data. + if (use_columns_from_insert_query && !have_asteriks && !columns_names.empty()) + { + /// For input function we should check if input format supports reading subset of columns. + if (table_function_ptr->getName() == "input") + use_columns_from_insert_query = FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(getInsertFormat()); + else + use_columns_from_insert_query = table_function_ptr->supportsReadingSubsetOfColumns(); + } } + + if (use_columns_from_insert_query) + table_function_ptr->setStructureHint(structure_hint); } res = table_function_ptr->execute(table_expression, shared_from_this(), table_function_ptr->getName()); @@ -1432,12 +1480,21 @@ String Context::getDefaultFormat() const return default_format.empty() ? "TabSeparated" : default_format; } - void Context::setDefaultFormat(const String & name) { default_format = name; } +String Context::getInsertFormat() const +{ + return insert_format; +} + +void Context::setInsertFormat(const String & name) +{ + insert_format = name; +} + MultiVersion::Version Context::getMacros() const { return shared->macros.get(); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index a9984e32c1b..0cc4b17968f 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -232,6 +233,9 @@ private: String default_format; /// Format, used when server formats data by itself and if query does not have FORMAT specification. /// Thus, used in HTTP interface. If not specified - then some globally default format is used. + + String insert_format; /// Format, used in insert query. + TemporaryTablesMapping external_tables_mapping; Scalars scalars; /// Used to store constant values which are different on each instance during distributed plan, such as _shard_num. @@ -592,7 +596,9 @@ public: const QueryFactoriesInfo & getQueryFactoriesInfo() const { return query_factories_info; } void addQueryFactoriesInfo(QueryLogFactories factory_type, const String & created_object) const; - StoragePtr executeTableFunction(const ASTPtr & table_expression); + /// For table functions s3/file/url/hdfs/input we can use structure from + /// insertion table depending on select expression. + StoragePtr executeTableFunction(const ASTPtr & table_expression, const ASTSelectQuery * select_query_hint = nullptr); void addViewSource(const StoragePtr & storage); StoragePtr getViewSource() const; @@ -621,6 +627,9 @@ public: String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned. void setDefaultFormat(const String & name); + String getInsertFormat() const; + void setInsertFormat(const String & name); + MultiVersion::Version getMacros() const; void setMacros(std::unique_ptr && macros); diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index f4a98ada199..b88bb5d1caf 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -173,12 +173,13 @@ using RenameQualifiedIdentifiersVisitor = InDepthNodeVisitorgetQueryContext()->executeTableFunction(left_table_expression); + return context->getQueryContext()->executeTableFunction(left_table_expression, &select_query); StorageID table_id = StorageID::createEmpty(); if (left_db_and_table) diff --git a/src/Interpreters/JoinedTables.h b/src/Interpreters/JoinedTables.h index 9d01c081e9f..7562dbc9ac5 100644 --- a/src/Interpreters/JoinedTables.h +++ b/src/Interpreters/JoinedTables.h @@ -22,7 +22,7 @@ using StorageMetadataPtr = std::shared_ptr; class JoinedTables { public: - JoinedTables(ContextPtr context, const ASTSelectQuery & select_query, bool include_all_columns_ = false); + JoinedTables(ContextPtr context, const ASTSelectQuery & select_query_, bool include_all_columns_ = false); void reset(const ASTSelectQuery & select_query); @@ -52,6 +52,7 @@ private: /// Legacy (duplicated left table values) ASTPtr left_table_expression; std::optional left_db_and_table; + const ASTSelectQuery & select_query; }; } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 7a633242904..253de6ea3ac 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -450,6 +450,7 @@ static std::tuple executeQueryImpl( if (insert_query) { + context->setInsertFormat(insert_query->format); if (insert_query->data) query_end = insert_query->data; else @@ -579,7 +580,7 @@ static std::tuple executeQueryImpl( insert_query->tryFindInputFunction(input_function); if (input_function) { - StoragePtr storage = context->executeTableFunction(input_function); + StoragePtr storage = context->executeTableFunction(input_function, insert_query->select->as()); auto & input_storage = dynamic_cast(*storage); auto input_metadata_snapshot = input_storage.getInMemoryMetadataPtr(); auto pipe = getSourceFromASTInsertQuery( diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4b9a87b93f1..d586946da04 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -54,15 +54,17 @@ public: virtual ColumnsDescription getActualTableStructure(ContextPtr /*context*/) const = 0; /// Check if table function needs a structure hint from SELECT query in case of - /// INSERT INTO FUNCTION ... SELECT ... + /// INSERT INTO FUNCTION ... SELECT ... and INSERT INTO ... SELECT ... FROM table_function(...) /// It's used for schema inference. virtual bool needStructureHint() const { return false; } /// Set a structure hint from SELECT query in case of - /// INSERT INTO FUNCTION ... SELECT ... + /// INSERT INTO FUNCTION ... SELECT ... and INSERT INTO ... SELECT ... FROM table_function(...) /// This hint could be used not to repeat schema in function arguments. virtual void setStructureHint(const ColumnsDescription &) {} + virtual bool supportsReadingSubsetOfColumns() { return true; } + /// Create storage according to the query. StoragePtr execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}, bool use_global_context = false) const; diff --git a/src/TableFunctions/ITableFunctionFileLike.cpp b/src/TableFunctions/ITableFunctionFileLike.cpp index 8be2341b81d..d62e44a16cc 100644 --- a/src/TableFunctions/ITableFunctionFileLike.cpp +++ b/src/TableFunctions/ITableFunctionFileLike.cpp @@ -34,6 +34,11 @@ String ITableFunctionFileLike::getFormatFromFirstArgument() return FormatFactory::instance().getFormatFromFileName(filename, true); } +bool ITableFunctionFileLike::supportsReadingSubsetOfColumns() +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format); +} + void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context) { /// Parse args diff --git a/src/TableFunctions/ITableFunctionFileLike.h b/src/TableFunctions/ITableFunctionFileLike.h index c2f32eb0aa3..589fce67638 100644 --- a/src/TableFunctions/ITableFunctionFileLike.h +++ b/src/TableFunctions/ITableFunctionFileLike.h @@ -18,6 +18,8 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + bool supportsReadingSubsetOfColumns() override; + protected: void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 0bf33007760..44b6f087d6b 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -146,6 +146,11 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) return parseColumnsListFromString(configuration.structure, context); } +bool TableFunctionS3::supportsReadingSubsetOfColumns() +{ + return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(configuration.format); +} + StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { Poco::URI uri (configuration.url); diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index be84bc4d8ab..05d3ebea316 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -30,6 +30,8 @@ public: void setStructureHint(const ColumnsDescription & structure_hint_) override { structure_hint = structure_hint_; } + bool supportsReadingSubsetOfColumns() override; + protected: friend class TableFunctionS3Cluster; diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference new file mode 100644 index 00000000000..0ca28640270 --- /dev/null +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.reference @@ -0,0 +1,9 @@ +\N 1 +1 2 +\N 42 +\N 42 +\N 42 +\N 42 +\N +\N +\N diff --git a/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql new file mode 100644 index 00000000000..a609dc361fe --- /dev/null +++ b/tests/queries/0_stateless/02458_use_structure_from_insertion_table.sql @@ -0,0 +1,39 @@ +-- Tags: no-parallel, no-fasttest + +insert into function file(02458_data.jsonl) select NULL as x, 42 as y settings engine_file_truncate_on_insert=1; +insert into function file(02458_data.jsoncompacteachrow) select NULL as x, 42 as y settings engine_file_truncate_on_insert=1; +drop table if exists test; +create table test (x Nullable(UInt32), y UInt32) engine=Memory(); + +set use_structure_from_insertion_table_in_table_functions=2; +insert into test select * from file(02458_data.jsonl); +insert into test select x, 1 from file(02458_data.jsonl); +insert into test select x, y from file(02458_data.jsonl); +insert into test select x + 1, y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, z from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select * from file(02458_data.jsoncompacteachrow); +insert into test select x, 1 from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x + 1, y from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select x, z from file(02458_data.jsoncompacteachrow); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select * from input() format CSV 1,2 +insert into test select x, y from input() format CSV 1,2 -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select x, y from input() format JSONEachRow {"x" : null, "y" : 42} + +select * from test order by y; + +drop table test; +create table test (x Nullable(UInt32)) engine=Memory(); +insert into test select * from file(02458_data.jsonl); +insert into test select x from file(02458_data.jsonl); +insert into test select y from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} +insert into test select y as x from file(02458_data.jsonl); -- {serverError ONLY_NULLS_WHILE_READING_SCHEMA} + +insert into test select c1 from input() format CSV 1,2; -- {serverError CANNOT_EXTRACT_TABLE_STRUCTURE} +insert into test select x from input() format JSONEachRow {"x" : null, "y" : 42} + +select * from test order by x; + +drop table test; From c11e4bfbbfbc36999814e5983e19e41b1eecd2ea Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Fri, 14 Oct 2022 15:13:26 +0000 Subject: [PATCH 044/526] Fix build --- src/Functions/vectorFunctions.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 007875a8b81..78f35342d60 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -555,7 +555,7 @@ public: { const auto & cur_types = tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", @@ -607,7 +607,7 @@ public: { const auto & cur_types = first_tuple->getElements(); - for (auto & type : cur_types) + for (const auto & type : cur_types) if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", From 2b150a575e263d7c91dbbc25f489452d901383c4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Oct 2022 15:15:03 +0000 Subject: [PATCH 045/526] Add docs --- docs/en/operations/settings/settings.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 31609fe24be..82fb825df45 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3403,6 +3403,17 @@ Use schema from cache for URL with last modification time validation (for urls w Default value: `true`. +## use_structure_from_insertion_table_in_table_functions {use_structure_from_insertion_table_in_table_functions} + +Use structure from insertion table instead of schema inference from data. + +Possible values: +- 0 - disabled +- 1 - enabled +- 2 - auto + +Default value: 2. + ## compatibility {#compatibility} This setting changes other settings according to provided ClickHouse version. From 65a0b9fcc60d0254e5f2d2b020af73c1b03b3b41 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Fri, 14 Oct 2022 23:32:13 +0800 Subject: [PATCH 046/526] fix test --- .../00502_custom_partitioning_local.reference | 2 +- .../00502_custom_partitioning_local.sql | 2 +- .../02117_show_create_table_system.reference | 133 +----------------- 3 files changed, 4 insertions(+), 133 deletions(-) diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index fff28819e74..be02d3eeefc 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 1 2 1 +default not_partitioned all all_1_2_1 324 default 1 2 1 default *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index c85a978af68..df4785af90d 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT `path`, disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ad27b86c6f5..ee2ccc8b7d1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -139,7 +139,9 @@ CREATE TABLE system.detached_parts `table` String, `partition_id` Nullable(String), `name` String, + `bytes_on_disk` UInt64, `disk` String, + `path` String, `reason` Nullable(String), `min_block_number` Nullable(Int64), `max_block_number` Nullable(Int64), @@ -974,134 +976,3 @@ CREATE TABLE system.settings_profiles ) ENGINE = SystemSettingsProfiles COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.stack_trace -( - `thread_name` String, - `thread_id` UInt64, - `query_id` String, - `trace` Array(UInt64) -) -ENGINE = SystemStackTrace -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.storage_policies -( - `policy_name` String, - `volume_name` String, - `volume_priority` UInt64, - `disks` Array(String), - `volume_type` String, - `max_data_part_size` UInt64, - `move_factor` Float32, - `prefer_not_to_merge` UInt8 -) -ENGINE = SystemStoragePolicies -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.table_engines -( - `name` String, - `supports_settings` UInt8, - `supports_skipping_indices` UInt8, - `supports_projections` UInt8, - `supports_sort_order` UInt8, - `supports_ttl` UInt8, - `supports_replication` UInt8, - `supports_deduplication` UInt8, - `supports_parallel_insert` UInt8 -) -ENGINE = SystemTableEngines -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.table_functions -( - `name` String, - `description` String -) -ENGINE = SystemTableFunctions -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.tables -( - `database` String, - `name` String, - `uuid` UUID, - `engine` String, - `is_temporary` UInt8, - `data_paths` Array(String), - `metadata_path` String, - `metadata_modification_time` DateTime, - `dependencies_database` Array(String), - `dependencies_table` Array(String), - `create_table_query` String, - `engine_full` String, - `as_select` String, - `partition_key` String, - `sorting_key` String, - `primary_key` String, - `sampling_key` String, - `storage_policy` String, - `total_rows` Nullable(UInt64), - `total_bytes` Nullable(UInt64), - `lifetime_rows` Nullable(UInt64), - `lifetime_bytes` Nullable(UInt64), - `comment` String, - `has_own_data` UInt8, - `loading_dependencies_database` Array(String), - `loading_dependencies_table` Array(String), - `loading_dependent_database` Array(String), - `loading_dependent_table` Array(String), - `table` String -) -ENGINE = SystemTables -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.time_zones -( - `time_zone` String -) -ENGINE = SystemTimeZones -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.user_directories -( - `name` String, - `type` String, - `params` String, - `precedence` UInt64 -) -ENGINE = SystemUserDirectories -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.users -( - `name` String, - `id` UUID, - `storage` String, - `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6), - `auth_params` String, - `host_ip` Array(String), - `host_names` Array(String), - `host_names_regexp` Array(String), - `host_names_like` Array(String), - `default_roles_all` UInt8, - `default_roles_list` Array(String), - `default_roles_except` Array(String), - `grantees_any` UInt8, - `grantees_list` Array(String), - `grantees_except` Array(String), - `default_database` String -) -ENGINE = SystemUsers -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.warnings -( - `message` String -) -ENGINE = SystemWarnings -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.zeros -( - `zero` UInt8 -) -ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' -CREATE TABLE system.zeros_mt -( - `zero` UInt8 -) -ENGINE = SystemZeros -COMMENT 'SYSTEM TABLE is built on the fly.' From 11a9b5977a30ec409ef2113ae0e1ac636b34a9f0 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 14 Oct 2022 19:22:05 +0200 Subject: [PATCH 047/526] Fix typos --- src/Interpreters/Context.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6ca6e987436..b5002b1df19 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1246,8 +1246,8 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const { const auto * expression_list = select_query_hint->select()->as(); Names columns_names; - bool have_asteriks = false; - /// First, check if we have only identifiers, asteriks and literals in select expression, + bool have_asterisk = false; + /// First, check if we have only identifiers, asterisk and literals in select expression, /// and if no, we cannot use the structure from insertion table. for (const auto & expression : expression_list->children) { @@ -1257,7 +1257,7 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } else if (expression->as()) { - have_asteriks = true; + have_asterisk = true; } else if (!expression->as()) { @@ -1276,10 +1276,10 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const } } - /// If we don't have asteriks but only subset of columns, we should use + /// If we don't have asterisk but only subset of columns, we should use /// structure from insertion table only in case when table function /// supports reading subset of columns from data. - if (use_columns_from_insert_query && !have_asteriks && !columns_names.empty()) + if (use_columns_from_insert_query && !have_asterisk && !columns_names.empty()) { /// For input function we should check if input format supports reading subset of columns. if (table_function_ptr->getName() == "input") From 66f6c0a68322f182c80fb86ad0ad9fc646a25ae5 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sat, 15 Oct 2022 22:43:55 +0800 Subject: [PATCH 048/526] fix test --- .../02117_show_create_table_system.reference | 131 ++++++++++++++++++ 1 file changed, 131 insertions(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ee2ccc8b7d1..4d705e4b0d1 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -976,3 +976,134 @@ CREATE TABLE system.settings_profiles ) ENGINE = SystemSettingsProfiles COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.stack_trace +( + `thread_name` String, + `thread_id` UInt64, + `query_id` String, + `trace` Array(UInt64) +) +ENGINE = SystemStackTrace +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.storage_policies +( + `policy_name` String, + `volume_name` String, + `volume_priority` UInt64, + `disks` Array(String), + `volume_type` String, + `max_data_part_size` UInt64, + `move_factor` Float32, + `prefer_not_to_merge` UInt8 +) +ENGINE = SystemStoragePolicies +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_engines +( + `name` String, + `supports_settings` UInt8, + `supports_skipping_indices` UInt8, + `supports_projections` UInt8, + `supports_sort_order` UInt8, + `supports_ttl` UInt8, + `supports_replication` UInt8, + `supports_deduplication` UInt8, + `supports_parallel_insert` UInt8 +) +ENGINE = SystemTableEngines +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.table_functions +( + `name` String, + `description` String +) +ENGINE = SystemTableFunctions +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.tables +( + `database` String, + `name` String, + `uuid` UUID, + `engine` String, + `is_temporary` UInt8, + `data_paths` Array(String), + `metadata_path` String, + `metadata_modification_time` DateTime, + `dependencies_database` Array(String), + `dependencies_table` Array(String), + `create_table_query` String, + `engine_full` String, + `as_select` String, + `partition_key` String, + `sorting_key` String, + `primary_key` String, + `sampling_key` String, + `storage_policy` String, + `total_rows` Nullable(UInt64), + `total_bytes` Nullable(UInt64), + `lifetime_rows` Nullable(UInt64), + `lifetime_bytes` Nullable(UInt64), + `comment` String, + `has_own_data` UInt8, + `loading_dependencies_database` Array(String), + `loading_dependencies_table` Array(String), + `loading_dependent_database` Array(String), + `loading_dependent_table` Array(String), + `table` String +) +ENGINE = SystemTables +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.time_zones +( + `time_zone` String +) +ENGINE = SystemTimeZones +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.user_directories +( + `name` String, + `type` String, + `params` String, + `precedence` UInt64 +) +ENGINE = SystemUserDirectories +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.users +( + `name` String, + `id` UUID, + `storage` String, + `auth_type` Enum8('no_password' = 0, 'plaintext_password' = 1, 'sha256_password' = 2, 'double_sha1_password' = 3, 'ldap' = 4, 'kerberos' = 5, 'ssl_certificate' = 6), + `auth_params` String, + `host_ip` Array(String), + `host_names` Array(String), + `host_names_regexp` Array(String), + `host_names_like` Array(String), + `default_roles_all` UInt8, + `default_roles_list` Array(String), + `default_roles_except` Array(String), + `grantees_any` UInt8, + `grantees_list` Array(String), + `grantees_except` Array(String), + `default_database` String +) +ENGINE = SystemUsers +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.warnings +( + `message` String +) +ENGINE = SystemWarnings +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros +( + `zero` UInt8 +) +ENGINE = SystemZeros +COMMENT 'SYSTEM TABLE is built on the fly.' +CREATE TABLE system.zeros_mt +( + `zero` UInt8 +) +ENGINE = SystemZeros +COMMENT 'SYSTEM TABLE is built on the fly.' From 2a356715da9a44f9273daab8fc3e6e71c9a52d08 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sun, 16 Oct 2022 16:46:00 +0800 Subject: [PATCH 049/526] fix test --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 13 ++++++++++++- tests/integration/test_partition/test.py | 2 +- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e50cee4b654..8ceebae333c 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -112,7 +112,18 @@ UInt32 DataPartStorageOnDisk::getRefCount(const String & file_name) const static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from) { if (disk->isFile(from)) - return disk->getFileSize(from); + { + try + { + return disk->getFileSize(from); + } + catch (...) + { + /// Files of detached part may be not exist, and then set file size is 0. + return 0; + } + } + std::vector files; disk->listFiles(from, files); UInt64 res = 0; diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index f3df66631a5..18e59becdd4 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select * from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except `path`, disk from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From 9ec206f8f74a6e6fb94a581dee30dc87b7bcd7d8 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sun, 16 Oct 2022 19:52:53 +0800 Subject: [PATCH 050/526] fix test --- .../MergeTree/DataPartStorageOnDisk.cpp | 25 +++++++++---------- .../00502_custom_partitioning_local.reference | 2 +- .../00502_custom_partitioning_local.sql | 2 +- 3 files changed, 14 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 8ceebae333c..600544b0b20 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -112,18 +112,7 @@ UInt32 DataPartStorageOnDisk::getRefCount(const String & file_name) const static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from) { if (disk->isFile(from)) - { - try - { - return disk->getFileSize(from); - } - catch (...) - { - /// Files of detached part may be not exist, and then set file size is 0. - return 0; - } - } - + return disk->getFileSize(from); std::vector files; disk->listFiles(from, files); UInt64 res = 0; @@ -134,7 +123,17 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { - return calculateTotalSizeOnDiskImpl(disk, from); + try + { + /// Files of detached part may be not exist, and then set file size is 0. + if (!disk->exists(from)) + return 0; + return calculateTotalSizeOnDiskImpl(disk, from); + } + catch (...) + { + return 0; + } } UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index be02d3eeefc..226a39f628e 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 324 default 1 2 1 default +default not_partitioned all all_1_2_1 324 1 2 1 *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index df4785af90d..87a6331402b 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT `path`, disk FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (`path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; From 9e62570d61a3ece8e32468a2479b0ac047051602 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 17 Oct 2022 22:57:11 +0800 Subject: [PATCH 051/526] fix test --- src/Storages/MergeTree/DataPartStorageOnDisk.cpp | 12 +----------- src/Storages/System/StorageSystemDetachedParts.cpp | 13 ++++++++++++- .../00502_custom_partitioning_local.reference | 2 +- .../0_stateless/00502_custom_partitioning_local.sql | 2 +- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index 600544b0b20..e50cee4b654 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -123,17 +123,7 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { - try - { - /// Files of detached part may be not exist, and then set file size is 0. - if (!disk->exists(from)) - return 0; - return calculateTotalSizeOnDiskImpl(disk, from); - } - catch (...) - { - return 0; - } + return calculateTotalSizeOnDiskImpl(disk, from); } UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 9220a053e04..1bc55c922c1 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -58,7 +58,18 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - new_columns[i++]->insert(DataPartStorageOnDisk::calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); + + size_t bytes_on_disk = 0; + try + { + /// Files of detached part may be not exist, and then set file size is 0. + bytes_on_disk = DataPartStorageOnDisk::calculateTotalSizeOnDisk( + p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path); + } + catch (...) + {} + + new_columns[i++]->insert(bytes_on_disk); new_columns[i++]->insert(p.disk->getName()); new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.reference b/tests/queries/0_stateless/00502_custom_partitioning_local.reference index 226a39f628e..fff28819e74 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.reference +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.reference @@ -9,7 +9,7 @@ Sum before DETACH PARTITION: Sum after DETACH PARTITION: 0 system.detached_parts after DETACH PARTITION: -default not_partitioned all all_1_2_1 324 1 2 1 +default not_partitioned all all_1_2_1 1 2 1 *** Partitioned by week *** Parts before OPTIMIZE: 1999-12-27 19991227_1_1_0 diff --git a/tests/queries/0_stateless/00502_custom_partitioning_local.sql b/tests/queries/0_stateless/00502_custom_partitioning_local.sql index 87a6331402b..3d5f71429fe 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_local.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_local.sql @@ -18,7 +18,7 @@ ALTER TABLE not_partitioned DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned; SELECT 'system.detached_parts after DETACH PARTITION:'; -SELECT system.detached_parts.* EXCEPT (`path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; +SELECT system.detached_parts.* EXCEPT (bytes_on_disk, `path`, disk) FROM system.detached_parts WHERE database = currentDatabase() AND table = 'not_partitioned'; DROP TABLE not_partitioned; From b26361c6476ec8ae4081da877b8d5b3f54c4f6e3 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 17 Oct 2022 23:02:39 +0800 Subject: [PATCH 052/526] fix test --- src/Storages/System/StorageSystemDetachedParts.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 1bc55c922c1..53526a02459 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -59,7 +59,7 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - size_t bytes_on_disk = 0; + UInt64 bytes_on_disk = 0; try { /// Files of detached part may be not exist, and then set file size is 0. From 10acf43db4d863e8991c5195dfc37bb069d78c56 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 18 Oct 2022 09:20:59 +0800 Subject: [PATCH 053/526] fix test --- tests/integration/test_partition/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index 18e59becdd4..f333011fa37 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select system.detached_parts.* except `path`, disk from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`, disk) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From 163b3c9f5263e810aba2998487af45839d5432e6 Mon Sep 17 00:00:00 2001 From: chen Date: Tue, 18 Oct 2022 14:53:21 +0800 Subject: [PATCH 054/526] fix test fix test --- tests/integration/test_partition/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_partition/test.py b/tests/integration/test_partition/test.py index f333011fa37..03c4ffbbd36 100644 --- a/tests/integration/test_partition/test.py +++ b/tests/integration/test_partition/test.py @@ -379,7 +379,7 @@ def test_system_detached_parts(drop_detached_parts_table): ) res = q( - "select system.detached_parts.* except (bytes_on_disk, `path`, disk) from system.detached_parts where table like 'sdp_%' order by table, name" + "select system.detached_parts.* except (bytes_on_disk, `path`) from system.detached_parts where table like 'sdp_%' order by table, name" ) assert ( res == "default\tsdp_0\tall\tall_1_1_0\tdefault\t\t1\t1\t0\n" From a50c0a7f85f7cef911a946d4ba9a54b61023ad97 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 18 Oct 2022 07:23:00 +0000 Subject: [PATCH 055/526] Better test --- .../02457_tuple_of_intervals.reference | 21 +++++++--- .../0_stateless/02457_tuple_of_intervals.sql | 42 ++++++++++++++----- 2 files changed, 48 insertions(+), 15 deletions(-) diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index dd190dce891..d5ffbc33dc0 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -1,16 +1,27 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toIntervalYear(1)) --- +-1 +2022-10-12 +2022-10-10 +(2) +(0) +2022-10-12 +2022-10-10 +2022-10-12 +(2) Tuple(IntervalSecond) +(0) Tuple(IntervalSecond) +--- 3 IntervalSecond (1,2) Tuple(IntervalHour, IntervalSecond) (1,1,1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (2,1) Tuple(IntervalSecond, IntervalHour) ---- -3 IntervalSecond (-1,-2) Tuple(IntervalHour, IntervalSecond) (-1,-1,-1) Tuple(IntervalSecond, IntervalHour, IntervalSecond) (-2,-1) Tuple(IntervalSecond, IntervalHour) --- -1 -1 -1 -1 +1 2022-03-01 +1 2022-02-28 +1 2023-07-11 00:01:59 +1 2021-07-31 23:00:00 +1 2021-06-10 23:59:59.000 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index d4065ab98f8..494914d4d4f 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -2,13 +2,29 @@ EXPLAIN SYNTAX SELECT INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR'; SELECT '---'; +SELECT negate(INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); +SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); +SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); +SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; +SELECT tuple(INTERVAL 1 DAY) - '2022-10-11'::Date; -- { serverError 43 } + +WITH tuple(INTERVAL 1 SECOND) + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH tuple(INTERVAL 1 SECOND) - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); +WITH INTERVAL 1 SECOND + tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } +WITH INTERVAL 1 SECOND - tuple(INTERVAL 1 SECOND) as expr SELECT expr, toTypeName(expr); -- { serverError 43 } + +SELECT '---'; + WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 HOUR + INTERVAL 1 SECOND + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 HOUR + INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH INTERVAL 1 SECOND + INTERVAL 1 SECOND + INTERVAL 1 HOUR as expr SELECT expr, toTypeName(expr); -SELECT '---'; - WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 HOUR - INTERVAL 1 SECOND - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); WITH - INTERVAL 1 SECOND - INTERVAL 1 HOUR - INTERVAL 1 SECOND as expr SELECT expr, toTypeName(expr); @@ -16,27 +32,33 @@ WITH - INTERVAL 1 SECOND - INTERVAL 1 SECOND - INTERVAL 1 HOUR as expr SELECT ex SELECT '---'; -WITH '2022-10-11'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, - '2022-10-11'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, - '2022-10-11'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, - '2022-10-11'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +WITH '2022-01-30'::Date + INTERVAL 1 MONTH + INTERVAL 1 DAY AS e1, + '2022-01-30'::Date + (INTERVAL 1 MONTH + INTERVAL 1 DAY) AS e2, + '2022-01-30'::Date + (INTERVAL 1 MONTH, INTERVAL 1 DAY) AS e3, + '2022-01-30'::Date + INTERVAL '1 MONTH 1 DAY' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; + +WITH '2022-01-30'::Date + INTERVAL 1 DAY + INTERVAL 1 MONTH AS e1, + '2022-01-30'::Date + (INTERVAL 1 DAY + INTERVAL 1 MONTH) AS e2, + '2022-01-30'::Date + (INTERVAL 1 DAY, INTERVAL 1 MONTH) AS e3, + '2022-01-30'::Date + INTERVAL '1 DAY 1 MONTH' AS e4 +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::Date + INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR AS e1, '2022-10-11'::Date + (INTERVAL -1 SECOND + INTERVAL 2 MINUTE + INTERVAL -3 MONTH + INTERVAL 1 YEAR) AS e2, '2022-10-11'::Date + (INTERVAL -1 SECOND, INTERVAL 2 MINUTE, INTERVAL -3 MONTH, INTERVAL 1 YEAR) AS e3, '2022-10-11'::Date + INTERVAL '-1 SECOND 2 MINUTE -3 MONTH 1 YEAR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime - INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR AS e1, '2022-10-11'::DateTime + (- INTERVAL 1 QUARTER - INTERVAL -3 WEEK - INTERVAL 1 YEAR - INTERVAL 1 HOUR) AS e2, '2022-10-11'::DateTime - (INTERVAL 1 QUARTER, INTERVAL -3 WEEK, INTERVAL 1 YEAR, INTERVAL 1 HOUR) AS e3, '2022-10-11'::DateTime - INTERVAL '1 QUARTER -3 WEEK 1 YEAR 1 HOUR' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; WITH '2022-10-11'::DateTime64 - INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND AS e1, '2022-10-11'::DateTime64 + (- INTERVAL 1 YEAR - INTERVAL 4 MONTH - INTERVAL 1 SECOND) AS e2, '2022-10-11'::DateTime64 - (INTERVAL 1 YEAR, INTERVAL 4 MONTH, INTERVAL 1 SECOND) AS e3, '2022-10-11'::DateTime64 - INTERVAL '1 YEAR 4 MONTH 1 SECOND' AS e4 -SELECT e1 == e2 AND e2 == e3 AND e3 == e4; \ No newline at end of file +SELECT e1 == e2 AND e2 == e3 AND e3 == e4, e1; \ No newline at end of file From 798e6281b4e7ceac3e5b5ca0de1c150c364be039 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:19:07 +0000 Subject: [PATCH 056/526] Fix test --- tests/queries/0_stateless/01825_type_json_schema_inference.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.sh b/tests/queries/0_stateless/01825_type_json_schema_inference.sh index 36991bd8069..f6917ab18ad 100755 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.sh +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.sh @@ -46,7 +46,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (obj JSON, map Map(String echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": 2}}' > $filename -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 1 ${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1" From 32ad28cbe507aadea9f16df70d01129b1ef15d0b Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Oct 2022 17:37:26 +0800 Subject: [PATCH 057/526] optimize code --- .../MergeTree/DataPartStorageOnDisk.cpp | 5 -- .../MergeTree/DataPartStorageOnDisk.h | 2 - .../System/StorageSystemDetachedParts.cpp | 47 ++++++++++++++----- 3 files changed, 35 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp index e50cee4b654..e2a2f3f793f 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.cpp +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.cpp @@ -121,11 +121,6 @@ static UInt64 calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & return res; } -UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) -{ - return calculateTotalSizeOnDiskImpl(disk, from); -} - UInt64 DataPartStorageOnDisk::calculateTotalSizeOnDisk() const { return calculateTotalSizeOnDiskImpl(volume->getDisk(), fs::path(root_path) / part_dir); diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 79988e9baab..91ed8bbfb43 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -113,8 +113,6 @@ public: DataPartStorageBuilderPtr getBuilder() const override; - static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from); - private: VolumePtr volume; std::string root_path; diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 53526a02459..980529032c4 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -31,6 +31,40 @@ StorageSystemDetachedParts::StorageSystemDetachedParts(const StorageID & table_i }}); setInMemoryMetadata(storage_metadata); } +static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & from, UInt64 & total_size) +{ + /// Files or directories of detached part may not exist. Only count the size of existing files. + if (disk->isFile(from)) + { + try + { + total_size += disk->getFileSize(from); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + return; + } + std::vector files; + try + { + disk->listFiles(from, files); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + for (const auto & file : files) + calculateTotalSizeOnDiskImpl(disk, fs::path(from) / file, total_size); +} + +static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) +{ + UInt64 total_size = 0; + calculateTotalSizeOnDiskImpl(disk, from, total_size); + return total_size; +} Pipe StorageSystemDetachedParts::read( const Names & /* column_names */, @@ -58,18 +92,7 @@ Pipe StorageSystemDetachedParts::read( new_columns[i++]->insert(info.table); new_columns[i++]->insert(p.valid_name ? p.partition_id : Field()); new_columns[i++]->insert(p.dir_name); - - UInt64 bytes_on_disk = 0; - try - { - /// Files of detached part may be not exist, and then set file size is 0. - bytes_on_disk = DataPartStorageOnDisk::calculateTotalSizeOnDisk( - p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path); - } - catch (...) - {} - - new_columns[i++]->insert(bytes_on_disk); + new_columns[i++]->insert(calculateTotalSizeOnDisk(p.disk, fs::path(info.data->getRelativeDataPath()) / detached_part_path)); new_columns[i++]->insert(p.disk->getName()); new_columns[i++]->insert((fs::path(info.data->getFullPathOnDisk(p.disk)) / detached_part_path).string()); new_columns[i++]->insert(p.valid_name ? p.prefix : Field()); From 330777a0ccf787eeff887a72892cf163ff0efb1c Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Wed, 19 Oct 2022 17:44:36 +0800 Subject: [PATCH 058/526] fix --- src/Storages/MergeTree/DataPartStorageOnDisk.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/MergeTree/DataPartStorageOnDisk.h b/src/Storages/MergeTree/DataPartStorageOnDisk.h index 91ed8bbfb43..adf1b78cdfb 100644 --- a/src/Storages/MergeTree/DataPartStorageOnDisk.h +++ b/src/Storages/MergeTree/DataPartStorageOnDisk.h @@ -112,7 +112,6 @@ public: void changeRootPath(const std::string & from_root, const std::string & to_root) override; DataPartStorageBuilderPtr getBuilder() const override; - private: VolumePtr volume; std::string root_path; @@ -128,7 +127,6 @@ private: MergeTreeDataPartState state, Poco::Logger * log, bool is_projection) const; - }; class DataPartStorageBuilderOnDisk final : public IDataPartStorageBuilder From 77ea58d5392a6e741240fbafd7515e8213bc6277 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 21 Oct 2022 17:58:20 +0300 Subject: [PATCH 059/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 0728d3e1127..5c9ab1af24e 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db >> server.log 2>&1 & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db 2>> server.log & server_pid=$! kill -0 $server_pid From 6242e93c81c22a1306c4dd35c263f65e22048fe9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 24 Oct 2022 02:11:08 +0000 Subject: [PATCH 060/526] Fixes & better tests --- src/Functions/vectorFunctions.cpp | 8 ++++---- .../0_stateless/02457_tuple_of_intervals.reference | 5 +++++ tests/queries/0_stateless/02457_tuple_of_intervals.sql | 7 +++++++ 3 files changed, 16 insertions(+), 4 deletions(-) diff --git a/src/Functions/vectorFunctions.cpp b/src/Functions/vectorFunctions.cpp index 78f35342d60..20835f59cc1 100644 --- a/src/Functions/vectorFunctions.cpp +++ b/src/Functions/vectorFunctions.cpp @@ -434,7 +434,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (!isDate(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + if (!isDateOrDate32(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception{ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of first argument of function {}. Should be a date or a date with time", arguments[0].type->getName(), getName()}; @@ -545,7 +545,7 @@ public: if (!isInterval(arguments[1])) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0]->getName(), getName()); + arguments[1]->getName(), getName()); DataTypes types; @@ -559,7 +559,7 @@ public: if (!isInterval(type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of Tuple element of first argument of function {}, must be Interval", - types.back()->getName(), getName()); + type->getName(), getName()); types = cur_types; } @@ -582,7 +582,7 @@ public: if (!isInterval(arguments[1].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of second argument of function {}, must be Interval", - arguments[0].type->getName(), getName()); + arguments[1].type->getName(), getName()); Columns tuple_columns; diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.reference b/tests/queries/0_stateless/02457_tuple_of_intervals.reference index d5ffbc33dc0..e635aec1163 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.reference +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.reference @@ -5,6 +5,11 @@ SELECT (toIntervalSecond(-1), toIntervalMinute(2), toIntervalMonth(-3), toInterv 2022-10-10 (2) (0) +2022-11-12 +2022-09-10 +(1,2) +(1,0) +--- 2022-10-12 2022-10-10 2022-10-12 diff --git a/tests/queries/0_stateless/02457_tuple_of_intervals.sql b/tests/queries/0_stateless/02457_tuple_of_intervals.sql index 494914d4d4f..be9ccb50d92 100644 --- a/tests/queries/0_stateless/02457_tuple_of_intervals.sql +++ b/tests/queries/0_stateless/02457_tuple_of_intervals.sql @@ -8,6 +8,13 @@ SELECT subtractTupleOfIntervals('2022-10-11'::Date, tuple(INTERVAL 1 DAY)); SELECT addInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); SELECT subtractInterval(tuple(INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT addTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT subtractTupleOfIntervals('2022-10-11'::Date, (INTERVAL 1 DAY, INTERVAL 1 MONTH)); +SELECT addInterval((INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); +SELECT subtractInterval(tuple(INTERVAL 1 DAY, INTERVAL 1 SECOND), INTERVAL 1 SECOND); + +SELECT '---'; + SELECT '2022-10-11'::Date + tuple(INTERVAL 1 DAY); SELECT '2022-10-11'::Date - tuple(INTERVAL 1 DAY); SELECT tuple(INTERVAL 1 DAY) + '2022-10-11'::Date; From 61563d4f19b2d358c94e4044f48267ab94160574 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 24 Oct 2022 11:52:20 +0800 Subject: [PATCH 061/526] better --- .../System/StorageSystemDetachedParts.cpp | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 980529032c4..11c4f7843ac 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -44,19 +44,22 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr { tryLogCurrentException(__PRETTY_FUNCTION__); } - return; } - std::vector files; - try + else { - disk->listFiles(from, files); + DirectoryIteratorPtr it; + try + { + it = disk->iterateDirectory(from); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + + for (; it->isValid(); it->next()) + calculateTotalSizeOnDiskImpl(disk, fs::path(from) / it->name(), total_size); } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - for (const auto & file : files) - calculateTotalSizeOnDiskImpl(disk, fs::path(from) / file, total_size); } static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) From 54013009d3649051eacf21debae7093ae4968621 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 24 Oct 2022 08:20:11 +0000 Subject: [PATCH 062/526] Code migration from #41734 to align diff and git blame. Initial implementation was different and it changed the entire ReplicatedMergeTreeSink::commitPart() which change history provided by git blame. Then RetriesControl.retryLoop() was introduced later which significantly reduces the diff since it's like while() used before. So, check outing the current version will keep more original history in git blame, which is useful here --- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Core/Settings.h | 5 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 48 +- .../MergeTree/EphemeralLockInZooKeeper.h | 12 +- .../PartMovesBetweenShardsOrchestrator.cpp | 1 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 216 ++++++--- .../MergeTree/ReplicatedMergeTreeSink.h | 16 +- src/Storages/MergeTree/ZooKeeperRetries.h | 250 ++++++++++ .../MergeTree/ZooKeeperWithFaultInjection.h | 429 ++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 84 ++-- src/Storages/StorageReplicatedMergeTree.h | 35 +- tests/clickhouse-test | 2 + .../__init__.py | 0 .../test_inserts_with_keeper_retries/test.py | 127 ++++++ .../00121_drop_column_zookeeper.sql | 6 +- ...partitioning_replicated_zookeeper_long.sql | 41 +- .../00652_replicated_mutations_zookeeper.sh | 24 +- ...replicated_without_partition_zookeeper.sql | 8 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 9 +- ...ated_minimalistic_part_header_zookeeper.sh | 1 + .../01037_zookeeper_check_table_empty_pk.sql | 9 +- ...eeper_system_mutations_with_parts_names.sh | 3 +- ...eeper_mutations_and_insert_quorum_long.sql | 11 +- .../01135_default_and_alter_zookeeper.sql | 4 +- ...mutation_stuck_after_replace_partition.sql | 2 + .../0_stateless/01158_zookeeper_log_long.sql | 6 +- ...artition_key_replicated_zookeeper_long.sql | 2 + ...1417_freeze_partition_verbose_zookeeper.sh | 8 +- ...replicated_detach_drop_and_quorum_long.sql | 1 + ...01451_replicated_detach_drop_part_long.sql | 9 +- .../01459_manual_write_to_replicas.sh | 10 +- .../01459_manual_write_to_replicas_quorum.sh | 6 +- ...check_many_parallel_quorum_inserts_long.sh | 4 +- ...arallel_quorum_insert_no_replicas_long.sql | 13 +- ..._execute_merges_on_single_replica_long.sql | 2 + ...primary_key_without_order_by_zookeeper.sql | 12 +- ...6_replicated_mutations_empty_partition.sql | 7 +- .../01593_concurrent_alter_mutations_kill.sh | 6 +- ..._part_and_deduplication_zookeeper_long.sql | 5 +- ...kground_checker_blather_zookeeper_long.sql | 8 +- .../01810_max_part_removal_threads_long.sh | 3 + .../0_stateless/01825_type_json_3.sql.j2 | 2 + .../01870_modulo_partition_key.sql | 9 +- .../02228_merge_tree_insert_memory_usage.sql | 5 +- .../02340_parts_refcnt_mergetree.sh | 4 +- .../02369_lost_part_intersecting_merges.sh | 14 +- .../02448_clone_replica_lost_part.sql | 2 + ...456_keeper_retries_during_insert.reference | 2 + .../02456_keeper_retries_during_insert.sql | 26 ++ 50 files changed, 1293 insertions(+), 220 deletions(-) create mode 100644 src/Storages/MergeTree/ZooKeeperRetries.h create mode 100644 src/Storages/MergeTree/ZooKeeperWithFaultInjection.h create mode 100644 tests/integration/test_inserts_with_keeper_retries/__init__.py create mode 100644 tests/integration/test_inserts_with_keeper_retries/test.py create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.reference create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.sql diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index aad5131fcb5..62615afb4f7 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -34,7 +34,7 @@ using TestKeeperRequestPtr = std::shared_ptr; class TestKeeper final : public IKeeper { public: - TestKeeper(const zkutil::ZooKeeperArgs & args_); + explicit TestKeeper(const zkutil::ZooKeeperArgs & args_); ~TestKeeper() override; bool isExpired() const override { return expired; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index f7d5bccc0e0..e2d086a720b 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -145,7 +145,7 @@ public: using Ptr = std::shared_ptr; - ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); + explicit ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); /** Config of the form: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0b8d24b1abc..87814a7aff8 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -656,6 +656,11 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ + M(UInt64, insert_keeper_max_retries, 20, "Max retries for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ + M(Float, insert_keeper_fault_injection_probability, 0.1f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ + M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 7abed125b7a..f2492e67437 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -12,17 +13,24 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_) - : zookeeper(&zookeeper_), path_prefix(path_prefix_), holder_path(holder_path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & holder_path_) + : zookeeper(zookeeper_), path_prefix(path_prefix_), holder_path(holder_path_) { /// Write the path to the secondary node in the main node. path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential); if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); + + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "Path created: path={} path_prefix={} holder_path={}", + path, + path_prefix, + holder_path); } std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path) { /// The /abandonable_lock- name is for backward compatibility. String holder_path_prefix = temp_path + "/abandonable_lock-"; @@ -31,7 +39,7 @@ std::optional createEphemeralLockInZooKeeper( /// Let's create an secondary ephemeral node. if (deduplication_path.empty()) { - holder_path = zookeeper_.create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential); + holder_path = zookeeper_->create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential); } else { @@ -41,11 +49,15 @@ std::optional createEphemeralLockInZooKeeper( ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); ops.emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; - Coordination::Error e = zookeeper_.tryMulti(ops, responses); + Coordination::Error e = zookeeper_->tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { - if (responses[0]->error == Coordination::Error::ZNODEEXISTS) + if (!responses.empty() && responses.front()->error == Coordination::Error::ZNODEEXISTS) { + LOG_DEBUG( + &Poco::Logger::get("createEphemeralLockInZooKeeper"), + "Deduplication path already exists: deduplication_path={}", + deduplication_path); return {}; } else @@ -85,9 +97,31 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { unlock(); } + catch (const zkutil::KeeperException & e) + { + if (Coordination::isHardwareError(e.code)) + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper communication error during unlock: code={} message='{}'", + e.code, + e.message()); + else if (e.code == Coordination::Error::ZNONODE) + /// To avoid additional round-trip for unlocking, + /// ephemeral node can be deleted explicitly as part of another multi op request to ZK + /// and marked as such via assumeUnlocked() if we got successful response. + /// But it's possible that the multi op request can be executed on server side, and client will not get response due to network issue. + /// In such case, assumeUnlocked() will not be called, so we'll get ZNONODE error here since the noded is already deleted + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper node was already deleted: code={} message={}", + e.code, + e.message()); + else + tryLogCurrentException("EphemeralLockInZooKeeper"); + } catch (...) { - tryLogCurrentException("~EphemeralLockInZooKeeper"); + tryLogCurrentException("EphemeralLockInZooKeeper"); } } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 276ffab5254..aac524ac65e 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -12,6 +12,8 @@ namespace DB { +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; namespace ErrorCodes { @@ -25,13 +27,13 @@ namespace ErrorCodes class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); protected: - EphemeralLockInZooKeeper() = delete; - EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_); + EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & holder_path_); public: + EphemeralLockInZooKeeper() = delete; EphemeralLockInZooKeeper(EphemeralLockInZooKeeper && rhs) noexcept { *this = std::move(rhs); @@ -86,14 +88,14 @@ public: ~EphemeralLockInZooKeeper(); private: - zkutil::ZooKeeper * zookeeper = nullptr; + ZooKeeperWithFaultInjectionPtr zookeeper; String path_prefix; String path; String holder_path; }; std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); /// Acquires block number locks in all partitions. diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index d5f35ea1b3c..52e11ab97a5 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 158cbfca9fd..2ff1a3e093b 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -8,7 +8,6 @@ #include #include - namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; @@ -82,7 +81,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; /// Allow to verify that the session in ZooKeeper is still alive. -static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) +static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) { if (!zookeeper) throw Exception("No ZooKeeper session.", ErrorCodes::NO_ZOOKEEPER); @@ -91,7 +90,7 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -101,6 +100,7 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); Strings exists_paths; + exists_paths.reserve(replicas.size()); for (const auto & replica : replicas) if (replica != storage.replica_name) exists_paths.emplace_back(fs::path(storage.zookeeper_path) / "replicas" / replica / "is_active"); @@ -108,20 +108,28 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z auto exists_result = zookeeper->exists(exists_paths); auto get_results = zookeeper->get(Strings{storage.replica_path + "/is_active", storage.replica_path + "/host"}); + Coordination::Error keeper_error = Coordination::Error::ZOK; size_t active_replicas = 1; /// Assume current replica is active (will check below) for (size_t i = 0; i < exists_paths.size(); ++i) { - auto status = exists_result[i]; - if (status.error == Coordination::Error::ZOK) + auto error = exists_result[i].error; + if (error == Coordination::Error::ZOK) ++active_replicas; + else if (Coordination::isHardwareError(error)) + keeper_error = error; } size_t replicas_number = replicas.size(); size_t quorum_size = getQuorumSize(replicas_number); if (active_replicas < quorum_size) + { + if (Coordination::isHardwareError(keeper_error)) + throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); + throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); + } /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. @@ -155,15 +163,34 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + const auto & settings = context->getSettingsRef(); + zookeeper_retries_info = ZooKeeperRetriesInfo( + "ReplicatedMergeTreeSink::consume", + log, + settings.insert_keeper_max_retries, + settings.insert_keeper_retry_initial_backoff_ms, + settings.insert_keeper_retry_max_backoff_ms); + + ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( + settings.insert_keeper_fault_injection_probability, + settings.insert_keeper_fault_injection_seed, + storage.getZooKeeper(), + "ReplicatedMergeTreeSink::consume", + log); /** If write is with quorum, then we check that the required number of replicas is now live, * and also that for all previous parts for which quorum is required, this quorum is reached. * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - size_t replicas_num = checkQuorumPrecondition(zookeeper); + size_t replicas_num = 0; + ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info); + quorum_retries_ctl.retryLoop( + [&]() + { + zookeeper->setKeeper(storage.getZooKeeper()); + replicas_num = checkQuorumPrecondition(zookeeper); + }); deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); @@ -173,7 +200,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; - const Settings & settings = context->getSettingsRef(); for (auto & current_block : part_blocks) { @@ -253,7 +279,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) +void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!delayed_chunk) return; @@ -291,8 +317,8 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt { /// NOTE: No delay in this case. That's Ok. - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + assertSessionIsNotExpired(storage.getZooKeeper()); + auto zookeeper = std::make_shared(storage.getZooKeeper()); size_t replicas_num = checkQuorumPrecondition(zookeeper); @@ -312,7 +338,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } void ReplicatedMergeTreeSink::commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, DataPartStorageBuilderPtr builder, @@ -324,8 +350,6 @@ void ReplicatedMergeTreeSink::commitPart( /// /// metadata_snapshot->check(part->getColumns()); - assertSessionIsNotExpired(zookeeper); - String temporary_part_relative_path = part->data_part_storage->getPartDirectory(); /// There is one case when we need to retry transaction in a loop. @@ -335,8 +359,40 @@ void ReplicatedMergeTreeSink::commitPart( bool is_already_existing_part = false; - while (true) + /// for retries due to keeper error + bool part_committed_locally_but_zookeeper = false; + Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; + + ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info); + retries_ctl.retryLoop([&]() { + zookeeper->setKeeper(storage.getZooKeeper()); + + /// if we are in retry, check if last iteration was actually successful + /// we could get network error on latest keeper operation in iteration + /// but operation could be completed by keeper server + if (retries_ctl.isRetry()) + { + if (part_committed_locally_but_zookeeper) + { + /// check that info about the part was actually written in zk + if (zookeeper->exists(fs::path(storage.replica_path) / "parts" / part->name)) + { + LOG_DEBUG(log, "Part was successfully committed on previous iteration: part_id={}", part->name); + } + else + { + retries_ctl.setUserError( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Insert failed due to zookeeper error. Please retry. Reason: {}", + Coordination::errorMessage(write_part_info_keeper_error)); + } + + retries_ctl.stopRetries(); + return; + } + } + /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. /// Also, make deduplication check. If a duplicate is detected, no nodes are created. @@ -344,7 +400,8 @@ void ReplicatedMergeTreeSink::commitPart( /// Allocate new block number and check for duplicates bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; - auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); + std::optional block_number_lock + = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); /// Prepare transaction to ZooKeeper /// It will simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. @@ -444,6 +501,8 @@ void ReplicatedMergeTreeSink::commitPart( { is_already_existing_part = true; + /// TODO: looks unclear, - if deduplication is off then block_id will be empty, - so what will happened here then? + /// This block was already written to some replica. Get the part name for it. /// Note: race condition with DROP PARTITION operation is possible. User will get "No node" exception and it is Ok. existing_part_name = zookeeper->get(storage.zookeeper_path + "/blocks/" + block_id); @@ -463,7 +522,13 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_value, replicas_num); + if (!retries_ctl.callAndCatchAll( + [&]() + { + waitForQuorum( + zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_value, replicas_num); + })) + return; } else { @@ -472,6 +537,7 @@ void ReplicatedMergeTreeSink::commitPart( return; } + LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", block_id, existing_part_name); @@ -503,8 +569,7 @@ void ReplicatedMergeTreeSink::commitPart( } catch (const Exception & e) { - if (e.code() != ErrorCodes::DUPLICATE_DATA_PART - && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } @@ -521,11 +586,24 @@ void ReplicatedMergeTreeSink::commitPart( part->name); } - storage.lockSharedData(*part, false, {}); + try + { + storage.lockSharedData(*part, zookeeper, false, {}); + } + catch (const Exception &) + { + /// todo: add 'rename back to temp state' func + transaction.rollbackPartsToTemporaryState(); + + part->is_temp = true; + part->renameTo(temporary_part_relative_path, false, builder); + builder->commit(); + + throw; + } Coordination::Responses responses; Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT - if (multi_code == Coordination::Error::ZOK) { transaction.commit(); @@ -535,18 +613,26 @@ void ReplicatedMergeTreeSink::commitPart( if (block_number_lock) block_number_lock->assumeUnlocked(); } - else if (multi_code == Coordination::Error::ZCONNECTIONLOSS - || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) + else if (Coordination::isHardwareError(multi_code)) { + write_part_info_keeper_error = multi_code; /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ + * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. + */ transaction.commit(); - storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); + part_committed_locally_but_zookeeper = true; + + /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action + retries_ctl.actionAfterLastFailedRetry( + [&storage = storage, part_name = part->name]() + { storage.enqueuePartForCheck(part_name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); }); /// We do not know whether or not data has been inserted. - throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), - ErrorCodes::UNKNOWN_STATUS_OF_INSERT); + retries_ctl.setUserError( + ErrorCodes::UNKNOWN_STATUS_OF_INSERT, + "Unknown status, client must retry. Reason: {}", + Coordination::errorMessage(multi_code)); + return; } else if (Coordination::isUserError(multi_code)) { @@ -567,62 +653,71 @@ void ReplicatedMergeTreeSink::commitPart( builder->commit(); /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part - /// than it will be ignored on the next itration. + /// than it will be ignored on the next iteration. ++loop_counter; if (loop_counter == max_iterations) { part->is_duplicate = true; /// Part is duplicate, just remove it from local FS throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); } - continue; + retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout + return; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); - throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected logical error while adding block {} with ID '{}': {}, path {}", + block_number, + block_id, + Coordination::errorMessage(multi_code), + failed_op_path); } } - else if (Coordination::isHardwareError(multi_code)) - { - storage.unlockSharedData(*part); - transaction.rollback(); - throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); - } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); - throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", + block_number, + block_id, + Coordination::errorMessage(multi_code)); } - - break; - } + }); if (isQuorumEnabled()) { - if (is_already_existing_part) + ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); + quorum_retries_ctl.retryLoop([&]() { - /// We get duplicate part without fetch - /// Check if this quorum insert is parallel or not - if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) - storage.updateQuorum(part->name, true); - else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) - storage.updateQuorum(part->name, false); - } + zookeeper->setKeeper(storage.getZooKeeper()); - waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value, replicas_num); + if (is_already_existing_part) + { + /// We get duplicate part without fetch + /// Check if this quorum insert is parallel or not + if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) + storage.updateQuorum(part->name, true); + else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) + storage.updateQuorum(part->name, false); + } + + if (!quorum_retries_ctl.callAndCatchAll( + [&]() + { waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value, replicas_num); })) + return; + }); } } @@ -636,12 +731,13 @@ void ReplicatedMergeTreeSink::onStart() void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); + /// todo: check this place, afaiu, it can be called after Generate apart of Consume assertSessionIsNotExpired(zookeeper); - finishDelayedChunk(zookeeper); + finishDelayedChunk(std::make_shared(zookeeper)); } void ReplicatedMergeTreeSink::waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, const std::string & is_active_node_value, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index ab729e6edec..5af599544b5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace Poco { class Logger; } @@ -60,6 +62,7 @@ public: } private: + ZooKeeperRetriesInfo zookeeper_retries_info; struct QuorumInfo { String status_path; @@ -72,11 +75,11 @@ private: /// Checks active replicas. /// Returns total number of replicas. - size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); + size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, DataPartStorageBuilderPtr part_builder, @@ -85,8 +88,11 @@ private: /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. void waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, - const std::string & quorum_path, const std::string & is_active_node_value, size_t replicas_num) const; + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const std::string & part_name, + const std::string & quorum_path, + const std::string & is_active_node_value, + size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; @@ -118,7 +124,7 @@ private: struct DelayedChunk; std::unique_ptr delayed_chunk; - void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); + void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); }; } diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h new file mode 100644 index 00000000000..e10478e3113 --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -0,0 +1,250 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int OK; +} + +struct ZooKeeperRetriesInfo +{ + ZooKeeperRetriesInfo() = default; + ZooKeeperRetriesInfo(std::string name_, Poco::Logger * logger_, UInt64 max_retries_, UInt64 initial_backoff_ms_, UInt64 max_backoff_ms_) + : name(std::move(name_)) + , logger(logger_) + , max_retries(max_retries_) + , curr_backoff_ms(std::min(initial_backoff_ms_, max_backoff_ms_)) + , max_backoff_ms(max_backoff_ms_) + { + } + + std::string name; + Poco::Logger * logger = nullptr; + UInt64 max_retries = 0; + UInt64 curr_backoff_ms = 0; + UInt64 max_backoff_ms = 0; + UInt64 retry_count = 0; +}; + +class ZooKeeperRetriesControl +{ +public: + ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_) : name(std::move(name_)), retries_info(retries_info_) + { + } + + bool canTry() + { + ++iteration_count; + /// first iteration is ordinary execution, no further checks needed + if (0 == iteration_count) + return true; + + if (unconditional_retry) + { + unconditional_retry = false; + return true; + } + + /// iteration succeeded -> no need to retry + if (iteration_succeeded) + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", + retries_info.name, + name, + iteration_count, + retries_info.retry_count); + return false; + } + + if (stop_retries) + { + logLastError("stop retries on request"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + if (retries_info.retry_count >= retries_info.max_retries) + { + logLastError("retry limit is reached"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + /// retries + ++retries_info.retry_count; + logLastError("will retry due to error"); + sleepForMilliseconds(retries_info.curr_backoff_ms); + retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); + + /// reset the flag, it will be set to false in case of error + iteration_succeeded = true; + + return true; + } + + void retryLoop(auto && f) + { + while (canTry()) + { + try + { + f(); + } + catch (const zkutil::KeeperException & e) + { + if (!Coordination::isHardwareError(e.code)) + throw; + + setKeeperError(e.code, e.message()); + } + } + } + + bool callAndCatchAll(auto && f) + { + try + { + f(); + return true; + } + catch (const zkutil::KeeperException & e) + { + setKeeperError(e.code, e.message()); + } + catch (const Exception & e) + { + setUserError(e.code(), e.what()); + } + return false; + } + + void setUserError(int code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setUserError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + user_error.code = code; + user_error.message = std::move(message); + keeper_error = KeeperError{}; + } + + template + void setUserError(int code, fmt::format_string fmt, Args &&... args) + { + setUserError(code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setKeeperError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + keeper_error.code = code; + keeper_error.message = std::move(message); + user_error = UserError{}; + } + + void stopRetries() { stop_retries = true; } + + void requestUnconditionalRetry() { unconditional_retry = true; } + + bool isLastRetry() const { return retries_info.retry_count >= retries_info.max_retries; } + + bool isRetry() const { return retries_info.retry_count > 0; } + + Coordination::Error getLastKeeperErrorCode() const { return keeper_error.code; } + + void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } + +private: + struct KeeperError + { + using Code = Coordination::Error; + Code code = Code::ZOK; + std::string message; + }; + + struct UserError + { + int code = ErrorCodes::OK; + std::string message; + }; + + void throwIfError() const + { + if (user_error.code != ErrorCodes::OK) + throw Exception(user_error.code, user_error.message); + + if (keeper_error.code != KeeperError::Code::ZOK) + throw zkutil::KeeperException(keeper_error.code, keeper_error.message); + } + + void logLastError(std::string header) + { + if (user_error.code == ErrorCodes::OK) + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + keeper_error.code, + keeper_error.message); + } + else + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + user_error.code, + user_error.message); + } + } + + + std::string name; + ZooKeeperRetriesInfo & retries_info; + Int64 iteration_count = -1; + UserError user_error; + KeeperError keeper_error; + std::function action_after_last_failed_retry = []() {}; + bool unconditional_retry = false; + bool iteration_succeeded = true; + bool stop_retries = false; +}; + +} diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h new file mode 100644 index 00000000000..c1dbbe7bcc1 --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -0,0 +1,429 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class RandomFaultInjection +{ +public: + RandomFaultInjection(double probability, UInt64 seed_) : rndgen(seed_), distribution(probability) { } + + void beforeOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); + } + void afterOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); + } + +private: + std::mt19937_64 rndgen; + std::bernoulli_distribution distribution; +}; + +/// +/// ZooKeeperWithFailtInjection mimics ZooKeeper interface and inject failures according to failure policy if set +/// +class ZooKeeperWithFaultInjection +{ + using zk = zkutil::ZooKeeper; + + zk::Ptr keeper; + std::unique_ptr fault_policy; + std::string name; + Poco::Logger * logger = nullptr; + UInt64 calls_total = 0; + UInt64 calls_without_fault_injection = 0; + const UInt64 seed = 0; + const std::function noop_cleanup = []() {}; + + ZooKeeperWithFaultInjection( + zk::Ptr const & keeper_, + double fault_injection_probability, + UInt64 fault_injection_seed, + std::string name_, + Poco::Logger * logger_ = nullptr) + : keeper(keeper_), name(std::move(name_)), logger(logger_), seed(fault_injection_seed) + { + if (fault_injection_probability > .0) + fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); + + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFailtInjection created: name={} seed={} fault_probability={}", + name, + seed, + fault_injection_probability); + } + +public: + using Ptr = std::shared_ptr; + + static ZooKeeperWithFaultInjection::Ptr createInstance( + double fault_injection_probability, UInt64 fault_injection_seed, const zk::Ptr & zookeeper, std::string name, Poco::Logger * logger) + { + /// validate all parameters here, constructor just accept everything + + if (fault_injection_probability < 0.0) + fault_injection_probability = .0; + else if (fault_injection_probability > 1.0) + fault_injection_probability = 1.0; + + if (0 == fault_injection_seed) + fault_injection_seed = randomSeed(); + + if (fault_injection_probability > 0.0) + return std::shared_ptr( + new ZooKeeperWithFaultInjection(zookeeper, fault_injection_probability, fault_injection_seed, std::move(name), logger)); + + /// if no fault injection provided, create instance which will not log anything + return std::make_shared(zookeeper); + } + + explicit ZooKeeperWithFaultInjection(zk::Ptr const & keeper_) : keeper(keeper_) { } + + ~ZooKeeperWithFaultInjection() + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFailtInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", + name, + seed, + calls_total, + calls_without_fault_injection, + calls_total - calls_without_fault_injection, + float(calls_total - calls_without_fault_injection) / calls_total); + } + + void setKeeper(zk::Ptr const & keeper_) { keeper = keeper_; } + bool isNull() const { return keeper.get() == nullptr; } + + /// + /// mirror ZooKeeper interface + /// + + Strings getChildren( + const std::string & path, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access( + "getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }, noop_cleanup); + } + + Coordination::Error tryGetChildren( + const std::string & path, + Strings & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access( + "tryGetChildren", + path, + [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }, + [&](Coordination::Error) {}); + } + + zk::FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}) + { + return access( + "asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }, noop_cleanup); + } + + zk::FutureGet asyncTryGet(const std::string & path) + { + return access( + "asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }, noop_cleanup); + } + + bool tryGet( + const std::string & path, + std::string & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::Error * code = nullptr) + { + return access( + "tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }, noop_cleanup); + } + + Coordination::Error tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses) + { + return access( + "tryMulti", + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMulti(requests, responses); }, + [&](Coordination::Error err) + { + if (err == Coordination::Error::ZOK) + faultInjectionCleanup("tryMulti", requests, responses); + }); + } + + Coordination::Error tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses) + { + constexpr auto no_throw = true; + constexpr auto inject_failure_before_op = false; + return access( + "tryMultiNoThrow", + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMultiNoThrow(requests, responses); }, + [&](Coordination::Error err) + { + if (err == Coordination::Error::ZOK) + faultInjectionCleanup("tryMultiNoThrow", requests, responses); + }); + } + + std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access( + "get", path, [&]() { return keeper->get(path, stat, watch); }, [](const std::string &) {}); + } + + zkutil::ZooKeeper::MultiGetResponse get(const std::vector & paths) + { + return access( + "get", paths.front(), [&]() { return keeper->get(paths); }, noop_cleanup); + } + + bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access( + "exists", path, [&]() { return keeper->exists(path, stat, watch); }, noop_cleanup); + } + + zkutil::ZooKeeper::MultiExistsResponse exists(const std::vector & paths) + { + return access( + "exists", paths.front(), [&]() { return keeper->exists(paths); }, noop_cleanup); + } + + std::string create(const std::string & path, const std::string & data, int32_t mode) + { + return access( + "create", + path, + [&]() { return keeper->create(path, data, mode); }, + [&](std::string const & result_path) + { + try + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + { + keeper->remove(result_path); + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFailtInjection cleanup: seed={} func={} path={}", seed, "create", result_path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFailtInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", + seed, + "create", + result_path, + e.code, + e.message()); + } + }); + } + + Coordination::Responses multi(const Coordination::Requests & requests) + { + return access( + "multi", + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->multi(requests); }, + [&](Coordination::Responses const & responses) { faultInjectionCleanup("multi", requests, responses); }); + } + + void createAncestors(const std::string & path) + { + access( + "createAncestors", path, [&]() { return keeper->createAncestors(path); }, noop_cleanup); + } + + Coordination::Error tryRemove(const std::string & path, int32_t version = -1) + { + return access( + "tryRemove", path, [&]() { return keeper->tryRemove(path, version); }, [&](Coordination::Error) {}); + } + +private: + template + void faultInjectionAfter(const Result & res, FaultCleanup fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->afterOperation(); + } + catch (const zkutil::KeeperException &) + { + if constexpr (std::is_same_v>) + fault_cleanup(); + else + { + if constexpr (std::is_same_v) + fault_cleanup(res); + else if constexpr (std::is_same_v) + fault_cleanup(res); + else if constexpr (std::is_same_v) + fault_cleanup(res); + else + fault_cleanup(); + } + + throw; + } + } + + void faultInjectionCleanup(const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses) + { + if (responses.empty()) + return; + + if (responses.size() != requests.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Number of responses doesn't match number of requests: method={} requests={} responses={}", + method, + requests.size(), + responses.size()); + + /// find create request with ephemeral flag + std::vector> create_requests; + for (size_t i = 0; i < requests.size(); ++i) + { + const auto * create_req = dynamic_cast(requests[i].get()); + if (create_req && create_req->is_ephemeral) + create_requests.emplace_back(i, create_req); + } + + for (auto && [i, req] : create_requests) + { + const auto * create_resp = dynamic_cast(responses.at(i).get()); + if (!create_resp) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Response should be CreateResponse: method={} index={} path={}", method, i, req->path); + + keeper->remove(create_resp->path_created); + } + } + + template < + typename Result, + bool no_throw_access = false, + bool inject_failure_before_op = true, + int inject_failure_after_op = true, + typename FaultCleanup> + Result access(const char * func_name, const std::string & path, auto && operation, FaultCleanup fault_after_op_cleanup) + { + try + { + ++calls_total; + + if (!keeper) + throw zkutil::KeeperException( + "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); + + if constexpr (inject_failure_before_op) + { + if (unlikely(fault_policy)) + fault_policy->beforeOperation(); + } + + if constexpr (!std::is_same_v) + { + Result res = operation(); + + /// if connectivity error occurred w/o fault injection -> just return it + if constexpr (std::is_same_v) + { + if (Coordination::isHardwareError(res)) + return res; + } + + if constexpr (inject_failure_after_op) + faultInjectionAfter(res, fault_after_op_cleanup); + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFailtInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + + return res; + } + else + { + operation(); + + if constexpr (inject_failure_after_op) + { + void * stub = nullptr; /// just for template overloading + faultInjectionAfter(stub, fault_after_op_cleanup); + } + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFailtInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFailtInjection call FAILED: seed={} func={} path={} code={} message={} ", + seed, + func_name, + path, + e.code, + e.message()); + + keeper.reset(); + + /// for try*NoThrow() methods + if constexpr (no_throw_access) + return e.code; + + if constexpr (std::is_same_v) + { + /// try*() methods throws at least on hardware error and return only on logical errors + /// todo: the methods return only on subset of logical errors, and throw on another errors + /// to mimic the methods exactly - we need to specify errors on which to return for each such method + if (Coordination::isHardwareError(e.code)) + throw; + + return e.code; + } + else + throw; + } + } +}; + +using ZooKeeperWithFaultInjectionPtr = ZooKeeperWithFaultInjection::Ptr; +} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef56ade2d58..5f2c5d628af 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4467,8 +4467,6 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { const auto storage_settings_ptr = getSettings(); - assertNotReadonly(); - const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; @@ -4983,8 +4981,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St Int64 mutation_version; { - auto zookeeper = getZooKeeper(); - delimiting_block_lock = allocateBlockNumber(partition_id, zookeeper); + delimiting_block_lock = allocateBlockNumber(partition_id, getZooKeeper()); right = delimiting_block_lock->getNumber(); /// Make sure we cover all parts in drop range. /// There might be parts with mutation version greater than current block number @@ -5265,7 +5262,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c } -bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const +bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const { { std::lock_guard lock(existing_nodes_cache_mutex); @@ -5273,7 +5270,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons return true; } - bool res = getZooKeeper()->exists(path); + bool res = zookeeper->exists(path); if (res) { @@ -5285,9 +5282,22 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons } -std::optional -StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path, const String & zookeeper_path_prefix) const +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const zkutil::ZooKeeperPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const +{ + return allocateBlockNumber( + partition_id, std::make_shared(zookeeper), zookeeper_block_id_path, zookeeper_path_prefix); +} + + +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const { String zookeeper_table_path; if (zookeeper_path_prefix.empty()) @@ -5298,7 +5308,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; - if (!existsNodeCached(partition_path)) + if (!existsNodeCached(zookeeper, partition_path)) { Coordination::Requests ops; ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent)); @@ -5314,10 +5324,9 @@ StorageReplicatedMergeTree::allocateBlockNumber( } return createEphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_path); + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", zookeeper, zookeeper_block_id_path); } - Strings StorageReplicatedMergeTree::tryWaitForAllReplicasToProcessLogEntry( const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout) { @@ -7002,7 +7011,7 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, - MutableDataPartPtr & part, + const DataPartPtr & part, const String & block_id_path) const { const String & part_name = part->name; @@ -7580,11 +7589,28 @@ void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_nam String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; LOG_TRACE(log, "Set zookeeper temporary ephemeral lock {}", zookeeper_node); - createZeroCopyLockNode(zookeeper, zookeeper_node, zkutil::CreateMode::Ephemeral, false); + createZeroCopyLockNode( + std::make_shared(zookeeper), zookeeper_node, zkutil::CreateMode::Ephemeral, false); } } -void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + bool replace_existing_lock, + std::optional hardlinked_files) const +{ + auto zookeeper = tryGetZooKeeper(); + if (zookeeper) + return lockSharedData(part, std::make_shared(zookeeper), replace_existing_lock, hardlinked_files); + else + return lockSharedData(part, std::make_shared(nullptr), replace_existing_lock, hardlinked_files); +} + +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const { auto settings = getSettings(); @@ -7594,8 +7620,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, if (!part.data_part_storage->supportZeroCopyReplication()) return; - zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); - if (!zookeeper) + if (zookeeper->isNull()) return; String id = part.getUniqueId(); @@ -7629,7 +7654,14 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, } } -std::pair StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +{ + return unlockSharedData(part, std::make_shared(nullptr)); +} + +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const { auto settings = getSettings(); if (!settings->allow_remote_fs_zero_copy_replication) @@ -7678,11 +7710,10 @@ std::pair StorageReplicatedMergeTree::unlockSharedData(const IMer /// We remove parts during table shutdown. If exception happen, restarting thread will be already turned /// off and nobody will reconnect our zookeeper connection. In this case we use zookeeper connection from /// context. - zkutil::ZooKeeperPtr zookeeper; if (shutdown_called.load()) - zookeeper = getZooKeeperIfTableShutDown(); + zookeeper->setKeeper(getZooKeeperIfTableShutDown()); else - zookeeper = getZooKeeper(); + zookeeper->setKeeper(getZooKeeper()); /// It can happen that we didn't had the connection to zookeeper during table creation, but actually /// table is completely dropped, so we can drop it without any additional checks. @@ -7707,7 +7738,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -7769,7 +7800,7 @@ std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr std::pair StorageReplicatedMergeTree::unlockSharedDataByID( String part_id, const String & table_uuid, const String & part_name, - const String & replica_name_, const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, + const String & replica_name_, const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) { boost::replace_all(part_id, "/", "_"); @@ -7788,7 +7819,8 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); - auto [has_parent, parent_not_to_remove] = getParentLockedBlobs(zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( + zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8305,7 +8337,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP void StorageReplicatedMergeTree::createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, int32_t mode, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode, bool replace_existing_lock, const String & path_to_set_hardlinked_files, const NameSet & hardlinked_files) { /// In rare case other replica can remove path between createAncestors and createIfNotExists @@ -8422,7 +8454,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St id, table_uuid, part_name, detached_replica_name, toString(disk->getDataSourceDescription().type), - zookeeper, local_context->getReplicatedMergeTreeSettings(), + std::make_shared(zookeeper), local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), detached_zookeeper_path, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 436defebe1d..3d8084d9493 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -83,6 +83,9 @@ namespace DB * as the time will take the time of creation the appropriate part on any of the replicas. */ +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -267,6 +270,11 @@ public: /// Lock part in zookeeper for use shared data in several nodes void lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const override; + void lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const; void lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const; @@ -274,13 +282,23 @@ public: /// Return true if data unlocked /// Return false if data is still used by another node std::pair unlockSharedData(const IMergeTreeDataPart & part) const override; + std::pair + unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const; /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static std::pair unlockSharedDataByID(String part_id, const String & table_uuid, const String & part_name, const String & replica_name_, - const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, - const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version); + static std::pair unlockSharedDataByID( + String part_id, + const String & table_uuid, + const String & part_name, + const String & replica_name_, + const std::string & disk_type, + const ZooKeeperWithFaultInjectionPtr & zookeeper_, + const MergeTreeSettings & settings, + Poco::Logger * logger, + const String & zookeeper_path_old, + MergeTreeDataFormatVersion data_format_version); /// Fetch part only if some replica has it on shared storage like S3 DataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; @@ -534,7 +552,7 @@ private: bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; - void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const; + void getCommitPartOps(Coordination::Requests & ops, const DataPartPtr & part, const String & block_id_path = "") const; /// Adds actions to `ops` that remove a part from ZooKeeper. /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). @@ -711,6 +729,11 @@ private: std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; + std::optional allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path = "", + const String & zookeeper_path_prefix = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. @@ -748,7 +771,7 @@ private: /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. mutable std::unordered_set existing_nodes_cache; mutable std::mutex existing_nodes_cache_mutex; - bool existsNodeCached(const std::string & path) const; + bool existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const; void getClearBlocksInPartitionOps(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); /// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range. @@ -833,7 +856,7 @@ private: const String & part_name, const String & zookeeper_path_old); static void createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false, const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {}); diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20e63412d91..3e58eef9e78 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,6 +456,8 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), + "insert_keeper_fault_injection_probability": lambda: round(random.uniform(0.0, 0.1),2), + "insert_keeper_fault_injection_seed": lambda: random.randint(1, 0xFFFFFFFFFFFFFFFF), } @staticmethod diff --git a/tests/integration/test_inserts_with_keeper_retries/__init__.py b/tests/integration/test_inserts_with_keeper_retries/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py new file mode 100644 index 00000000000..24299b7ee87 --- /dev/null +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -0,0 +1,127 @@ +#!/usr/bin/env python3 + +import pytest +import time +import threading +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.client import QueryRuntimeException +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", with_zookeeper=True) +node2 = cluster.add_instance("node2", with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_replica_inserts_with_keeper_restart(started_cluster): + try: + settings = { + "insert_quorum": "2", + } + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + node2.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '1') ORDER BY tuple()" + ) + + p = Pool(3) + zk_stopped_event = threading.Event() + + def zoo_restart(zk_stopped_event): + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + zk_stopped_event.set() + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + + job = p.apply_async(zoo_restart, (zk_stopped_event,)) + + zk_stopped_event.wait(60) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10)", + settings=settings, + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)", + settings=settings, + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(20, 10)", + settings=settings, + ) + + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "30\n" + assert node2.query("SELECT COUNT() FROM r") == "30\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") + node2.query("DROP TABLE IF EXISTS r SYNC") + + +@pytest.mark.skip(reason="Unfortunately it showed to be flaky. Disabled for now") +def test_replica_inserts_with_keeper_disconnect(started_cluster): + try: + settings = { + "insert_quorum": "2", + } + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + node2.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '1') ORDER BY tuple()" + ) + + p = Pool(1) + disconnect_event = threading.Event() + + def keeper_disconnect(node, event): + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + event.set() + time.sleep(5) + + job = p.apply_async( + keeper_disconnect, + ( + node1, + disconnect_event, + ), + ) + disconnect_event.wait(60) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10)", + settings=settings, + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)", + settings=settings, + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(20, 10)", + settings=settings, + ) + + assert node1.query("SELECT COUNT() FROM r") == "30\n" + assert node2.query("SELECT COUNT() FROM r") == "30\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") + node2.query("DROP TABLE IF EXISTS r SYNC") diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index f62f11c60fd..ed1f654f847 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,14 +1,14 @@ -- Tags: zookeeper, no-replicated-database -- Tag no-replicated-database: Old syntax is not allowed -DROP TABLE IF EXISTS alter_00121; +DROP TABLE IF EXISTS alter_00121 SYNC; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); @@ -23,4 +23,4 @@ SELECT * FROM alter_00121 ORDER BY d; ALTER TABLE alter_00121 DROP COLUMN x; SELECT * FROM alter_00121 ORDER BY d; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 8267a451728..2883215ff94 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -1,11 +1,12 @@ -- Tags: long, replica SET replication_alter_partitions_sync = 2; +SET insert_keeper_fault_injection_probability=0; SELECT '*** Not partitioned ***'; -DROP TABLE IF EXISTS not_partitioned_replica1_00502; -DROP TABLE IF EXISTS not_partitioned_replica2_00502; +DROP TABLE IF EXISTS not_partitioned_replica1_00502 SYNC; +DROP TABLE IF EXISTS not_partitioned_replica2_00502 SYNC; CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; @@ -25,13 +26,13 @@ ALTER TABLE not_partitioned_replica1_00502 DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned_replica2_00502; -DROP TABLE not_partitioned_replica1_00502; -DROP TABLE not_partitioned_replica2_00502; +DROP TABLE not_partitioned_replica1_00502 SYNC; +DROP TABLE not_partitioned_replica2_00502 SYNC; SELECT '*** Partitioned by week ***'; -DROP TABLE IF EXISTS partitioned_by_week_replica1; -DROP TABLE IF EXISTS partitioned_by_week_replica2; +DROP TABLE IF EXISTS partitioned_by_week_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_week_replica2 SYNC; CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; @@ -52,13 +53,13 @@ ALTER TABLE partitioned_by_week_replica1 DROP PARTITION '1999-12-27'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_week_replica2; -DROP TABLE partitioned_by_week_replica1; -DROP TABLE partitioned_by_week_replica2; +DROP TABLE partitioned_by_week_replica1 SYNC; +DROP TABLE partitioned_by_week_replica2 SYNC; SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); @@ -79,13 +80,13 @@ ALTER TABLE partitioned_by_tuple_replica1_00502 DETACH PARTITION ID '20000101-1' SELECT 'Sum after DETACH PARTITION:'; SELECT sum(y) FROM partitioned_by_tuple_replica2_00502; -DROP TABLE partitioned_by_tuple_replica1_00502; -DROP TABLE partitioned_by_tuple_replica2_00502; +DROP TABLE partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00502 SYNC; SELECT '*** Partitioned by String ***'; -DROP TABLE IF EXISTS partitioned_by_string_replica1; -DROP TABLE IF EXISTS partitioned_by_string_replica2; +DROP TABLE IF EXISTS partitioned_by_string_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_string_replica2 SYNC; CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; @@ -105,13 +106,13 @@ ALTER TABLE partitioned_by_string_replica1 DROP PARTITION 'bbb'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_string_replica2; -DROP TABLE partitioned_by_string_replica1; -DROP TABLE partitioned_by_string_replica2; +DROP TABLE partitioned_by_string_replica1 SYNC; +DROP TABLE partitioned_by_string_replica2 SYNC; SELECT '*** Table without columns with fixed size ***'; -DROP TABLE IF EXISTS without_fixed_size_columns_replica1; -DROP TABLE IF EXISTS without_fixed_size_columns_replica2; +DROP TABLE IF EXISTS without_fixed_size_columns_replica1 SYNC; +DROP TABLE IF EXISTS without_fixed_size_columns_replica2 SYNC; CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; @@ -130,5 +131,5 @@ ALTER TABLE without_fixed_size_columns_replica1 DROP PARTITION 1; SELECT 'After DROP PARTITION:'; SELECT * FROM without_fixed_size_columns_replica2 ORDER BY s; -DROP TABLE without_fixed_size_columns_replica1; -DROP TABLE without_fixed_size_columns_replica2; +DROP TABLE without_fixed_size_columns_replica1 SYNC; +DROP TABLE without_fixed_size_columns_replica2 SYNC; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index 7a6c7609660..a1e7d8727c7 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2 SYNC" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" @@ -19,9 +19,9 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 1, 'a')" -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 2, 'b'), ('2000-01-01', 3, 'c'), ('2000-01-01', 4, 'd') \ ('2000-02-01', 2, 'b'), ('2000-02-01', 3, 'c'), ('2000-02-01', 4, 'd')" @@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE s = 'd' SETT ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTINGS mutations_sync = 2" # Insert more data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" ${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" @@ -49,8 +49,8 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partiti ${CLICKHOUSE_CLIENT} --query="SELECT '*** Test mutations cleaner ***'" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2 SYNC" # Create 2 replicas with finished_mutations_to_keep = 2 ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r1') ORDER BY x SETTINGS \ @@ -63,7 +63,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE cleanup_delay_period_random_add = 0" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" # Add some mutations and wait for their execution ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" @@ -80,8 +80,8 @@ sleep 1.5 # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner_r2' ORDER BY mutation_id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2 SYNC" diff --git a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql index 4cdd75f520c..8f694345d93 100644 --- a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql @@ -2,8 +2,8 @@ SET optimize_on_insert = 0; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w); CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w); @@ -21,5 +21,5 @@ OPTIMIZE TABLE partitioned_by_tuple_replica1_00661 FINAL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00661; SELECT * FROM partitioned_by_tuple_replica2_00661 ORDER BY d, x, w, y; -DROP TABLE partitioned_by_tuple_replica1_00661; -DROP TABLE partitioned_by_tuple_replica2_00661; +DROP TABLE partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00661 SYNC; diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 4637b210194..3f384380f9b 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE IF EXISTS fetches_r1; - DROP TABLE IF EXISTS fetches_r2" + DROP TABLE IF EXISTS fetches_r1 SYNC; + DROP TABLE IF EXISTS fetches_r2 SYNC" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r1') ORDER BY x" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r2') ORDER BY x \ @@ -18,6 +18,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate prefer_fetch_merged_part_size_threshold=0" ${CLICKHOUSE_CLIENT} -n --query=" + SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); INSERT INTO fetches_r1 VALUES (3)" @@ -51,5 +52,5 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutate ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE fetches_r1; - DROP TABLE fetches_r2" + DROP TABLE fetches_r1 SYNC; + DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 6f609065c01..5fc3fa460e6 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -12,6 +12,7 @@ $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 703f8824055..b836f806170 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,9 +1,10 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET check_query_single_value_result = 0; SET send_logs_level = 'fatal'; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeTree() ORDER BY tuple(); @@ -11,9 +12,9 @@ INSERT INTO mt_without_pk VALUES (1, 2); CHECK TABLE mt_without_pk; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); @@ -21,4 +22,4 @@ INSERT INTO replicated_mt_without_pk VALUES (1, 2); CHECK TABLE replicated_mt_without_pk; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; diff --git a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh index a438cf7badc..68c511b80ac 100755 --- a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh +++ b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh @@ -52,7 +52,8 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations" -${CLICKHOUSE_CLIENT} --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" +# test relays on part ids, which are non-deterministic with keeper fault injections, so disable it +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" ${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM replicated_table_for_mutations" diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index 43ab053655a..e4acfed8a0c 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -1,8 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); @@ -10,6 +10,9 @@ CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGI -- Should not be larger then 600e6 (default timeout in clickhouse-test) SET insert_quorum=2, insert_quorum_parallel=0, insert_quorum_timeout=300e3; +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + INSERT INTO mutations_and_quorum1 VALUES ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'), ('2019-04-01', 'test4'), ('2019-05-01', 'test1'), ('2019-06-01', 'test2'), ('2019-07-01', 'test3'), ('2019-08-01', 'test4'), ('2019-09-01', 'test1'), ('2019-10-01', 'test2'), ('2019-11-01', 'test3'), ('2019-12-01', 'test4'); ALTER TABLE mutations_and_quorum1 DELETE WHERE something = 'test1' SETTINGS mutations_sync=2; @@ -19,5 +22,5 @@ SELECT COUNT() FROM mutations_and_quorum2; SELECT COUNT() FROM system.mutations WHERE database = currentDatabase() AND table like 'mutations_and_quorum%' and is_done = 0; -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index c370726c72f..2096942630e 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; CREATE TABLE default_table ( @@ -26,4 +26,4 @@ ALTER TABLE default_table MODIFY COLUMN enum_column Enum8('undefined' = 0, 'fox' SHOW CREATE TABLE default_table; -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; diff --git a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql index 687c8051eed..bd11b24d568 100644 --- a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql +++ b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql @@ -1,5 +1,7 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + set send_logs_level='error'; drop table if exists mt; drop table if exists rmt sync; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 501ab805a5a..6701c8b840f 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -1,7 +1,9 @@ -- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts -- Tag no-replicated-database: Fails due to additional replicas or shards -drop table if exists rmt; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + +drop table if exists rmt sync; -- cleanup code will perform extra Exists -- (so the .reference will not match) create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; @@ -30,7 +32,7 @@ from system.zookeeper_log where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks%' and op_num not in (1, 12, 500)) order by xid, type, request_idx; -drop table rmt; +drop table rmt sync; system flush logs; select 'duration_ms'; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 8bf5d4f2cf8..6d569ca6352 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -1,5 +1,7 @@ -- Tags: long, replica +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + DROP TABLE IF EXISTS test; DROP TABLE IF EXISTS test2; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 01678d1b500..4629450c1f9 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -12,9 +12,9 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \ @@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name FROM table" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated DETACH PARTITION '3';" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated ATTACH PARTITION '3' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $ATTACH_OUT_STRUCTURE" \ @@ -38,4 +38,4 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name, old_part_name FROM table" # teardown -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql index 0155b83db31..eea231c9f58 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql @@ -1,5 +1,6 @@ -- Tags: long, replica, no-replicated-database +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index 9f9d1db78f1..bf7a471fa40 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,10 +1,11 @@ -- Tags: long, replica, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS replica1; -DROP TABLE IF EXISTS replica2; +DROP TABLE IF EXISTS replica1 SYNC; +DROP TABLE IF EXISTS replica2 SYNC; CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; @@ -48,5 +49,5 @@ SELECT v FROM replica1 ORDER BY v; SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); -DROP TABLE replica1; -DROP TABLE replica2; +DROP TABLE replica1 SYNC; +DROP TABLE replica2 SYNC; diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 13c772e75d2..c05d813ca7f 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -11,7 +11,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -20,7 +20,11 @@ function thread { for x in {0..99}; do # sometimes we can try to commit obsolete part if fetches will be quite fast, # so supress warning messages like "Tried to commit obsolete part ... covered by ..." - $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas + # (2) keeper fault injection for inserts because + # it can be a cause of deduplicated parts be visible to SELECTs for sometime (until cleanup thread remove them), + # so the same SELECT on different replicas can return different results, i.e. test output will be non-deterministic + # (see #9712) + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas done } @@ -37,5 +41,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 6eabc9ae1b5..209e18e3329 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -22,7 +22,7 @@ valid_exceptions_to_retry='Quorum for previous write has not been satisfied yet| function thread { for x in {0..99}; do while true; do - $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break + $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break done done } @@ -40,5 +40,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index 030ae017e71..a2ecef64bbb 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_quorum_many', 'r$i') ORDER BY x; " done @@ -20,7 +20,7 @@ done function thread { i=0 retries=300 while [[ $i -lt $retries ]]; do # server can be dead - $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break + $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --insert_keeper_max_retries=100 --insert_keeper_retry_max_backoff_ms=10 --query "INSERT INTO r$1 SELECT $2" && break ((++i)) sleep 0.1 done diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql index 2f0c59ab64b..86bef193f4d 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql @@ -16,6 +16,9 @@ CREATE TABLE r2 ( ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SET insert_quorum_parallel=1; SET insert_quorum=3; @@ -79,11 +82,11 @@ SYSTEM STOP FETCHES r2; SET insert_quorum_timeout=0; -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } -- retry should fail despite the insert_deduplicate enabled -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } SELECT * FROM r2 WHERE key=4; SYSTEM START FETCHES r2; @@ -99,5 +102,5 @@ SELECT 'insert happened'; SELECT COUNT() FROM r1; SELECT COUNT() FROM r2; -DROP TABLE IF EXISTS r1; -DROP TABLE IF EXISTS r2; +DROP TABLE IF EXISTS r1 SYNC; +DROP TABLE IF EXISTS r2 SYNC; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index cf06af0113d..f217b6094b2 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -2,6 +2,8 @@ -- Tag no-replicated-database: Fails due to additional replicas or shards -- Tag no-parallel: static zk path +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql index af2a46cd77f..b55b4871363 100644 --- a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql +++ b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; CREATE TABLE merge_tree_pk ( @@ -24,9 +24,9 @@ ATTACH TABLE merge_tree_pk; SELECT * FROM merge_tree_pk FINAL ORDER BY key, value; -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; CREATE TABLE merge_tree_pk_sql ( @@ -60,9 +60,9 @@ SELECT * FROM merge_tree_pk_sql FINAL ORDER BY key, value; SHOW CREATE TABLE merge_tree_pk_sql; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; CREATE TABLE replicated_merge_tree_pk_sql ( @@ -99,4 +99,4 @@ ATTACH TABLE replicated_merge_tree_pk_sql; SHOW CREATE TABLE replicated_merge_tree_pk_sql; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index e52e0c94c3c..a5f301b1353 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -1,6 +1,6 @@ -- Tags: replica -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; CREATE TABLE replicated_mutations_empty_partitions ( @@ -11,7 +11,8 @@ ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_rep ORDER BY key PARTITION by key; -INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number) FROM numbers(10); +-- insert_keeper* settings are adjusted since several actual inserts are happening behind one statement due to partitioning i.e. inserts in different partitions +INSERT INTO replicated_mutations_empty_partitions SETTINGS insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=10 SELECT number, toString(number) FROM numbers(10); SELECT count(distinct value) FROM replicated_mutations_empty_partitions; @@ -31,4 +32,4 @@ SELECT sum(value) FROM replicated_mutations_empty_partitions; SHOW CREATE TABLE replicated_mutations_empty_partitions; -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index acaa2cfcd25..d68f9bc1837 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_kill (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_kill', '1') ORDER BY key PARTITION BY key % 100 SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" -$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" function alter_thread { @@ -67,4 +67,4 @@ done $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE concurrent_mutate_kill" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_mutate_kill FINAL" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_mutate_kill" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql index c77f29d89c2..e2926d9a8ac 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql @@ -1,7 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS partitioned_table; +SET insert_keeper_fault_injection_probability=0; +DROP TABLE IF EXISTS partitioned_table SYNC; CREATE TABLE partitioned_table ( key UInt64, @@ -47,4 +48,4 @@ SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AN SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; -DROP TABLE IF EXISTS partitioned_table; +DROP TABLE IF EXISTS partitioned_table SYNC; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index dad5cdbf1f3..87e1a039488 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -1,7 +1,7 @@ -- Tags: long, zookeeper -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE IF EXISTS i20203_1 SYNC; +DROP TABLE IF EXISTS i20203_2 SYNC; CREATE TABLE i20203_1 (a Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') @@ -26,5 +26,5 @@ WHERE table = 'i20203_2' AND database = currentDatabase(); ATTACH TABLE i20203_1; -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE i20203_1 SYNC; +DROP TABLE i20203_2 SYNC; diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index 85662438f33..4e4255fe9c2 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -51,6 +51,9 @@ $CLICKHOUSE_CLIENT -nm -q """ partition by key%100 settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; + SET insert_keeper_max_retries=1000; + SET insert_keeper_retry_max_backoff_ms=10; + insert into rep_data_01810 select * from numbers(100); drop table rep_data_01810 settings log_queries=1; system flush logs; diff --git a/tests/queries/0_stateless/01825_type_json_3.sql.j2 b/tests/queries/0_stateless/01825_type_json_3.sql.j2 index 62d86c3efd4..0fbf7a936d8 100644 --- a/tests/queries/0_stateless/01825_type_json_3.sql.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.sql.j2 @@ -2,6 +2,8 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_json_3; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 1de23c13a65..46cd09622b1 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -1,3 +1,6 @@ +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SELECT 'simple partition key:'; DROP TABLE IF EXISTS table1 SYNC; CREATE TABLE table1 (id Int64, v UInt64) @@ -15,7 +18,7 @@ select 'where id % 200 < 0:'; select id from table1 where id % 200 < 0 order by id; SELECT 'tuple as partition key:'; -DROP TABLE IF EXISTS table2; +DROP TABLE IF EXISTS table2 SYNC; CREATE TABLE table2 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id; @@ -24,7 +27,7 @@ INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10); SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p; SELECT 'recursive modulo partition key:'; -DROP TABLE IF EXISTS table3; +DROP TABLE IF EXISTS table3 SYNC; CREATE TABLE table3 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id; @@ -38,7 +41,7 @@ SELECT 'After detach:'; SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; SELECT 'Indexes:'; -DROP TABLE IF EXISTS table4; +DROP TABLE IF EXISTS table4 SYNC; CREATE TABLE table4 (id Int64, v UInt64, s String, INDEX a (id * 2, s) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v; diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 635da491aa0..8924627a717 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,4 +1,5 @@ -- Tags: long, no-parallel +SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush @@ -8,8 +9,8 @@ insert into data_02228 select number, 1, number from numbers_mt(100e3) settings insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; -drop table if exists data_rep_02228; +drop table if exists data_rep_02228 SYNC; create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } -drop table data_rep_02228; +drop table data_rep_02228 SYNC; diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index e8111bf7a03..9ea924377b2 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -10,7 +10,7 @@ function check_refcnt_for_table() local table=$1 && shift $CLICKHOUSE_CLIENT -q "system stop merges $table" - $CLICKHOUSE_CLIENT -q "insert into $table select number, number%4 from numbers(200)" + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" local query_id query_id="$table-$(random_str 10)" @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nmq " check_refcnt_for_table data_02340 $CLICKHOUSE_CLIENT -nmq " - drop table if exists data_02340_rep; + drop table if exists data_02340_rep sync; create table data_02340_rep (key Int, part Int) engine=ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340_rep diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index 11caf1e45de..d61052392c7 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -5,14 +5,14 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists rmt1;" -$CLICKHOUSE_CLIENT -q "drop table if exists rmt2;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" $CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n;" $CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (1);" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (2);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" @@ -32,7 +32,7 @@ $CLICKHOUSE_CLIENT -q "select * from rmt1;" 2>/dev/null $CLICKHOUSE_CLIENT -q "detach table rmt1;" $CLICKHOUSE_CLIENT -q "attach table rmt1;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (3);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" $CLICKHOUSE_CLIENT -q "system start merges rmt2;" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" @@ -42,5 +42,5 @@ $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" $CLICKHOUSE_CLIENT -q "select 3, *, _part from rmt1 order by n;" $CLICKHOUSE_CLIENT -q "select 4, *, _part from rmt2 order by n;" -$CLICKHOUSE_CLIENT -q "drop table rmt1;" -$CLICKHOUSE_CLIENT -q "drop table rmt2;" +$CLICKHOUSE_CLIENT -q "drop table rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table rmt2 sync;" diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 371f7389837..0db816332a1 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -1,5 +1,7 @@ -- Tags: long +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference new file mode 100644 index 00000000000..f5f1cf8ac64 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference @@ -0,0 +1,2 @@ +1 +11 diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql new file mode 100644 index 00000000000..774370bc132 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql @@ -0,0 +1,26 @@ +-- Tags: replica + +DROP TABLE IF EXISTS keeper_retries_r1 SYNC; +DROP TABLE IF EXISTS keeper_retries_r2 SYNC; + +CREATE TABLE keeper_retries_r1(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r1') ORDER BY tuple (); +CREATE TABLE keeper_retries_r2(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r2') ORDER BY tuple(); + +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (1); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (2); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=10 VALUES (3); -- { serverError KEEPER_EXCEPTION } + +SET insert_quorum=2; +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (11); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (12); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=1 VALUES (13); -- { serverError KEEPER_EXCEPTION } + +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.05, insert_keeper_fault_injection_seed=1 VALUES (21); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.2, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=2 VALUES (22); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.3, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=3 VALUES (23); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.4, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=4 VALUES (24); + +SELECT * FROM keeper_retries_r1 order by a; + +DROP TABLE keeper_retries_r1 SYNC; +DROP TABLE keeper_retries_r2 SYNC; From ec8d5b0ddd63810a5ad9d5529c75aaf73b9194ff Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Oct 2022 08:12:24 +0000 Subject: [PATCH 063/526] Add support for testing different products --- tests/jepsen.clickhouse-keeper/project.clj | 6 +- .../resources/keeper_config.xml | 2 +- .../keeper}/bench.clj | 2 +- .../keeper}/constants.clj | 2 +- .../keeper}/counter.clj | 4 +- .../keeper}/db.clj | 8 +- .../keeper}/main.clj | 121 ++---------------- .../keeper}/nemesis.clj | 6 +- .../keeper}/queue.clj | 4 +- .../keeper}/register.clj | 4 +- .../keeper}/set.clj | 4 +- .../keeper}/unique.clj | 4 +- .../keeper}/utils.clj | 4 +- .../keeper}/zookeeperdb.clj | 4 +- .../src/jepsen/clickhouse/main.clj | 105 +++++++++++++++ .../src/jepsen/clickhouse/server/main.clj | 7 + 16 files changed, 152 insertions(+), 135 deletions(-) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/bench.clj (96%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/constants.clj (94%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/counter.clj (95%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/db.clj (97%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/main.clj (63%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/nemesis.clj (97%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/queue.clj (97%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/register.clj (96%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/set.clj (95%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/unique.clj (93%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/utils.clj (98%) rename tests/jepsen.clickhouse-keeper/src/jepsen/{clickhouse_keeper => clickhouse/keeper}/zookeeperdb.clj (95%) create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse-keeper/project.clj index 98049835cb1..1f4a762753b 100644 --- a/tests/jepsen.clickhouse-keeper/project.clj +++ b/tests/jepsen.clickhouse-keeper/project.clj @@ -1,10 +1,10 @@ -(defproject jepsen.keeper "0.1.0-SNAPSHOT" +(defproject jepsen.clickhouse "0.1.0-SNAPSHOT" :injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))] - :description "A jepsen tests for ClickHouse Keeper" + :description "A jepsen tests for ClickHouse" :url "https://clickhouse.com/" :license {:name "EPL-2.0" :url "https://www.eclipse.org/legal/epl-2.0/"} - :main jepsen.clickhouse-keeper.main + :main jepsen.clickhouse.main :plugins [[lein-cljfmt "0.7.0"]] :dependencies [[org.clojure/clojure "1.10.1"] [jepsen "0.2.7"] diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml index 2ab747fbd71..20ca74c1cf5 100644 --- a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml @@ -1,5 +1,5 @@ - :: + 0.0.0.0 trace diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/bench.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/bench.clj similarity index 96% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/bench.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/bench.clj index 040d2eaa77b..d0d30e05650 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/bench.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/bench.clj @@ -1,4 +1,4 @@ -(ns jepsen.clickhouse-keeper.bench +(ns jepsen.clickhouse.keeper.bench (:require [clojure.tools.logging :refer :all] [jepsen [client :as client]]) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/constants.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj similarity index 94% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/constants.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj index cd62d66e652..0c0116deef3 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/constants.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj @@ -1,4 +1,4 @@ -(ns jepsen.clickhouse-keeper.constants) +(ns jepsen.clickhouse.keeper.constants) (def common-prefix "/home/robot-clickhouse") diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj similarity index 95% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj index 60b29bd799a..3abb4843287 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/counter.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj @@ -1,11 +1,11 @@ -(ns jepsen.clickhouse-keeper.counter +(ns jepsen.clickhouse.keeper.counter (:require [clojure.tools.logging :refer :all] [jepsen [checker :as checker] [client :as client] [generator :as gen]] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj similarity index 97% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj index c354e36e430..861bb434957 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj @@ -1,11 +1,11 @@ -(ns jepsen.clickhouse-keeper.db +(ns jepsen.clickhouse.keeper.db (:require [clojure.tools.logging :refer :all] [jepsen [control :as c] [db :as db] [util :as util :refer [meh]]] - [jepsen.clickhouse-keeper.constants :refer :all] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.constants :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [clojure.java.io :as io] [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu])) @@ -30,7 +30,7 @@ [dest])) nil) -(ns jepsen.clickhouse-keeper.db) +(ns jepsen.clickhouse.keeper.db) (defn get-clickhouse-url [url] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj similarity index 63% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj index 46fc8651bfe..efbceb314d7 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj @@ -1,112 +1,17 @@ -(ns jepsen.control.sshj - (:require [jepsen.control [core :as core] - [sshj :as sshj]] - [slingshot.slingshot :refer [try+ throw+]]) - (:import (net.schmizz.sshj SSHClient - DefaultConfig) - (net.schmizz.sshj.transport.verification PromiscuousVerifier) - (java.util.concurrent Semaphore))) - -(defrecord SSHJRemote [concurrency-limit - conn-spec - ^SSHClient client - ^Semaphore semaphore] - core/Remote - (connect [this conn-spec] - (if (:dummy conn-spec) - (assoc this :conn-spec conn-spec) - (try+ (let [c (as-> (SSHClient.) client - (do - (if (:strict-host-key-checking conn-spec) - (.loadKnownHosts client) - (.addHostKeyVerifier client (PromiscuousVerifier.))) - (.connect client (:host conn-spec) (:port conn-spec)) - (auth! client conn-spec) - client))] - (assoc this - :conn-spec conn-spec - :client c - :semaphore (Semaphore. concurrency-limit true))) - (catch Exception e - ; SSHJ wraps InterruptedException in its own exceptions, so we - ; have to see through that and rethrow properly. - (let [cause (util/ex-root-cause e)] - (when (instance? InterruptedException cause) - (throw cause))) - (throw+ (assoc conn-spec - :type :jepsen.control/session-error - :message "Error opening SSH session. Verify username, password, and node hostnames are correct.")))))) - - (disconnect! [this] - (when-let [c client] - (.close c))) - - (execute! [this ctx action] - ; (info :permits (.availablePermits semaphore)) - (when (:dummy conn-spec) - (throw+ {:type :jepsen.control/dummy})) - (.acquire semaphore) - (sshj/with-errors conn-spec ctx - (try - (with-open [session (.startSession client)] - (let [cmd (.exec session (:cmd action)) - ; Feed it input - _ (when-let [input (:in action)] - (let [stream (.getOutputStream cmd)] - (bs/transfer input stream) - (send-eof! client session) - (.close stream))) - ; Read output - out (.toString (IOUtils/readFully (.getInputStream cmd))) - err (.toString (IOUtils/readFully (.getErrorStream cmd))) - ; Wait on command - _ (.join cmd)] - ; Return completion - (assoc action - :out out - :err err - ; There's also a .getExitErrorMessage that might be - ; interesting here? - :exit (.getExitStatus cmd)))) - (finally - (.release semaphore))))) - - (upload! [this ctx local-paths remote-path _opts] - (when (:dummy conn-spec) - (throw+ {:type :jepsen.control/dummy})) - (with-errors conn-spec ctx - (with-open [sftp (.newSFTPClient client)] - (.put sftp (FileSystemFile. local-paths) remote-path)))) - - (download! [this ctx remote-paths local-path _opts] - (when (:dummy conn-spec) - (throw+ {:type :jepsen.control/dummy})) - (with-errors conn-spec ctx - (with-open [sftp (.newSFTPClient client)] - (.get sftp remote-paths (FileSystemFile. local-path)))))) - -(defn remote - "Constructs an SSHJ remote." - [] - (-> (SSHJRemote. concurrency-limit nil nil nil) - ; We *can* use our own SCP, but shelling out is faster. - scp/remote - retry/remote)) - -(ns jepsen.clickhouse-keeper.main +(ns jepsen.clickhouse.keeper.main (:require [clojure.tools.logging :refer :all] - [jepsen.clickhouse-keeper.utils :refer :all] [clojure.pprint :refer [pprint]] - [jepsen.clickhouse-keeper.set :as set] - [jepsen.clickhouse-keeper.db :refer :all] - [jepsen.clickhouse-keeper.zookeeperdb :refer :all] - [jepsen.clickhouse-keeper.nemesis :as custom-nemesis] - [jepsen.clickhouse-keeper.register :as register] - [jepsen.clickhouse-keeper.unique :as unique] - [jepsen.clickhouse-keeper.queue :as queue] - [jepsen.clickhouse-keeper.counter :as counter] - [jepsen.clickhouse-keeper.bench :as bench] - [jepsen.clickhouse-keeper.constants :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] + [jepsen.clickhouse.keeper.set :as set] + [jepsen.clickhouse.keeper.db :refer :all] + [jepsen.clickhouse.keeper.zookeeperdb :refer :all] + [jepsen.clickhouse.keeper.nemesis :as custom-nemesis] + [jepsen.clickhouse.keeper.register :as register] + [jepsen.clickhouse.keeper.unique :as unique] + [jepsen.clickhouse.keeper.queue :as queue] + [jepsen.clickhouse.keeper.counter :as counter] + [jepsen.clickhouse.keeper.bench :as bench] + [jepsen.clickhouse.keeper.constants :refer :all] [clojure.string :as str] [jepsen [checker :as checker] @@ -281,7 +186,7 @@ (map test-fn (all-test-options cli (cart [lightweight-workloads useful-nemesises]))) (map test-fn (all-test-options cli (cart [all-workloads all-nemesises]))))) -(defn -main +(defn main "Handles command line arguments. Can either run a test, or a web server for browsing results." [& args] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj similarity index 97% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/nemesis.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj index caf59d3a25f..1048d19e28f 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/nemesis.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj @@ -1,12 +1,12 @@ -(ns jepsen.clickhouse-keeper.nemesis +(ns jepsen.clickhouse.keeper.nemesis (:require [clojure.tools.logging :refer :all] [jepsen [nemesis :as nemesis] [control :as c] [generator :as gen]] - [jepsen.clickhouse-keeper.constants :refer :all] - [jepsen.clickhouse-keeper.utils :refer :all])) + [jepsen.clickhouse.keeper.constants :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all])) (defn random-node-killer-nemesis [] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/queue.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj similarity index 97% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/queue.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj index 1c306a0ff04..3c7c16de90e 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/queue.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj @@ -1,4 +1,4 @@ -(ns jepsen.clickhouse-keeper.queue +(ns jepsen.clickhouse.keeper.queue (:require [clojure.tools.logging :refer :all] [jepsen @@ -7,7 +7,7 @@ [generator :as gen]] [knossos.model :as model] [jepsen.checker.timeline :as timeline] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj similarity index 96% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj index a1605192b51..ab3c39be945 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/register.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj @@ -1,4 +1,4 @@ -(ns jepsen.clickhouse-keeper.register +(ns jepsen.clickhouse.keeper.register (:require [jepsen [checker :as checker] [client :as client] @@ -6,7 +6,7 @@ [generator :as gen]] [jepsen.checker.timeline :as timeline] [knossos.model :as model] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/set.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj similarity index 95% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/set.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj index b992a6abcbb..12450ab5ff6 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/set.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj @@ -1,11 +1,11 @@ -(ns jepsen.clickhouse-keeper.set +(ns jepsen.clickhouse.keeper.set (:require [clojure.tools.logging :refer :all] [jepsen [checker :as checker] [client :as client] [generator :as gen]] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/unique.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/unique.clj similarity index 93% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/unique.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/unique.clj index 752240722d8..89c5f9ccb3a 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/unique.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/unique.clj @@ -1,11 +1,11 @@ -(ns jepsen.clickhouse-keeper.unique +(ns jepsen.clickhouse.keeper.unique (:require [clojure.tools.logging :refer :all] [jepsen [checker :as checker] [client :as client] [generator :as gen]] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj similarity index 98% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj index 3625b24b4f9..bf1a82faf21 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/utils.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj @@ -1,10 +1,10 @@ -(ns jepsen.clickhouse-keeper.utils +(ns jepsen.clickhouse.keeper.utils (:require [clojure.string :as str] [zookeeper.data :as data] [zookeeper :as zk] [zookeeper.internal :as zi] [jepsen.control.util :as cu] - [jepsen.clickhouse-keeper.constants :refer :all] + [jepsen.clickhouse.keeper.constants :refer :all] [jepsen.control :as c] [clojure.tools.logging :refer :all] [clojure.java.io :as io]) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/zookeeperdb.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/zookeeperdb.clj similarity index 95% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/zookeeperdb.clj rename to tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/zookeeperdb.clj index 7cb88cd1fd9..6712b35fb24 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse_keeper/zookeeperdb.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/zookeeperdb.clj @@ -1,6 +1,6 @@ -(ns jepsen.clickhouse-keeper.zookeeperdb +(ns jepsen.clickhouse.keeper.zookeeperdb (:require [clojure.tools.logging :refer :all] - [jepsen.clickhouse-keeper.utils :refer :all] + [jepsen.clickhouse.keeper.utils :refer :all] [clojure.java.io :as io] [jepsen [control :as c] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj new file mode 100644 index 00000000000..74e47115b48 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj @@ -0,0 +1,105 @@ +(ns jepsen.control.sshj + (:require [jepsen.control [core :as core] + [sshj :as sshj]] + [slingshot.slingshot :refer [try+ throw+]]) + (:import (net.schmizz.sshj SSHClient + DefaultConfig) + (net.schmizz.sshj.transport.verification PromiscuousVerifier) + (java.util.concurrent Semaphore))) + +(defrecord SSHJRemote [concurrency-limit + conn-spec + ^SSHClient client + ^Semaphore semaphore] + core/Remote + (connect [this conn-spec] + (if (:dummy conn-spec) + (assoc this :conn-spec conn-spec) + (try+ (let [c (as-> (SSHClient.) client + (do + (if (:strict-host-key-checking conn-spec) + (.loadKnownHosts client) + (.addHostKeyVerifier client (PromiscuousVerifier.))) + (.connect client (:host conn-spec) (:port conn-spec)) + (auth! client conn-spec) + client))] + (assoc this + :conn-spec conn-spec + :client c + :semaphore (Semaphore. concurrency-limit true))) + (catch Exception e + ; SSHJ wraps InterruptedException in its own exceptions, so we + ; have to see through that and rethrow properly. + (let [cause (util/ex-root-cause e)] + (when (instance? InterruptedException cause) + (throw cause))) + (throw+ (assoc conn-spec + :type :jepsen.control/session-error + :message "Error opening SSH session. Verify username, password, and node hostnames are correct.")))))) + + (disconnect! [this] + (when-let [c client] + (.close c))) + + (execute! [this ctx action] + ; (info :permits (.availablePermits semaphore)) + (when (:dummy conn-spec) + (throw+ {:type :jepsen.control/dummy})) + (.acquire semaphore) + (sshj/with-errors conn-spec ctx + (try + (with-open [session (.startSession client)] + (let [cmd (.exec session (:cmd action)) + ; Feed it input + _ (when-let [input (:in action)] + (let [stream (.getOutputStream cmd)] + (bs/transfer input stream) + (send-eof! client session) + (.close stream))) + ; Read output + out (.toString (IOUtils/readFully (.getInputStream cmd))) + err (.toString (IOUtils/readFully (.getErrorStream cmd))) + ; Wait on command + _ (.join cmd)] + ; Return completion + (assoc action + :out out + :err err + ; There's also a .getExitErrorMessage that might be + ; interesting here? + :exit (.getExitStatus cmd)))) + (finally + (.release semaphore))))) + + (upload! [this ctx local-paths remote-path _opts] + (when (:dummy conn-spec) + (throw+ {:type :jepsen.control/dummy})) + (with-errors conn-spec ctx + (with-open [sftp (.newSFTPClient client)] + (.put sftp (FileSystemFile. local-paths) remote-path)))) + + (download! [this ctx remote-paths local-path _opts] + (when (:dummy conn-spec) + (throw+ {:type :jepsen.control/dummy})) + (with-errors conn-spec ctx + (with-open [sftp (.newSFTPClient client)] + (.get sftp remote-paths (FileSystemFile. local-path)))))) + +(defn remote + "Constructs an SSHJ remote." + [] + (-> (SSHJRemote. concurrency-limit nil nil nil) + ; We *can* use our own SCP, but shelling out is faster. + scp/remote + retry/remote)) + +(ns jepsen.clickhouse.main + (:require [jepsen.clickhouse.keeper.main] + [jepsen.clickhouse.server.main])) + +(defn -main + [product & args] + (case product + "keeper" (apply jepsen.clickhouse.keeper.main/main args) + "server" (apply jepsen.clickhouse.server.main/main args) + (throw (Exception. (str "Unknown option specified: " product))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj new file mode 100644 index 00000000000..1b7ba0190d0 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj @@ -0,0 +1,7 @@ +(ns jepsen.clickhouse.server.main) + +(defn main + "Handles command line arguments. Can either run a test, or a web server for + browsing results." + [& args] + (print "Not yet implemented" args)) From 5336addd0f5d82db9578b1401f6b4c7e344e7aad Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 24 Oct 2022 09:45:24 +0000 Subject: [PATCH 064/526] Disable fault injections by default + enable fault injection for functional tests --- src/Core/Settings.h | 2 +- tests/clickhouse-test | 2 -- tests/config/install.sh | 1 + tests/config/users.d/insert_keeper_retries.xml | 7 +++++++ 4 files changed, 9 insertions(+), 3 deletions(-) create mode 100644 tests/config/users.d/insert_keeper_retries.xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 87814a7aff8..1fb9c2c0ade 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -659,7 +659,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, insert_keeper_max_retries, 20, "Max retries for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ - M(Float, insert_keeper_fault_injection_probability, 0.1f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ + M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 3e58eef9e78..20e63412d91 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,8 +456,6 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "insert_keeper_fault_injection_probability": lambda: round(random.uniform(0.0, 0.1),2), - "insert_keeper_fault_injection_seed": lambda: random.randint(1, 0xFFFFFFFFFFFFFFFF), } @staticmethod diff --git a/tests/config/install.sh b/tests/config/install.sh index 9a172f0be15..dce38b23668 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -70,6 +70,7 @@ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml new file mode 100644 index 00000000000..e4420d53fb0 --- /dev/null +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -0,0 +1,7 @@ + + + + 0.01 + + + From 93d48a24032e5e0f4036a0d96a5293e65f898e0b Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 24 Oct 2022 11:04:45 +0000 Subject: [PATCH 065/526] add functions for powerBI connection --- src/Functions/UTCTimestamp.cpp | 117 +++++++++++++++++ src/Functions/timeDiff.cpp | 228 +++++++++++++++++++++++++++++++++ 2 files changed, 345 insertions(+) create mode 100644 src/Functions/UTCTimestamp.cpp create mode 100644 src/Functions/timeDiff.cpp diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp new file mode 100644 index 00000000000..9b114754061 --- /dev/null +++ b/src/Functions/UTCTimestamp.cpp @@ -0,0 +1,117 @@ +#include + +#include +#include +#include +#include + +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +namespace +{ + +/// Get the UTC time. (It is a constant, it is evaluated once for the entire query.) +class ExecutableFunctionUTCTimestamp : public IExecutableFunction +{ +public: + explicit ExecutableFunctionUTCTimestamp(time_t time_) : time_value(time_) {} + + String getName() const override { return "UTCTimestamp"; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeDateTime().createColumnConst( + input_rows_count, + static_cast(time_value)); + } + +private: + time_t time_value; +}; + +class FunctionBaseUTCTimestamp : public IFunctionBase +{ +public: + explicit FunctionBaseUTCTimestamp(time_t time_, DataTypes argument_types_, DataTypePtr return_type_) + : time_value(time_), argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} + + String getName() const override { return "UTCTimestamp"; } + + const DataTypes & getArgumentTypes() const override + { + return argument_types; + } + + const DataTypePtr & getResultType() const override + { + return return_type; + } + + ExecutableFunctionPtr prepare(const ColumnsWithTypeAndName &) const override + { + return std::make_unique(time_value); + } + + bool isDeterministic() const override { return false; } + bool isDeterministicInScopeOfQuery() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + +private: + time_t time_value; + DataTypes argument_types; + DataTypePtr return_type; +}; + +class UTCTimestampOverloadResolver : public IFunctionOverloadResolver +{ +public: + static constexpr auto name = "UTCTimestamp"; + + String getName() const override { return name; } + + bool isDeterministic() const override { return false; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (arguments.size() > 0) + { + throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return std::make_shared(); + } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override + { + if (arguments.size() > 0) + { + throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + + return std::make_unique(time(nullptr), DataTypes(), std::make_shared("UTC")); + } +}; + +} + +/// UTC_timestamp for MySQL interface support +REGISTER_FUNCTION(UTCTimestamp) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive); +} + +} diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp new file mode 100644 index 00000000000..9081351998f --- /dev/null +++ b/src/Functions/timeDiff.cpp @@ -0,0 +1,228 @@ +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int ILLEGAL_COLUMN; +} + +namespace +{ + +/** TimeDiff(t1, t2) + * t1 and t2 can be Date or DateTime + */ +class FunctionTimeDiff : public IFunction +{ + using ColumnDateTime64 = ColumnDecimal; +public: + static constexpr auto name = "TimeDiff"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override + { + return name; + } + + bool isVariadic() const override { return true; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) + throw Exception("First argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) + throw Exception("Second argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + const IColumn & x = *arguments[0].column; + const IColumn & y = *arguments[1].column; + + size_t rows = input_rows_count; + auto res = ColumnInt64::create(rows); + + dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); + + return res; + } + +private: + template + void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + } + + template + void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); + + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); + } + + template + void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); + + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); + } + + template + void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); + + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); + } + + template + Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const + { + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); + } + + template + static UInt32 getScale(const T & v) + { + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; + } + template + static auto stripDecimalFieldValue(T && v) + { + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; + } +}; + +} + +REGISTER_FUNCTION(TimeDiff) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + +} From e0c87b2ae2c6f2eef84a9b4cdf642b77a1fd47ea Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 24 Oct 2022 12:53:10 +0000 Subject: [PATCH 066/526] small fix --- src/Functions/UTCTimestamp.cpp | 2 +- src/Functions/timeDiff.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 9b114754061..218c7fa5c1c 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -79,7 +79,7 @@ public: bool isDeterministic() const override { return false; } - bool isVariadic() const override { return true; } + bool isVariadic() const override { return false; } size_t getNumberOfArguments() const override { return 0; } static FunctionOverloadResolverPtr create(ContextPtr) { return std::make_unique(); } diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp index 9081351998f..0353dbf7c51 100644 --- a/src/Functions/timeDiff.cpp +++ b/src/Functions/timeDiff.cpp @@ -47,9 +47,9 @@ public: return name; } - bool isVariadic() const override { return true; } + bool isVariadic() const override { return false; } bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 0; } + size_t getNumberOfArguments() const override { return 2; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { From 5bc9be8a5f21f42b05f97e30deb13d17597e062f Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 24 Oct 2022 11:24:33 +0000 Subject: [PATCH 067/526] Start defining tests for server --- docker/test/keeper-jepsen/run.sh | 2 +- tests/jepsen.clickhouse-keeper/project.clj | 2 + .../resources/config.xml | 18 ++ .../resources/users.xml | 110 ++++++++++ .../src/jepsen/clickhouse/constants.clj | 21 ++ .../jepsen/clickhouse/keeper/constants.clj | 20 -- .../src/jepsen/clickhouse/keeper/counter.clj | 5 +- .../src/jepsen/clickhouse/keeper/db.clj | 124 +---------- .../src/jepsen/clickhouse/keeper/main.clj | 5 +- .../src/jepsen/clickhouse/keeper/nemesis.clj | 9 +- .../src/jepsen/clickhouse/keeper/queue.clj | 5 +- .../src/jepsen/clickhouse/keeper/register.clj | 3 +- .../src/jepsen/clickhouse/keeper/set.clj | 5 +- .../src/jepsen/clickhouse/keeper/utils.clj | 81 +------- .../src/jepsen/clickhouse/main.clj | 11 +- .../src/jepsen/clickhouse/server/client.clj | 49 +++++ .../src/jepsen/clickhouse/server/db.clj | 45 ++++ .../src/jepsen/clickhouse/server/main.clj | 94 ++++++++- .../src/jepsen/clickhouse/server/register.clj | 56 +++++ .../src/jepsen/clickhouse/server/utils.clj | 27 +++ .../src/jepsen/clickhouse/utils.clj | 194 ++++++++++++++++++ 21 files changed, 658 insertions(+), 228 deletions(-) create mode 100644 tests/jepsen.clickhouse-keeper/resources/config.xml create mode 100644 tests/jepsen.clickhouse-keeper/resources/users.xml create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj delete mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj diff --git a/docker/test/keeper-jepsen/run.sh b/docker/test/keeper-jepsen/run.sh index adf99c029a9..220f1cf7459 100644 --- a/docker/test/keeper-jepsen/run.sh +++ b/docker/test/keeper-jepsen/run.sh @@ -17,6 +17,6 @@ fi cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse-keeper" -(lein run test-all --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --snapshot-distance 100 --stale-log-gap 100 --reserved-log-items 10 --lightweight-run --clickhouse-source "$CLICKHOUSE_PACKAGE" -q --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log" +(lein run keeper test-all --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --snapshot-distance 100 --stale-log-gap 100 --reserved-log-items 10 --lightweight-run --clickhouse-source "$CLICKHOUSE_PACKAGE" -q --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log" mv store "$TEST_OUTPUT/" diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse-keeper/project.clj index 1f4a762753b..6c714604b56 100644 --- a/tests/jepsen.clickhouse-keeper/project.clj +++ b/tests/jepsen.clickhouse-keeper/project.clj @@ -9,6 +9,8 @@ :dependencies [[org.clojure/clojure "1.10.1"] [jepsen "0.2.7"] [zookeeper-clj "0.9.4"] + [org.clojure/java.jdbc "0.7.12"] [com.hierynomus/sshj "0.34.0"] + [com.clickhouse/clickhouse-jdbc "0.3.2-patch11"] [org.apache.zookeeper/zookeeper "3.6.1" :exclusions [org.slf4j/slf4j-log4j12]]] :repl-options {:init-ns jepsen.clickhouse-keeper.main}) diff --git a/tests/jepsen.clickhouse-keeper/resources/config.xml b/tests/jepsen.clickhouse-keeper/resources/config.xml new file mode 100644 index 00000000000..a0b1b33b26d --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/resources/config.xml @@ -0,0 +1,18 @@ + + 0.0.0.0 + 9000 + 8123 + + + trace + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + never + + + + + users.xml + + + diff --git a/tests/jepsen.clickhouse-keeper/resources/users.xml b/tests/jepsen.clickhouse-keeper/resources/users.xml new file mode 100644 index 00000000000..5e2ff51bf4d --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/resources/users.xml @@ -0,0 +1,110 @@ + + + + + + + + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj new file mode 100644 index 00000000000..887fc04265c --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj @@ -0,0 +1,21 @@ +(ns jepsen.clickhouse.constants) + +(def root-folder "/home/robot-clickhouse") + +(def binary-name "clickhouse") + +(def binary-path (str root-folder "/" binary-name)) +(def pid-file-path (str root-folder "/clickhouse.pid")) + +(def data-dir (str root-folder "/db")) +(def logs-dir (str root-folder "/logs")) +(def configs-dir (str root-folder "/config")) +(def sub-configs-dir (str configs-dir "/config.d")) + +(def coordination-data-dir (str data-dir "/coordination")) +(def coordination-snapshots-dir (str coordination-data-dir "/snapshots")) +(def coordination-logs-dir (str coordination-data-dir "/logs")) + +(def stderr-file (str logs-dir "/stderr.log")) + +(def binaries-cache-dir (str root-folder "/binaries")) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj deleted file mode 100644 index 0c0116deef3..00000000000 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/constants.clj +++ /dev/null @@ -1,20 +0,0 @@ -(ns jepsen.clickhouse.keeper.constants) - -(def common-prefix "/home/robot-clickhouse") - -(def binary-name "clickhouse") - -(def binary-path (str common-prefix "/" binary-name)) -(def pid-file-path (str common-prefix "/clickhouse.pid")) - -(def data-dir (str common-prefix "/db")) -(def logs-dir (str common-prefix "/logs")) -(def configs-dir (str common-prefix "/config")) -(def sub-configs-dir (str configs-dir "/config.d")) -(def coordination-data-dir (str data-dir "/coordination")) -(def coordination-snapshots-dir (str coordination-data-dir "/snapshots")) -(def coordination-logs-dir (str coordination-data-dir "/logs")) - -(def stderr-file (str logs-dir "/stderr.log")) - -(def binaries-cache-dir (str common-prefix "/binaries")) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj index 3abb4843287..0e2e97670f9 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj @@ -6,6 +6,7 @@ [client :as client] [generator :as gen]] [jepsen.clickhouse.keeper.utils :refer :all] + [jepsen.clickhouse.utils :as chu] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) @@ -22,12 +23,12 @@ :nodename node)) (setup! [this test] - (exec-with-retries 30 (fn [] + (chu/exec-with-retries 30 (fn [] (zk-create-if-not-exists conn root-path "")))) (invoke! [this test op] (case (:f op) - :read (exec-with-retries 30 (fn [] + :read (chu/exec-with-retries 30 (fn [] (assoc op :type :ok :value (count (zk-list conn root-path))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj index 861bb434957..41344d2c0cf 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj @@ -2,10 +2,10 @@ (:require [clojure.tools.logging :refer :all] [jepsen [control :as c] - [db :as db] [util :as util :refer [meh]]] - [jepsen.clickhouse.keeper.constants :refer :all] + [jepsen.clickhouse.constants :refer :all] [jepsen.clickhouse.keeper.utils :refer :all] + [jepsen.clickhouse.utils :as chu] [clojure.java.io :as io] [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu])) @@ -32,65 +32,6 @@ (ns jepsen.clickhouse.keeper.db) -(defn get-clickhouse-url - [url] - (non-precise-cached-wget! url)) - -(defn get-clickhouse-scp - [path] - (c/upload path (str common-prefix "/clickhouse"))) - -(defn download-clickhouse - [source] - (info "Downloading clickhouse from" source) - (cond - (clojure.string/starts-with? source "http") (get-clickhouse-url source) - (.exists (io/file source)) (get-clickhouse-scp source) - :else (throw (Exception. (str "Don't know how to download clickhouse from" source))))) - -(defn unpack-deb - [path] - (do - (c/exec :dpkg :-x path common-prefix) - (c/exec :rm :-f path) - (c/exec :mv (str common-prefix "/usr/bin/clickhouse") common-prefix) - (c/exec :rm :-rf (str common-prefix "/usr") (str common-prefix "/etc")))) - -(defn unpack-tgz - [path] - (do - (c/exec :mkdir :-p (str common-prefix "/unpacked")) - (c/exec :tar :-zxvf path :-C (str common-prefix "/unpacked")) - (c/exec :rm :-f path) - (let [subdir (c/exec :ls (str common-prefix "/unpacked"))] - (c/exec :mv (str common-prefix "/unpacked/" subdir "/usr/bin/clickhouse") common-prefix) - (c/exec :rm :-fr (str common-prefix "/unpacked"))))) - -(defn chmod-binary - [path] - (info "Binary path chmod" path) - (c/exec :chmod :+x path)) - -(defn install-downloaded-clickhouse - [path] - (cond - (clojure.string/ends-with? path ".deb") (unpack-deb path) - (clojure.string/ends-with? path ".tgz") (unpack-tgz path) - (clojure.string/ends-with? path "clickhouse") (chmod-binary path) - :else (throw (Exception. (str "Don't know how to install clickhouse from path" path))))) - -(defn prepare-dirs - [] - (do - (c/exec :mkdir :-p common-prefix) - (c/exec :mkdir :-p data-dir) - (c/exec :mkdir :-p coordination-data-dir) - (c/exec :mkdir :-p logs-dir) - (c/exec :mkdir :-p configs-dir) - (c/exec :mkdir :-p sub-configs-dir) - (c/exec :touch stderr-file) - (c/exec :chown :-R :root common-prefix))) - (defn cluster-config [test node config-template] (let [nodes (:nodes test) @@ -108,63 +49,12 @@ [test node] (c/exec :echo (cluster-config test node (slurp (io/resource "keeper_config.xml"))) :> (str configs-dir "/keeper_config.xml"))) -(defn collect-traces +(defn extra-setup [test node] - (let [pid (c/exec :pidof "clickhouse")] - (c/exec :timeout :-s "KILL" "60" :gdb :-ex "set pagination off" :-ex (str "set logging file " logs-dir "/gdb.log") :-ex - "set logging on" :-ex "backtrace" :-ex "thread apply all backtrace" - :-ex "backtrace" :-ex "detach" :-ex "quit" :--pid pid :|| :true))) + (do + (info "Installing configs") + (install-configs test node))) (defn db [version reuse-binary] - (reify db/DB - (setup! [_ test node] - (c/su - (do - (info "Preparing directories") - (prepare-dirs) - (if (or (not (cu/exists? binary-path)) (not reuse-binary)) - (do (info "Downloading clickhouse") - (install-downloaded-clickhouse (download-clickhouse version))) - (info "Binary already exsist on path" binary-path "skipping download")) - (info "Installing configs") - (install-configs test node) - (info "Starting server") - (start-clickhouse! node test) - (info "ClickHouse started")))) - - (teardown! [_ test node] - (info node "Tearing down clickhouse") - (c/su - (kill-clickhouse! node test) - (if (not reuse-binary) - (c/exec :rm :-rf binary-path)) - (c/exec :rm :-rf pid-file-path) - (c/exec :rm :-rf data-dir) - (c/exec :rm :-rf logs-dir) - (c/exec :rm :-rf configs-dir))) - - db/LogFiles - (log-files [_ test node] - (c/su - ;(if (cu/exists? pid-file-path) - ;(do - ; (info node "Collecting traces") - ; (collect-traces test node)) - ;(info node "Pid files doesn't exists")) - (kill-clickhouse! node test) - (if (cu/exists? coordination-data-dir) - (do - (info node "Coordination files exists, going to compress") - (c/cd data-dir - (c/exec :tar :czf "coordination.tar.gz" "coordination")))) - (if (cu/exists? (str logs-dir)) - (do - (info node "Logs exist, going to compress") - (c/cd common-prefix - (c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing"))) - (let [common-logs [(str common-prefix "/logs.tar.gz") (str data-dir "/coordination.tar.gz")] - gdb-log (str logs-dir "/gdb.log")] - (if (cu/exists? (str logs-dir "/gdb.log")) - (conj common-logs gdb-log) - common-logs))))) + (chu/db version reuse-binary start-clickhouse! extra-setup)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj index efbceb314d7..87f437d3d4f 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj @@ -11,7 +11,8 @@ [jepsen.clickhouse.keeper.queue :as queue] [jepsen.clickhouse.keeper.counter :as counter] [jepsen.clickhouse.keeper.bench :as bench] - [jepsen.clickhouse.keeper.constants :refer :all] + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.utils :as chu] [clojure.string :as str] [jepsen [checker :as checker] @@ -69,7 +70,7 @@ :validate [#(and (number? %) (pos? %)) "Must be a positive number"]] [nil "--ops-per-key NUM" "Maximum number of operations on any given key." :default 100 - :parse-fn parse-long + :parse-fn chu/parse-long :validate [pos? "Must be a positive integer."]] [nil, "--lightweight-run" "Subset of workloads/nemesises which is simple to validate"] [nil, "--reuse-binary" "Use already downloaded binary if it exists, don't remove it on shutdown"] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj index 1048d19e28f..2505a4b8463 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj @@ -5,21 +5,22 @@ [nemesis :as nemesis] [control :as c] [generator :as gen]] - [jepsen.clickhouse.keeper.constants :refer :all] + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.utils :as chu] [jepsen.clickhouse.keeper.utils :refer :all])) (defn random-node-killer-nemesis [] (nemesis/node-start-stopper rand-nth - (fn start [test node] (kill-clickhouse! node test)) + (fn start [test node] (chu/kill-clickhouse! node test)) (fn stop [test node] (start-clickhouse! node test)))) (defn all-nodes-killer-nemesis [] (nemesis/node-start-stopper identity - (fn start [test node] (kill-clickhouse! node test)) + (fn start [test node] (chu/kill-clickhouse! node test)) (fn stop [test node] (start-clickhouse! node test)))) (defn random-node-hammer-time-nemesis @@ -62,7 +63,7 @@ (c/on-nodes test nodes (fn [test node] (c/su - (kill-clickhouse! node test) + (chu/kill-clickhouse! node test) (corruption-op path) (start-clickhouse! node test)))) (assoc op :type :info, :value :corrupted)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj index 3c7c16de90e..866bd0b3e4e 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj @@ -8,6 +8,7 @@ [knossos.model :as model] [jepsen.checker.timeline :as timeline] [jepsen.clickhouse.keeper.utils :refer :all] + [jepsen.clickhouse.utils :as chu] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) @@ -24,7 +25,7 @@ :nodename node)) (setup! [this test] - (exec-with-retries 30 (fn [] + (chu/exec-with-retries 30 (fn [] (zk-create-if-not-exists conn root-path "")))) (invoke! [this test op] @@ -43,7 +44,7 @@ (catch Exception _ (assoc op :type :info, :error :connect-error))) :drain ; drain via delete is to long, just list all nodes - (exec-with-retries 30 (fn [] + (chu/exec-with-retries 30 (fn [] (zk-sync conn) (assoc op :type :ok :value (into #{} (map #(str %1) (zk-list conn root-path)))))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj index ab3c39be945..72cf113a9c3 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj @@ -5,6 +5,7 @@ [independent :as independent] [generator :as gen]] [jepsen.checker.timeline :as timeline] + [jepsen.clickhouse.utils :as chu] [knossos.model :as model] [jepsen.clickhouse.keeper.utils :refer :all] [zookeeper :as zk]) @@ -27,7 +28,7 @@ zk-k (zk-path k)] (case (:f op) :read (try - (assoc op :type :ok, :value (independent/tuple k (parse-long (:data (zk-get-str conn zk-k))))) + (assoc op :type :ok, :value (independent/tuple k (chu/parse-long (:data (zk-get-str conn zk-k))))) (catch Exception _ (assoc op :type :fail, :error :connect-error))) :write (try (do (zk-set conn zk-k v) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj index 12450ab5ff6..75da7497c67 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj @@ -6,6 +6,7 @@ [client :as client] [generator :as gen]] [jepsen.clickhouse.keeper.utils :refer :all] + [jepsen.clickhouse.utils :as chu] [zookeeper :as zk]) (:import (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) @@ -18,12 +19,12 @@ :nodename node)) (setup! [this test] - (exec-with-retries 30 (fn [] + (chu/exec-with-retries 30 (fn [] (zk-create-if-not-exists conn k "#{}")))) (invoke! [this test op] (case (:f op) - :read (exec-with-retries 30 (fn [] + :read (chu/exec-with-retries 30 (fn [] (zk-sync conn) (assoc op :type :ok diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj index bf1a82faf21..6767c144e01 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj @@ -3,33 +3,13 @@ [zookeeper.data :as data] [zookeeper :as zk] [zookeeper.internal :as zi] - [jepsen.control.util :as cu] - [jepsen.clickhouse.keeper.constants :refer :all] - [jepsen.control :as c] - [clojure.tools.logging :refer :all] - [clojure.java.io :as io]) + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.utils :as chu] + [clojure.tools.logging :refer :all]) (:import (org.apache.zookeeper.data Stat) (org.apache.zookeeper CreateMode ZooKeeper) - (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException) - (java.security MessageDigest))) - -(defn exec-with-retries - [retries f & args] - (let [res (try {:value (apply f args)} - (catch Exception e - (if (zero? retries) - (throw e) - {:exception e})))] - (if (:exception res) - (do (Thread/sleep 1000) (recur (dec retries) f args)) - (:value res)))) - -(defn parse-long - "Parses a string to a Long. Passes through `nil` and empty strings." - [s] - (if (and s (> (count s) 0)) - (Long/parseLong s))) + (org.apache.zookeeper ZooKeeper KeeperException KeeperException$BadVersionException))) (defn parse-and-get-counter [path] @@ -45,7 +25,7 @@ (defn zk-connect [host port timeout] - (exec-with-retries 30 (fn [] (zk/connect (str host ":" port) :timeout-msec timeout)))) + (chu/exec-with-retries 30 (fn [] (zk/connect (str host ":" port) :timeout-msec timeout)))) (defn zk-create-range [conn n] @@ -77,7 +57,7 @@ (defn zk-cas [conn path old-value new-value] (let [current-value (zk-get-str conn path)] - (if (= (parse-long (:data current-value)) old-value) + (if (= (chu/parse-long (:data current-value)) old-value) (do (zk-set conn path new-value (:version (:stat current-value))) true)))) @@ -154,29 +134,13 @@ (zk-connect (name node) 9181 30000) (catch Exception _ false))) -(defn wait-clickhouse-alive! - [node test & {:keys [maxtries] :or {maxtries 30}}] - (loop [i 0] - (cond (> i maxtries) false - (clickhouse-alive? node test) true - :else (do (Thread/sleep 1000) (recur (inc i)))))) - -(defn kill-clickhouse! - [node test] - (info "Killing server on node" node) - (c/su - (cu/stop-daemon! binary-path pid-file-path) - (c/exec :rm :-fr (str data-dir "/status")))) - (defn start-clickhouse! [node test] (info "Starting server on node" node) - (c/su - (cu/start-daemon! - {:pidfile pid-file-path - :logfile stderr-file - :chdir data-dir} - binary-path + (chu/start-clickhouse! + node + test + clickhouse-alive? :keeper :--config (str configs-dir "/keeper_config.xml") :-- @@ -184,27 +148,4 @@ :--logger.errorlog (str logs-dir "/clickhouse-keeper.err.log") :--keeper_server.snapshot_storage_path coordination-snapshots-dir :--keeper_server.log_storage_path coordination-logs-dir - :--path coordination-data-dir) - (wait-clickhouse-alive! node test))) - -(defn md5 [^String s] - (let [algorithm (MessageDigest/getInstance "MD5") - raw (.digest algorithm (.getBytes s))] - (format "%032x" (BigInteger. 1 raw)))) - -(defn non-precise-cached-wget! - [url] - (let [encoded-url (md5 url) - expected-file-name (.getName (io/file url)) - dest-folder (str binaries-cache-dir "/" encoded-url) - dest-file (str dest-folder "/clickhouse") - dest-symlink (str common-prefix "/" expected-file-name) - wget-opts (concat cu/std-wget-opts [:-O dest-file])] - (when-not (cu/exists? dest-file) - (info "Downloading" url) - (do (c/exec :mkdir :-p dest-folder) - (c/cd dest-folder - (cu/wget-helper! wget-opts url)))) - (c/exec :rm :-rf dest-symlink) - (c/exec :ln :-s dest-file dest-symlink) - dest-symlink)) + :--path coordination-data-dir)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj index 74e47115b48..5e6ed5d53c0 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj @@ -98,8 +98,9 @@ [jepsen.clickhouse.server.main])) (defn -main - [product & args] - (case product - "keeper" (apply jepsen.clickhouse.keeper.main/main args) - "server" (apply jepsen.clickhouse.server.main/main args) - (throw (Exception. (str "Unknown option specified: " product))))) + [f & args] + (cond + (= f "keeper") (apply jepsen.clickhouse.keeper.main/main args) + (= f "server") (apply jepsen.clickhouse.server.main/main args) + (some #(= f %) ["test" "test-all"]) (apply jepsen.clickhouse.keeper.main/main f args) ;; backwards compatibility + :unknown (throw (Exception. (str "Unknown option specified: " f))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj new file mode 100644 index 00000000000..a1e3b6d9687 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj @@ -0,0 +1,49 @@ +(ns jepsen.clickhouse.server.client + (:require [clojure.java.jdbc :as j] + [jepsen.reconnect :as rc])) + +(defn db-spec + [node] + {:dbtype "clickhouse" + :dbname "default" + :classname "com.clickhouse.ClickhouseDriver" + :host (name node) + :port 8123 + :jdbcCompliant false}) + +(defn open-connection + [node] + (let [spec (db-spec node) + connection (j/get-connection spec) + added-connection (j/add-connection spec connection)] + (assert added-connection) + added-connection)) + +(defn close-connection + "Close connection" + [connection] + (when-let [c (j/db-find-connection connection)] + (.close c)) + (dissoc connection :connection)) + +(defn client + "Client JDBC" + [node] + (rc/open! + (rc/wrapper + {:name (name node) + :open #(open-connection node) + :close close-connection + :log? true}))) + +(defmacro with-connection + "Like jepsen.reconnect/with-conn, but also asserts that the connection has + not been closed. If it has, throws an ex-info with :type :conn-not-ready. + Delays by 1 second to allow time for the DB to recover." + [[c client] & body] + `(rc/with-conn [~c ~client] + (when (.isClosed (j/db-find-connection ~c)) + (Thread/sleep 1000) + (throw (ex-info "Connection not yet ready." + {:type :conn-not-ready}))) + ~@body)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj new file mode 100644 index 00000000000..9e91f3e1354 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj @@ -0,0 +1,45 @@ +(ns jepsen.control.scp) + +;; We need to overwrite Jepsen's implementation of scp! because it +;; doesn't use strict-host-key-checking + +(defn scp! + "Runs an SCP command by shelling out. Takes a conn-spec (used for port, key, + etc), a seq of sources, and a single destination, all as strings." + [conn-spec sources dest] + (apply util/sh "scp" "-rpC" + "-P" (str (:port conn-spec)) + (concat (when-let [k (:private-key-path conn-spec)] + ["-i" k]) + (if-not (:strict-host-key-checking conn-spec) + ["-o StrictHostKeyChecking=no"]) + sources + [dest])) + nil) + +(ns jepsen.clickhouse.keeper.db) + +(ns jepsen.clickhouse.server.db + (:require [clojure.tools.logging :refer :all] + [clojure.java.io :as io] + [jepsen + [control :as c] + [db :as db]] + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.server.utils :refer :all] + [jepsen.clickhouse.utils :as chu])) + +(defn install-configs + [test node] + (c/exec :echo (slurp (io/resource "config.xml")) :> (str configs-dir "/config.xml")) + (c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml"))) + +(defn extra-setup + [test node] + (do + (info "Installing configs") + (install-configs test node))) + +(defn db + [version reuse-binary] + (chu/db version reuse-binary start-clickhouse! extra-setup)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj index 1b7ba0190d0..b4e66443c1a 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj @@ -1,7 +1,97 @@ -(ns jepsen.clickhouse.server.main) +(ns jepsen.clickhouse.server.main + (:require [clojure.tools.logging :refer :all] + [clojure.pprint :refer [pprint]] + [clojure.string :as str] + [jepsen + [checker :as checker] + [cli :as cli] + [generator :as gen] + [tests :as tests] + [util :as util :refer [meh]]] + [jepsen.clickhouse.server.db :refer :all] + [jepsen.clickhouse.server.register :as register] + [jepsen.control.util :as cu] + [jepsen.os.ubuntu :as ubuntu] + [jepsen.checker.timeline :as timeline] + [clojure.java.io :as io])) + +(def workloads + "A map of workload names to functions that construct workloads, given opts." + {"register" register/workload}) + +(def custom-nemesises + {}) + +(def cli-opts + "Additional command line options." + [["-w" "--workload NAME" "What workload should we run?" + :default "set" + :validate [workloads (cli/one-of workloads)]] + ;[nil "--nemesis NAME" "Which nemesis will poison our lives?" + ; :default "random-node-killer" + ; :validate [custom-nemesises (cli/one-of custom-nemesises)]] + ["-r" "--rate HZ" "Approximate number of requests per second, per thread." + :default 10 + :parse-fn read-string + :validate [#(and (number? %) (pos? %)) "Must be a positive number"]] + [nil, "--reuse-binary" "Use already downloaded binary if it exists, don't remove it on shutdown"] + ["-c" "--clickhouse-source URL" "URL for clickhouse deb or tgz package"]]) + +(defn get-db + [opts] + (db (:clickhouse-source opts) (boolean (:reuse-binary opts)))) + +(defn clickhouse-func-tests + [opts] + (info "Test opts\n" (with-out-str (pprint opts))) + (let [quorum (boolean (:quorum opts)) + workload ((get workloads (:workload opts)) opts)] + (merge tests/noop-test + opts + {:name (str "clickhouse-server-" (name (:workload opts))) + :os ubuntu/os + :db (get-db opts) + :pure-generators true + :client (:client workload) + :checker (checker/compose + {:perf (checker/perf) + :workload (:checker workload)}) + :generator (gen/phases + (->> (:generator workload) + (gen/stagger (/ (:rate opts))) + (gen/time-limit (:time-limit opts))) + (gen/clients (:final-generator workload)))}))) + +(defn clickhouse-server-test + "Given an options map from the command line runner (e.g. :nodes, :ssh, + :concurrency, ...), constructs a test map." + [opts] + (clickhouse-func-tests opts)) + +(def all-workloads (keys workloads)) + +(defn all-test-options + "Takes base cli options, a collection of nemeses, workloads, and a test count, + and constructs a sequence of test options." + [cli] + (take (:test-count cli) + (shuffle (for [[workload] all-workloads] + (assoc cli + :workload workload + :test-count 1))))) + +(defn all-tests + "Turns CLI options into a sequence of tests." + [test-fn cli] + (map test-fn (all-test-options cli))) (defn main "Handles command line arguments. Can either run a test, or a web server for browsing results." [& args] - (print "Not yet implemented" args)) + (cli/run! (merge (cli/single-test-cmd {:test-fn clickhouse-server-test + :opt-spec cli-opts}) + (cli/test-all-cmd {:tests-fn (partial all-tests clickhouse-server-test) + :opt-spec cli-opts}) + (cli/serve-cmd)) + args)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj new file mode 100644 index 00000000000..6789169db86 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj @@ -0,0 +1,56 @@ +(ns jepsen.clickhouse.server.register + (:require [jepsen + [checker :as checker] + [client :as client] + [independent :as independent] + [generator :as gen]] + [jepsen.checker.timeline :as timeline] + [jepsen.clickhouse.utils :as chu] + [jepsen.clickhouse.server.client :as chc] + [clojure.tools.logging :refer :all] + [clojure.java.jdbc :as j] + [knossos.model :as model])) + +(defn r [_ _] {:type :invoke, :f :read, :value nil}) +(defn w [_ _] {:type :invoke, :f :write, :value (rand-int 5)}) +(defn cas [_ _] {:type :invoke, :f :cas, :value [(rand-int 5) (rand-int 5)]}) + +(defrecord RegisterClient [table-created? conn] + client/Client + (open! [this test node] + (assoc this :conn (chc/client node))) + + (setup! [this test] + (locking table-created? + (when (compare-and-set! table-created? false true) + (chc/with-connection [c conn] + (j/query c "DROP TABLE IF EXISTS register") + (j/query c "CREATE TABLE register (id Int64, value Int64) ENGINE=MergeTree ORDER BY id") + (info (j/query c "SHOW CREATE TABLE register")))))) + + (invoke! [_ test op] + (print "invoke")) + + (teardown! [this test] + nil) + + (close! [_ test] + (print "close"))) + +(defn workload + "Tests linearizable reads, writes, and compare-and-set operations on + independent keys." + [opts] + {:client (RegisterClient. (atom false) nil) + :checker (independent/checker + (checker/compose + {:linear (checker/linearizable {:model (model/cas-register) + :algorithm :linear}) + :perf (checker/perf) + :timeline (timeline/html)})) + :generator (independent/concurrent-generator + 10 + (range) + (fn [k] + (->> (gen/mix [r w cas]) + (gen/limit (:ops-per-key opts)))))}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj new file mode 100644 index 00000000000..6e99da769a4 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj @@ -0,0 +1,27 @@ +(ns jepsen.clickhouse.server.utils + (:require [jepsen.clickhouse.utils :as chu] + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.server.client :as chc] + [clojure.tools.logging :refer :all] + [clojure.java.jdbc :as jdbc])) + +(defn clickhouse-alive? + [node test] + (try + (let [c (chc/open-connection node)] + (jdbc/query c "SELECT 1") + (chc/close-connection c)) + (catch Exception _ false))) + +(defn start-clickhouse! + [node test] + (chu/start-clickhouse! + node + test + clickhouse-alive? + :server + :--config (str configs-dir "/config.xml") + :-- + :--logger.log (str logs-dir "/clickhouse.log") + :--logger.errorlog (str logs-dir "/clickhouse.err.log") + :--path data-dir)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj new file mode 100644 index 00000000000..e8c48a9011a --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj @@ -0,0 +1,194 @@ +(ns jepsen.clickhouse.utils + (:require [jepsen.control.util :as cu] + [jepsen + [control :as c] + [db :as db]] + [jepsen.clickhouse.constants :refer :all] + [clojure.tools.logging :refer :all] + [clojure.java.io :as io]) + (:import (java.security MessageDigest))) + +(defn exec-with-retries + [retries f & args] + (let [res (try {:value (apply f args)} + (catch Exception e + (if (zero? retries) + (throw e) + {:exception e})))] + (if (:exception res) + (do (Thread/sleep 1000) (recur (dec retries) f args)) + (:value res)))) + +(defn parse-long + "Parses a string to a Long. Passes through `nil` and empty strings." + [s] + (if (and s (> (count s) 0)) + (Long/parseLong s))) + +(defn md5 [^String s] + (let [algorithm (MessageDigest/getInstance "MD5") + raw (.digest algorithm (.getBytes s))] + (format "%032x" (BigInteger. 1 raw)))) + +(defn non-precise-cached-wget! + [url] + (let [encoded-url (md5 url) + expected-file-name (.getName (io/file url)) + dest-folder (str binaries-cache-dir "/" encoded-url) + dest-file (str root-folder "/clickhouse") + dest-symlink (str root-folder "/" expected-file-name) + wget-opts (concat cu/std-wget-opts [:-O dest-file])] + (when-not (cu/exists? dest-file) + (info "Downloading" url) + (do (c/exec :mkdir :-p dest-folder) + (c/cd dest-folder + (cu/wget-helper! wget-opts url)))) + (c/exec :rm :-rf dest-symlink) + (c/exec :ln :-s dest-file dest-symlink) + dest-symlink)) + +(defn get-clickhouse-url + [url] + (non-precise-cached-wget! url root-folder binaries-cache-dir)) + +(defn get-clickhouse-scp + [path] + (c/upload path (str root-folder "/clickhouse"))) + +(defn download-clickhouse + [source] + (info "Downloading clickhouse from" source) + (cond + (clojure.string/starts-with? source "http") (get-clickhouse-url source root-folder binaries-cache-dir) + (.exists (io/file source)) (get-clickhouse-scp source root-folder) + :else (throw (Exception. (str "Don't know how to download clickhouse from" source))))) + +(defn unpack-deb + [path] + (do + (c/exec :dpkg :-x path root-folder) + (c/exec :rm :-f path) + (c/exec :mv (str root-folder "/usr/bin/clickhouse") root-folder) + (c/exec :rm :-rf (str root-folder "/usr") (str root-folder "/etc")))) + +(defn unpack-tgz + [path] + (do + (c/exec :mkdir :-p (str root-folder "/unpacked")) + (c/exec :tar :-zxvf path :-C (str root-folder "/unpacked")) + (c/exec :rm :-f path) + (let [subdir (c/exec :ls (str root-folder "/unpacked"))] + (c/exec :mv (str root-folder "/unpacked/" subdir "/usr/bin/clickhouse") root-folder) + (c/exec :rm :-fr (str root-folder "/unpacked"))))) + +(defn chmod-binary + [path] + (info "Binary path chmod" path) + (c/exec :chmod :+x path)) + +(defn install-downloaded-clickhouse + [path] + (cond + (clojure.string/ends-with? path ".deb") (unpack-deb path root-folder) + (clojure.string/ends-with? path ".tgz") (unpack-tgz path root-folder) + (clojure.string/ends-with? path "clickhouse") (chmod-binary path) + :else (throw (Exception. (str "Don't know how to install clickhouse from path" path))))) + +(defn collect-traces + [test node] + (let [pid (c/exec :pidof "clickhouse")] + (c/exec :timeout :-s "KILL" "60" :gdb :-ex "set pagination off" :-ex (str "set logging file " logs-dir "/gdb.log") :-ex + "set logging on" :-ex "backtrace" :-ex "thread apply all backtrace" + :-ex "backtrace" :-ex "detach" :-ex "quit" :--pid pid :|| :true))) + +(defn wait-clickhouse-alive! + [node test clickhouse-alive? & {:keys [maxtries] :or {maxtries 30}}] + (loop [i 0] + (cond (> i maxtries) false + (clickhouse-alive? node test) true + :else (do (Thread/sleep 1000) (recur (inc i)))))) + +(defn kill-clickhouse! + [node test] + (info "Killing server on node" node) + (c/su + (cu/stop-daemon! binary-path pid-file-path) + (c/exec :rm :-fr (str data-dir "/status")))) + +(defn start-clickhouse! + [node test clickhouse-alive? & binary-args] + (info "Starting server on node" node) + (c/su + (cu/start-daemon! + {:pidfile pid-file-path + :logfile stderr-file + :chdir data-dir} + binary-path + binary-args) + (wait-clickhouse-alive! node test clickhouse-alive?))) + +(defn prepare-dirs + [] + (do + (c/exec :mkdir :-p root-folder) + (c/exec :mkdir :-p data-dir) + (c/exec :mkdir :-p coordination-data-dir) + (c/exec :mkdir :-p logs-dir) + (c/exec :mkdir :-p configs-dir) + (c/exec :mkdir :-p sub-configs-dir) + (c/exec :touch stderr-file) + (c/exec :chown :-R :root root-folder))) + +(defn db + [version reuse-binary start-clickhouse! extra-setup] + (reify db/DB + (setup! [_ test node] + (c/su + (do + (info "Preparing directories") + (prepare-dirs) + (if (or (not (cu/exists? binary-path)) (not reuse-binary)) + (do (info "Downloading clickhouse") + (let [clickhouse-path (download-clickhouse version root-folder binaries-cache-dir)] + (install-downloaded-clickhouse clickhouse-path))) + (info "Binary already exsist on path" binary-path "skipping download")) + (extra-setup test node) + (info "Starting server") + (start-clickhouse! node test) + (info "ClickHouse started")))) + + (teardown! [_ test node] + (info node "Tearing down clickhouse") + (c/su + (kill-clickhouse! node test) + (if (not reuse-binary) + (c/exec :rm :-rf binary-path)) + (c/exec :rm :-rf pid-file-path) + (c/exec :rm :-rf data-dir) + (c/exec :rm :-rf logs-dir) + (c/exec :rm :-rf configs-dir))) + + db/LogFiles + (log-files [_ test node] + (c/su + ;(if (cu/exists? pid-file-path) + ;(do + ; (info node "Collecting traces") + ; (collect-traces test node logs-dir)) + ;(info node "Pid files doesn't exists")) + (kill-clickhouse! node test) + (if (cu/exists? coordination-data-dir) + (do + (info node "Coordination files exists, going to compress") + (c/cd data-dir + (c/exec :tar :czf "coordination.tar.gz" "coordination")))) + (if (cu/exists? (str logs-dir)) + (do + (info node "Logs exist, going to compress") + (c/cd root-folder + (c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing"))) + (let [common-logs [(str root-folder "/logs.tar.gz") (str data-dir "/coordination.tar.gz")] + gdb-log (str logs-dir "/gdb.log")] + (if (cu/exists? (str logs-dir "/gdb.log")) + (conj common-logs gdb-log) + common-logs))))) From 1d405788090f20e30747939666a51934e87a4e12 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Oct 2022 18:35:18 +0200 Subject: [PATCH 068/526] Remove cruft from Docker --- docker/server/Dockerfile.ubuntu | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c6254b898ed..3a04728864f 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -80,6 +80,16 @@ RUN arch=${TARGETARCH:-amd64} \ && mkdir -p /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client \ && chmod ugo+Xrw -R /var/lib/clickhouse /var/log/clickhouse-server /etc/clickhouse-server /etc/clickhouse-client +# Remove as much of Ubuntu as possible. +# ClickHouse does not need Ubuntu. It can run on top of Linux kernel without any OS distribution. +# ClickHouse does not need Docker at all. ClickHouse is above all that. +# It does not care about Ubuntu, Docker, or other cruft and you should neither. +# The fact that this Docker image is based on Ubuntu is just a misconception. +# Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. +# ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. + +RUN sudo apt remove -y libksba8 + # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From 163001382b07844b36f559f2c4373bcdce6d6836 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Tue, 25 Oct 2022 09:43:29 +0800 Subject: [PATCH 069/526] Fix no progress indication on INSERT FROM INFILE --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d7de8ec799..9a5002b17af 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1173,7 +1173,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && !std_in.eof(); - if (need_render_progress && have_data_in_stdin) + if (need_render_progress) { /// Set total_bytes_to_read for current fd. FileProgress file_progress(0, std_in.getFileSize()); From d0a7f855ef19e3baf7de96d04ec99497308d212c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 25 Oct 2022 05:32:11 +0300 Subject: [PATCH 070/526] Update docker/server/Dockerfile.ubuntu Co-authored-by: Nikita Mikhaylov --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 3a04728864f..9deec86b746 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -88,7 +88,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. # ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. -RUN sudo apt remove -y libksba8 +RUN apt-get remove --purge -y libksba8 && apt-get autoremove # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From 8ef07bac89120e7d1e35b2ad61815f5b17384e59 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Tue, 25 Oct 2022 17:40:54 +0200 Subject: [PATCH 071/526] Failing case corrected and test added --- src/Functions/FunctionsJSON.cpp | 28 ++++++++++++++----- ...xtract_low_cardinality_from_json.reference | 2 ++ ...2473_extract_low_cardinality_from_json.sql | 3 ++ 3 files changed, 26 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02473_extract_low_cardinality_from_json.reference create mode 100644 tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 2f0a0889847..9e44de2cb52 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1303,13 +1303,27 @@ public: static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { - ColumnString & col_str = assert_cast(dest); - auto & chars = col_str.getChars(); - WriteBufferFromVector buf(chars, AppendModeTag()); - traverse(element, buf); - buf.finalize(); - chars.push_back(0); - col_str.getOffsets().push_back(chars.size()); + if (dest.getDataType() == TypeIndex::LowCardinality) + { + ColumnString::Chars chars; + WriteBufferFromVector buf(chars, AppendModeTag()); + chars.push_back(0); + traverse(element, buf); + buf.finalize(); + std::string str = reinterpret_cast(chars.data()); + chars.push_back(0); + assert_cast(dest).insertData(str.data(), str.size()); + } + else + { + ColumnString & col_str = assert_cast(dest); + auto & chars = col_str.getChars(); + WriteBufferFromVector buf(chars, AppendModeTag()); + traverse(element, buf); + buf.finalize(); + chars.push_back(0); + col_str.getOffsets().push_back(chars.size()); + } return true; } diff --git a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.reference b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.reference new file mode 100644 index 00000000000..73eab066599 --- /dev/null +++ b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.reference @@ -0,0 +1,2 @@ +('{"b":{"c":1,"d":"str"}}','','','') +('{"b":{"c":1,"d":"str"}}','','','') diff --git a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql new file mode 100644 index 00000000000..c9810c77720 --- /dev/null +++ b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql @@ -0,0 +1,3 @@ +-- Tags: no-fasttest +SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); +SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a String, b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); \ No newline at end of file From a9a799ac89816b7a8a82c227aeb5533892168d0e Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 26 Oct 2022 05:20:34 +0000 Subject: [PATCH 072/526] Add test --- .../0_stateless/02473_infile_progress.py | 42 +++++++++++++++++++ .../02473_infile_progress.reference | 0 2 files changed, 42 insertions(+) create mode 100755 tests/queries/0_stateless/02473_infile_progress.py create mode 100644 tests/queries/0_stateless/02473_infile_progress.reference diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py new file mode 100755 index 00000000000..2d6493fe4a5 --- /dev/null +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python3 +# Tags: no-replicated-database, no-parallel, no-fasttest + +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, "helpers")) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +# log=sys.stdout + +with client(name="client>", log=log) as client1: + filename = os.environ["CLICKHOUSE_TMP"] + "/infile_progress.tsv" + + client1.expect(prompt) + client1.send("DROP TABLE IF EXISTS test.infile_progress") + client1.expect(prompt) + client1.send(f"SELECT number FROM numbers(5) INTO OUTFILE '{filename}'") + client1.expect(prompt) + client1.send( + "CREATE TABLE test.infile_progress (a Int32) Engine=MergeTree order by tuple()" + ) + client1.expect(prompt) + client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") + client1.expect("Progress: 5.00 rows.*\)") + client1.expect(prompt) + + # send Ctrl-C + client1.send("\x03", eol="") + match = client1.expect("(%s)|([#\$] )" % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send("DROP TABLE test.infile_progress") + client1.expect(prompt) + + os.remove(filename) diff --git a/tests/queries/0_stateless/02473_infile_progress.reference b/tests/queries/0_stateless/02473_infile_progress.reference new file mode 100644 index 00000000000..e69de29bb2d From 8db31beee61e4f28fd5ab4e824751a2afa2cfe6b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 30 Jul 2022 09:19:58 +0300 Subject: [PATCH 073/526] Remove covered parts for fetched part Here is an example that I found on production, simplified. Consider the following queue (nothing of this had been processed on this replica): - GET_PART all_0_0_0 (queue-0000000001) - GET_PART all_1_1_0 (queue-0000000002) ... - GET_PART all_0_1_1 (queue-0000000003) - GET_PART all_2_2_0 (queue-0000000004) ... - MERGE_PARTS from [all_0_1_1, all_2_2_0] to all_0_2_2 (queue-0000000005) And now queue-0000000005 started to executing (either because of reording, or because at that time GET_PART fails), and it does not have any required parts, so it will fetch them, but not all_0_0_0 and all_1_1_0, so this replica delay will set to the time of min(queue-0000000001, queue-0000000002), while it is not true, since it already have parts that covers those parts. and since MERGE_PARTS takes 30min, it increased the replica delay eventually to 30min, for the time range of 30min, which is pretty huge. Signed-off-by: Azat Khuzhin --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 12 +++++- .../MergeTree/ReplicatedMergeTreeQueue.h | 6 ++- src/Storages/StorageReplicatedMergeTree.cpp | 40 ++++++++++++++++--- src/Storages/StorageReplicatedMergeTree.h | 5 ++- 4 files changed, 51 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d6d937ce66f..e60403149b5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1015,7 +1015,8 @@ bool ReplicatedMergeTreeQueue::checkReplaceRangeCanBeRemoved(const MergeTreePart void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & covering_entry) + const std::optional & covering_entry, + const String & fetch_entry_znode) { /// TODO is it possible to simplify it? Queue to_wait; @@ -1029,10 +1030,17 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( [[maybe_unused]] bool called_from_alter_query_directly = covering_entry && covering_entry->replace_range_entry && covering_entry->replace_range_entry->columns_version < 0; [[maybe_unused]] bool called_for_broken_part = !covering_entry; - assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part); + assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part || !fetch_entry_znode.empty()); for (Queue::iterator it = queue.begin(); it != queue.end();) { + /// Skipping currently processing entry + if (!fetch_entry_znode.empty() && (*it)->znode_name == fetch_entry_znode) + { + ++it; + continue; + } + auto type = (*it)->type; bool is_simple_producing_op = type == LogEntry::GET_PART || type == LogEntry::ATTACH_PART || diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 2ebdd604af2..32421f91b04 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -336,8 +336,10 @@ public: * And also wait for the completion of their execution, if they are now being executed. * covering_entry is as an entry that caused removal of entries in range (usually, DROP_RANGE) */ - void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, const MergeTreePartInfo & part_info, - const std::optional & covering_entry); + void removePartProducingOpsInRange(zkutil::ZooKeeperPtr zookeeper, + const MergeTreePartInfo & part_info, + const std::optional & covering_entry, + const String & fetch_entry_znode); /** In the case where there are not enough parts to perform the merge in part_name * - move actions with merged parts to the end of the queue diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index ef56ade2d58..0e4e73f0e3c 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1731,8 +1731,18 @@ bool StorageReplicatedMergeTree::executeFetch(LogEntry & entry, bool need_to_che if (!entry.actual_new_part_name.empty()) LOG_DEBUG(log, "Will fetch part {} instead of {}", entry.actual_new_part_name, entry.new_part_name); - if (!fetchPart(part_name, metadata_snapshot, fs::path(zookeeper_path) / "replicas" / replica, false, entry.quorum)) + String source_replica_path = fs::path(zookeeper_path) / "replicas" / replica; + if (!fetchPart(part_name, + metadata_snapshot, + source_replica_path, + /* to_detached= */ false, + entry.quorum, + /* zookeeper_ */ nullptr, + /* try_fetch_shared= */ true, + entry.znode_name)) + { return false; + } } catch (Exception & e) { @@ -1815,7 +1825,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) LOG_TRACE(log, "Executing DROP_RANGE {}", entry.new_part_name); auto drop_range_info = MergeTreePartInfo::fromPartName(entry.new_part_name, format_version); getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range_info.partition_id, drop_range_info.max_block); - queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry); + queue.removePartProducingOpsInRange(getZooKeeper(), drop_range_info, entry, /* fetch_entry_znode= */ {}); part_check_thread.cancelRemovedPartsCheck(drop_range_info); /// Delete the parts contained in the range to be deleted. @@ -1884,7 +1894,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) if (replace) { getContext()->getMergeList().cancelInPartition(getStorageID(), drop_range.partition_id, drop_range.max_block); - queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry); + queue.removePartProducingOpsInRange(getZooKeeper(), drop_range, entry, /* fetch_entry_znode= */ {}); part_check_thread.cancelRemovedPartsCheck(drop_range); } else @@ -3431,7 +3441,7 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n /// so GET_PART all_1_42_5 (and all source parts) is useless. The only thing we can do is to fetch all_1_42_5_63. /// 2. If all_1_42_5_63 is lost, then replication may stuck waiting for all_1_42_5_63 to appear, /// because we may have some covered parts (more precisely, parts with the same min and max blocks) - queue.removePartProducingOpsInRange(zookeeper, broken_part_info, {}); + queue.removePartProducingOpsInRange(zookeeper, broken_part_info, /* covering_entry= */ {}, /* fetch_entry_znode= */ {}); String part_path = fs::path(replica_path) / "parts" / part_name; @@ -3831,8 +3841,15 @@ bool StorageReplicatedMergeTree::partIsLastQuorumPart(const MergeTreePartInfo & } -bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const StorageMetadataPtr & metadata_snapshot, - const String & source_replica_path, bool to_detached, size_t quorum, zkutil::ZooKeeper::Ptr zookeeper_, bool try_fetch_shared) +bool StorageReplicatedMergeTree::fetchPart( + const String & part_name, + const StorageMetadataPtr & metadata_snapshot, + const String & source_replica_path, + bool to_detached, + size_t quorum, + zkutil::ZooKeeper::Ptr zookeeper_, + bool try_fetch_shared, + String entry_znode) { auto zookeeper = zookeeper_ ? zookeeper_ : getZooKeeper(); const auto part_info = MergeTreePartInfo::fromPartName(part_name, format_version); @@ -4031,6 +4048,17 @@ bool StorageReplicatedMergeTree::fetchPart(const String & part_name, const Stora ProfileEvents::increment(ProfileEvents::ObsoleteReplicatedParts); } + /// It is possible that fetched parts may cover other parts (see + /// findReplicaHavingCoveringPart()), and if those covered parts + /// cannot be executed right now (due to MERGE_PARTS that covers + /// them is in progress), replica delay will be increased until + /// those entries will be executed (if covered operations + /// finishes) in other words until MERGE_PARTS is in progress, + /// while this can take awhile. + /// + /// So let's just remove them from the queue. + queue.removePartProducingOpsInRange(zookeeper, part->info, /* covering_entry= */ {}, entry_znode); + write_part_log({}); } else diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 436defebe1d..b15350d077b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -672,11 +672,12 @@ private: bool fetchPart( const String & part_name, const StorageMetadataPtr & metadata_snapshot, - const String & replica_path, + const String & source_replica_path, bool to_detached, size_t quorum, zkutil::ZooKeeper::Ptr zookeeper_ = nullptr, - bool try_fetch_shared = true); + bool try_fetch_shared = true, + String entry_znode = ""); /** Download the specified part from the specified replica. * Used for replace local part on the same s3-shared part in hybrid storage. From fcbc217a7dc6e5cd7ee654ff5ee698df0c4baea9 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 25 Oct 2022 16:38:36 +0800 Subject: [PATCH 074/526] enable limits for functions using FunctionTokens --- src/Functions/FunctionsStringArray.h | 170 +++++++++++------- src/Functions/URL/URLHierarchy.cpp | 6 + src/Functions/URL/URLPathHierarchy.cpp | 6 + .../URL/extractURLParameterNames.cpp | 7 + src/Functions/URL/extractURLParameters.cpp | 6 + .../01866_split_by_regexp.reference | 5 + .../0_stateless/01866_split_by_regexp.sql | 6 + .../0_stateless/02185_split_by_char.reference | 11 +- .../0_stateless/02185_split_by_char.sql | 10 +- 9 files changed, 157 insertions(+), 70 deletions(-) diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 95c06ae74ca..832399c30c8 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -32,12 +32,12 @@ namespace ErrorCodes /** Functions that split strings into an array of strings or vice versa. * - * splitByChar(sep, s) - * splitByString(sep, s) - * splitByRegexp(regexp, s) + * splitByChar(sep, s[, max_split]) + * splitByString(sep, s[, max_split]) + * splitByRegexp(regexp, s[, max_split]) * - * splitByWhitespace(s) - split the string by whitespace characters - * splitByNonAlpha(s) - split the string by whitespace and punctuation characters + * splitByWhitespace(s[, max_split]) - split the string by whitespace characters + * splitByNonAlpha(s[, max_split]) - split the string by whitespace and punctuation characters * * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. * - first subpattern, if regexp has subpattern; @@ -98,6 +98,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return {}; + } + /// Get the next token, if any, or return false. bool get(Pos & token_begin, Pos & token_end) { @@ -157,6 +163,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return {}; + } + /// Get the next token, if any, or return false. bool get(Pos & token_begin, Pos & token_end) { @@ -216,6 +228,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return {}; + } + /// Get the next token, if any, or return false. bool get(Pos & token_begin, Pos & token_end) { @@ -242,10 +260,7 @@ class SplitByCharImpl private: Pos pos; Pos end; - char sep; - std::optional max_split; - UInt64 curr_split = 0; public: static constexpr auto name = "splitByChar"; @@ -268,13 +283,6 @@ public: if (!isString(arguments[1])) throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (arguments.size() == 3 && !isNativeInteger(arguments[2])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Third argument for function '{}' must be integer, got '{}' instead", - getName(), - arguments[2]->getName()); } void init(const ColumnsWithTypeAndName & arguments) @@ -292,39 +300,6 @@ public: throw Exception("Illegal separator for function " + getName() + ". Must be exactly one byte.", ErrorCodes::BAD_ARGUMENTS); sep = sep_str[0]; - - if (arguments.size() > 2) - { - if (!((max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])) - || (max_split = getMaxSplit(arguments[2])))) - { - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of third argument of function {}", - arguments[2].column->getName(), - getName()); - } - } - } - - template - std::optional getMaxSplit(const ColumnWithTypeAndName & argument) - { - const auto * col = checkAndGetColumnConst>(argument.column.get()); - if (!col) - return std::nullopt; - - auto value = col->template getValue(); - if (value < 0) - throw Exception( - ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of third argument of function {}", argument.column->getName(), getName()); - return value; } /// Returns the position of the argument, that is the column of strings @@ -333,6 +308,12 @@ public: return 1; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return 2; + } + void set(Pos pos_, Pos end_) { pos = pos_; @@ -346,19 +327,12 @@ public: return false; token_begin = pos; - if (unlikely(max_split && curr_split >= *max_split)) - { - token_end = end; - pos = nullptr; - return true; - } - pos = reinterpret_cast(memchr(pos, sep, end - pos)); + if (pos) { token_end = pos; ++pos; - ++curr_split; } else token_end = end; @@ -405,6 +379,12 @@ public: return 1; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return {}; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { @@ -454,12 +434,13 @@ private: Pos pos; Pos end; + public: static constexpr auto name = "splitByRegexp"; static String getName() { return name; } - static bool isVariadic() { return false; } - static size_t getNumberOfArguments() { return 2; } + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } /// Check the type of function arguments. static void checkArguments(const DataTypes & arguments) @@ -479,7 +460,6 @@ public: if (!col->getValue().empty()) re = std::make_shared(Regexps::createRegexp(col->getValue())); - } /// Returns the position of the argument that is the column of strings @@ -488,6 +468,12 @@ public: return 1; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return 2; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { @@ -573,6 +559,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return {}; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { @@ -630,6 +622,16 @@ public: { Generator::checkArguments(arguments); + const auto max_split_pos = Generator::getMaxSplitArgumentPosition(); + if (max_split_pos) + if (arguments.size() > *max_split_pos && !isNativeInteger(arguments[*max_split_pos])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "{}-th argument for function '{}' must be integer, got '{}' instead", + *max_split_pos + 1, + getName(), + arguments[*max_split_pos]->getName()); + return std::make_shared(std::make_shared()); } @@ -639,6 +641,10 @@ public: generator.init(arguments); const auto & array_argument = arguments[generator.getStringsArgumentPosition()]; + /// Whether we need to limit max tokens returned by Generator::get + /// If max_split is std::nullopt, no limit is applied. + auto max_split = getMaxSplit(arguments); + const ColumnString * col_str = checkAndGetColumn(array_argument.column.get()); const ColumnConst * col_const_str = checkAndGetColumnConstStringOrFixedString(array_argument.column.get()); @@ -672,9 +678,8 @@ public: Pos end = reinterpret_cast(&src_chars[current_src_offset]) - 1; generator.set(pos, end); - size_t j = 0; - while (generator.get(token_begin, token_end)) + while (generator.get(token_begin, token_end) && !(max_split && j >= *max_split)) { size_t token_size = token_end - token_begin; @@ -702,7 +707,7 @@ public: Pos token_begin = nullptr; Pos token_end = nullptr; - while (generator.get(token_begin, token_end)) + while (generator.get(token_begin, token_end) && !(max_split && dst.size() >= *max_split)) dst.push_back(String(token_begin, token_end - token_begin)); return result_type->createColumnConst(col_const_str->size(), dst); @@ -713,6 +718,47 @@ public: + " of arguments of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); } + +private: + template + std::optional getMaxSplitImpl(const ColumnWithTypeAndName & argument) const + { + const auto * col = checkAndGetColumnConst>(argument.column.get()); + if (!col) + return {}; + + auto value = col->template getValue(); + return static_cast(value); + } + + std::optional getMaxSplit(const ColumnsWithTypeAndName & arguments) const + { + const auto pos = Generator::getMaxSplitArgumentPosition(); + if (!pos) + return {}; + + if (arguments.size() <= *pos) + return {}; + + std::optional max_split; + if (!((max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) + || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) + || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) + || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])))) + throw Exception( + ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {}, which is {}-th argument of function {}", + arguments[*pos].column->getName(), + *pos + 1, + getName()); + + /// If max_split is negative or zero, tokenize will be applied as many times as possible, which is equivalent to + /// no max_split argument in function + if (max_split && *max_split <= 0) + return {}; + + return *max_split; + } }; diff --git a/src/Functions/URL/URLHierarchy.cpp b/src/Functions/URL/URLHierarchy.cpp index c0c3a7e5b37..f8b7cbb3858 100644 --- a/src/Functions/URL/URLHierarchy.cpp +++ b/src/Functions/URL/URLHierarchy.cpp @@ -38,6 +38,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return std::nullopt; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { diff --git a/src/Functions/URL/URLPathHierarchy.cpp b/src/Functions/URL/URLPathHierarchy.cpp index c00e61e67fd..9af592b8db7 100644 --- a/src/Functions/URL/URLPathHierarchy.cpp +++ b/src/Functions/URL/URLPathHierarchy.cpp @@ -37,6 +37,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return std::nullopt; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { diff --git a/src/Functions/URL/extractURLParameterNames.cpp b/src/Functions/URL/extractURLParameterNames.cpp index f274547ed4e..2cb6af7ae12 100644 --- a/src/Functions/URL/extractURLParameterNames.cpp +++ b/src/Functions/URL/extractURLParameterNames.cpp @@ -35,6 +35,13 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return std::nullopt; + } + + void init(const ColumnsWithTypeAndName & /*arguments*/) {} /// Called for each next string. diff --git a/src/Functions/URL/extractURLParameters.cpp b/src/Functions/URL/extractURLParameters.cpp index 15b564fc947..f7a2fd10b77 100644 --- a/src/Functions/URL/extractURLParameters.cpp +++ b/src/Functions/URL/extractURLParameters.cpp @@ -37,6 +37,12 @@ public: return 0; } + /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. + static std::optional getMaxSplitArgumentPosition() + { + return std::nullopt; + } + /// Called for each next string. void set(Pos pos_, Pos end_) { diff --git a/tests/queries/0_stateless/01866_split_by_regexp.reference b/tests/queries/0_stateless/01866_split_by_regexp.reference index a3ae2f35a5f..1b873bf5621 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.reference +++ b/tests/queries/0_stateless/01866_split_by_regexp.reference @@ -5,3 +5,8 @@ ['gbye','bug'] [''] [] +['one','two','three',''] +['one','two','three',''] +['one','two','three',''] +['one'] +['one','two'] diff --git a/tests/queries/0_stateless/01866_split_by_regexp.sql b/tests/queries/0_stateless/01866_split_by_regexp.sql index e472fb68d94..c6d4b12380e 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.sql +++ b/tests/queries/0_stateless/01866_split_by_regexp.sql @@ -3,3 +3,9 @@ select splitByRegexp('', 'abcde'); select splitByRegexp('<[^<>]*>', x) from (select arrayJoin(['

hello

world

', 'gbyebug']) x); select splitByRegexp('ab', ''); select splitByRegexp('', ''); + +SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC'); +SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0); +SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1); +SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1); +SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2); diff --git a/tests/queries/0_stateless/02185_split_by_char.reference b/tests/queries/0_stateless/02185_split_by_char.reference index 6afd0824a73..d8dd65ed63d 100644 --- a/tests/queries/0_stateless/02185_split_by_char.reference +++ b/tests/queries/0_stateless/02185_split_by_char.reference @@ -1,7 +1,10 @@ ['1','2','3'] -['1,2,3'] -['1','2,3'] +['1'] +['1','2'] ['1','2','3'] ['1','2','3'] -['expr1','1+1=2'] -['expr2','2+2=4=1+3'] +['1','2','3'] +['1','2','3'] +['1','2','3'] +['expr1'] +['expr2'] diff --git a/tests/queries/0_stateless/02185_split_by_char.sql b/tests/queries/0_stateless/02185_split_by_char.sql index c45f3de07eb..6ab1fc19079 100644 --- a/tests/queries/0_stateless/02185_split_by_char.sql +++ b/tests/queries/0_stateless/02185_split_by_char.sql @@ -1,10 +1,12 @@ select splitByChar(',', '1,2,3'); -select splitByChar(',', '1,2,3', 0); select splitByChar(',', '1,2,3', 1); select splitByChar(',', '1,2,3', 2); select splitByChar(',', '1,2,3', 3); +select splitByChar(',', '1,2,3', 4); +select splitByChar(',', '1,2,3', 0); +select splitByChar(',', '1,2,3', -1); +select splitByChar(',', '1,2,3', -2); +SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3'); -select splitByChar(',', '1,2,3', -2); -- { serverError 44 } select splitByChar(',', '1,2,3', ''); -- { serverError 43 } - -SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3') +select splitByChar(',', '1,2,3', ''); -- { serverError 43 } From 05b77b3e345a3161105edcc8d1e05992d06aa457 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Wed, 26 Oct 2022 16:25:25 +0800 Subject: [PATCH 075/526] fix comment --- src/Functions/FunctionsStringArray.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 832399c30c8..42334b68251 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -32,12 +32,12 @@ namespace ErrorCodes /** Functions that split strings into an array of strings or vice versa. * - * splitByChar(sep, s[, max_split]) - * splitByString(sep, s[, max_split]) - * splitByRegexp(regexp, s[, max_split]) + * splitByChar(sep, s) + * splitByString(sep, s) + * splitByRegexp(regexp, s) * - * splitByWhitespace(s[, max_split]) - split the string by whitespace characters - * splitByNonAlpha(s[, max_split]) - split the string by whitespace and punctuation characters + * splitByWhitespace(s) - split the string by whitespace characters + * splitByNonAlpha(s) - split the string by whitespace and punctuation characters * * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. * - first subpattern, if regexp has subpattern; From 241049763bedfeeef29e62c7c5b8698dd7fc4a7f Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Wed, 26 Oct 2022 11:50:45 +0200 Subject: [PATCH 076/526] Fixed failing tests --- src/Functions/FunctionsJSON.cpp | 17 ++++++++++++++--- .../02473_extract_low_cardinality_from_json.sql | 1 - ...2474_extract_fixedstring_from_json.reference | 10 ++++++++++ .../02474_extract_fixedstring_from_json.sql | 8 ++++++++ 4 files changed, 32 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference create mode 100644 tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 9e44de2cb52..d0fd6e2397c 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1330,12 +1330,23 @@ public: // We use insertResultToFixedStringColumn in case we are inserting raw data in a FixedString column static bool insertResultToFixedStringColumn(IColumn & dest, const Element & element, std::string_view) { - ColumnFixedString & col_str = assert_cast(dest); - auto & chars = col_str.getChars(); + ColumnFixedString::Chars chars; WriteBufferFromVector buf(chars, AppendModeTag()); traverse(element, buf); buf.finalize(); - col_str.insertDefault(); + + auto & col_str = assert_cast(dest); + + if (chars.size() > col_str.getN()) + return false; + + chars.push_back(0); + std::string str = reinterpret_cast(chars.data()); + + auto padded_str = str + std::string(col_str.getN() - std::min(col_str.getN(), str.length()), '\0'); + col_str.insertData(str.data(), str.size()); + + return true; } diff --git a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql index c9810c77720..0a0ae398227 100644 --- a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql +++ b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql @@ -1,3 +1,2 @@ --- Tags: no-fasttest SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a String, b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); \ No newline at end of file diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference new file mode 100644 index 00000000000..783d12fcf1a --- /dev/null +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.reference @@ -0,0 +1,10 @@ +\0\0\0\0\0\0\0\0\0\0\0 +{"a":123456} +\0\0\0\0\0 +123456 +\0\0\0\0\0 +123456 +\0\0\0\0\0 +\0\0\0\0\0 +131231 +131231 diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql new file mode 100644 index 00000000000..cfc47e00cba --- /dev/null +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql @@ -0,0 +1,8 @@ +SELECT JSONExtract('{"a": 123456}', 'FixedString(11)'); +SELECT JSONExtract('{"a": 123456}', 'FixedString(12)'); +SELECT JSONExtract('{"a": "123456"}', 'a', 'FixedString(5)'); +SELECT JSONExtract('{"a": "123456"}', 'a', 'FixedString(6)'); +SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(5)'); +SELECT JSONExtract('{"a": 123456}', 'a', 'FixedString(6)'); +SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(5))') FROM numbers(2); +SELECT JSONExtract(materialize('{"a": 131231}'), 'a', 'LowCardinality(FixedString(6))') FROM numbers(2); From 6a8fa2d4a5e922fe9dadd322429e5a88e4df1d30 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 26 Oct 2022 18:45:23 +0200 Subject: [PATCH 077/526] Added new field allow_readonly in system.table_functions to allow using table functions in readonly mode. Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. * Added TableFunctionProperties similar to AggregateFunctionProperties. * The functions allowed in readonly mode are in this set table_functions_allowed_in_readonly_mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. --- .../en/sql-reference/table-functions/index.md | 4 ++ .../System/StorageSystemTableFunctions.cpp | 27 ++++++++++--- src/TableFunctions/ITableFunction.cpp | 5 ++- src/TableFunctions/ITableFunction.h | 9 +++++ src/TableFunctions/TableFunctionFactory.cpp | 40 ++++++++++++++----- src/TableFunctions/TableFunctionFactory.h | 24 +++++++++-- ...02473_functions_in_readonly_mode.reference | 3 ++ .../02473_functions_in_readonly_mode.sql | 5 +++ 8 files changed, 97 insertions(+), 20 deletions(-) create mode 100644 tests/queries/0_stateless/02473_functions_in_readonly_mode.reference create mode 100644 tests/queries/0_stateless/02473_functions_in_readonly_mode.sql diff --git a/docs/en/sql-reference/table-functions/index.md b/docs/en/sql-reference/table-functions/index.md index d09adcd13d6..94b23bc695c 100644 --- a/docs/en/sql-reference/table-functions/index.md +++ b/docs/en/sql-reference/table-functions/index.md @@ -39,3 +39,7 @@ You can’t use table functions if the [allow_ddl](../../operations/settings/per | [s3](../../sql-reference/table-functions/s3.md) | Creates a [S3](../../engines/table-engines/integrations/s3.md)-engine table. | | [sqlite](../../sql-reference/table-functions/sqlite.md) | Creates a [sqlite](../../engines/table-engines/integrations/sqlite.md)-engine table. | +:::note +Only these table functions are enabled in readonly mode : +null, view, viewIfPermitted, numbers, numbers_mt, generateRandom, values, cluster, clusterAllReplicas +::: \ No newline at end of file diff --git a/src/Storages/System/StorageSystemTableFunctions.cpp b/src/Storages/System/StorageSystemTableFunctions.cpp index 308cbc5686d..07a504edc5e 100644 --- a/src/Storages/System/StorageSystemTableFunctions.cpp +++ b/src/Storages/System/StorageSystemTableFunctions.cpp @@ -1,16 +1,23 @@ #include - #include +#include + namespace DB { +namespace ErrorCodes +{ + extern const int UNKNOWN_FUNCTION; +} + NamesAndTypesList StorageSystemTableFunctions::getNamesAndTypes() { return - { - {"name", std::make_shared()}, - {"description", std::make_shared()} - }; + { + {"name", std::make_shared()}, + {"description", std::make_shared()}, + {"allow_readonly", std::make_shared()} + }; } void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, ContextPtr, const SelectQueryInfo &) const @@ -20,7 +27,15 @@ void StorageSystemTableFunctions::fillData(MutableColumns & res_columns, Context for (const auto & function_name : functions_names) { res_columns[0]->insert(function_name); - res_columns[1]->insert(factory.getDocumentation(function_name).description); + + auto properties = factory.tryGetProperties(function_name); + if (properties) + { + res_columns[1]->insert(properties->documentation.description); + res_columns[2]->insert(properties->allow_readonly); + } + else + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", function_name); } } diff --git a/src/TableFunctions/ITableFunction.cpp b/src/TableFunctions/ITableFunction.cpp index 82b6230dc30..da0de7e47f6 100644 --- a/src/TableFunctions/ITableFunction.cpp +++ b/src/TableFunctions/ITableFunction.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace ProfileEvents @@ -25,8 +26,8 @@ StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr conte ProfileEvents::increment(ProfileEvents::TableFunctionExecute); AccessFlags required_access = getSourceAccessType(); - String function_name = getName(); - if ((function_name != "null") && (function_name != "view") && (function_name != "viewIfPermitted")) + auto table_function_properties = TableFunctionFactory::instance().tryGetProperties(getName()); + if (!(table_function_properties && table_function_properties->allow_readonly)) required_access |= AccessType::CREATE_TEMPORARY_TABLE; context->checkAccess(required_access); diff --git a/src/TableFunctions/ITableFunction.h b/src/TableFunctions/ITableFunction.h index 4b9a87b93f1..a05edcd32c8 100644 --- a/src/TableFunctions/ITableFunction.h +++ b/src/TableFunctions/ITableFunction.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -79,6 +80,14 @@ private: virtual const char * getStorageTypeName() const = 0; }; +/// Properties of table function that are independent of argument types and parameters. +struct TableFunctionProperties +{ + Documentation documentation; + bool allow_readonly = false; +}; + + using TableFunctionPtr = std::shared_ptr; diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 5ed22e39300..79802d2ec77 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -16,16 +16,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +static const NameSet table_functions_allowed_in_readonly_mode +{ + "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" +}; void TableFunctionFactory::registerFunction( const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness) { - if (!table_functions.emplace(name, TableFunctionFactoryData{creator, doc}).second) + bool allowed_in_readonly_mode = table_functions_allowed_in_readonly_mode.contains(name); + + if (!table_functions.emplace(name, TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive - && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, doc}).second) + && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } @@ -59,13 +65,13 @@ TableFunctionPtr TableFunctionFactory::tryGet( auto it = table_functions.find(name); if (table_functions.end() != it) { - res = it->second.first(); + res = it->second.creator(); } else { it = case_insensitive_table_functions.find(Poco::toLower(name)); if (case_insensitive_table_functions.end() != it) - res = it->second.first(); + res = it->second.creator(); } if (!res) @@ -86,13 +92,29 @@ bool TableFunctionFactory::isTableFunctionName(const std::string & name) const return table_functions.contains(name); } -Documentation TableFunctionFactory::getDocumentation(const std::string & name) const +std::optional TableFunctionFactory::tryGetProperties(const String & name) const { - auto it = table_functions.find(name); - if (it == table_functions.end()) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, "Unknown table function {}", name); + return tryGetPropertiesImpl(name); +} - return it->second.second; +std::optional TableFunctionFactory::tryGetPropertiesImpl(const String & name_param) const +{ + String name = getAliasToOrName(name_param); + Value found; + + /// Find by exact match. + if (auto it = table_functions.find(name); it != table_functions.end()) + { + found = it->second; + } + + if (auto jt = case_insensitive_table_functions.find(Poco::toLower(name)); jt != case_insensitive_table_functions.end()) + found = jt->second; + + if (found.creator) + return found.properties; + + return {}; } TableFunctionFactory & TableFunctionFactory::instance() diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 8ff352ff9ac..24b46d40de0 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -18,7 +17,24 @@ namespace DB class Context; using TableFunctionCreator = std::function; -using TableFunctionFactoryData = std::pair; + +struct TableFunctionFactoryData +{ + TableFunctionCreator creator; + TableFunctionProperties properties; + + TableFunctionFactoryData() = default; + TableFunctionFactoryData(const TableFunctionFactoryData &) = default; + TableFunctionFactoryData & operator = (const TableFunctionFactoryData &) = default; + + template + requires (!std::is_same_v) + TableFunctionFactoryData(Creator creator_, TableFunctionProperties properties_ = {}) /// NOLINT + : creator(std::forward(creator_)), properties(std::move(properties_)) + { + } +}; + /** Lets you get a table function by its name. */ @@ -48,7 +64,7 @@ public: /// Returns nullptr if not found. TableFunctionPtr tryGet(const std::string & name, ContextPtr context) const; - Documentation getDocumentation(const std::string & name) const; + std::optional tryGetProperties(const String & name) const; bool isTableFunctionName(const std::string & name) const; @@ -61,6 +77,8 @@ private: String getFactoryName() const override { return "TableFunctionFactory"; } + std::optional tryGetPropertiesImpl(const String & name) const; + TableFunctions table_functions; TableFunctions case_insensitive_table_functions; }; diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference new file mode 100644 index 00000000000..4977168f515 --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference @@ -0,0 +1,3 @@ +0 +(123,'str') +0 diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql new file mode 100644 index 00000000000..7cf55ad714c --- /dev/null +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -0,0 +1,5 @@ +SELECT * from numbers(1); +select * from format(JSONEachRow, '{"x" : [123, "str"]}'); +SELECT * from numbers(1) SETTINGS readonly=1; +select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } +set readonly=0; \ No newline at end of file From 6d8e2db082bc6226afc26ea9145b503356ff011d Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 26 Oct 2022 18:49:14 +0200 Subject: [PATCH 078/526] Removed extra line from test - 42414 Enable table functions in readonly mode --- tests/queries/0_stateless/02473_functions_in_readonly_mode.sql | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql index 7cf55ad714c..eb2631168a8 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -1,5 +1,4 @@ SELECT * from numbers(1); select * from format(JSONEachRow, '{"x" : [123, "str"]}'); SELECT * from numbers(1) SETTINGS readonly=1; -select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } -set readonly=0; \ No newline at end of file +select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file From 649e8d302c460d825c77533063f05367e817c44f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 26 Oct 2022 17:41:44 +0000 Subject: [PATCH 079/526] Sync with retiring branch igor/insert_zk_retries --- .../MergeTree/EphemeralLockInZooKeeper.cpp | 7 - .../MergeTree/ReplicatedMergeTreeSink.cpp | 12 +- src/Storages/MergeTree/ZooKeeperRetries.h | 121 ++++++------ .../MergeTree/ZooKeeperWithFaultInjection.h | 175 ++++++++++++------ .../test_inserts_with_keeper_retries/test.py | 57 ++---- ...compact_part_replicated_zookeeper_long.sql | 4 + ...18_long_unsuccessful_mutation_zookeeper.sh | 9 +- .../01509_parallel_quorum_and_merge_long.sh | 13 +- 8 files changed, 217 insertions(+), 181 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index f2492e67437..b6c455e76b3 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -20,13 +20,6 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential); if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); - - LOG_DEBUG( - &Poco::Logger::get("EphemeralLockInZooKeeper"), - "Path created: path={} path_prefix={} holder_path={}", - path, - path_prefix, - holder_path); } std::optional createEphemeralLockInZooKeeper( diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 2ff1a3e093b..54b7df81654 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -317,8 +317,9 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt { /// NOTE: No delay in this case. That's Ok. - assertSessionIsNotExpired(storage.getZooKeeper()); - auto zookeeper = std::make_shared(storage.getZooKeeper()); + auto origin_zookeeper = storage.getZooKeeper(); + assertSessionIsNotExpired(origin_zookeeper); + auto zookeeper = std::make_shared(origin_zookeeper); size_t replicas_num = checkQuorumPrecondition(zookeeper); @@ -398,7 +399,7 @@ void ReplicatedMergeTreeSink::commitPart( /// Also, make deduplication check. If a duplicate is detected, no nodes are created. /// Allocate new block number and check for duplicates - bool deduplicate_block = !block_id.empty(); + const bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; std::optional block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); @@ -501,8 +502,6 @@ void ReplicatedMergeTreeSink::commitPart( { is_already_existing_part = true; - /// TODO: looks unclear, - if deduplication is off then block_id will be empty, - so what will happened here then? - /// This block was already written to some replica. Get the part name for it. /// Note: race condition with DROP PARTITION operation is possible. User will get "No node" exception and it is Ok. existing_part_name = zookeeper->get(storage.zookeeper_path + "/blocks/" + block_id); @@ -694,7 +693,8 @@ void ReplicatedMergeTreeSink::commitPart( block_id, Coordination::errorMessage(multi_code)); } - }); + }, + [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); if (isQuorumEnabled()) { diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index e10478e3113..24bfad22f69 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -39,76 +39,34 @@ public: { } - bool canTry() + void retryLoop(auto && f) { - ++iteration_count; - /// first iteration is ordinary execution, no further checks needed - if (0 == iteration_count) - return true; - - if (unconditional_retry) - { - unconditional_retry = false; - return true; - } - - /// iteration succeeded -> no need to retry - if (iteration_succeeded) - { - if (retries_info.logger) - LOG_DEBUG( - retries_info.logger, - "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", - retries_info.name, - name, - iteration_count, - retries_info.retry_count); - return false; - } - - if (stop_retries) - { - logLastError("stop retries on request"); - action_after_last_failed_retry(); - throwIfError(); - return false; - } - - if (retries_info.retry_count >= retries_info.max_retries) - { - logLastError("retry limit is reached"); - action_after_last_failed_retry(); - throwIfError(); - return false; - } - - /// retries - ++retries_info.retry_count; - logLastError("will retry due to error"); - sleepForMilliseconds(retries_info.curr_backoff_ms); - retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); - - /// reset the flag, it will be set to false in case of error - iteration_succeeded = true; - - return true; + retryLoop(f, []() {}); } - void retryLoop(auto && f) + void retryLoop(auto && f, auto && iteration_cleanup) { while (canTry()) { try { f(); + iteration_cleanup(); } catch (const zkutil::KeeperException & e) { + iteration_cleanup(); + if (!Coordination::isHardwareError(e.code)) throw; setKeeperError(e.code, e.message()); } + catch (...) + { + iteration_cleanup(); + throw; + } } } @@ -194,6 +152,61 @@ private: std::string message; }; + bool canTry() + { + ++iteration_count; + /// first iteration is ordinary execution, no further checks needed + if (0 == iteration_count) + return true; + + if (unconditional_retry) + { + unconditional_retry = false; + return true; + } + + /// iteration succeeded -> no need to retry + if (iteration_succeeded) + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", + retries_info.name, + name, + iteration_count, + retries_info.retry_count); + return false; + } + + if (stop_retries) + { + logLastError("stop retries on request"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + if (retries_info.retry_count >= retries_info.max_retries) + { + logLastError("retry limit is reached"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + /// retries + ++retries_info.retry_count; + logLastError("will retry due to error"); + sleepForMilliseconds(retries_info.curr_backoff_ms); + retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); + + /// reset the flag, it will be set to false in case of error + iteration_succeeded = true; + + return true; + } + void throwIfError() const { if (user_error.code != ErrorCodes::OK) @@ -203,7 +216,7 @@ private: throw zkutil::KeeperException(keeper_error.code, keeper_error.message); } - void logLastError(std::string header) + void logLastError(std::string_view header) { if (user_error.code == ErrorCodes::OK) { diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h index c1dbbe7bcc1..07b37f4c8f0 100644 --- a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -43,13 +43,15 @@ class ZooKeeperWithFaultInjection using zk = zkutil::ZooKeeper; zk::Ptr keeper; + zk::Ptr keeper_prev; std::unique_ptr fault_policy; std::string name; Poco::Logger * logger = nullptr; UInt64 calls_total = 0; UInt64 calls_without_fault_injection = 0; const UInt64 seed = 0; - const std::function noop_cleanup = []() {}; + + std::vector ephemeral_nodes; ZooKeeperWithFaultInjection( zk::Ptr const & keeper_, @@ -124,8 +126,7 @@ public: const zkutil::EventPtr & watch = nullptr, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) { - return access( - "getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }, noop_cleanup); + return access("getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }); } Coordination::Error tryGetChildren( @@ -135,23 +136,17 @@ public: const zkutil::EventPtr & watch = nullptr, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) { - return access( - "tryGetChildren", - path, - [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }, - [&](Coordination::Error) {}); + return access("tryGetChildren", path, [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }); } zk::FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}) { - return access( - "asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }, noop_cleanup); + return access("asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }); } zk::FutureGet asyncTryGet(const std::string & path) { - return access( - "asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }, noop_cleanup); + return access("asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }); } bool tryGet( @@ -161,65 +156,78 @@ public: const zkutil::EventPtr & watch = nullptr, Coordination::Error * code = nullptr) { - return access( - "tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }, noop_cleanup); + return access("tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }); } Coordination::Error tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses) { - return access( - "tryMulti", + constexpr auto method = "tryMulti"; + auto error = access( + method, !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->tryMulti(requests, responses); }, - [&](Coordination::Error err) + [&](const Coordination::Error & err) { if (err == Coordination::Error::ZOK) - faultInjectionCleanup("tryMulti", requests, responses); + faultInjectionCleanup(method, requests, responses); }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; } Coordination::Error tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses) { + constexpr auto method = "tryMultiNoThrow"; constexpr auto no_throw = true; constexpr auto inject_failure_before_op = false; - return access( - "tryMultiNoThrow", + auto error = access( + method, !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->tryMultiNoThrow(requests, responses); }, - [&](Coordination::Error err) + [&](const Coordination::Error & err) { if (err == Coordination::Error::ZOK) - faultInjectionCleanup("tryMultiNoThrow", requests, responses); + faultInjectionCleanup(method, requests, responses); }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; } std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) { - return access( - "get", path, [&]() { return keeper->get(path, stat, watch); }, [](const std::string &) {}); + return access("get", path, [&]() { return keeper->get(path, stat, watch); }); } zkutil::ZooKeeper::MultiGetResponse get(const std::vector & paths) { - return access( - "get", paths.front(), [&]() { return keeper->get(paths); }, noop_cleanup); + return access("get", !paths.empty() ? paths.front() : "", [&]() { return keeper->get(paths); }); } bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) { - return access( - "exists", path, [&]() { return keeper->exists(path, stat, watch); }, noop_cleanup); + return access("exists", path, [&]() { return keeper->exists(path, stat, watch); }); } zkutil::ZooKeeper::MultiExistsResponse exists(const std::vector & paths) { - return access( - "exists", paths.front(), [&]() { return keeper->exists(paths); }, noop_cleanup); + return access("exists", !paths.empty() ? paths.front() : "", [&]() { return keeper->exists(paths); }); } std::string create(const std::string & path, const std::string & data, int32_t mode) { - return access( + auto path_created = access( "create", path, [&]() { return keeper->create(path, data, mode); }, @@ -247,11 +255,20 @@ public: e.message()); } }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy)) + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + ephemeral_nodes.push_back(path_created); + } + + return path_created; } Coordination::Responses multi(const Coordination::Requests & requests) { - return access( + return access( "multi", !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->multi(requests); }, @@ -260,19 +277,35 @@ public: void createAncestors(const std::string & path) { - access( - "createAncestors", path, [&]() { return keeper->createAncestors(path); }, noop_cleanup); + access("createAncestors", path, [&]() { return keeper->createAncestors(path); }); } Coordination::Error tryRemove(const std::string & path, int32_t version = -1) { - return access( - "tryRemove", path, [&]() { return keeper->tryRemove(path, version); }, [&](Coordination::Error) {}); + return access("tryRemove", path, [&]() { return keeper->tryRemove(path, version); }); + } + + void cleanupEphemeralNodes() + { + for (const auto & path : ephemeral_nodes) + { + try + { + if (keeper_prev) + keeper_prev->tryRemove(path); + } + catch (...) + { + if (unlikely(logger)) + tryLogCurrentException(logger, "Exception during ephemeral nodes clean up"); + } + } + + ephemeral_nodes.clear(); } private: - template - void faultInjectionAfter(const Result & res, FaultCleanup fault_cleanup) + void faultInjectionAfter(std::function fault_cleanup) { try { @@ -281,25 +314,13 @@ private: } catch (const zkutil::KeeperException &) { - if constexpr (std::is_same_v>) - fault_cleanup(); - else - { - if constexpr (std::is_same_v) - fault_cleanup(res); - else if constexpr (std::is_same_v) - fault_cleanup(res); - else if constexpr (std::is_same_v) - fault_cleanup(res); - else - fault_cleanup(); - } - + fault_cleanup(); throw; } } - void faultInjectionCleanup(const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses) + void doForEachEphemeralNode( + const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses, auto && action) { if (responses.empty()) return; @@ -328,17 +349,38 @@ private: throw Exception( ErrorCodes::LOGICAL_ERROR, "Response should be CreateResponse: method={} index={} path={}", method, i, req->path); - keeper->remove(create_resp->path_created); + action(create_resp->path_created); } } + void faultInjectionCleanup(const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses) + { + doForEachEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); + } + + template + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template <> + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template + using FaultCleanupType = typename FaultCleanupTypeImpl::Type; + template < - typename Result, bool no_throw_access = false, bool inject_failure_before_op = true, int inject_failure_after_op = true, - typename FaultCleanup> - Result access(const char * func_name, const std::string & path, auto && operation, FaultCleanup fault_after_op_cleanup) + typename Operation, + typename Result = std::invoke_result_t> + Result + access(const char * func_name, const std::string & path, Operation operation, FaultCleanupType fault_after_op_cleanup = {}) { try { @@ -366,7 +408,14 @@ private: } if constexpr (inject_failure_after_op) - faultInjectionAfter(res, fault_after_op_cleanup); + { + faultInjectionAfter( + [&] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(res); + }); + } ++calls_without_fault_injection; @@ -381,8 +430,12 @@ private: if constexpr (inject_failure_after_op) { - void * stub = nullptr; /// just for template overloading - faultInjectionAfter(stub, fault_after_op_cleanup); + faultInjectionAfter( + [&fault_after_op_cleanup] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(); + }); } ++calls_without_fault_injection; @@ -403,6 +456,8 @@ private: e.code, e.message()); + if (keeper) + keeper_prev = keeper; keeper.reset(); /// for try*NoThrow() methods diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py index 24299b7ee87..09849998185 100644 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -12,7 +12,6 @@ from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", with_zookeeper=True) -node2 = cluster.add_instance("node2", with_zookeeper=True) @pytest.fixture(scope="module") @@ -28,17 +27,11 @@ def started_cluster(): def test_replica_inserts_with_keeper_restart(started_cluster): try: - settings = { - "insert_quorum": "2", - } node1.query( "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" ) - node2.query( - "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '1') ORDER BY tuple()" - ) - p = Pool(3) + p = Pool(1) zk_stopped_event = threading.Event() def zoo_restart(zk_stopped_event): @@ -48,45 +41,28 @@ def test_replica_inserts_with_keeper_restart(started_cluster): job = p.apply_async(zoo_restart, (zk_stopped_event,)) - zk_stopped_event.wait(60) + zk_stopped_event.wait(90) + node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10)") node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10)", - settings=settings, - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)", - settings=settings, - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(20, 10)", - settings=settings, + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)" ) job.wait() p.close() p.join() - assert node1.query("SELECT COUNT() FROM r") == "30\n" - assert node2.query("SELECT COUNT() FROM r") == "30\n" + assert node1.query("SELECT COUNT() FROM r") == "20\n" finally: node1.query("DROP TABLE IF EXISTS r SYNC") - node2.query("DROP TABLE IF EXISTS r SYNC") -@pytest.mark.skip(reason="Unfortunately it showed to be flaky. Disabled for now") def test_replica_inserts_with_keeper_disconnect(started_cluster): try: - settings = { - "insert_quorum": "2", - } node1.query( "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" ) - node2.query( - "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '1') ORDER BY tuple()" - ) p = Pool(1) disconnect_event = threading.Event() @@ -95,7 +71,6 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): with PartitionManager() as pm: pm.drop_instance_zk_connections(node) event.set() - time.sleep(5) job = p.apply_async( keeper_disconnect, @@ -104,24 +79,18 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): disconnect_event, ), ) - disconnect_event.wait(60) + disconnect_event.wait(90) + node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10)") node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10)", - settings=settings, - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)", - settings=settings, - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(20, 10)", - settings=settings, + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)" ) - assert node1.query("SELECT COUNT() FROM r") == "30\n" - assert node2.query("SELECT COUNT() FROM r") == "30\n" + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "20\n" finally: node1.query("DROP TABLE IF EXISTS r SYNC") - node2.query("DROP TABLE IF EXISTS r SYNC") diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql index 875bd1f96de..78adbee612f 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql @@ -1,5 +1,9 @@ -- Tags: long, replica +-- in case of keeper fault injection on insert, set bigger number of retries because partitions +set insert_keeper_max_retries=100; +set insert_keeper_retry_max_backoff_ms=10; + -- Testing basic functionality with compact parts set replication_alter_partitions_sync = 2; drop table if exists mt_compact; diff --git a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh index 34fa822b6ea..f7615974237 100755 --- a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh +++ b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" $CLICKHOUSE_CLIENT --query " CREATE TABLE mutation_table( @@ -17,9 +17,10 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY key % 10 " -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table VALUES(0, 'hello')" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table VALUES(0, 'hello')" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM mutation_table" @@ -71,4 +72,4 @@ $CLICKHOUSE_CLIENT --query "SELECT is_done, parts_to_do FROM system.mutations wh $CLICKHOUSE_CLIENT --query "SELECT type, new_part_name FROM system.replication_queue WHERE table='mutation_table' and database='$CLICKHOUSE_DATABASE'" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" diff --git a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh index 9325cac0ae6..445706e35bf 100755 --- a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh +++ b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0" @@ -19,9 +19,10 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q2 (x UInt64) ENGINE=ReplicatedMerg $CLICKHOUSE_CLIENT -q "SYSTEM STOP REPLICATION QUEUES parallel_q2" -$CLICKHOUSE_CLIENT -q "INSERT INTO parallel_q1 VALUES (1)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO parallel_q1 VALUES (1)" -$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --query="INSERT INTO parallel_q1 VALUES (2)" & +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --insert_keeper_fault_injection_probability=0 --query="INSERT INTO parallel_q1 VALUES (2)" & part_count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM system.parts WHERE table='parallel_q1' and database='${CLICKHOUSE_DATABASE}'") @@ -66,5 +67,5 @@ $CLICKHOUSE_CLIENT --query="SELECT event_type FROM system.part_log WHERE table=' $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q2" $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" From 4cabe1f57c70e4b856d786fc25d0f40ee6fcec6c Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Tue, 2 Aug 2022 13:45:06 +0300 Subject: [PATCH 080/526] Added applied row-level policies to `system.query_log`. --- src/Access/Common/RowPolicyDefs.h | 6 +++ src/Access/ContextAccess.cpp | 6 +-- src/Access/ContextAccess.h | 4 +- src/Access/EnabledRowPolicies.cpp | 39 +++++++++++-------- src/Access/EnabledRowPolicies.h | 20 ++++++++-- src/Access/RowPolicy.h | 2 + src/Access/RowPolicyCache.cpp | 7 +++- src/Interpreters/Context.cpp | 6 ++- src/Interpreters/Context.h | 3 +- .../IInterpreterUnionOrSelectQuery.cpp | 8 +++- .../IInterpreterUnionOrSelectQuery.h | 3 +- src/Interpreters/InterpreterSelectQuery.cpp | 22 ++++++++--- src/Interpreters/InterpreterSelectQuery.h | 4 +- src/Interpreters/QueryLog.cpp | 29 +++++++++++--- src/Interpreters/QueryLog.h | 5 +++ 15 files changed, 121 insertions(+), 43 deletions(-) diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h index 792884c56df..11b5e46e9c5 100644 --- a/src/Access/Common/RowPolicyDefs.h +++ b/src/Access/Common/RowPolicyDefs.h @@ -43,6 +43,12 @@ enum class RowPolicyFilterType MAX }; +enum class RowPolicyOrderType +{ + Restrictive = 0, + Permissive = 1 +}; + String toString(RowPolicyFilterType type); struct RowPolicyFilterTypeInfo diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 4e409946666..a1d25ee2791 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -379,12 +379,12 @@ std::shared_ptr ContextAccess::getEnabledRowPolicies() return no_row_policies; } -ASTPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const +RowPolicyFilter ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const { std::lock_guard lock{mutex}; if (enabled_row_policies) - return enabled_row_policies->getFilter(database, table_name, filter_type, combine_with_expr); - return nullptr; + return enabled_row_policies->getFilter(database, table_name, filter_type, combine_with_filter); + return combine_with_filter; } std::shared_ptr ContextAccess::getQuota() const diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index ce1ea2d1220..dbfffb8bda5 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -87,7 +87,7 @@ public: /// Returns the row policy filter for a specified table. /// The function returns nullptr if there is no filter to apply. - ASTPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr = nullptr) const; + RowPolicyFilter getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter = {}) const; /// Returns the quota to track resource consumption. std::shared_ptr getQuota() const; diff --git a/src/Access/EnabledRowPolicies.cpp b/src/Access/EnabledRowPolicies.cpp index e4f592884fe..626f7013941 100644 --- a/src/Access/EnabledRowPolicies.cpp +++ b/src/Access/EnabledRowPolicies.cpp @@ -6,12 +6,19 @@ namespace DB { + +void RowPolicyFilter::optimize() +{ + bool value; + if (tryGetLiteralBool(expression.get(), value) && value) + expression.reset(); /// The condition is always true, no need to check it. +} + size_t EnabledRowPolicies::Hash::operator()(const MixedFiltersKey & key) const { return std::hash{}(key.database) - std::hash{}(key.table_name) + static_cast(key.filter_type); } - EnabledRowPolicies::EnabledRowPolicies() : params() { } @@ -23,7 +30,7 @@ EnabledRowPolicies::EnabledRowPolicies(const Params & params_) : params(params_) EnabledRowPolicies::~EnabledRowPolicies() = default; -ASTPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const +RowPolicyFilter EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { /// We don't lock `mutex` here. auto loaded = mixed_filters.load(); @@ -31,26 +38,26 @@ ASTPtr EnabledRowPolicies::getFilter(const String & database, const String & tab if (it == loaded->end()) return {}; - auto filter = it->second.ast; - - bool value; - if (tryGetLiteralBool(filter.get(), value) && value) - return nullptr; /// The condition is always true, no need to check it. + RowPolicyFilter filter = {it->second.ast, it->second.policies}; + filter.optimize(); return filter; } -ASTPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const +RowPolicyFilter EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const { - ASTPtr filter = getFilter(database, table_name, filter_type); - if (filter && combine_with_expr) - filter = makeASTForLogicalAnd({filter, combine_with_expr}); - else if (!filter) - filter = combine_with_expr; + RowPolicyFilter filter = getFilter(database, table_name, filter_type); + if (filter.expression && combine_with_filter.expression) + { + filter.expression = makeASTForLogicalAnd({filter.expression, combine_with_filter.expression}); + } + else if (!filter.expression) + { + filter.expression = combine_with_filter.expression; + } - bool value; - if (tryGetLiteralBool(filter.get(), value) && value) - return nullptr; /// The condition is always true, no need to check it. + std::copy(combine_with_filter.policies.begin(), combine_with_filter.policies.end(), std::back_inserter(filter.policies)); + filter.optimize(); return filter; } diff --git a/src/Access/EnabledRowPolicies.h b/src/Access/EnabledRowPolicies.h index a3d3f009408..ee066759aae 100644 --- a/src/Access/EnabledRowPolicies.h +++ b/src/Access/EnabledRowPolicies.h @@ -1,12 +1,16 @@ #pragma once #include +#include #include #include + #include #include -#include + #include +#include +#include namespace DB @@ -15,6 +19,15 @@ class IAST; using ASTPtr = std::shared_ptr; +struct RowPolicyFilter +{ + ASTPtr expression; + std::vector policies; + + void optimize(); +}; + + /// Provides fast access to row policies' conditions for a specific user and tables. class EnabledRowPolicies { @@ -39,8 +52,8 @@ public: /// Returns prepared filter for a specific table and operations. /// The function can return nullptr, that means there is no filters applied. /// The returned filter can be a combination of the filters defined by multiple row policies. - ASTPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; - ASTPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const; + RowPolicyFilter getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; + RowPolicyFilter getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const; private: friend class RowPolicyCache; @@ -61,6 +74,7 @@ private: { ASTPtr ast; std::shared_ptr> database_and_table_name; + std::vector policies; }; struct Hash diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 99e6f1992f5..dcf1fce54ff 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -35,6 +35,8 @@ struct RowPolicy : public IAccessEntity void setPermissive(bool permissive_ = true) { setRestrictive(!permissive_); } bool isPermissive() const { return !isRestrictive(); } + RowPolicyOrderType getOrderType() const { return isRestrictive() ? RowPolicyOrderType::Restrictive : RowPolicyOrderType::Permissive; } + /// Sets that the policy is restrictive. /// A row is only accessible if at least one of the permissive policies passes, /// in addition to all the restrictive policies. diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 34549e3fa62..60d73802dcc 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -212,6 +212,7 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) { FiltersMixer mixer; std::shared_ptr> database_and_table_name; + std::vector policies; }; std::unordered_map mixers; @@ -232,7 +233,10 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) auto & mixer = mixers[key]; mixer.database_and_table_name = info.database_and_table_name; if (match) + { mixer.mixer.add(info.parsed_filters[filter_type_i], policy.isRestrictive()); + mixer.policies.push_back(info.policy); + } } } } @@ -241,8 +245,9 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) for (auto & [key, mixer] : mixers) { auto & mixed_filter = (*mixed_filters)[key]; - mixed_filter.database_and_table_name = mixer.database_and_table_name; + mixed_filter.database_and_table_name = std::move(mixer.database_and_table_name); mixed_filter.ast = std::move(mixer.mixer).getResult(access_control.isEnabledUsersWithoutRowPoliciesCanReadRows()); + mixed_filter.policies = std::move(mixer.policies); } enabled.mixed_filters.store(mixed_filters); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 9c949a17e64..095ac3d9327 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -987,10 +987,12 @@ std::shared_ptr Context::getAccess() const return access ? access : ContextAccess::getFullAccess(); } -ASTPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const +RowPolicyFilter Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { auto lock = getLock(); - auto row_filter_of_initial_user = row_policies_of_initial_user ? row_policies_of_initial_user->getFilter(database, table_name, filter_type) : nullptr; + RowPolicyFilter row_filter_of_initial_user; + if (row_policies_of_initial_user) + row_filter_of_initial_user = row_policies_of_initial_user->getFilter(database, table_name, filter_type); return getAccess()->getRowPolicyFilter(database, table_name, filter_type, row_filter_of_initial_user); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index eeb9e8da148..daefa91176c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -516,7 +517,7 @@ public: std::shared_ptr getAccess() const; - ASTPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; + RowPolicyFilter getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; /// Finds and sets extra row policies to be used based on `client_info.initial_user`, /// if the initial user exists. diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index 6319949655b..ce6cbb8b7af 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -15,12 +15,16 @@ namespace DB { -void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const +void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +{ + extendQueryLogElemImplImpl(elem, ast, context_); +} + +void IInterpreterUnionOrSelectQuery::extendQueryLogElemImplImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const { elem.query_kind = "Select"; } - QueryPipelineBuilder IInterpreterUnionOrSelectQuery::buildQueryPipeline() { QueryPlan query_plan; diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index a1c86f9de85..e294a6e5a7d 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -44,7 +44,8 @@ public: size_t getMaxStreams() const { return max_streams; } - void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const override; + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + void extendQueryLogElemImplImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const; /// Returns whether the query uses the view source from the Context /// The view source is a virtual storage that currently only materialized views use to replace the source table diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d8ac263e3d1..ef398bd2d69 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #include @@ -112,7 +113,7 @@ namespace ErrorCodes /// Assumes `storage` is set and the table filter (row-level security) is not empty. FilterDAGInfoPtr generateFilterActions( const StorageID & table_id, - const ASTPtr & row_policy_filter, + const ASTPtr & row_policy_filter_expression, const ContextPtr & context, const StoragePtr & storage, const StorageSnapshotPtr & storage_snapshot, @@ -133,9 +134,9 @@ FilterDAGInfoPtr generateFilterActions( auto expr_list = select_ast->select(); /// The first column is our filter expression. - /// the row_policy_filter should be cloned, because it may be changed by TreeRewriter. + /// the row_policy_filter_expression should be cloned, because it may be changed by TreeRewriter. /// which make it possible an invalid expression, although it may be valid in whole select. - expr_list->children.push_back(row_policy_filter->clone()); + expr_list->children.push_back(row_policy_filter_expression->clone()); /// Keep columns that are required after the filter actions. for (const auto & column_str : prerequisite_columns) @@ -613,10 +614,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.clear(); /// Fix source_header for filter actions. - if (row_policy_filter) + if (row_policy_filter.expression) { filter_info = generateFilterActions( - table_id, row_policy_filter, context, storage, storage_snapshot, metadata_snapshot, required_columns, + table_id, row_policy_filter.expression, context, storage, storage_snapshot, metadata_snapshot, required_columns, prepared_sets); query_info.filter_asts.push_back(row_policy_filter); @@ -1866,6 +1867,17 @@ void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t repl context->getClientInfo().number_of_current_replica = replica_num; } +void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +{ + extendQueryLogElemImplImpl(elem, ast, context_); + + for (const auto & row_policy : row_policy_filter.policies) + { + elem.row_policies_names.push_back(row_policy->getShortName()); + elem.row_policies_order_types.push_back(row_policy->getOrderType()); + } +} + bool InterpreterSelectQuery::shouldMoveToPrewhere() { const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 3b933547422..62aeaee1bfc 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -153,6 +153,8 @@ private: const StorageMetadataPtr & metadata_snapshot_ = nullptr, PreparedSetsPtr prepared_sets_ = nullptr); + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } void addPrewhereAliasActions(); @@ -209,7 +211,7 @@ private: /// Is calculated in getSampleBlock. Is used later in readImpl. ExpressionAnalysisResult analysis_result; /// For row-level security. - ASTPtr row_policy_filter; + RowPolicyFilter row_policy_filter; FilterDAGInfoPtr filter_info; /// For additional_filter setting. diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 72d9ce44102..187d9ad7e13 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -1,4 +1,5 @@ -#include +#include + #include #include #include @@ -15,13 +16,15 @@ #include #include #include -#include -#include #include #include #include #include +#include + +#include + namespace DB { @@ -37,6 +40,13 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)} }); + auto row_policy_order_type = std::make_shared( + DataTypeEnum8::Values + { + {"Restrictive", static_cast(RowPolicyOrderType::Restrictive)}, + {"Permissive", static_cast(RowPolicyOrderType::Permissive)}, + }); + return { {"type", std::move(query_status_datatype)}, @@ -118,6 +128,9 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"used_functions", std::make_shared(std::make_shared())}, {"used_storages", std::make_shared(std::make_shared())}, {"used_table_functions", std::make_shared(std::make_shared())}, + + {"used_row_policies.name", std::make_shared(std::make_shared(std::make_shared()))}, + {"used_row_policies.type", std::make_shared(row_policy_order_type)}, {"transaction_id", getTransactionIDDataType()}, }; @@ -236,13 +249,15 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_function_factory_objects = typeid_cast(*columns[i++]); auto & column_storage_factory_objects = typeid_cast(*columns[i++]); auto & column_table_function_factory_objects = typeid_cast(*columns[i++]); + auto & column_row_policies_names = typeid_cast(*columns[i++]); + auto & column_row_policies_order_types = typeid_cast(*columns[i++]); - auto fill_column = [](const std::unordered_set & data, ColumnArray & column) + auto fill_column = [](const auto & data, ColumnArray & column) { size_t size = 0; - for (const auto & name : data) + for (const auto & value : data) { - column.getData().insertData(name.data(), name.size()); + column.getData().insert(value); ++size; } auto & offsets = column.getOffsets(); @@ -258,6 +273,8 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(used_functions, column_function_factory_objects); fill_column(used_storages, column_storage_factory_objects); fill_column(used_table_functions, column_table_function_factory_objects); + fill_column(row_policies_names, column_row_policies_names); + fill_column(row_policies_order_types, column_row_policies_order_types); } columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index bb912ccc817..c2e9ee883ec 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -7,6 +8,7 @@ #include #include + namespace ProfileEvents { class Counters; @@ -86,6 +88,9 @@ struct QueryLogElement std::shared_ptr profile_counters; std::shared_ptr query_settings; + std::vector row_policies_names; + std::vector row_policies_order_types; + TransactionID tid; static std::string name() { return "QueryLog"; } From c28e439c3321a8ca0ef20d9301f57d4abbf904ea Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Sat, 13 Aug 2022 16:03:16 +0300 Subject: [PATCH 081/526] Review fixes. --- src/Access/Common/RowPolicyDefs.h | 6 ---- src/Access/RowPolicy.h | 2 -- .../IInterpreterUnionOrSelectQuery.cpp | 3 +- .../InterpreterSelectIntersectExceptQuery.cpp | 24 ++++++++++++++ .../InterpreterSelectIntersectExceptQuery.h | 2 ++ src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++-- src/Interpreters/InterpreterSelectQuery.h | 6 ++-- .../InterpreterSelectWithUnionQuery.cpp | 25 +++++++++++++++ .../InterpreterSelectWithUnionQuery.h | 2 ++ src/Interpreters/QueryLog.cpp | 32 ++++++++++++------- src/Interpreters/QueryLog.h | 5 +-- .../02131_row_policies_combination.reference | 24 ++++++++++++++ .../02131_row_policies_combination.sql | 6 ++++ 13 files changed, 124 insertions(+), 29 deletions(-) diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h index 11b5e46e9c5..792884c56df 100644 --- a/src/Access/Common/RowPolicyDefs.h +++ b/src/Access/Common/RowPolicyDefs.h @@ -43,12 +43,6 @@ enum class RowPolicyFilterType MAX }; -enum class RowPolicyOrderType -{ - Restrictive = 0, - Permissive = 1 -}; - String toString(RowPolicyFilterType type); struct RowPolicyFilterTypeInfo diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index dcf1fce54ff..99e6f1992f5 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -35,8 +35,6 @@ struct RowPolicy : public IAccessEntity void setPermissive(bool permissive_ = true) { setRestrictive(!permissive_); } bool isPermissive() const { return !isRestrictive(); } - RowPolicyOrderType getOrderType() const { return isRestrictive() ? RowPolicyOrderType::Restrictive : RowPolicyOrderType::Permissive; } - /// Sets that the policy is restrictive. /// A row is only accessible if at least one of the permissive policies passes, /// in addition to all the restrictive policies. diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index ce6cbb8b7af..a6236a95e00 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -6,12 +6,13 @@ #include #include #include -#include #include #include +#include #include #include + namespace DB { diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index a134f7bb913..ac758732315 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -1,7 +1,10 @@ +#include + #include #include #include #include +#include #include #include #include @@ -188,4 +191,25 @@ void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } +void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +{ + extendQueryLogElemImplImpl(elem, ast, context_); + + const auto & access_control = context_->getAccessControl(); + + for (auto & interpreter : nested_interpreters) + { + if (auto select_interpreter = dynamic_cast(interpreter.get())) + { + auto policies = select_interpreter->getUsedRowPolicies(); + for (const auto & row_policy : policies) + { + auto name = row_policy->getFullName().toString(); + std::optional id = access_control.find(name); + elem.used_row_policies.emplace(std::move(name), std::move(*id)); + } + } + } +} + } diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h index 79ee32e514a..e93ab7bd2e9 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.h +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.h @@ -35,6 +35,8 @@ public: void ignoreWithTotals() override; + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + private: static String getName() { return "SelectIntersectExceptQuery"; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index ef398bd2d69..e87d7f2dc4d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1,3 +1,5 @@ +#include + #include #include @@ -620,7 +622,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( table_id, row_policy_filter.expression, context, storage, storage_snapshot, metadata_snapshot, required_columns, prepared_sets); - query_info.filter_asts.push_back(row_policy_filter); + query_info.filter_asts.push_back(row_policy_filter.expression); } if (query_info.additional_filter_ast) @@ -1867,14 +1869,22 @@ void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t repl context->getClientInfo().number_of_current_replica = replica_num; } +const std::vector & InterpreterSelectQuery::getUsedRowPolicies() const +{ + return row_policy_filter.policies; +} + void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const { extendQueryLogElemImplImpl(elem, ast, context_); + const auto & access_control = context_->getAccessControl(); + for (const auto & row_policy : row_policy_filter.policies) { - elem.row_policies_names.push_back(row_policy->getShortName()); - elem.row_policies_order_types.push_back(row_policy->getOrderType()); + auto name = row_policy->getFullName().toString(); + std::optional id = access_control.find(name); + elem.used_row_policies.emplace(std::move(name), std::move(*id)); } } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 62aeaee1bfc..a64b7cdcc29 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -129,6 +129,10 @@ public: FilterDAGInfoPtr getAdditionalQueryInfo() const { return additional_filter_info; } + const std::vector & getUsedRowPolicies() const; + + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + static SortDescription getSortDescription(const ASTSelectQuery & query, const ContextPtr & context); static UInt64 getLimitForSorting(const ASTSelectQuery & query, const ContextPtr & context); @@ -153,8 +157,6 @@ private: const StorageMetadataPtr & metadata_snapshot_ = nullptr, PreparedSetsPtr prepared_sets_ = nullptr); - void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; - ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } void addPrewhereAliasActions(); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index a679b17a5bd..e10b450dca5 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -1,8 +1,11 @@ +#include + #include #include #include #include #include +#include #include #include #include @@ -22,6 +25,7 @@ #include + namespace DB { @@ -382,4 +386,25 @@ void InterpreterSelectWithUnionQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } +void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +{ + extendQueryLogElemImplImpl(elem, ast, context_); + + const auto & access_control = context_->getAccessControl(); + + for (auto & interpreter : nested_interpreters) + { + if (auto select_interpreter = dynamic_cast(interpreter.get())) + { + auto policies = select_interpreter->getUsedRowPolicies(); + for (const auto & row_policy : policies) + { + auto name = row_policy->getFullName().toString(); + std::optional id = access_control.find(name); + elem.used_row_policies.emplace(std::move(name), std::move(*id)); + } + } + } +} + } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.h b/src/Interpreters/InterpreterSelectWithUnionQuery.h index ff763ec6490..2ec4fbfceaf 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -47,6 +47,8 @@ public: bool supportsTransactions() const override { return true; } + void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; + private: std::vector> nested_interpreters; diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 187d9ad7e13..6dd85a4cb78 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -40,13 +41,6 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"ExceptionWhileProcessing", static_cast(EXCEPTION_WHILE_PROCESSING)} }); - auto row_policy_order_type = std::make_shared( - DataTypeEnum8::Values - { - {"Restrictive", static_cast(RowPolicyOrderType::Restrictive)}, - {"Permissive", static_cast(RowPolicyOrderType::Permissive)}, - }); - return { {"type", std::move(query_status_datatype)}, @@ -130,7 +124,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"used_table_functions", std::make_shared(std::make_shared())}, {"used_row_policies.name", std::make_shared(std::make_shared(std::make_shared()))}, - {"used_row_policies.type", std::make_shared(row_policy_order_type)}, + {"used_row_policies.uuid", std::make_shared(std::make_shared())}, {"transaction_id", getTransactionIDDataType()}, }; @@ -250,7 +244,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_storage_factory_objects = typeid_cast(*columns[i++]); auto & column_table_function_factory_objects = typeid_cast(*columns[i++]); auto & column_row_policies_names = typeid_cast(*columns[i++]); - auto & column_row_policies_order_types = typeid_cast(*columns[i++]); + auto & column_row_policies_uuids = typeid_cast(*columns[i++]); auto fill_column = [](const auto & data, ColumnArray & column) { @@ -273,8 +267,24 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(used_functions, column_function_factory_objects); fill_column(used_storages, column_storage_factory_objects); fill_column(used_table_functions, column_table_function_factory_objects); - fill_column(row_policies_names, column_row_policies_names); - fill_column(row_policies_order_types, column_row_policies_order_types); + + { + size_t size = 0; + Array uuid_array; + uuid_array.reserve(used_row_policies.size()); + + for (const auto & [name, uuid] : used_row_policies) + { + column_row_policies_names.getData().insert(name); + uuid_array.emplace_back(uuid); + ++size; + } + + auto & column_row_policies_names_offsets = column_row_policies_names.getOffsets(); + column_row_policies_names_offsets.push_back(column_row_policies_names_offsets.back() + size); + + column_row_policies_uuids.insert(uuid_array); + } } columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index c2e9ee883ec..88be93aa2f8 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -75,6 +74,7 @@ struct QueryLogElement std::unordered_set used_functions; std::unordered_set used_storages; std::unordered_set used_table_functions; + std::set> used_row_policies; Int32 exception_code{}; // because ErrorCodes are int String exception; @@ -88,9 +88,6 @@ struct QueryLogElement std::shared_ptr profile_counters; std::shared_ptr query_settings; - std::vector row_policies_names; - std::vector row_policies_order_types; - TransactionID tid; static std::string name() { return "QueryLog"; } diff --git a/tests/queries/0_stateless/02131_row_policies_combination.reference b/tests/queries/0_stateless/02131_row_policies_combination.reference index b76028d5077..0d7afc265a3 100644 --- a/tests/queries/0_stateless/02131_row_policies_combination.reference +++ b/tests/queries/0_stateless/02131_row_policies_combination.reference @@ -31,3 +31,27 @@ None 2 3 4 +Check system.query_log +SELECT \'None\'; [] [] +SELECT * FROM 02131_rptable; [] [] +SELECT \'R1: x == 1\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_1'] ['Permissive'] +SELECT \'R1, R2: (x == 1) OR (x == 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_1','02131_filter_2'] ['Permissive','Permissive'] +SELECT \'R1, R2, R3: (x == 1) OR (x == 2) OR (x == 3)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_3','02131_filter_1','02131_filter_2'] ['Permissive','Permissive','Permissive'] +SELECT \'R1, R2, R3, R4: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_3','02131_filter_1','02131_filter_4','02131_filter_2'] ['Permissive','Permissive','Restrictive','Permissive'] +SELECT \'R1, R2, R3, R4, R5: ((x == 1) OR (x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_5','02131_filter_3','02131_filter_1','02131_filter_4','02131_filter_2'] ['Restrictive','Permissive','Permissive','Restrictive','Permissive'] +SELECT \'R2, R3, R4, R5: ((x == 2) OR (x == 3)) AND (x <= 2) AND (x >= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_5','02131_filter_3','02131_filter_4','02131_filter_2'] ['Restrictive','Permissive','Restrictive','Permissive'] +SELECT \'R3, R4, R5: (x == 3) AND (x <= 2) AND (x >= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_5','02131_filter_3','02131_filter_4'] ['Restrictive','Permissive','Restrictive'] +SELECT \'R4, R5: (x <= 2) AND (x >= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_5','02131_filter_4'] ['Restrictive','Restrictive'] +SELECT \'R5: (x >= 2)\'; [] [] +SELECT * FROM 02131_rptable; ['02131_filter_5'] ['Restrictive'] +SELECT \'None\'; [] [] +SELECT * FROM 02131_rptable; [] [] +SELECT \'Check system.query_log\'; [] [] diff --git a/tests/queries/0_stateless/02131_row_policies_combination.sql b/tests/queries/0_stateless/02131_row_policies_combination.sql index b5be672bb1b..745ab1288c7 100644 --- a/tests/queries/0_stateless/02131_row_policies_combination.sql +++ b/tests/queries/0_stateless/02131_row_policies_combination.sql @@ -52,3 +52,9 @@ SELECT 'None'; SELECT * FROM 02131_rptable; DROP TABLE 02131_rptable; + +SELECT 'Check system.query_log'; + +SYSTEM FLUSH LOGS; + +SELECT query, used_row_policies.name, used_row_policies.type FROM system.query_log WHERE event_date >= yesterday() AND current_database == currentDatabase() AND type == 'QueryStart' AND query_kind == 'Select'; From 0c485d786c0deb87e64a776e5defd06e87fcc09b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Thu, 18 Aug 2022 10:47:29 +0300 Subject: [PATCH 082/526] Better includes. --- src/Interpreters/Context.h | 2 +- src/Interpreters/InterpreterSelectQuery.h | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index daefa91176c..42480b5817c 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -46,6 +45,7 @@ struct User; using UserPtr = std::shared_ptr; struct EnabledRolesInfo; class EnabledRowPolicies; +struct RowPolicyFilter; class EnabledQuota; struct QuotaUsage; class AccessFlags; diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index a64b7cdcc29..23b0f81fae6 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -2,6 +2,7 @@ #include +#include #include #include #include @@ -23,6 +24,7 @@ class Logger; namespace DB { + class SubqueryForSet; class InterpreterSelectWithUnionQuery; class Context; @@ -34,6 +36,9 @@ using GroupingSetsParamsList = std::vector; struct TreeRewriterResult; using TreeRewriterResultPtr = std::shared_ptr; +struct RowPolicy; +using RowPolicyPtr = std::shared_ptr; + /** Interprets the SELECT query. Returns the stream of blocks with the results of the query before `to_stage` stage. */ From 1ec199b5391b3a6a1ecb3a430c857dd7891fb196 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Thu, 20 Oct 2022 10:46:16 +0300 Subject: [PATCH 083/526] Style fix. --- src/Interpreters/QueryLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 6dd85a4cb78..85cd09fa618 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -122,7 +122,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"used_functions", std::make_shared(std::make_shared())}, {"used_storages", std::make_shared(std::make_shared())}, {"used_table_functions", std::make_shared(std::make_shared())}, - + {"used_row_policies.name", std::make_shared(std::make_shared(std::make_shared()))}, {"used_row_policies.uuid", std::make_shared(std::make_shared())}, From 0707d883112d73ea75426abec243fc7e4d32d576 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Fri, 21 Oct 2022 18:07:17 +0300 Subject: [PATCH 084/526] Removed `extendQueryLogElemImplImpl`. --- src/Interpreters/IInterpreterUnionOrSelectQuery.cpp | 7 +------ src/Interpreters/IInterpreterUnionOrSelectQuery.h | 1 - src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp | 4 ++-- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- src/Interpreters/InterpreterSelectWithUnionQuery.cpp | 4 ++-- 5 files changed, 7 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp index a6236a95e00..5df3abbb32e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.cpp @@ -16,12 +16,7 @@ namespace DB { -void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const -{ - extendQueryLogElemImplImpl(elem, ast, context_); -} - -void IInterpreterUnionOrSelectQuery::extendQueryLogElemImplImpl(QueryLogElement & elem, const ASTPtr &, ContextPtr) const +void IInterpreterUnionOrSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { elem.query_kind = "Select"; } diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index e294a6e5a7d..81554340669 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -45,7 +45,6 @@ public: size_t getMaxStreams() const { return max_streams; } void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; - void extendQueryLogElemImplImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const; /// Returns whether the query uses the view source from the Context /// The view source is a virtual storage that currently only materialized views use to replace the source table diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index ac758732315..d455a3ce6fb 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -191,9 +191,9 @@ void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const { - extendQueryLogElemImplImpl(elem, ast, context_); + elem.query_kind = "Select"; const auto & access_control = context_->getAccessControl(); diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index e87d7f2dc4d..9b0056800cb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1874,9 +1874,9 @@ const std::vector & InterpreterSelectQuery::getUsedRowPolicies() c return row_policy_filter.policies; } -void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const { - extendQueryLogElemImplImpl(elem, ast, context_); + elem.query_kind = "Select"; const auto & access_control = context_->getAccessControl(); diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index e10b450dca5..19b744be97a 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -386,9 +386,9 @@ void InterpreterSelectWithUnionQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context_) const +void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const { - extendQueryLogElemImplImpl(elem, ast, context_); + elem.query_kind = "Select"; const auto & access_control = context_->getAccessControl(); From 6d5d31e49c391a95835ea64f3cbbb78321dfd25f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Fri, 21 Oct 2022 19:13:36 +0300 Subject: [PATCH 085/526] Removed UUID's. --- .../InterpreterSelectIntersectExceptQuery.cpp | 7 ++---- src/Interpreters/InterpreterSelectQuery.cpp | 7 ++---- .../InterpreterSelectWithUnionQuery.cpp | 7 ++---- src/Interpreters/QueryLog.cpp | 23 ++----------------- src/Interpreters/QueryLog.h | 2 +- 5 files changed, 9 insertions(+), 37 deletions(-) diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index d455a3ce6fb..7a6ba69d8a5 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -191,12 +191,10 @@ void InterpreterSelectIntersectExceptQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const +void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { elem.query_kind = "Select"; - const auto & access_control = context_->getAccessControl(); - for (auto & interpreter : nested_interpreters) { if (auto select_interpreter = dynamic_cast(interpreter.get())) @@ -205,8 +203,7 @@ void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogEleme for (const auto & row_policy : policies) { auto name = row_policy->getFullName().toString(); - std::optional id = access_control.find(name); - elem.used_row_policies.emplace(std::move(name), std::move(*id)); + elem.used_row_policies.emplace(std::move(name)); } } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9b0056800cb..867a90c7e00 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1874,17 +1874,14 @@ const std::vector & InterpreterSelectQuery::getUsedRowPolicies() c return row_policy_filter.policies; } -void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const +void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { elem.query_kind = "Select"; - const auto & access_control = context_->getAccessControl(); - for (const auto & row_policy : row_policy_filter.policies) { auto name = row_policy->getFullName().toString(); - std::optional id = access_control.find(name); - elem.used_row_policies.emplace(std::move(name), std::move(*id)); + elem.used_row_policies.emplace(std::move(name)); } } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 19b744be97a..2e223d3a0b9 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -386,12 +386,10 @@ void InterpreterSelectWithUnionQuery::ignoreWithTotals() interpreter->ignoreWithTotals(); } -void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr context_) const +void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { elem.query_kind = "Select"; - const auto & access_control = context_->getAccessControl(); - for (auto & interpreter : nested_interpreters) { if (auto select_interpreter = dynamic_cast(interpreter.get())) @@ -400,8 +398,7 @@ void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & e for (const auto & row_policy : policies) { auto name = row_policy->getFullName().toString(); - std::optional id = access_control.find(name); - elem.used_row_policies.emplace(std::move(name), std::move(*id)); + elem.used_row_policies.emplace(std::move(name)); } } } diff --git a/src/Interpreters/QueryLog.cpp b/src/Interpreters/QueryLog.cpp index 85cd09fa618..224ca9f11a8 100644 --- a/src/Interpreters/QueryLog.cpp +++ b/src/Interpreters/QueryLog.cpp @@ -123,8 +123,7 @@ NamesAndTypesList QueryLogElement::getNamesAndTypes() {"used_storages", std::make_shared(std::make_shared())}, {"used_table_functions", std::make_shared(std::make_shared())}, - {"used_row_policies.name", std::make_shared(std::make_shared(std::make_shared()))}, - {"used_row_policies.uuid", std::make_shared(std::make_shared())}, + {"used_row_policies", std::make_shared(std::make_shared(std::make_shared()))}, {"transaction_id", getTransactionIDDataType()}, }; @@ -244,7 +243,6 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const auto & column_storage_factory_objects = typeid_cast(*columns[i++]); auto & column_table_function_factory_objects = typeid_cast(*columns[i++]); auto & column_row_policies_names = typeid_cast(*columns[i++]); - auto & column_row_policies_uuids = typeid_cast(*columns[i++]); auto fill_column = [](const auto & data, ColumnArray & column) { @@ -267,24 +265,7 @@ void QueryLogElement::appendToBlock(MutableColumns & columns) const fill_column(used_functions, column_function_factory_objects); fill_column(used_storages, column_storage_factory_objects); fill_column(used_table_functions, column_table_function_factory_objects); - - { - size_t size = 0; - Array uuid_array; - uuid_array.reserve(used_row_policies.size()); - - for (const auto & [name, uuid] : used_row_policies) - { - column_row_policies_names.getData().insert(name); - uuid_array.emplace_back(uuid); - ++size; - } - - auto & column_row_policies_names_offsets = column_row_policies_names.getOffsets(); - column_row_policies_names_offsets.push_back(column_row_policies_names_offsets.back() + size); - - column_row_policies_uuids.insert(uuid_array); - } + fill_column(used_row_policies, column_row_policies_names); } columns[i++]->insert(Tuple{tid.start_csn, tid.local_tid, tid.host_id}); diff --git a/src/Interpreters/QueryLog.h b/src/Interpreters/QueryLog.h index 88be93aa2f8..75cda9719a3 100644 --- a/src/Interpreters/QueryLog.h +++ b/src/Interpreters/QueryLog.h @@ -74,7 +74,7 @@ struct QueryLogElement std::unordered_set used_functions; std::unordered_set used_storages; std::unordered_set used_table_functions; - std::set> used_row_policies; + std::set used_row_policies; Int32 exception_code{}; // because ErrorCodes are int String exception; From d17b7387f9b06a30674afd3df96f9d891fedecfb Mon Sep 17 00:00:00 2001 From: Vladimir Chebotaryov Date: Mon, 24 Oct 2022 10:58:14 +0300 Subject: [PATCH 086/526] Reworked changes to `std::shared_ptr`. --- src/Access/ContextAccess.cpp | 2 +- src/Access/ContextAccess.h | 2 +- src/Access/EnabledRowPolicies.cpp | 47 +++++++++++-------- src/Access/EnabledRowPolicies.h | 19 ++++---- src/Access/RowPolicyCache.cpp | 9 ++-- src/Interpreters/Context.cpp | 4 +- src/Interpreters/Context.h | 3 +- .../InterpreterSelectIntersectExceptQuery.cpp | 11 +++-- src/Interpreters/InterpreterSelectQuery.cpp | 12 ++--- src/Interpreters/InterpreterSelectQuery.h | 4 +- .../InterpreterSelectWithUnionQuery.cpp | 11 +++-- 11 files changed, 69 insertions(+), 55 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index a1d25ee2791..7584dd2f2b1 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -379,7 +379,7 @@ std::shared_ptr ContextAccess::getEnabledRowPolicies() return no_row_policies; } -RowPolicyFilter ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const +RowPolicyFilterPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter) const { std::lock_guard lock{mutex}; if (enabled_row_policies) diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index dbfffb8bda5..84ef0ab722d 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -87,7 +87,7 @@ public: /// Returns the row policy filter for a specified table. /// The function returns nullptr if there is no filter to apply. - RowPolicyFilter getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter = {}) const; + RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter = {}) const; /// Returns the quota to track resource consumption. std::shared_ptr getQuota() const; diff --git a/src/Access/EnabledRowPolicies.cpp b/src/Access/EnabledRowPolicies.cpp index 626f7013941..c00dcf9e3a7 100644 --- a/src/Access/EnabledRowPolicies.cpp +++ b/src/Access/EnabledRowPolicies.cpp @@ -7,11 +7,10 @@ namespace DB { -void RowPolicyFilter::optimize() +bool RowPolicyFilter::empty() const { bool value; - if (tryGetLiteralBool(expression.get(), value) && value) - expression.reset(); /// The condition is always true, no need to check it. + return !expression || (tryGetLiteralBool(expression.get(), value) && value); } size_t EnabledRowPolicies::Hash::operator()(const MixedFiltersKey & key) const @@ -30,7 +29,7 @@ EnabledRowPolicies::EnabledRowPolicies(const Params & params_) : params(params_) EnabledRowPolicies::~EnabledRowPolicies() = default; -RowPolicyFilter EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const +RowPolicyFilterPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { /// We don't lock `mutex` here. auto loaded = mixed_filters.load(); @@ -38,26 +37,36 @@ RowPolicyFilter EnabledRowPolicies::getFilter(const String & database, const Str if (it == loaded->end()) return {}; - RowPolicyFilter filter = {it->second.ast, it->second.policies}; - filter.optimize(); - - return filter; + return it->second; } -RowPolicyFilter EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const +RowPolicyFilterPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter) const { - RowPolicyFilter filter = getFilter(database, table_name, filter_type); - if (filter.expression && combine_with_filter.expression) + RowPolicyFilterPtr filter = getFilter(database, table_name, filter_type); + if (filter && combine_with_filter) { - filter.expression = makeASTForLogicalAnd({filter.expression, combine_with_filter.expression}); - } - else if (!filter.expression) - { - filter.expression = combine_with_filter.expression; - } + auto new_filter = std::make_shared(*filter); - std::copy(combine_with_filter.policies.begin(), combine_with_filter.policies.end(), std::back_inserter(filter.policies)); - filter.optimize(); + if (filter->empty()) + { + new_filter->expression = combine_with_filter->expression; + } + else if (combine_with_filter->empty()) + { + new_filter->expression = filter->expression; + } + else + { + new_filter->expression = makeASTForLogicalAnd({filter->expression, combine_with_filter->expression}); + } + + std::copy(combine_with_filter->policies.begin(), combine_with_filter->policies.end(), std::back_inserter(new_filter->policies)); + filter = new_filter; + } + else if (!filter) + { + filter = combine_with_filter; + } return filter; } diff --git a/src/Access/EnabledRowPolicies.h b/src/Access/EnabledRowPolicies.h index ee066759aae..b8e6b2e0549 100644 --- a/src/Access/EnabledRowPolicies.h +++ b/src/Access/EnabledRowPolicies.h @@ -18,13 +18,17 @@ namespace DB class IAST; using ASTPtr = std::shared_ptr; +struct RowPolicyFilter; +using RowPolicyFilterPtr = std::shared_ptr; + struct RowPolicyFilter { ASTPtr expression; + std::shared_ptr> database_and_table_name; std::vector policies; - void optimize(); + bool empty() const; }; @@ -52,8 +56,8 @@ public: /// Returns prepared filter for a specific table and operations. /// The function can return nullptr, that means there is no filters applied. /// The returned filter can be a combination of the filters defined by multiple row policies. - RowPolicyFilter getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; - RowPolicyFilter getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const RowPolicyFilter & combine_with_filter) const; + RowPolicyFilterPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; + RowPolicyFilterPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, RowPolicyFilterPtr combine_with_filter) const; private: friend class RowPolicyCache; @@ -70,19 +74,12 @@ private: friend bool operator!=(const MixedFiltersKey & left, const MixedFiltersKey & right) { return left.toTuple() != right.toTuple(); } }; - struct MixedFiltersResult - { - ASTPtr ast; - std::shared_ptr> database_and_table_name; - std::vector policies; - }; - struct Hash { size_t operator()(const MixedFiltersKey & key) const; }; - using MixedFiltersMap = std::unordered_map; + using MixedFiltersMap = std::unordered_map; const Params params; mutable boost::atomic_shared_ptr mixed_filters; diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index 60d73802dcc..1036df92609 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -244,10 +244,11 @@ void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) auto mixed_filters = boost::make_shared(); for (auto & [key, mixer] : mixers) { - auto & mixed_filter = (*mixed_filters)[key]; - mixed_filter.database_and_table_name = std::move(mixer.database_and_table_name); - mixed_filter.ast = std::move(mixer.mixer).getResult(access_control.isEnabledUsersWithoutRowPoliciesCanReadRows()); - mixed_filter.policies = std::move(mixer.policies); + auto mixed_filter = std::make_shared(); + mixed_filter->database_and_table_name = std::move(mixer.database_and_table_name); + mixed_filter->expression = std::move(mixer.mixer).getResult(access_control.isEnabledUsersWithoutRowPoliciesCanReadRows()); + mixed_filter->policies = std::move(mixer.policies); + mixed_filters->emplace(key, std::move(mixed_filter)); } enabled.mixed_filters.store(mixed_filters); diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 095ac3d9327..2d3d378e926 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -987,10 +987,10 @@ std::shared_ptr Context::getAccess() const return access ? access : ContextAccess::getFullAccess(); } -RowPolicyFilter Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const +RowPolicyFilterPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { auto lock = getLock(); - RowPolicyFilter row_filter_of_initial_user; + RowPolicyFilterPtr row_filter_of_initial_user; if (row_policies_of_initial_user) row_filter_of_initial_user = row_policies_of_initial_user->getFilter(database, table_name, filter_type); return getAccess()->getRowPolicyFilter(database, table_name, filter_type, row_filter_of_initial_user); diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 42480b5817c..0eaec215588 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -46,6 +46,7 @@ using UserPtr = std::shared_ptr; struct EnabledRolesInfo; class EnabledRowPolicies; struct RowPolicyFilter; +using RowPolicyFilterPtr = std::shared_ptr; class EnabledQuota; struct QuotaUsage; class AccessFlags; @@ -517,7 +518,7 @@ public: std::shared_ptr getAccess() const; - RowPolicyFilter getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; + RowPolicyFilterPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; /// Finds and sets extra row policies to be used based on `client_info.initial_user`, /// if the initial user exists. diff --git a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp index 7a6ba69d8a5..2b802c188e2 100644 --- a/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp +++ b/src/Interpreters/InterpreterSelectIntersectExceptQuery.cpp @@ -199,11 +199,14 @@ void InterpreterSelectIntersectExceptQuery::extendQueryLogElemImpl(QueryLogEleme { if (auto select_interpreter = dynamic_cast(interpreter.get())) { - auto policies = select_interpreter->getUsedRowPolicies(); - for (const auto & row_policy : policies) + auto filter = select_interpreter->getRowPolicyFilter(); + if (filter) { - auto name = row_policy->getFullName().toString(); - elem.used_row_policies.emplace(std::move(name)); + for (const auto & row_policy : filter->policies) + { + auto name = row_policy->getFullName().toString(); + elem.used_row_policies.emplace(std::move(name)); + } } } } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 867a90c7e00..2948d756986 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -616,13 +616,13 @@ InterpreterSelectQuery::InterpreterSelectQuery( query_info.filter_asts.clear(); /// Fix source_header for filter actions. - if (row_policy_filter.expression) + if (row_policy_filter && !row_policy_filter->empty()) { filter_info = generateFilterActions( - table_id, row_policy_filter.expression, context, storage, storage_snapshot, metadata_snapshot, required_columns, + table_id, row_policy_filter->expression, context, storage, storage_snapshot, metadata_snapshot, required_columns, prepared_sets); - query_info.filter_asts.push_back(row_policy_filter.expression); + query_info.filter_asts.push_back(row_policy_filter->expression); } if (query_info.additional_filter_ast) @@ -1869,16 +1869,16 @@ void InterpreterSelectQuery::setProperClientInfo(size_t replica_num, size_t repl context->getClientInfo().number_of_current_replica = replica_num; } -const std::vector & InterpreterSelectQuery::getUsedRowPolicies() const +RowPolicyFilterPtr InterpreterSelectQuery::getRowPolicyFilter() const { - return row_policy_filter.policies; + return row_policy_filter; } void InterpreterSelectQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & /*ast*/, ContextPtr /*context_*/) const { elem.query_kind = "Select"; - for (const auto & row_policy : row_policy_filter.policies) + for (const auto & row_policy : row_policy_filter->policies) { auto name = row_policy->getFullName().toString(); elem.used_row_policies.emplace(std::move(name)); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 23b0f81fae6..761eea8e1b8 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -134,7 +134,7 @@ public: FilterDAGInfoPtr getAdditionalQueryInfo() const { return additional_filter_info; } - const std::vector & getUsedRowPolicies() const; + RowPolicyFilterPtr getRowPolicyFilter() const; void extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr context) const override; @@ -218,7 +218,7 @@ private: /// Is calculated in getSampleBlock. Is used later in readImpl. ExpressionAnalysisResult analysis_result; /// For row-level security. - RowPolicyFilter row_policy_filter; + RowPolicyFilterPtr row_policy_filter; FilterDAGInfoPtr filter_info; /// For additional_filter setting. diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 2e223d3a0b9..10f9d1c0b51 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -394,11 +394,14 @@ void InterpreterSelectWithUnionQuery::extendQueryLogElemImpl(QueryLogElement & e { if (auto select_interpreter = dynamic_cast(interpreter.get())) { - auto policies = select_interpreter->getUsedRowPolicies(); - for (const auto & row_policy : policies) + auto filter = select_interpreter->getRowPolicyFilter(); + if (filter) { - auto name = row_policy->getFullName().toString(); - elem.used_row_policies.emplace(std::move(name)); + for (const auto & row_policy : filter->policies) + { + auto name = row_policy->getFullName().toString(); + elem.used_row_policies.emplace(std::move(name)); + } } } } From 41dc5b30b4d1a16a4a397bc8fc034c7091adf18f Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 27 Oct 2022 10:52:59 +0200 Subject: [PATCH 087/526] Updated test to use TSV in format, and fixed test 02117_show_create_table_system - 42414 Enable table functions in readonly mode --- .../0_stateless/02117_show_create_table_system.reference | 3 ++- .../0_stateless/02473_functions_in_readonly_mode.reference | 2 +- .../queries/0_stateless/02473_functions_in_readonly_mode.sql | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index ad27b86c6f5..65362e9f35d 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -1013,7 +1013,8 @@ COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.table_functions ( `name` String, - `description` String + `description` String, + `allow_readonly` UInt8 ) ENGINE = SystemTableFunctions COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference index 4977168f515..500004a06b2 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.reference @@ -1,3 +1,3 @@ 0 -(123,'str') +123 0 diff --git a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql index eb2631168a8..c5c82d2e2bf 100644 --- a/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql +++ b/tests/queries/0_stateless/02473_functions_in_readonly_mode.sql @@ -1,4 +1,4 @@ SELECT * from numbers(1); -select * from format(JSONEachRow, '{"x" : [123, "str"]}'); +SELECT * from format('TSV', '123'); SELECT * from numbers(1) SETTINGS readonly=1; -select * from format(JSONEachRow, '{"x" : [123, "str"]}') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file +SELECT * from format('TSV', '123') SETTINGS readonly=1; -- { serverError READONLY } \ No newline at end of file From cc65def3520c23278e6ea738c78813a59c01fd8a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 27 Oct 2022 12:54:26 +0000 Subject: [PATCH 088/526] Add set test --- .../src/jepsen/clickhouse/server/client.clj | 13 +++++ .../src/jepsen/clickhouse/server/main.clj | 7 ++- .../src/jepsen/clickhouse/server/set.clj | 55 +++++++++++++++++++ 3 files changed, 73 insertions(+), 2 deletions(-) create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj index a1e3b6d9687..661c4842f22 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj @@ -1,7 +1,10 @@ (ns jepsen.clickhouse.server.client (:require [clojure.java.jdbc :as j] + [clojure.tools.logging :refer :all] [jepsen.reconnect :as rc])) +(def operation-timeout "Default operation timeout in ms" 10000) + (defn db-spec [node] {:dbtype "clickhouse" @@ -47,3 +50,13 @@ (throw (ex-info "Connection not yet ready." {:type :conn-not-ready}))) ~@body)) + +(defmacro with-exception + "Takes an operation and a body. Evaluates body, catches exceptions, and maps + them to ops with :type :info and a descriptive :error." + [op & body] + `(try ~@body + (catch Exception e# + (if-let [message# (.getMessage e#)] + (assoc ~op :type :fail, :error message#) + (throw e#))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj index b4e66443c1a..541a92d6eaa 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj @@ -9,7 +9,9 @@ [tests :as tests] [util :as util :refer [meh]]] [jepsen.clickhouse.server.db :refer :all] - [jepsen.clickhouse.server.register :as register] + [jepsen.clickhouse.server + [register :as register] + [set :as set]] [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu] [jepsen.checker.timeline :as timeline] @@ -17,7 +19,8 @@ (def workloads "A map of workload names to functions that construct workloads, given opts." - {"register" register/workload}) + {"register" register/workload + "set" set/workload}) (def custom-nemesises {}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj new file mode 100644 index 00000000000..2abf0b2b081 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj @@ -0,0 +1,55 @@ +(ns jepsen.clickhouse.server.set + (:require + [clojure.tools.logging :refer :all] + [clojure.java.jdbc :as j] + [jepsen + [util :as util] + [reconnect :as rc] + [checker :as checker] + [client :as client] + [generator :as gen]] + [jepsen.clickhouse.server.client :as chc] + [jepsen.clickhouse.utils :as chu])) + +(defrecord SetClient [table-created? conn] + client/Client + (open! [this test node] + (assoc this :conn (chc/client node))) + + (setup! [this test] + (locking table-created? + (when (compare-and-set! table-created? false true) + (chc/with-connection [c conn] + (j/query c "DROP TABLE IF EXISTS set") + (j/query c "CREATE TABLE set (value Int64) Engine=MergeTree ORDER BY value"))))) + + (invoke! [this test op] + (chc/with-exception op + (chc/with-connection [c conn] + (util/with-retry [] + (case (:f op) + :add (do + (j/query c (str "INSERT INTO set VALUES (" (:value op) ")")) + (assoc op :type :ok)) + :read (->> (j/query c "SELECT value FROM set") + (mapv :value) + (assoc op :type :ok, :value))))))) + + (teardown! [_ test] + (util/timeout chc/operation-timeout + (chc/with-connection [c conn] + (j/query c ["DROP TABLE set"])))) + + (close! [_ test] + (rc/close! conn))) + +(defn workload + "A generator, client, and checker for a set test." + [opts] + {:client (SetClient. (atom false) nil) + :checker (checker/compose + {:set (checker/set) + :perf (checker/perf)}) + :generator (->> (range) + (map (fn [x] {:type :invoke, :f :add, :value x}))) + :final-generator (gen/once {:type :invoke, :f :read, :value nil})}) From f773436de567a0b90a6cf8fc7384adae75a87dbd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 27 Oct 2022 13:34:07 +0000 Subject: [PATCH 089/526] Fix flaky test --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- src/Storages/MergeTree/ZooKeeperRetries.h | 3 ++- .../0_stateless/01158_zookeeper_log_long.reference | 12 +++++------- .../queries/0_stateless/01158_zookeeper_log_long.sql | 2 +- 4 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 54b7df81654..2bfa0344761 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -166,7 +166,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) const auto & settings = context->getSettingsRef(); zookeeper_retries_info = ZooKeeperRetriesInfo( "ReplicatedMergeTreeSink::consume", - log, + settings.insert_keeper_max_retries ? log : nullptr, settings.insert_keeper_max_retries, settings.insert_keeper_retry_initial_backoff_ms, settings.insert_keeper_retry_max_backoff_ms); diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index 24bfad22f69..c000e297ceb 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -168,7 +168,8 @@ private: /// iteration succeeded -> no need to retry if (iteration_succeeded) { - if (retries_info.logger) + /// avoid unnecessary logs, - print something only in case of retries + if (retries_info.logger && iteration_count > 1) LOG_DEBUG( retries_info.logger, "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index ee2b465b3d3..d7620d92621 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -1,9 +1,9 @@ log -::1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 -::1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 -::1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 -::1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 -::1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 +::ffff:127.0.0.1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 +::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 +::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 +::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 +::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 parts Request 0 Multi 0 0 \N 5 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 @@ -17,8 +17,6 @@ Response 0 Remove /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 Response 0 Remove /test/01158/default/rmt/temp/abandonable_lock-0000000000 0 0 -1 0 3 ZOK \N \N 0 0 0 0 Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0 Response 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 5 ZOK \N \N /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 0 0 -Request 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 \N \N \N 0 0 0 0 -Response 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 ZOK \N \N 0 0 96 0 blocks Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0 diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 6701c8b840f..eb549e1a37a 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -22,7 +22,7 @@ select 'parts'; select type, has_watch, op_num, path, is_ephemeral, is_sequential, version, requests_size, request_idx, error, watch_type, watch_state, path_created, stat_version, stat_cversion, stat_dataLength, stat_numChildren from system.zookeeper_log -where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path='/test/01158/' || currentDatabase() || '/rmt/replicas/1/parts/all_0_0_0') +where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path='/test/01158/' || currentDatabase() || '/rmt/replicas/1/parts/all_0_0_0' and op_num not in(3)) order by xid, type, request_idx; select 'blocks'; From a601c166ad43e96366cb975a0d862725d9a6dd69 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 27 Oct 2022 21:31:53 +0000 Subject: [PATCH 090/526] Add clean up for faults injected before zk request + for multi-op request, it create responses in multi response + some polishing --- .../MergeTree/EphemeralLockInZooKeeper.cpp | 2 +- .../MergeTree/ZooKeeperWithFaultInjection.h | 94 ++++++++++++++----- 2 files changed, 69 insertions(+), 27 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index b6c455e76b3..2e3b2fde194 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -45,7 +45,7 @@ std::optional createEphemeralLockInZooKeeper( Coordination::Error e = zookeeper_->tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { - if (!responses.empty() && responses.front()->error == Coordination::Error::ZNODEEXISTS) + if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { LOG_DEBUG( &Poco::Logger::get("createEphemeralLockInZooKeeper"), diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h index 07b37f4c8f0..1e0b730c32a 100644 --- a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -1,9 +1,9 @@ #pragma once #include -#include #include #include #include +#include #include namespace DB @@ -58,11 +58,10 @@ class ZooKeeperWithFaultInjection double fault_injection_probability, UInt64 fault_injection_seed, std::string name_, - Poco::Logger * logger_ = nullptr) + Poco::Logger * logger_) : keeper(keeper_), name(std::move(name_)), logger(logger_), seed(fault_injection_seed) { - if (fault_injection_probability > .0) - fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); + fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); if (unlikely(logger)) LOG_TRACE( @@ -166,16 +165,23 @@ public: method, !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->tryMulti(requests, responses); }, - [&](const Coordination::Error & err) + [&](const Coordination::Error & original_error) { - if (err == Coordination::Error::ZOK) - faultInjectionCleanup(method, requests, responses); + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); }); - /// collect ephemeral nodes to clean up + + /// collect ephemeral nodes when no fault was injected (to clean up on demand) if (unlikely(fault_policy) && Coordination::Error::ZOK == error) { - doForEachEphemeralNode( + doForEachCreatedEphemeralNode( method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); } return error; @@ -190,16 +196,22 @@ public: method, !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->tryMultiNoThrow(requests, responses); }, - [&](const Coordination::Error & err) + [&](const Coordination::Error & original_error) { - if (err == Coordination::Error::ZOK) - faultInjectionCleanup(method, requests, responses); + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); }); - /// collect ephemeral nodes to clean up + /// collect ephemeral nodes when no fault was injected (to clean up later) if (unlikely(fault_policy) && Coordination::Error::ZOK == error) { - doForEachEphemeralNode( + doForEachCreatedEphemeralNode( method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); } return error; @@ -256,7 +268,7 @@ public: } }); - /// collect ephemeral nodes to clean up + /// collect ephemeral nodes when no fault was injected (to clean up later) if (unlikely(fault_policy)) { if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) @@ -268,11 +280,20 @@ public: Coordination::Responses multi(const Coordination::Requests & requests) { - return access( - "multi", + constexpr auto method = "multi"; + auto result = access( + method, !requests.empty() ? requests.front()->getPath() : "", [&]() { return keeper->multi(requests); }, - [&](Coordination::Responses const & responses) { faultInjectionCleanup("multi", requests, responses); }); + [&](Coordination::Responses & responses) { faultInjectionPostAction(method, requests, responses); }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy)) + { + doForEachCreatedEphemeralNode( + method, requests, result, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return result; } void createAncestors(const std::string & path) @@ -305,6 +326,19 @@ public: } private: + void faultInjectionBefore(std::function fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->beforeOperation(); + } + catch (const zkutil::KeeperException &) + { + fault_cleanup(); + throw; + } + } void faultInjectionAfter(std::function fault_cleanup) { try @@ -319,7 +353,7 @@ private: } } - void doForEachEphemeralNode( + void doForEachCreatedEphemeralNode( const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses, auto && action) { if (responses.empty()) @@ -353,15 +387,15 @@ private: } } - void faultInjectionCleanup(const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses) + void faultInjectionPostAction(const char * method, const Coordination::Requests & requests, Coordination::Responses & responses) { - doForEachEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); + doForEachCreatedEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); } template struct FaultCleanupTypeImpl { - using Type = std::function; + using Type = std::function; }; template <> @@ -379,8 +413,12 @@ private: int inject_failure_after_op = true, typename Operation, typename Result = std::invoke_result_t> - Result - access(const char * func_name, const std::string & path, Operation operation, FaultCleanupType fault_after_op_cleanup = {}) + Result access( + const char * func_name, + const std::string & path, + Operation operation, + FaultCleanupType fault_after_op_cleanup = {}, + FaultCleanupType fault_before_op_cleanup = {}) { try { @@ -392,8 +430,12 @@ private: if constexpr (inject_failure_before_op) { - if (unlikely(fault_policy)) - fault_policy->beforeOperation(); + faultInjectionBefore( + [&] + { + if (fault_before_op_cleanup) + fault_before_op_cleanup(); + }); } if constexpr (!std::is_same_v) From c0809643915b2840a02e7962b946d05bca7619a4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:39:01 +0800 Subject: [PATCH 091/526] correct 'notLike' in key condition --- src/Storages/MergeTree/KeyCondition.cpp | 6 ++++-- src/Storages/MergeTree/KeyCondition.h | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9b579512522..9bd9f3c8853 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -55,7 +55,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -String extractFixedPrefixFromLikePattern(const String & like_pattern) +String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) { String fixed_prefix; @@ -68,6 +68,8 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern) case '%': [[fallthrough]]; case '_': + if (perfect_prefix_match && std::find_if(pos+1, end, [](const char c) { return c != '%' && c != '_'; }) != end) + return ""; return fixed_prefix; case '\\': @@ -567,7 +569,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get()); + String prefix = extractFixedPrefixFromLikePattern(value.get(), true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index d00a25a1077..da1a74af90d 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -481,6 +481,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(const String & like_pattern); +String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match = false); } From 9c860f0e0623a19b77a8f4f474df680836406ef4 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:42:39 +0800 Subject: [PATCH 092/526] fix test style check --- .../0_stateless/002458_key_condition_not_like_prefix.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql index 211fa5662e7..2c1402df27e 100644 --- a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql +++ b/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql @@ -2,4 +2,4 @@ CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); -SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; \ No newline at end of file +SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; From 9ea9039b0f80b22f229f98c036437814b41bfb73 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Fri, 28 Oct 2022 13:45:08 +0800 Subject: [PATCH 093/526] fix test name --- ...ix.reference => 02458_key_condition_not_like_prefix.reference} | 0 ...ot_like_prefix.sql => 02458_key_condition_not_like_prefix.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{002458_key_condition_not_like_prefix.reference => 02458_key_condition_not_like_prefix.reference} (100%) rename tests/queries/0_stateless/{002458_key_condition_not_like_prefix.sql => 02458_key_condition_not_like_prefix.sql} (100%) diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference similarity index 100% rename from tests/queries/0_stateless/002458_key_condition_not_like_prefix.reference rename to tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference diff --git a/tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql similarity index 100% rename from tests/queries/0_stateless/002458_key_condition_not_like_prefix.sql rename to tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql From f31ce5debef0888a1befe9d643f03188837dc701 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Oct 2022 06:24:22 +0000 Subject: [PATCH 094/526] Fix for messed up test output file --- .../0_stateless/01158_zookeeper_log_long.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index d7620d92621..3473a28911d 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -1,9 +1,9 @@ log -::ffff:127.0.0.1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 -::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 -::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 -::ffff:127.0.0.1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 -::ffff:127.0.0.1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 +::1 Response 0 Watch /test/01158/default/rmt/log 0 0 \N 0 0 ZOK CHILD CONNECTED 0 0 0 0 +::1 Request 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 \N \N \N 0 0 0 0 +::1 Response 0 Create /test/01158/default/rmt/log 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/log 0 0 0 0 +::1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 +::1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 parts Request 0 Multi 0 0 \N 5 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 From 77a7411bd0756ea5dbd0a48b458ebdf23e31e396 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 07:34:23 +0000 Subject: [PATCH 095/526] start adding nemesis --- .../src/jepsen/clickhouse/keeper/nemesis.clj | 13 +++-------- .../src/jepsen/clickhouse/nemesis.clj | 23 +++++++++++++++++++ .../src/jepsen/clickhouse/server/client.clj | 17 +++++++++----- .../src/jepsen/clickhouse/server/main.clj | 10 +++++++- .../src/jepsen/clickhouse/server/nemesis.clj | 8 +++++++ 5 files changed, 54 insertions(+), 17 deletions(-) create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj create mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj index 2505a4b8463..76e4227cf72 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj @@ -5,6 +5,7 @@ [nemesis :as nemesis] [control :as c] [generator :as gen]] + [jepsen.clickhouse.nemesis :as chnem] [jepsen.clickhouse.constants :refer :all] [jepsen.clickhouse.utils :as chu] [jepsen.clickhouse.keeper.utils :refer :all])) @@ -23,14 +24,6 @@ (fn start [test node] (chu/kill-clickhouse! node test)) (fn stop [test node] (start-clickhouse! node test)))) -(defn random-node-hammer-time-nemesis - [] - (nemesis/hammer-time "clickhouse")) - -(defn all-nodes-hammer-time-nemesis - [] - (nemesis/hammer-time identity "clickhouse")) - (defn select-last-file [path] (last (clojure.string/split @@ -141,9 +134,9 @@ :generator (start-stop-generator 1 10)} "simple-partitioner" {:nemesis (nemesis/partition-random-halves) :generator (start-stop-generator 5 5)} - "random-node-hammer-time" {:nemesis (random-node-hammer-time-nemesis) + "random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) :generator (start-stop-generator 5 5)} - "all-nodes-hammer-time" {:nemesis (all-nodes-hammer-time-nemesis) + "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) :generator (start-stop-generator 1 10)} "logs-corruptor" {:nemesis (logs-corruption-nemesis) :generator (corruption-generator)} diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj new file mode 100644 index 00000000000..116fd1648b2 --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj @@ -0,0 +1,23 @@ +(ns jepsen.clickhouse.nemesis + (:require + [clojure.tools.logging :refer :all] + [jepsen + [nemesis :as nemesis] + [generator :as gen]])) + +(defn random-node-hammer-time-nemesis + [] + (nemesis/hammer-time "clickhouse")) + +(defn all-nodes-hammer-time-nemesis + [] + (nemesis/hammer-time identity "clickhouse")) + +(defn start-stop-generator + [time-corrupt time-ok] + (->> + (cycle [(gen/sleep time-ok) + {:type :info, :f :start} + (gen/sleep time-corrupt) + {:type :info, :f :stop}]))) + diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj index 661c4842f22..92b112ffda5 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj @@ -1,6 +1,7 @@ (ns jepsen.clickhouse.server.client (:require [clojure.java.jdbc :as j] [clojure.tools.logging :refer :all] + [jepsen.util :as util] [jepsen.reconnect :as rc])) (def operation-timeout "Default operation timeout in ms" 10000) @@ -16,11 +17,15 @@ (defn open-connection [node] - (let [spec (db-spec node) - connection (j/get-connection spec) - added-connection (j/add-connection spec connection)] - (assert added-connection) - added-connection)) + (util/timeout 30000 + (throw (RuntimeException. + (str "Connection to " node " timed out"))) + (util/retry 0.1 + (let [spec (db-spec node) + connection (j/get-connection spec) + added-connection (j/add-connection spec connection)] + (assert added-connection) + added-connection)))) (defn close-connection "Close connection" @@ -35,7 +40,7 @@ (rc/open! (rc/wrapper {:name (name node) - :open #(open-connection node) + :open (partial open-connection node) :close close-connection :log? true}))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj index 541a92d6eaa..a87c719462a 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj @@ -12,6 +12,7 @@ [jepsen.clickhouse.server [register :as register] [set :as set]] + [jepsen.clickhouse.server.nemesis :as custom-nemesis] [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu] [jepsen.checker.timeline :as timeline] @@ -48,13 +49,15 @@ [opts] (info "Test opts\n" (with-out-str (pprint opts))) (let [quorum (boolean (:quorum opts)) - workload ((get workloads (:workload opts)) opts)] + workload ((get workloads (:workload opts)) opts) + current-nemesis (get custom-nemesis/custom-nemesises "random-node-hammer-time")] (merge tests/noop-test opts {:name (str "clickhouse-server-" (name (:workload opts))) :os ubuntu/os :db (get-db opts) :pure-generators true + :nemesis (:nemesis current-nemesis) :client (:client workload) :checker (checker/compose {:perf (checker/perf) @@ -62,7 +65,12 @@ :generator (gen/phases (->> (:generator workload) (gen/stagger (/ (:rate opts))) + (gen/nemesis (:generator current-nemesis)) (gen/time-limit (:time-limit opts))) + (gen/log "Healing cluster") + (gen/nemesis (gen/once {:type :info, :f :stop})) + (gen/log "Waiting for recovery") + (gen/sleep 10) (gen/clients (:final-generator workload)))}))) (defn clickhouse-server-test diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj new file mode 100644 index 00000000000..3512409dd9e --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj @@ -0,0 +1,8 @@ +(ns jepsen.clickhouse.server.nemesis + (:require [jepsen.clickhouse.nemesis :as chnem])) + +(def custom-nemesises + {"random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) + :generator (chnem/start-stop-generator 5 5)} + "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) + :generator (chnem/start-stop-generator 1 10)}}) From c37b1542545ab4a3362afdd266a531a7eeca00ff Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 28 Oct 2022 12:37:59 +0200 Subject: [PATCH 096/526] Added reverted files and fixes for failing fuzzer tests --- src/Functions/FunctionsJSON.cpp | 106 +++++++++++++----- .../performance/low_cardinality_from_json.xml | 73 ++++++++++++ .../02452_check_low_cardinality.reference | 7 ++ .../02452_check_low_cardinality.sql | 54 +++++++++ ...ct_fixed_string_from_nested_json.reference | 1 + ..._extract_fixed_string_from_nested_json.sql | 6 + 6 files changed, 219 insertions(+), 28 deletions(-) create mode 100644 tests/performance/low_cardinality_from_json.xml create mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.reference create mode 100644 tests/queries/0_stateless/02452_check_low_cardinality.sql create mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference create mode 100644 tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index ea2d2cc94af..64c5ff3c2c8 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -20,18 +20,19 @@ #include #include -#include -#include -#include -#include -#include -#include +#include #include #include +#include +#include #include #include -#include +#include #include +#include +#include +#include +#include #include #include @@ -695,8 +696,16 @@ public: else return false; - auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(value); + if (dest.getDataType() == TypeIndex::LowCardinality) + { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(reinterpret_cast(&value), sizeof(value)); + } + else + { + auto & col_vec = assert_cast &>(dest); + col_vec.insertValue(value); + } return true; } }; @@ -773,8 +782,17 @@ public: return JSONExtractRawImpl::insertResultToColumn(dest, element, {}); auto str = element.getString(); - ColumnString & col_str = assert_cast(dest); - col_str.insertData(str.data(), str.size()); + + if (dest.getDataType() == TypeIndex::LowCardinality) + { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(str.data(), str.size()); + } + else + { + ColumnString & col_str = assert_cast(dest); + col_str.insertData(str.data(), str.size()); + } return true; } }; @@ -803,25 +821,33 @@ struct JSONExtractTree } }; - class LowCardinalityNode : public Node + class LowCardinalityFixedStringNode : public Node { public: - LowCardinalityNode(DataTypePtr dictionary_type_, std::unique_ptr impl_) - : dictionary_type(dictionary_type_), impl(std::move(impl_)) {} + explicit LowCardinalityFixedStringNode(const size_t fixed_length_) : fixed_length(fixed_length_) { } bool insertResultToColumn(IColumn & dest, const Element & element) override { - auto from_col = dictionary_type->createColumn(); - if (impl->insertResultToColumn(*from_col, element)) - { - std::string_view value = from_col->getDataAt(0).toView(); - assert_cast(dest).insertData(value.data(), value.size()); - return true; - } - return false; + // If element is an object we delegate the insertion to JSONExtractRawImpl + if (element.isObject()) + return JSONExtractRawImpl::insertResultToLowCardinalityFixedStringColumn(dest, element, fixed_length); + else if (!element.isString()) + return false; + + auto str = element.getString(); + if (str.size() > fixed_length) + return false; + + // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. + // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) + // the data is padded here and written directly to the Low Cardinality Column + auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); + + assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + return true; } + private: - DataTypePtr dictionary_type; - std::unique_ptr impl; + const size_t fixed_length; }; class UUIDNode : public Node @@ -833,7 +859,15 @@ struct JSONExtractTree return false; auto uuid = parseFromString(element.getString()); - assert_cast(dest).insert(uuid); + if (dest.getDataType() == TypeIndex::LowCardinality) + { + ColumnLowCardinality & col_low = assert_cast(dest); + col_low.insertData(reinterpret_cast(&uuid), sizeof(uuid)); + } + else + { + assert_cast(dest).insert(uuid); + } return true; } }; @@ -853,6 +887,7 @@ struct JSONExtractTree assert_cast &>(dest).insert(result); return true; } + private: DataTypePtr data_type; }; @@ -871,13 +906,18 @@ struct JSONExtractTree public: bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (!element.isString()) + if (element.isNull()) return false; - auto & col_str = assert_cast(dest); + + if (!element.isString()) + return JSONExtractRawImpl::insertResultToFixedStringColumn(dest, element, {}); + auto str = element.getString(); + auto & col_str = assert_cast(dest); if (str.size() > col_str.getN()) return false; col_str.insertData(str.data(), str.size()); + return true; } }; @@ -1099,9 +1139,19 @@ struct JSONExtractTree case TypeIndex::UUID: return std::make_unique(); case TypeIndex::LowCardinality: { + // The low cardinality case is treated in two different ways: + // For FixedString type, an especial class is implemented for inserting the data in the destination column, + // as the string length must be passed in order to check and pad the incoming data. + // For the rest of low cardinality types, the insertion is done in their corresponding class, adapting the data + // as needed for the insertData function of the ColumnLowCardinality. auto dictionary_type = typeid_cast(type.get())->getDictionaryType(); + if ((*dictionary_type).getTypeId() == TypeIndex::FixedString) + { + auto fixed_length = typeid_cast(dictionary_type.get())->getN(); + return std::make_unique(fixed_length); + } auto impl = build(function_name, dictionary_type); - return std::make_unique(dictionary_type, std::move(impl)); + return impl; } case TypeIndex::Decimal256: return std::make_unique>(type); case TypeIndex::Decimal128: return std::make_unique>(type); diff --git a/tests/performance/low_cardinality_from_json.xml b/tests/performance/low_cardinality_from_json.xml new file mode 100644 index 00000000000..ac6542ac503 --- /dev/null +++ b/tests/performance/low_cardinality_from_json.xml @@ -0,0 +1,73 @@ + + + + + string_json + + '{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}' + + + + int_json + + '{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}' + + + + uuid_json + + '{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}' + + + + low_cardinality_tuple_string + + 'Tuple(a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String) )' + + + + low_cardinality_tuple_fixed_string + + 'Tuple(a LowCardinality(FixedString(20)), b LowCardinality(FixedString(20)), c LowCardinality(FixedString(20)), d LowCardinality(FixedString(20)) )' + + + + low_cardinality_tuple_int8 + + 'Tuple(a LowCardinality(Int8), b LowCardinality(Int8), c LowCardinality(Int8), d LowCardinality(Int8) )' + + + + low_cardinality_tuple_int16 + + 'Tuple(a LowCardinality(Int16), b LowCardinality(Int16), c LowCardinality(Int16), d LowCardinality(Int16) )' + + + + low_cardinality_tuple_int32 + + 'Tuple(a LowCardinality(Int32), b LowCardinality(Int32), c LowCardinality(Int32), d LowCardinality(Int32) )' + + + + low_cardinality_tuple_int64 + + 'Tuple(a LowCardinality(Int64), b LowCardinality(Int64), c LowCardinality(Int64), d LowCardinality(Int64) )' + + + + low_cardinality_tuple_uuid + + 'Tuple(a LowCardinality(UUID), b LowCardinality(UUID), c LowCardinality(UUID), d LowCardinality(UUID) )' + + + + + SELECT 'fixed_string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_fixed_string})) FORMAT Null + SELECT 'string_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({string_json}), {low_cardinality_tuple_string})) FORMAT Null + SELECT 'int8_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int8})) FORMAT Null + SELECT 'int16_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int16})) FORMAT Null + SELECT 'int32_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int32})) FORMAT Null + SELECT 'int64_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({int_json}), {low_cardinality_tuple_int64})) FORMAT Null + SELECT 'uuid_json' FROM zeros(500000) WHERE NOT ignore(JSONExtract(materialize({uuid_json}), {low_cardinality_tuple_uuid})) FORMAT Null + \ No newline at end of file diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.reference b/tests/queries/0_stateless/02452_check_low_cardinality.reference new file mode 100644 index 00000000000..700778e02c7 --- /dev/null +++ b/tests/queries/0_stateless/02452_check_low_cardinality.reference @@ -0,0 +1,7 @@ +('hi','hello','hola','see you, bye, bye') +('hi\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hello\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','hola\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0','see you, bye, bye\0\0\0') +(11,0,0,0) +(11,2222,0,0) +(11,2222,33333333,0) +(11,2222,33333333,4444444444444444) +('2d49dc6e-ddce-4cd0-afb8-790956df54c4','2d49dc6e-ddce-4cd0-afb8-790956df54c3','2d49dc6e-ddce-4cd0-afb8-790956df54c1','2d49dc6e-ddce-4cd0-afb8-790956df54c1') diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql new file mode 100644 index 00000000000..166be281405 --- /dev/null +++ b/tests/queries/0_stateless/02452_check_low_cardinality.sql @@ -0,0 +1,54 @@ +DROP TABLE IF EXISTS test_low_cardinality_string; +DROP TABLE IF EXISTS test_low_cardinality_uuid; +DROP TABLE IF EXISTS test_low_cardinality_int; +CREATE TABLE test_low_cardinality_string (data String) ENGINE MergeTree ORDER BY data; +CREATE TABLE test_low_cardinality_uuid (data String) ENGINE MergeTree ORDER BY data; +CREATE TABLE test_low_cardinality_int (data String) ENGINE MergeTree ORDER BY data; +INSERT INTO test_low_cardinality_string (data) VALUES ('{"a": "hi", "b": "hello", "c": "hola", "d": "see you, bye, bye"}'); +INSERT INTO test_low_cardinality_int (data) VALUES ('{"a": 11, "b": 2222, "c": 33333333, "d": 4444444444444444}'); +INSERT INTO test_low_cardinality_uuid (data) VALUES ('{"a": "2d49dc6e-ddce-4cd0-afb8-790956df54c4", "b": "2d49dc6e-ddce-4cd0-afb8-790956df54c3", "c": "2d49dc6e-ddce-4cd0-afb8-790956df54c1", "d": "2d49dc6e-ddce-4cd0-afb8-790956df54c1"}'); +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(String), + b LowCardinality(String), + c LowCardinality(String), + d LowCardinality(String) + )') AS json FROM test_low_cardinality_string; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(FixedString(20)), + b LowCardinality(FixedString(20)), + c LowCardinality(FixedString(20)), + d LowCardinality(FixedString(20)) + )') AS json FROM test_low_cardinality_string; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int8), + b LowCardinality(Int8), + c LowCardinality(Int8), + d LowCardinality(Int8) + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int16), + b LowCardinality(Int16), + c LowCardinality(Int16), + d LowCardinality(Int16) + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int32), + b LowCardinality(Int32), + c LowCardinality(Int32), + d LowCardinality(Int32) + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(Int64), + b LowCardinality(Int64), + c LowCardinality(Int64), + d LowCardinality(Int64) + )') AS json FROM test_low_cardinality_int; +SELECT JSONExtract(data, 'Tuple( + a LowCardinality(UUID), + b LowCardinality(UUID), + c LowCardinality(UUID), + d LowCardinality(UUID) + )') AS json FROM test_low_cardinality_uuid; +DROP TABLE test_low_cardinality_string; +DROP TABLE test_low_cardinality_uuid; +DROP TABLE test_low_cardinality_int; diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference new file mode 100644 index 00000000000..3a528a24821 --- /dev/null +++ b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.reference @@ -0,0 +1 @@ +('{"b":{"c":1,"d":"str"}}\0') diff --git a/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql new file mode 100644 index 00000000000..449713d396f --- /dev/null +++ b/tests/queries/0_stateless/02455_extract_fixed_string_from_nested_json.sql @@ -0,0 +1,6 @@ +-- Tags: no-fasttest +DROP TABLE IF EXISTS test_fixed_string_nested_json; +CREATE TABLE test_fixed_string_nested_json (data String) ENGINE MergeTree ORDER BY data; +INSERT INTO test_fixed_string_nested_json (data) VALUES ('{"a" : {"b" : {"c" : 1, "d" : "str"}}}'); +SELECT JSONExtract(data, 'Tuple(a FixedString(24))') AS json FROM test_fixed_string_nested_json; +DROP TABLE test_fixed_string_nested_json; \ No newline at end of file From 976c6bac139494ed654a85b5a13d02ab23814bc1 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 11:05:56 +0000 Subject: [PATCH 097/526] Use ReplicatedMergeTree --- .../src/jepsen/clickhouse/keeper/main.clj | 15 ++--- .../src/jepsen/clickhouse/server/client.clj | 19 ++++--- .../src/jepsen/clickhouse/server/db.clj | 13 ++++- .../src/jepsen/clickhouse/server/main.clj | 33 ++++++----- .../src/jepsen/clickhouse/server/nemesis.clj | 2 +- .../src/jepsen/clickhouse/server/register.clj | 56 ------------------- .../src/jepsen/clickhouse/server/set.clj | 28 +++++----- .../src/jepsen/clickhouse/utils.clj | 7 +++ 8 files changed, 66 insertions(+), 107 deletions(-) delete mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj index 87f437d3d4f..d03448417af 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj @@ -163,19 +163,12 @@ "blind-node-partitioner" "blind-others-partitioner"]) -(defn cart [colls] - (if (empty? colls) - '(()) - (for [more (cart (rest colls)) - x (first colls)] - (cons x more)))) - (defn all-test-options "Takes base cli options, a collection of nemeses, workloads, and a test count, and constructs a sequence of test options." - [cli worload-nemeseis-collection] + [cli workload-nemesis-collection] (take (:test-count cli) - (shuffle (for [[workload nemesis] worload-nemeseis-collection] + (shuffle (for [[workload nemesis] workload-nemesis-collection] (assoc cli :nemesis nemesis :workload workload @@ -184,8 +177,8 @@ "Turns CLI options into a sequence of tests." [test-fn cli] (if (boolean (:lightweight-run cli)) - (map test-fn (all-test-options cli (cart [lightweight-workloads useful-nemesises]))) - (map test-fn (all-test-options cli (cart [all-workloads all-nemesises]))))) + (map test-fn (all-test-options cli (chu/cart [lightweight-workloads useful-nemesises]))) + (map test-fn (all-test-options cli (chu/cart [all-workloads all-nemesises]))))) (defn main "Handles command line arguments. Can either run a test, or a web server for diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj index 92b112ffda5..e051d361c73 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj @@ -13,6 +13,8 @@ :classname "com.clickhouse.ClickhouseDriver" :host (name node) :port 8123 + :connectTimeout 30 + :socketTimeout 30 :jdbcCompliant false}) (defn open-connection @@ -48,13 +50,16 @@ "Like jepsen.reconnect/with-conn, but also asserts that the connection has not been closed. If it has, throws an ex-info with :type :conn-not-ready. Delays by 1 second to allow time for the DB to recover." - [[c client] & body] - `(rc/with-conn [~c ~client] - (when (.isClosed (j/db-find-connection ~c)) - (Thread/sleep 1000) - (throw (ex-info "Connection not yet ready." - {:type :conn-not-ready}))) - ~@body)) + [[c client] final & body] + `(do + (when ~final + (rc/reopen! ~client)) + (rc/with-conn [~c ~client] + (when (.isClosed (j/db-find-connection ~c)) + (Thread/sleep 1000) + (throw (ex-info "Connection not yet ready." + {:type :conn-not-ready}))) + ~@body))) (defmacro with-exception "Takes an operation and a body. Evaluates body, catches exceptions, and maps diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj index 9e91f3e1354..e38c3c93759 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj @@ -29,10 +29,21 @@ [jepsen.clickhouse.server.utils :refer :all] [jepsen.clickhouse.utils :as chu])) +(defn replicated-merge-tree-config + [test node config-template] + (let [nodes (:nodes test) + replacement-map {#"\{server1\}" (get nodes 0) + #"\{server2\}" (get nodes 1) + #"\{server3\}" (get nodes 2) + #"\{server_id\}" (str (inc (.indexOf nodes node))) + #"\{replica_name\}" node}] + (reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map))) + (defn install-configs [test node] (c/exec :echo (slurp (io/resource "config.xml")) :> (str configs-dir "/config.xml")) - (c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml"))) + (c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml")) + (c/exec :echo (replicated-merge-tree-config test node (slurp (io/resource "replicated_merge_tree.xml"))) :> (str sub-configs-dir "/replicated_merge_tree.xml"))) (defn extra-setup [test node] diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj index a87c719462a..09e395ce183 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj @@ -8,11 +8,12 @@ [generator :as gen] [tests :as tests] [util :as util :refer [meh]]] - [jepsen.clickhouse.server.db :refer :all] [jepsen.clickhouse.server - [register :as register] + [db :refer :all] + [nemesis :as ch-nemesis]] + [jepsen.clickhouse.server [set :as set]] - [jepsen.clickhouse.server.nemesis :as custom-nemesis] + [jepsen.clickhouse.utils :as chu] [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu] [jepsen.checker.timeline :as timeline] @@ -20,20 +21,16 @@ (def workloads "A map of workload names to functions that construct workloads, given opts." - {"register" register/workload - "set" set/workload}) - -(def custom-nemesises - {}) + {"set" set/workload}) (def cli-opts "Additional command line options." [["-w" "--workload NAME" "What workload should we run?" :default "set" :validate [workloads (cli/one-of workloads)]] - ;[nil "--nemesis NAME" "Which nemesis will poison our lives?" - ; :default "random-node-killer" - ; :validate [custom-nemesises (cli/one-of custom-nemesises)]] + [nil "--nemesis NAME" "Which nemesis will poison our lives?" + :default "random-node-killer" + :validate [ch-nemesis/custom-nemeses (cli/one-of ch-nemesis/custom-nemeses)]] ["-r" "--rate HZ" "Approximate number of requests per second, per thread." :default 10 :parse-fn read-string @@ -50,10 +47,10 @@ (info "Test opts\n" (with-out-str (pprint opts))) (let [quorum (boolean (:quorum opts)) workload ((get workloads (:workload opts)) opts) - current-nemesis (get custom-nemesis/custom-nemesises "random-node-hammer-time")] + current-nemesis (get ch-nemesis/custom-nemeses (:nemesis opts))] (merge tests/noop-test opts - {:name (str "clickhouse-server-" (name (:workload opts))) + {:name (str "clickhouse-server-" (name (:workload opts)) "-" (name (:nemesis opts))) :os ubuntu/os :db (get-db opts) :pure-generators true @@ -81,20 +78,22 @@ (def all-workloads (keys workloads)) +(def all-nemeses (keys ch-nemesis/custom-nemeses)) + (defn all-test-options "Takes base cli options, a collection of nemeses, workloads, and a test count, and constructs a sequence of test options." - [cli] + [cli workload-nemesis-collection] (take (:test-count cli) - (shuffle (for [[workload] all-workloads] + (shuffle (for [[workload nemesis] workload-nemesis-collection] (assoc cli + :nemesis nemesis :workload workload :test-count 1))))) - (defn all-tests "Turns CLI options into a sequence of tests." [test-fn cli] - (map test-fn (all-test-options cli))) + (map test-fn (all-test-options cli (chu/cart [all-workloads all-nemeses])))) (defn main "Handles command line arguments. Can either run a test, or a web server for diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj index 3512409dd9e..faf16386526 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj @@ -1,7 +1,7 @@ (ns jepsen.clickhouse.server.nemesis (:require [jepsen.clickhouse.nemesis :as chnem])) -(def custom-nemesises +(def custom-nemeses {"random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) :generator (chnem/start-stop-generator 5 5)} "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj deleted file mode 100644 index 6789169db86..00000000000 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/register.clj +++ /dev/null @@ -1,56 +0,0 @@ -(ns jepsen.clickhouse.server.register - (:require [jepsen - [checker :as checker] - [client :as client] - [independent :as independent] - [generator :as gen]] - [jepsen.checker.timeline :as timeline] - [jepsen.clickhouse.utils :as chu] - [jepsen.clickhouse.server.client :as chc] - [clojure.tools.logging :refer :all] - [clojure.java.jdbc :as j] - [knossos.model :as model])) - -(defn r [_ _] {:type :invoke, :f :read, :value nil}) -(defn w [_ _] {:type :invoke, :f :write, :value (rand-int 5)}) -(defn cas [_ _] {:type :invoke, :f :cas, :value [(rand-int 5) (rand-int 5)]}) - -(defrecord RegisterClient [table-created? conn] - client/Client - (open! [this test node] - (assoc this :conn (chc/client node))) - - (setup! [this test] - (locking table-created? - (when (compare-and-set! table-created? false true) - (chc/with-connection [c conn] - (j/query c "DROP TABLE IF EXISTS register") - (j/query c "CREATE TABLE register (id Int64, value Int64) ENGINE=MergeTree ORDER BY id") - (info (j/query c "SHOW CREATE TABLE register")))))) - - (invoke! [_ test op] - (print "invoke")) - - (teardown! [this test] - nil) - - (close! [_ test] - (print "close"))) - -(defn workload - "Tests linearizable reads, writes, and compare-and-set operations on - independent keys." - [opts] - {:client (RegisterClient. (atom false) nil) - :checker (independent/checker - (checker/compose - {:linear (checker/linearizable {:model (model/cas-register) - :algorithm :linear}) - :perf (checker/perf) - :timeline (timeline/html)})) - :generator (independent/concurrent-generator - 10 - (range) - (fn [k] - (->> (gen/mix [r w cas]) - (gen/limit (:ops-per-key opts)))))}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj index 2abf0b2b081..e0abb5de44b 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj @@ -17,28 +17,28 @@ (assoc this :conn (chc/client node))) (setup! [this test] + (info (macroexpand (chc/with-connection [c conn] false (print "test")))) (locking table-created? (when (compare-and-set! table-created? false true) - (chc/with-connection [c conn] - (j/query c "DROP TABLE IF EXISTS set") - (j/query c "CREATE TABLE set (value Int64) Engine=MergeTree ORDER BY value"))))) + (chc/with-connection [c conn] false + (j/query c "DROP TABLE IF EXISTS set ON CLUSTER test_cluster") + (j/query c "CREATE TABLE set ON CLUSTER test_cluster (value Int64) Engine=ReplicatedMergeTree ORDER BY value"))))) (invoke! [this test op] (chc/with-exception op - (chc/with-connection [c conn] - (util/with-retry [] - (case (:f op) - :add (do - (j/query c (str "INSERT INTO set VALUES (" (:value op) ")")) - (assoc op :type :ok)) - :read (->> (j/query c "SELECT value FROM set") - (mapv :value) - (assoc op :type :ok, :value))))))) + (chc/with-connection [c conn] (= :read (:f op)) + (case (:f op) + :add (do + (j/query c (str "INSERT INTO set VALUES (" (:value op) ")")) + (assoc op :type :ok)) + :read (->> (j/query c "SELECT value FROM set") + (mapv :value) + (assoc op :type :ok, :value)))))) (teardown! [_ test] (util/timeout chc/operation-timeout - (chc/with-connection [c conn] - (j/query c ["DROP TABLE set"])))) + (chc/with-connection [c conn] false + (j/query c ["DROP TABLE set ON CLUSTER test_cluster"])))) (close! [_ test] (rc/close! conn))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj index e8c48a9011a..9e42a0b8323 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj @@ -25,6 +25,13 @@ (if (and s (> (count s) 0)) (Long/parseLong s))) +(defn cart [colls] + (if (empty? colls) + '(()) + (for [more (cart (rest colls)) + x (first colls)] + (cons x more)))) + (defn md5 [^String s] (let [algorithm (MessageDigest/getInstance "MD5") raw (.digest algorithm (.getBytes s))] From f41928878209598bc3c4c553dedf08b70a1e7f67 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 11:34:32 +0000 Subject: [PATCH 098/526] Add replicated merge tree config --- .../resources/replicated_merge_tree.xml | 95 +++++++++++++++++++ 1 file changed, 95 insertions(+) create mode 100644 tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml diff --git a/tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml b/tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml new file mode 100644 index 00000000000..43392c8da5b --- /dev/null +++ b/tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml @@ -0,0 +1,95 @@ + + /clickhouse/tables/{database}/{table} + {replica} + + + {replica_name} + + + + information + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + never + + + + 9181 + {server_id} + + + 10000 + 30000 + false + 120000 + trace + 1000 + 2000 + 4000 + false + + + + + 1 + {server1} + 9444 + + + 2 + {server2} + 9444 + + + 3 + {server3} + 9444 + + + + + + + {server1} + 9181 + + + {server2} + 9181 + + + {server3} + 9181 + + + + + + + + {server1} + 9000 + + + {server2} + 9000 + + + {server3} + 9000 + + + + + + 9009 + + + /clickhouse/task_queue/ddl + default + 1 + 604800 + 60 + 1000 + + From 602b1ebb15c4103c717be4a74fe53bd59c1eecb7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 28 Oct 2022 11:40:28 +0000 Subject: [PATCH 099/526] Fix --- .../src/jepsen/clickhouse/keeper/nemesis.clj | 24 +++++++------------ .../src/jepsen/clickhouse/server/set.clj | 1 - 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj index 76e4227cf72..6e54d1eee84 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj @@ -113,14 +113,6 @@ [] (nemesis/partitioner nemesis/bridge)) -(defn start-stop-generator - [time-corrupt time-ok] - (->> - (cycle [(gen/sleep time-ok) - {:type :info, :f :start} - (gen/sleep time-corrupt) - {:type :info, :f :stop}]))) - (defn corruption-generator [] (->> @@ -129,15 +121,15 @@ (def custom-nemesises {"random-node-killer" {:nemesis (random-node-killer-nemesis) - :generator (start-stop-generator 5 5)} + :generator (chnem/start-stop-generator 5 5)} "all-nodes-killer" {:nemesis (all-nodes-killer-nemesis) - :generator (start-stop-generator 1 10)} + :generator (chnem/start-stop-generator 1 10)} "simple-partitioner" {:nemesis (nemesis/partition-random-halves) - :generator (start-stop-generator 5 5)} + :generator (chnem/start-stop-generator 5 5)} "random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) - :generator (start-stop-generator 5 5)} + :generator (chnem/start-stop-generator 5 5)} "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) - :generator (start-stop-generator 1 10)} + :generator (chnem/start-stop-generator 1 10)} "logs-corruptor" {:nemesis (logs-corruption-nemesis) :generator (corruption-generator)} "snapshots-corruptor" {:nemesis (snapshots-corruption-nemesis) @@ -147,8 +139,8 @@ "drop-data-corruptor" {:nemesis (drop-all-corruption-nemesis) :generator (corruption-generator)} "bridge-partitioner" {:nemesis (partition-bridge-nemesis) - :generator (start-stop-generator 5 5)} + :generator (chnem/start-stop-generator 5 5)} "blind-node-partitioner" {:nemesis (blind-node-partition-nemesis) - :generator (start-stop-generator 5 5)} + :generator (chnem/start-stop-generator 5 5)} "blind-others-partitioner" {:nemesis (blind-others-partition-nemesis) - :generator (start-stop-generator 5 5)}}) + :generator (chnem/start-stop-generator 5 5)}}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj index e0abb5de44b..e624db1e5c5 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj +++ b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj @@ -17,7 +17,6 @@ (assoc this :conn (chc/client node))) (setup! [this test] - (info (macroexpand (chc/with-connection [c conn] false (print "test")))) (locking table-created? (when (compare-and-set! table-created? false true) (chc/with-connection [c conn] false From c53b96a11ce776f96e7cbd3b8cfe74f9fd495526 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Oct 2022 15:09:01 +0000 Subject: [PATCH 100/526] Add more comments --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 16 +++++++++++++--- src/Storages/MergeTree/ZooKeeperRetries.h | 1 + 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 8b226e2309d..16a5bbd587e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -368,11 +368,15 @@ void ReplicatedMergeTreeSink::commitPart( { zookeeper->setKeeper(storage.getZooKeeper()); - /// if we are in retry, check if last iteration was actually successful - /// we could get network error on latest keeper operation in iteration - /// but operation could be completed by keeper server if (retries_ctl.isRetry()) { + /// If we are retrying, check if last iteration was actually successful, + /// we could get network error on committing part to zk + /// but the operation could be completed by zk server + + /// If this flag is true, then part is in Active state, and we'll not retry anymore + /// we only check if part was committed to zk and return success or failure correspondingly + /// Note: if commit to zk failed then cleanup thread will mark the part as Outdated later if (part_committed_locally_but_zookeeper) { /// check that info about the part was actually written in zk @@ -616,9 +620,15 @@ void ReplicatedMergeTreeSink::commitPart( * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. */ transaction.commit(); + + /// Setting this flag is point of no return + /// On next retry, we'll just check if actually operation succeed or failed + /// and return ok or error correspondingly part_committed_locally_but_zookeeper = true; /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action + /// here lambda capture part name, it's ok since we'll not generate new one for this insert, + /// see comments around 'part_committed_locally_but_zookeeper' flag retries_ctl.actionAfterLastFailedRetry( [&storage = storage, part_name = part->name]() { storage.enqueuePartForCheck(part_name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); }); diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h index c000e297ceb..22ace074245 100644 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -136,6 +136,7 @@ public: Coordination::Error getLastKeeperErrorCode() const { return keeper_error.code; } + /// action will be called only once and only after latest failed retry void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } private: From d09deaf10f20c25116683c9baa7b34f27410e3bb Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Oct 2022 15:17:05 +0000 Subject: [PATCH 101/526] Fix style check --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 16a5bbd587e..ddf3cab7bba 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -627,7 +627,7 @@ void ReplicatedMergeTreeSink::commitPart( part_committed_locally_but_zookeeper = true; /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action - /// here lambda capture part name, it's ok since we'll not generate new one for this insert, + /// here lambda capture part name, it's ok since we'll not generate new one for this insert, /// see comments around 'part_committed_locally_but_zookeeper' flag retries_ctl.actionAfterLastFailedRetry( [&storage = storage, part_name = part->name]() From 5d41e7a6d5c68df4a633f69ee9618db377339d06 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 28 Oct 2022 20:03:23 +0000 Subject: [PATCH 102/526] Read-in-order over query plan (continuation) --- ...reuseStorageOrderingForWindowFunctions.cpp | 300 ++++++++++-------- 1 file changed, 168 insertions(+), 132 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 7fb0e99721c..d76090f5f16 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -193,10 +193,24 @@ void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns) } } -/// Here we try to find inner DAG inside outer DAG. -/// Build a map: inner.nodes -> outer.nodes. -// using NodesMap = std::unordered_map; -int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * outer) +struct MatchedTrees +{ + struct Monotonicity + { + int direction = 1; + bool strict = true; + }; + + struct Match + { + const ActionsDAG::Node * node = nullptr; + std::optional monotonicity; + }; + + using Matches = std::unordered_map; +}; + +MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag) { using Parents = std::set; std::unordered_map inner_parents; @@ -204,40 +218,45 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * { std::stack stack; - stack.push(inner); - inner_parents.emplace(inner, Parents()); - while (!stack.empty()) + for (const auto * out : inner_dag.getOutputs()) { - const auto * node = stack.top(); - stack.pop(); + if (inner_parents.contains(out)) + continue; - if (node->type == ActionsDAG::ActionType::INPUT) - inner_inputs.emplace(node->result_name, node); - - for (const auto * child : node->children) + stack.push(out); + inner_parents.emplace(out, Parents()); + while (!stack.empty()) { - auto [it, inserted] = inner_parents.emplace(child, Parents()); - it->second.emplace(node); + const auto * node = stack.top(); + stack.pop(); - if (inserted) - stack.push(child); + if (node->type == ActionsDAG::ActionType::INPUT) + inner_inputs.emplace(node->result_name, node); + + for (const auto * child : node->children) + { + auto [it, inserted] = inner_parents.emplace(child, Parents()); + it->second.emplace(node); + + if (inserted) + stack.push(child); + } } } } - std::unordered_map outer_to_inner; - std::unordered_map direction; - + struct Frame { - struct Frame - { - const ActionsDAG::Node * node; - ActionsDAG::NodeRawConstPtrs mapped_children; - int direction = 1; - }; + const ActionsDAG::Node * node; + ActionsDAG::NodeRawConstPtrs mapped_children; + }; + MatchedTrees::Matches matches; + + for (const auto * out : outer_dag.getOutputs()) + { std::stack stack; - stack.push(Frame{outer, {}}); + stack.push(Frame{out, {}}); while (!stack.empty()) { auto & frame = stack.top(); @@ -246,13 +265,13 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * while (frame.mapped_children.size() < frame.node->children.size()) { const auto * child = frame.node->children[frame.mapped_children.size()]; - auto it = outer_to_inner.find(child); - if (it == outer_to_inner.end()) + auto it = matches.find(child); + if (it == matches.end()) { stack.push(Frame{child, {}}); break; } - frame.mapped_children.push_back(it->second); + frame.mapped_children.push_back(it->second.node); } if (frame.mapped_children.size() < frame.node->children.size()) @@ -264,59 +283,22 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) mapped = it->second; - outer_to_inner.emplace(frame.node, mapped); + matches.emplace(frame.node, MatchedTrees::Match{.node = mapped}); } else if (frame.node->type == ActionsDAG::ActionType::ALIAS) { - outer_to_inner.emplace(frame.node, frame.mapped_children.at(0)); + matches.emplace(frame.node, matches[frame.node->children.at(0)]); } else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) { + auto & match = matches[frame.node]; + bool found_all_children = true; - size_t num_found_inner_roots = 0; for (const auto * child : frame.mapped_children) - { if (!child) found_all_children = false; - else if (child == inner) - ++num_found_inner_roots; - } - bool found_monotonic_wrapper = false; - if (num_found_inner_roots == 1) - { - if (frame.node->function_base->hasInformationAboutMonotonicity()) - { - size_t num_const_args = 0; - const ActionsDAG::Node * monotonic_child = nullptr; - for (const auto * child : frame.node->children) - { - if (child->column) - ++num_const_args; - else - monotonic_child = child; - } - - if (monotonic_child && num_const_args + 1 == frame.node->children.size()) - { - auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); - if (info.is_always_monotonic) - { - found_monotonic_wrapper = true; - outer_to_inner[frame.node] = inner; - - int cur_direction = info.is_positive ? 1 : -1; - auto it = direction.find(monotonic_child); - if (it != direction.end()) - cur_direction *= it->second; - - direction[frame.node] = cur_direction; - } - } - } - } - - if (!found_monotonic_wrapper && found_all_children && !frame.mapped_children.empty()) + if (found_all_children && !frame.mapped_children.empty()) { Parents container; Parents * intersection = &inner_parents[frame.mapped_children[0]]; @@ -351,8 +333,52 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * { auto func_name = frame.node->function_base->getName(); for (const auto * parent : *intersection) + { if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) - outer_to_inner[frame.node] = parent; + { + match.node = parent; + break; + } + } + } + } + + if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity()) + { + size_t num_const_args = 0; + const ActionsDAG::Node * monotonic_child = nullptr; + for (const auto * child : frame.node->children) + { + if (child->column) + ++num_const_args; + else + monotonic_child = child; + } + + if (monotonic_child && num_const_args + 1 == frame.node->children.size()) + { + const auto & child_match = matches[monotonic_child]; + if (child_match.node) + { + auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); + if (info.is_always_monotonic) + { + match.node = child_match.node; + + MatchedTrees::Monotonicity monotonicity; + monotonicity.direction *= info.is_positive ? 1 : -1; + monotonicity.strict = info.is_strict; + + if (child_match.monotonicity) + { + monotonicity.direction *= child_match.monotonicity->direction; + if (!child_match.monotonicity->strict) + monotonicity.strict = false; + } + + match.monotonicity = monotonicity; + } + } } } } @@ -361,84 +387,92 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * } } - if (outer_to_inner[outer] != inner) - return 0; - - int res = 1; - if (auto it = direction.find(outer); it != direction.end()) - res = it->second; - - return res; + return matches; } -SortDescription buildPrefixSortDescription( +InputOrderInfoPtr buildInputOrderInfo( const FixedColumns & fixed_columns, const ActionsDAGPtr & dag, const SortDescription & description, const ActionsDAG & sorting_key_dag, const Names & sorting_key_columns, - int & read_direction) + size_t limit) { SortDescription order_key_prefix_descr; order_key_prefix_descr.reserve(description.size()); + MatchedTrees::Matches matches; + if (dag) + matches = matchTrees(sorting_key_dag, *dag); + /// This is a result direction we will read from MergeTree /// 1 - in order, /// -1 - in reverse order, /// 0 - usual read, don't apply optimization /// /// So far, 0 means any direction is possible. It is ok for constant prefix. - read_direction = 0; + int read_direction = 0; + size_t next_descr_column = 0; + size_t next_sort_key = 0; - for (size_t i = 0, next_sort_key = 0; i < description.size() && next_sort_key < sorting_key_columns.size(); ++i) + for (; next_descr_column < description.size() && next_sort_key < sorting_key_columns.size(); ++next_sort_key) { - const auto & sort_column = description[i]; const auto & sorting_key_column = sorting_key_columns[next_sort_key]; + const auto & descr = description[next_descr_column]; /// If required order depend on collation, it cannot be matched with primary key order. /// Because primary keys cannot have collations. - if (sort_column.collator) - return order_key_prefix_descr; + if (descr.collator) + break; /// Direction for current sort key. int current_direction = 0; + bool strict_monotonic = true; + + const ActionsDAG::Node * sort_column_node = sorting_key_dag.tryFindInOutputs(sorting_key_column); + /// This should not happen. + if (!sort_column_node) + break; if (!dag) { - if (sort_column.column_name != sorting_key_column) - return order_key_prefix_descr; + if (sort_column_node->type != ActionsDAG::ActionType::INPUT) + break; - current_direction = sort_column.direction; - ++next_sort_key; + if (descr.column_name != sorting_key_column) + break; + + current_direction = descr.direction; } else { - const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(sort_column.column_name); + const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(descr.column_name); /// It is possible when e.g. sort by array joined column. if (!sort_node) - return order_key_prefix_descr; + break; - const ActionsDAG::Node * sort_column_node = sorting_key_dag.tryFindInOutputs(sorting_key_column); - /// This should not happen. - if (!sort_column_node) - return order_key_prefix_descr; + const auto & match = matches[sort_node]; - bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); + if (match.node) + { + /// We try to find the match first even if column is fixed. In this case, potentially more keys will match. + /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' + /// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1' + /// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order. - /// We try to find the match even if column is fixed. In this case, potentially more keys will match. - /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' - /// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1' - /// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order. - current_direction = isMonotonicSubtree(sort_column_node, sort_node) * sort_column.direction; - - if (current_direction == 0 || !is_fixed_column) - return order_key_prefix_descr; - - if (current_direction) - ++next_sort_key; - - if (is_fixed_column) - current_direction = 0; + current_direction = 1; + if (match.monotonicity) + { + current_direction *= match.monotonicity->direction; + strict_monotonic = match.monotonicity->strict; + } + } + else + { + bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); + if (!is_fixed_column) + break; + } } /// read_direction == 0 means we can choose any global direction. @@ -446,11 +480,23 @@ SortDescription buildPrefixSortDescription( if (current_direction && read_direction && current_direction != read_direction) break; - read_direction = current_direction; - order_key_prefix_descr.push_back(description[i]); + if (read_direction == 0) + read_direction = current_direction; + + if (current_direction) + { + order_key_prefix_descr.push_back(description[next_descr_column]); + ++next_descr_column; + + if (!strict_monotonic) + break; + } } - return order_key_prefix_descr; + if (read_direction == 0 || order_key_prefix_descr.empty()) + return nullptr; + + return std::make_shared(order_key_prefix_descr, next_sort_key, read_direction, limit); } void optimizeReadInOrder(QueryPlan::Node & node) @@ -475,30 +521,20 @@ void optimizeReadInOrder(QueryPlan::Node & node) buildSortingDAG(node.children.front(), dag, fixed_columns); const auto & description = sorting->getSortDescription(); + auto limit = sorting->getLimit(); const auto & sorting_key_columns = sorting_key.column_names; - int read_direction = 0; - auto prefix_description = buildPrefixSortDescription( + auto order_info = buildInputOrderInfo( fixed_columns, dag, description, sorting_key.expression->getActionsDAG(), sorting_key_columns, - read_direction); + limit); - /// It is possible that prefix_description is not empty, but read_direction is 0. - /// It means that some prefix of sorting key matched, but it was constant. - /// In this case, read-in-order is useless. - if (read_direction == 0 || prefix_description.empty()) + if (!order_info) return; - auto limit = sorting->getLimit(); - - auto order_info = std::make_shared( - SortDescription{}, - std::move(prefix_description), - read_direction, limit); - - reading->setQueryInfoInputOrderInfo(order_info); - sorting->convertToFinishSorting(order_info->order_key_prefix_descr); + reading->requestReadingInOrder(order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); + sorting->convertToFinishSorting(order_info->sort_description_for_merging); } size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) From 6cbf50a8295cbf7e21aabc461ebf2322d167ad75 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Fri, 28 Oct 2022 21:06:32 +0000 Subject: [PATCH 103/526] extract common part --- src/Functions/dateDiff.cpp | 335 ++++++++++++++++++++++--------------- src/Functions/timeDiff.cpp | 228 ------------------------- 2 files changed, 204 insertions(+), 359 deletions(-) delete mode 100644 src/Functions/timeDiff.cpp diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index b8bf3c11698..5673fd23173 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -33,6 +33,147 @@ namespace ErrorCodes namespace { +class DateDiffImpl +{ +using ColumnDateTime64 = ColumnDecimal; +public: +explicit DateDiffImpl(const String& name_) : name(name_) {} + +template +void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for first argument of function " + name + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + name + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception("Illegal column for second argument of function " + name + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); +} + +template +void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); + + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); +} + +template +void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); + + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); +} + +template +void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const +{ + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); + + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); +} + +template +Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const +{ + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); +} + +template +static UInt32 getScale(const T & v) +{ + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; +} +template +static auto stripDecimalFieldValue(T && v) +{ + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; +} +private: + String name; +}; + + /** dateDiff('unit', t1, t2, [timezone]) * t1 and t2 can be Date or DateTime * @@ -105,158 +246,85 @@ public: const auto & timezone_y = extractTimeZoneFromFunctionArguments(arguments, 3, 2); if (unit == "year" || unit == "yy" || unit == "yyyy") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "quarter" || unit == "qq" || unit == "q") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "month" || unit == "mm" || unit == "m") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "week" || unit == "wk" || unit == "ww") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "day" || unit == "dd" || unit == "d") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "hour" || unit == "hh" || unit == "h") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "minute" || unit == "mi" || unit == "n") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else if (unit == "second" || unit == "ss" || unit == "s") - dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); + impl.dispatchForColumns(x, y, timezone_x, timezone_y, res->getData()); else throw Exception("Function " + getName() + " does not support '" + unit + "' unit", ErrorCodes::BAD_ARGUMENTS); return res; } - private: - template - void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + DateDiffImpl impl{name}; +}; + + +/** TimeDiff(t1, t2) + * t1 and t2 can be Date or DateTime + */ +class FunctionTimeDiff : public IFunction +{ + using ColumnDateTime64 = ColumnDecimal; +public: + static constexpr auto name = "TimeDiff"; + static FunctionPtr create(ContextPtr) { return std::make_shared(); } + + String getName() const override { - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); + return name; } - template - void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + bool isVariadic() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); + if (arguments.size() != 2) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 2", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) + throw Exception("First argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) + throw Exception("Second argument for function " + getName() + " must be Date or DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return std::make_shared(); } - template - void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const + bool useDefaultImplementationForConstants() const override { return true; } + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); - - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); - } - - template - void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); - - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); - } - - template - void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); - - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); - } - - template - Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const - { - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); - } - - template - static UInt32 getScale(const T & v) - { - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); - - return 0; - } - template - static auto stripDecimalFieldValue(T && v) - { - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; + const IColumn & x = *arguments[0].column; + const IColumn & y = *arguments[1].column; + + size_t rows = input_rows_count; + auto res = ColumnInt64::create(rows); + + impl.dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); + + return res; } +private: + DateDiffImpl impl{name}; }; } @@ -266,4 +334,9 @@ REGISTER_FUNCTION(DateDiff) factory.registerFunction({}, FunctionFactory::CaseInsensitive); } +REGISTER_FUNCTION(TimeDiff) +{ + factory.registerFunction({}, FunctionFactory::CaseInsensitive); +} + } diff --git a/src/Functions/timeDiff.cpp b/src/Functions/timeDiff.cpp deleted file mode 100644 index 0353dbf7c51..00000000000 --- a/src/Functions/timeDiff.cpp +++ /dev/null @@ -1,228 +0,0 @@ -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include -#include -#include - -#include - -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ILLEGAL_COLUMN; -} - -namespace -{ - -/** TimeDiff(t1, t2) - * t1 and t2 can be Date or DateTime - */ -class FunctionTimeDiff : public IFunction -{ - using ColumnDateTime64 = ColumnDecimal; -public: - static constexpr auto name = "TimeDiff"; - static FunctionPtr create(ContextPtr) { return std::make_shared(); } - - String getName() const override - { - return name; - } - - bool isVariadic() const override { return false; } - bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; } - size_t getNumberOfArguments() const override { return 2; } - - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override - { - if (arguments.size() != 2) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 2", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) - throw Exception("First argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) - throw Exception("Second argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return std::make_shared(); - } - - bool useDefaultImplementationForConstants() const override { return true; } - ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override - { - const IColumn & x = *arguments[0].column; - const IColumn & y = *arguments[1].column; - - size_t rows = input_rows_count; - auto res = ColumnInt64::create(rows); - - dispatchForColumns(x, y, DateLUT::instance(), DateLUT::instance(), res->getData()); - - return res; - } - -private: - template - void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for first argument of function " + getName() + ", must be Date, DateTime or DateTime64", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception("Illegal column for second argument of function " + getName() + ", must be Date or DateTime", ErrorCodes::ILLEGAL_COLUMN); - } - - template - void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); - - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); - } - - template - void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); - - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); - } - - template - void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const - { - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); - - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); - } - - template - Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const - { - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); - } - - template - static UInt32 getScale(const T & v) - { - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); - - return 0; - } - template - static auto stripDecimalFieldValue(T && v) - { - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; - } -}; - -} - -REGISTER_FUNCTION(TimeDiff) -{ - factory.registerFunction({}, FunctionFactory::CaseInsensitive); -} - -} From 7419a3bd583ea476baa0b57ea8c66e647aa114a8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Sat, 29 Oct 2022 18:58:11 +0800 Subject: [PATCH 104/526] fix Signed-off-by: Lloyd-Pottiger --- CMakeLists.txt | 4 ++-- cmake/tools.cmake | 11 ++++++++--- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 7a04f347b2d..d10bc63c15e 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -202,7 +202,7 @@ option(ADD_GDB_INDEX_FOR_GOLD "Add .gdb-index to resulting binaries for gold lin if (NOT CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE") # Can be lld or ld-lld or lld-13 or /path/to/lld. - if (LINKER_NAME MATCHES "lld") + if (LINKER_NAME MATCHES "lld" AND OS_LINUX) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--gdb-index") set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--gdb-index") message (STATUS "Adding .gdb-index via --gdb-index linker option.") @@ -248,7 +248,7 @@ endif () # Create BuildID when using lld. For other linkers it is created by default. # (NOTE: LINKER_NAME can be either path or name, and in different variants) -if (LINKER_NAME MATCHES "lld") +if (LINKER_NAME MATCHES "lld" AND OS_LINUX) # SHA1 is not cryptographically secure but it is the best what lld is offering. set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--build-id=sha1") endif () diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 8a17d97cf13..23f34bb24cd 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -57,14 +57,19 @@ if (NOT LINKER_NAME) if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") - elseif (COMPILER_CLANG) + # llvm lld is a generic driver. + # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead + elseif (COMPILER_CLANG AND OS_LINUX) find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") find_program (GOLD_PATH NAMES "ld.gold" "gold") + elseif (COMPILER_CLANG AND OS_DARWIN) + find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld64.lld" "lld") + find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () endif() -if (OS_LINUX AND NOT LINKER_NAME) - # prefer lld linker over gold or ld on linux +if ((OS_LINUX OR OS_DARWIN) AND NOT LINKER_NAME) + # prefer lld linker over gold or ld on linux and macos if (LLD_PATH) if (COMPILER_GCC) # GCC driver requires one of supported linker names like "lld". From 330f18194a90fc473a5630117a7630d345d8a783 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Sun, 30 Oct 2022 14:44:13 +0000 Subject: [PATCH 105/526] fix test --- .../02415_all_new_functions_must_be_documented.reference | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index 040a8c8d317..ae701d9ae83 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -51,9 +51,11 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI +TimeDiff URLHash URLHierarchy URLPathHierarchy +UTCTimestamp UUIDNumToString UUIDStringToNum _CAST From 9f61ed4d50f911d31081f86af6bfcc0db2af8e3b Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Mon, 31 Oct 2022 13:23:27 +0100 Subject: [PATCH 106/526] Added no-fasttest tag for JSON tests --- tests/queries/0_stateless/02452_check_low_cardinality.sql | 1 + .../queries/0_stateless/02474_extract_fixedstring_from_json.sql | 1 + 2 files changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02452_check_low_cardinality.sql b/tests/queries/0_stateless/02452_check_low_cardinality.sql index 166be281405..e9cb8c800c7 100644 --- a/tests/queries/0_stateless/02452_check_low_cardinality.sql +++ b/tests/queries/0_stateless/02452_check_low_cardinality.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest DROP TABLE IF EXISTS test_low_cardinality_string; DROP TABLE IF EXISTS test_low_cardinality_uuid; DROP TABLE IF EXISTS test_low_cardinality_int; diff --git a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql index cfc47e00cba..336dda411da 100644 --- a/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql +++ b/tests/queries/0_stateless/02474_extract_fixedstring_from_json.sql @@ -1,3 +1,4 @@ +-- Tags: no-fasttest SELECT JSONExtract('{"a": 123456}', 'FixedString(11)'); SELECT JSONExtract('{"a": 123456}', 'FixedString(12)'); SELECT JSONExtract('{"a": "123456"}', 'a', 'FixedString(5)'); From c97c78e3fff7792e0ff6fd358df5f3415b7ab2e5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Mon, 31 Oct 2022 13:49:31 +0100 Subject: [PATCH 107/526] Updated normaliser to clone the alias ast. Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. Testing: * Added a test for or function with alias and const where 02475_or_function_alias_and_const_where.sql --- src/Interpreters/QueryNormalizer.cpp | 2 +- .../02475_or_function_alias_and_const_where.reference | 2 ++ .../0_stateless/02475_or_function_alias_and_const_where.sql | 2 ++ 3 files changed, 5 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference create mode 100644 tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 2a8b256c3d1..0f57a8f549c 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -118,7 +118,7 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) } } else - ast = alias_node; + ast = alias_node->clone(); } } diff --git a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference new file mode 100644 index 00000000000..b5d8e605a7d --- /dev/null +++ b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.reference @@ -0,0 +1,2 @@ +0 0 +0 0 diff --git a/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql new file mode 100644 index 00000000000..ddb0f70c6de --- /dev/null +++ b/tests/queries/0_stateless/02475_or_function_alias_and_const_where.sql @@ -0,0 +1,2 @@ +SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1; +SELECT (number = 1) AND (number = 2) AS value, sum(value) OVER () FROM numbers(1) WHERE 1 SETTINGS allow_experimental_analyzer=1; \ No newline at end of file From 81c1576e61fcac935bba37701fd5e072367be6c1 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 31 Oct 2022 13:03:13 +0000 Subject: [PATCH 108/526] add tests --- src/Functions/dateDiff.cpp | 290 +++++++++--------- .../02474_timeDiff_UTCTimestamp.reference | 9 + .../02474_timeDiff_UTCTimestamp.sql | 12 + 3 files changed, 168 insertions(+), 143 deletions(-) create mode 100644 tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference create mode 100644 tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index e62f0e2c517..c653db3ddf2 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -36,156 +36,157 @@ namespace class DateDiffImpl { -using ColumnDateTime64 = ColumnDecimal; public: -explicit DateDiffImpl(const String& name_) : name(name_) {} + using ColumnDateTime64 = ColumnDecimal; -template -void dispatchForColumns( - const IColumn & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * x_vec_16 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); - else if (const auto * x_vec_64 = checkAndGetColumn(&x)) - dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); - else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); - else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) - dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for first argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + explicit DateDiffImpl(const String & name_) : name(name_) {} -template -void dispatchForSecondColumn( - const LeftColumnType & x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); - else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); - else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) - vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + template + void dispatchForColumns( + const IColumn & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * x_vec_16 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_16, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_32_s = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_32_s, y, timezone_x, timezone_y, result); + else if (const auto * x_vec_64 = checkAndGetColumn(&x)) + dispatchForSecondColumn(*x_vec_64, y, timezone_x, timezone_y, result); + else if (const auto * x_const_16 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_16->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_32_s = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_32_s->getValue(), y, timezone_x, timezone_y, result); + else if (const auto * x_const_64 = checkAndGetColumnConst(&x)) + dispatchConstForSecondColumn(x_const_64->getValue>(), y, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for first argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } -template -void dispatchConstForSecondColumn( - T1 x, const IColumn & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - if (const auto * y_vec_16 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_16, timezone_x, timezone_y, result); - else if (const auto * y_vec_32 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32, timezone_x, timezone_y, result); - else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) - constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); - else if (const auto * y_vec_64 = checkAndGetColumn(&y)) - constantVector(x, *y_vec_64, timezone_x, timezone_y, result); - else - throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", - name); -} + template + void dispatchForSecondColumn( + const LeftColumnType & x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + vectorVector(x, *y_vec_64, timezone_x, timezone_y, result); + else if (const auto * y_const_16 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_16->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_32_s = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_32_s->getValue(), timezone_x, timezone_y, result); + else if (const auto * y_const_64 = checkAndGetColumnConst(&y)) + vectorConstant(x, y_const_64->getValue>(), timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } -template -void vectorVector( - const LeftColumnType & x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & x_data = x.getData(); - const auto & y_data = y.getData(); + template + void dispatchConstForSecondColumn( + T1 x, const IColumn & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + if (const auto * y_vec_16 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_16, timezone_x, timezone_y, result); + else if (const auto * y_vec_32 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32, timezone_x, timezone_y, result); + else if (const auto * y_vec_32_s = checkAndGetColumn(&y)) + constantVector(x, *y_vec_32_s, timezone_x, timezone_y, result); + else if (const auto * y_vec_64 = checkAndGetColumn(&y)) + constantVector(x, *y_vec_64, timezone_x, timezone_y, result); + else + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column for second argument of function {}, must be Date, Date32, DateTime or DateTime64", + name); + } - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); -} + template + void vectorVector( + const LeftColumnType & x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto & y_data = y.getData(); -template -void vectorConstant( - const LeftColumnType & x, T2 y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & x_data = x.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto y_value = stripDecimalFieldValue(y); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_data[i], timezone_x, timezone_y); + } - for (size_t i = 0, size = x.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); -} + template + void vectorConstant( + const LeftColumnType & x, T2 y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & x_data = x.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto y_value = stripDecimalFieldValue(y); -template -void constantVector( - T1 x, const RightColumnType & y, - const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, - ColumnInt64::Container & result) const -{ - const auto & y_data = y.getData(); - const auto transform_x = TransformDateTime64(getScale(x)); - const auto transform_y = TransformDateTime64(getScale(y)); - const auto x_value = stripDecimalFieldValue(x); + for (size_t i = 0, size = x.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_data[i], y_value, timezone_x, timezone_y); + } - for (size_t i = 0, size = y.size(); i < size; ++i) - result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); -} + template + void constantVector( + T1 x, const RightColumnType & y, + const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y, + ColumnInt64::Container & result) const + { + const auto & y_data = y.getData(); + const auto transform_x = TransformDateTime64(getScale(x)); + const auto transform_y = TransformDateTime64(getScale(y)); + const auto x_value = stripDecimalFieldValue(x); -template -Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const -{ - return static_cast(transform_y.execute(y, timezone_y)) - - static_cast(transform_x.execute(x, timezone_x)); -} + for (size_t i = 0, size = y.size(); i < size; ++i) + result[i] = calculate(transform_x, transform_y, x_value, y_data[i], timezone_x, timezone_y); + } -template -static UInt32 getScale(const T & v) -{ - if constexpr (std::is_same_v) - return v.getScale(); - else if constexpr (std::is_same_v>) - return v.getScale(); + template + Int64 calculate(const TransformX & transform_x, const TransformY & transform_y, T1 x, T2 y, const DateLUTImpl & timezone_x, const DateLUTImpl & timezone_y) const + { + return static_cast(transform_y.execute(y, timezone_y)) + - static_cast(transform_x.execute(x, timezone_x)); + } - return 0; -} -template -static auto stripDecimalFieldValue(T && v) -{ - if constexpr (std::is_same_v, DecimalField>) - return v.getValue(); - else - return v; -} + template + static UInt32 getScale(const T & v) + { + if constexpr (std::is_same_v) + return v.getScale(); + else if constexpr (std::is_same_v>) + return v.getScale(); + + return 0; + } + template + static auto stripDecimalFieldValue(T && v) + { + if constexpr (std::is_same_v, DecimalField>) + return v.getValue(); + else + return v; + } private: String name; }; @@ -321,13 +322,16 @@ public: + toString(arguments.size()) + ", should be 2", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isDate(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[1])) - throw Exception("First argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isDate(arguments[0]) && !isDate32(arguments[0]) && !isDateTime(arguments[0]) && !isDateTime64(arguments[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "First argument for function {} must be Date, Date32, DateTime or DateTime64", + getName()); - if (!isDate(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[2])) - throw Exception("Second argument for function " + getName() + " must be Date or DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + if (!isDate(arguments[1]) && !isDate32(arguments[1]) && !isDateTime(arguments[1]) && !isDateTime64(arguments[1])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Second argument for function {} must be Date, Date32, DateTime or DateTime64", + getName() + ); return std::make_shared(); } diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference new file mode 100644 index 00000000000..bc6ff2b709c --- /dev/null +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference @@ -0,0 +1,9 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql new file mode 100644 index 00000000000..eac5edae483 --- /dev/null +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql @@ -0,0 +1,12 @@ +-- all tests should be equal to zero as timediff is same as dateDiff('second', ... ) +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')); + +-- UTCTimestamp equals to now('UTC') +SELECT dateDiff('s', UTCTimestamp(), now('UTC')); +SELECT timeDiff(UTCTimestamp(), now('UTC')); From 375db5bd1581879c40a9ec9b106c0357aa9041c6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Oct 2022 13:54:03 +0000 Subject: [PATCH 109/526] Use read-in-order from query plan by default. --- src/Core/Settings.h | 1 + src/Interpreters/ExpressionAnalyzer.cpp | 2 +- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlanOptimizationSettings.cpp | 1 + .../QueryPlanOptimizationSettings.h | 3 ++ .../optimizePrimaryKeyCondition.cpp | 5 +-- ...reuseStorageOrderingForWindowFunctions.cpp | 35 +++++++++++++++++-- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- .../QueryPlan/ReadFromMergeTree.cpp | 16 ++++----- src/Processors/QueryPlan/SortingStep.h | 16 +++++---- ...monotonous_functions_in_order_by.reference | 4 +-- .../01576_alias_column_rewrite.reference | 4 +-- 12 files changed, 64 insertions(+), 27 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5c6ca1a1d57..1213b866b15 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -576,6 +576,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ + M(Bool, query_plan_read_in_order, true, "Use query plan for read-in-order optimisation", 0) \ M(UInt64, regexp_max_matches_per_row, 1000, "Max matches of any single regexp per row, used to safeguard 'extractAllGroupsHorizontal' against consuming too much memory with greedy RE.", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9daa42bf499..9ac0ea4f5d4 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1944,7 +1944,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } optimize_read_in_order = - settings.optimize_read_in_order + settings.optimize_read_in_order && (!settings.query_plan_read_in_order) && storage && query.orderBy() && !query_analyzer.hasAggregation() diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 0a085c0ce24..05266ccdbd2 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -13,7 +13,7 @@ namespace QueryPlanOptimizations void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); void optimizeReadInOrder(QueryPlan::Node & node); -void optimizePrimaryKeyCondition(QueryPlan::Node & root); +void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root); /// Optimization is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp index 2342f961751..88c0f346e9a 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -12,6 +12,7 @@ QueryPlanOptimizationSettings QueryPlanOptimizationSettings::fromSettings(const settings.max_optimizations_to_apply = from.query_plan_max_optimizations_to_apply; settings.filter_push_down = from.query_plan_filter_push_down; settings.distinct_in_order = from.optimize_distinct_in_order; + settings.read_in_order = from.optimize_read_in_order && from.query_plan_read_in_order; return settings; } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 0da89f28aad..56dd39fdf0a 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -24,6 +24,9 @@ struct QueryPlanOptimizationSettings /// if distinct in order optimization is enabled bool distinct_in_order = false; + /// If read-in-order optimisation is enabled + bool read_in_order = false; + static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); }; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index ebaf99dbd3f..b33ae646535 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -9,7 +9,7 @@ namespace DB::QueryPlanOptimizations { -void optimizePrimaryKeyCondition(QueryPlan::Node & root) +void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root) { struct Frame { @@ -33,7 +33,8 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) continue; } - optimizeReadInOrder(*frame.node); + if (optimization_settings.read_in_order) + optimizeReadInOrder(*frame.node); auto add_filter = [&](auto & storage) { diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index d76090f5f16..63f6a4838a2 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -73,7 +73,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr { if (!child->column) { - if (maybe_fixed_column) + if (!maybe_fixed_column) maybe_fixed_column = child; else is_singe = false; @@ -81,7 +81,10 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr } if (maybe_fixed_column && is_singe) + { + std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; fiexd_columns.insert(maybe_fixed_column); + } } } } @@ -402,9 +405,22 @@ InputOrderInfoPtr buildInputOrderInfo( order_key_prefix_descr.reserve(description.size()); MatchedTrees::Matches matches; + FixedColumns fixed_key_columns; + if (dag) + { matches = matchTrees(sorting_key_dag, *dag); + for (const auto & [node, match] : matches) + { + if (!match.monotonicity || match.monotonicity->strict) + { + if (match.node && fixed_columns.contains(node)) + fixed_key_columns.insert(match.node); + } + } + } + /// This is a result direction we will read from MergeTree /// 1 - in order, /// -1 - in reverse order, @@ -453,22 +469,32 @@ InputOrderInfoPtr buildInputOrderInfo( const auto & match = matches[sort_node]; - if (match.node) + // std::cerr << "====== Finding match for " << sort_node->result_name << ' ' << static_cast(sort_node) << std::endl; + + if (match.node && match.node == sort_column_node) { + // std::cerr << "====== Found direct match" << std::endl; + /// We try to find the match first even if column is fixed. In this case, potentially more keys will match. /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' /// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1' /// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order. - current_direction = 1; + current_direction = descr.direction; if (match.monotonicity) { current_direction *= match.monotonicity->direction; strict_monotonic = match.monotonicity->strict; } } + else if (fixed_key_columns.contains(sort_column_node)) + { + // std::cerr << "+++++++++ Found fixed key by match" << std::endl; + } else { + + // std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); if (!is_fixed_column) break; @@ -508,6 +534,9 @@ void optimizeReadInOrder(QueryPlan::Node & node) if (!sorting) return; + if (sorting->getType() != SortingStep::Type::Full) + return; + ReadFromMergeTree * reading = findReadingStep(node.children.front()); if (!reading) return; diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 5a92bcb41ee..d4e55955899 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -448,7 +448,7 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizePrimaryKeyCondition(*root); + QueryPlanOptimizations::optimizePrimaryKeyCondition(optimization_settings, *root); } void QueryPlan::explainEstimate(MutableColumns & columns) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c88a244b963..2e40ec4246d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1067,14 +1067,14 @@ void ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, break; sort_description.emplace_back(column_name, sort_direction); } - if (sort_description.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Sort description can't be empty when reading in order"); - - const size_t used_prefix_of_sorting_key_size = order_info->used_prefix_of_sorting_key_size; - if (sort_description.size() > used_prefix_of_sorting_key_size) - sort_description.resize(used_prefix_of_sorting_key_size); - output_stream->sort_description = std::move(sort_description); - output_stream->sort_scope = DataStream::SortScope::Stream; + if (!sort_description.empty()) + { + const size_t used_prefix_of_sorting_key_size = order_info->used_prefix_of_sorting_key_size; + if (sort_description.size() > used_prefix_of_sorting_key_size) + sort_description.resize(used_prefix_of_sorting_key_size); + output_stream->sort_description = std::move(sort_description); + output_stream->sort_scope = DataStream::SortScope::Stream; + } } ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 55ef59f86c1..000c4955b67 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -11,6 +11,13 @@ namespace DB class SortingStep : public ITransformingStep { public: + enum class Type + { + Full, + FinishSorting, + MergingSorted, + }; + /// Full SortingStep( const DataStream & input_stream, @@ -55,6 +62,8 @@ public: void convertToFinishSorting(SortDescription prefix_description); + Type getType() const { return type; } + private: void updateOutputStream() override; @@ -68,13 +77,6 @@ private: UInt64 limit_, bool skip_partial_sort = false); - enum class Type - { - Full, - FinishSorting, - MergingSorted, - }; - Type type; SortDescription prefix_description; diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference index 45a1a094c49..8c8bb73b801 100644 --- a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -17,7 +17,7 @@ ORDER BY toDate(timestamp) ASC LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - Sorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) ReadFromMergeTree (default.test_order_by) SELECT @@ -30,7 +30,7 @@ ORDER BY LIMIT 10 Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - Sorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) ReadFromMergeTree (default.test_order_by) SELECT diff --git a/tests/queries/0_stateless/01576_alias_column_rewrite.reference b/tests/queries/0_stateless/01576_alias_column_rewrite.reference index 830db82274f..2b7fdfaa642 100644 --- a/tests/queries/0_stateless/01576_alias_column_rewrite.reference +++ b/tests/queries/0_stateless/01576_alias_column_rewrite.reference @@ -28,12 +28,12 @@ Expression (Projection) ReadFromMergeTree (default.test_table) Expression (Projection) Limit (preliminary LIMIT (without OFFSET)) - Sorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) ReadFromMergeTree (default.test_table) Expression ((Projection + Before ORDER BY [lifted up part])) Limit (preliminary LIMIT (without OFFSET)) - Sorting + Sorting (Sorting for ORDER BY) Expression (Before ORDER BY) ReadFromMergeTree (default.test_table) optimize_aggregation_in_order From 068ae904484b04894c2c6e39761a1901aa1a912d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Oct 2022 14:01:51 +0000 Subject: [PATCH 110/526] Comment debug code. --- .../Optimizations/reuseStorageOrderingForWindowFunctions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 63f6a4838a2..adb1e3dcc45 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -82,7 +82,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr if (maybe_fixed_column && is_singe) { - std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; + // std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; fiexd_columns.insert(maybe_fixed_column); } } From 2d5050ea66309b57cad9605d9d3f1ca6ce834037 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 31 Oct 2022 18:20:58 +0000 Subject: [PATCH 111/526] Fix review comments --- .../MergeTree/PartMovesBetweenShardsOrchestrator.cpp | 1 - src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 1 - src/Storages/MergeTree/ZooKeeperWithFaultInjection.h | 9 +++++---- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ .../0_stateless/01158_zookeeper_log_long.reference | 2 ++ tests/queries/0_stateless/01158_zookeeper_log_long.sql | 3 ++- 6 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp index 52e11ab97a5..d5f35ea1b3c 100644 --- a/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp +++ b/src/Storages/MergeTree/PartMovesBetweenShardsOrchestrator.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index ddf3cab7bba..9da29d0becf 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -737,7 +737,6 @@ void ReplicatedMergeTreeSink::onStart() void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); - /// todo: check this place, afaiu, it can be called after Generate apart of Consume assertSessionIsNotExpired(zookeeper); finishDelayedChunk(std::make_shared(zookeeper)); } diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h index 1e0b730c32a..c2e689b0307 100644 --- a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -498,6 +498,7 @@ private: e.code, e.message()); + /// save valid pointer to clean up ephemeral nodes later if necessary if (keeper) keeper_prev = keeper; keeper.reset(); @@ -508,16 +509,16 @@ private: if constexpr (std::is_same_v) { - /// try*() methods throws at least on hardware error and return only on logical errors - /// todo: the methods return only on subset of logical errors, and throw on another errors + /// try*() methods throws at least on hardware error and return only on user errors + /// todo: the methods return only on subset of user errors, and throw on another errors /// to mimic the methods exactly - we need to specify errors on which to return for each such method if (Coordination::isHardwareError(e.code)) throw; return e.code; } - else - throw; + + throw; } } }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b7fcf8d72b2..48539b593af 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4460,6 +4460,8 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { const auto storage_settings_ptr = getSettings(); + assertNotReadonly(); + const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index 3473a28911d..ee2b465b3d3 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -17,6 +17,8 @@ Response 0 Remove /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 Response 0 Remove /test/01158/default/rmt/temp/abandonable_lock-0000000000 0 0 -1 0 3 ZOK \N \N 0 0 0 0 Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0 Response 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 5 ZOK \N \N /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 0 0 +Request 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 \N \N \N 0 0 0 0 +Response 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 ZOK \N \N 0 0 96 0 blocks Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0 diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index eb549e1a37a..d2212f42818 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -10,6 +10,7 @@ create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt' system sync replica rmt; insert into rmt values (1); insert into rmt values (1); +system sync replica rmt; system flush logs; select 'log'; @@ -22,7 +23,7 @@ select 'parts'; select type, has_watch, op_num, path, is_ephemeral, is_sequential, version, requests_size, request_idx, error, watch_type, watch_state, path_created, stat_version, stat_cversion, stat_dataLength, stat_numChildren from system.zookeeper_log -where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path='/test/01158/' || currentDatabase() || '/rmt/replicas/1/parts/all_0_0_0' and op_num not in(3)) +where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path='/test/01158/' || currentDatabase() || '/rmt/replicas/1/parts/all_0_0_0') order by xid, type, request_idx; select 'blocks'; From e99fd4ed256e50117d90a491835f88e06ea6a0c5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 31 Oct 2022 20:50:10 +0000 Subject: [PATCH 112/526] Add test --- ...reuseStorageOrderingForWindowFunctions.cpp | 93 ++++++-- ...rder_by_read_in_order_query_plan.reference | 217 ++++++++++++++++++ ...0940_order_by_read_in_order_query_plan.sql | 91 ++++++++ 3 files changed, 377 insertions(+), 24 deletions(-) create mode 100644 tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference create mode 100644 tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index adb1e3dcc45..01132e36502 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -82,7 +82,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr if (maybe_fixed_column && is_singe) { - // std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; + //std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; fiexd_columns.insert(maybe_fixed_column); } } @@ -136,7 +136,7 @@ void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, FixedColumns & } } -void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns) +void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { struct Frame { @@ -146,7 +146,7 @@ void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns) std::stack stack; std::unordered_set visited; - for (const auto & node : dag->getNodes()) + for (const auto & node : dag.getNodes()) { if (visited.contains(&node)) continue; @@ -181,16 +181,27 @@ void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns) { if (frame.node->function_base->isDeterministicInScopeOfQuery()) { + //std::cerr << "*** enreachFixedColumns check " << frame.node->result_name << std::endl; bool all_args_fixed_or_const = true; for (const auto * child : frame.node->children) - if (!child->column || !fixed_columns.contains(child)) + { + if (!child->column && !fixed_columns.contains(child)) + { + //std::cerr << "*** enreachFixedColumns fail " << child->result_name << ' ' << static_cast(child) << std::endl; all_args_fixed_or_const = false; + } + } if (all_args_fixed_or_const) + { + //std::cerr << "*** enreachFixedColumns add " << frame.node->result_name << ' ' << static_cast(frame.node) << std::endl; fixed_columns.insert(frame.node); + } } } } + + stack.pop(); } } } @@ -256,10 +267,13 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG MatchedTrees::Matches matches; - for (const auto * out : outer_dag.getOutputs()) + for (const auto & node : outer_dag.getNodes()) { + if (matches.contains(&node)) + continue; + std::stack stack; - stack.push(Frame{out, {}}); + stack.push(Frame{&node, {}}); while (!stack.empty()) { auto & frame = stack.top(); @@ -280,21 +294,24 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (frame.mapped_children.size() < frame.node->children.size()) continue; + auto & match = matches[frame.node]; + if (frame.node->type == ActionsDAG::ActionType::INPUT) { const ActionsDAG::Node * mapped = nullptr; if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end()) mapped = it->second; - matches.emplace(frame.node, MatchedTrees::Match{.node = mapped}); + match.node = mapped; } else if (frame.node->type == ActionsDAG::ActionType::ALIAS) { - matches.emplace(frame.node, matches[frame.node->children.at(0)]); + match = matches[frame.node->children.at(0)]; } else if (frame.node->type == ActionsDAG::ActionType::FUNCTION) { - auto & match = matches[frame.node]; + + //std::cerr << "... Processing " << frame.node->function_base->getName() << std::endl; bool found_all_children = true; for (const auto * child : frame.mapped_children) @@ -332,15 +349,30 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG intersection = &container; } + //std::cerr << ".. Candidate parents " << intersection->size() << std::endl; + if (!intersection->empty()) { auto func_name = frame.node->function_base->getName(); for (const auto * parent : *intersection) { + //std::cerr << ".. candidate " << parent->result_name << std::endl; if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) { - match.node = parent; - break; + const auto & childern = parent->children; + size_t num_children = childern.size(); + if (frame.mapped_children.size() == num_children) + { + bool all_children_matched = true; + for (size_t i = 0; all_children_matched && i < num_children; ++i) + all_children_matched = frame.mapped_children[i] == childern[i]; + + if (all_children_matched) + { + match.node = parent; + break; + } + } } } } @@ -366,8 +398,6 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); if (info.is_always_monotonic) { - match.node = child_match.node; - MatchedTrees::Monotonicity monotonicity; monotonicity.direction *= info.is_positive ? 1 : -1; monotonicity.strict = info.is_strict; @@ -379,6 +409,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG monotonicity.strict = false; } + match.node = child_match.node; match.monotonicity = monotonicity; } } @@ -413,12 +444,16 @@ InputOrderInfoPtr buildInputOrderInfo( for (const auto & [node, match] : matches) { + //std::cerr << "------- matching " << static_cast(node) << " " << node->result_name + // << " to " << static_cast(match.node) << " " << (match.node ? match.node->result_name : "") << std::endl; if (!match.monotonicity || match.monotonicity->strict) { if (match.node && fixed_columns.contains(node)) fixed_key_columns.insert(match.node); } } + + enreachFixedColumns(sorting_key_dag, fixed_key_columns); } /// This is a result direction we will read from MergeTree @@ -431,7 +466,7 @@ InputOrderInfoPtr buildInputOrderInfo( size_t next_descr_column = 0; size_t next_sort_key = 0; - for (; next_descr_column < description.size() && next_sort_key < sorting_key_columns.size(); ++next_sort_key) + while (next_descr_column < description.size() && next_sort_key < sorting_key_columns.size()) { const auto & sorting_key_column = sorting_key_columns[next_sort_key]; const auto & descr = description[next_descr_column]; @@ -459,6 +494,9 @@ InputOrderInfoPtr buildInputOrderInfo( break; current_direction = descr.direction; + + ++next_descr_column; + ++next_sort_key; } else { @@ -469,11 +507,11 @@ InputOrderInfoPtr buildInputOrderInfo( const auto & match = matches[sort_node]; - // std::cerr << "====== Finding match for " << sort_node->result_name << ' ' << static_cast(sort_node) << std::endl; + //std::cerr << "====== Finding match for " << sort_column_node->result_name << ' ' << static_cast(sort_column_node) << std::endl; if (match.node && match.node == sort_column_node) { - // std::cerr << "====== Found direct match" << std::endl; + //std::cerr << "====== Found direct match" << std::endl; /// We try to find the match first even if column is fixed. In this case, potentially more keys will match. /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' @@ -486,18 +524,25 @@ InputOrderInfoPtr buildInputOrderInfo( current_direction *= match.monotonicity->direction; strict_monotonic = match.monotonicity->strict; } + + ++next_descr_column; + ++next_sort_key; } else if (fixed_key_columns.contains(sort_column_node)) { - // std::cerr << "+++++++++ Found fixed key by match" << std::endl; + //std::cerr << "+++++++++ Found fixed key by match" << std::endl; + ++next_sort_key; } else { - // std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; + //std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); if (!is_fixed_column) break; + + order_key_prefix_descr.push_back(descr); + ++next_descr_column; } } @@ -510,13 +555,10 @@ InputOrderInfoPtr buildInputOrderInfo( read_direction = current_direction; if (current_direction) - { - order_key_prefix_descr.push_back(description[next_descr_column]); - ++next_descr_column; + order_key_prefix_descr.push_back(descr); - if (!strict_monotonic) - break; - } + if (current_direction && !strict_monotonic) + break; } if (read_direction == 0 || order_key_prefix_descr.empty()) @@ -549,6 +591,9 @@ void optimizeReadInOrder(QueryPlan::Node & node) FixedColumns fixed_columns; buildSortingDAG(node.children.front(), dag, fixed_columns); + if (dag && !fixed_columns.empty()) + enreachFixedColumns(*dag, fixed_columns); + const auto & description = sorting->getSortDescription(); auto limit = sorting->getLimit(); const auto & sorting_key_columns = sorting_key.column_names; diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference new file mode 100644 index 00000000000..99a844b287e --- /dev/null +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference @@ -0,0 +1,217 @@ +-- { echoOn } + +-- Exact match, single key +select * from tab order by (a + b) * c; +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC +select * from tab order by (a + b) * c desc; +4 4 4 4 +4 4 4 4 +3 3 3 3 +3 3 3 3 +2 2 2 2 +2 2 2 2 +1 1 1 1 +1 1 1 1 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC + Result sort description: multiply(plus(a, b), c) DESC +-- Exact match, full key +select * from tab order by (a + b) * c, sin(a / b); +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC +select * from tab order by (a + b) * c desc, sin(a / b) desc; +4 4 4 4 +4 4 4 4 +3 3 3 3 +3 3 3 3 +2 2 2 2 +2 2 2 2 +1 1 1 1 +1 1 1 1 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b) desc) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC, sin(divide(a, b)) DESC + Result sort description: multiply(plus(a, b), c) DESC, sin(divide(a, b)) DESC +-- Exact match, mixed direction +select * from tab order by (a + b) * c desc, sin(a / b); +4 4 4 4 +4 4 4 4 +3 3 3 3 +3 3 3 3 +2 2 2 2 +2 2 2 2 +1 1 1 1 +1 1 1 1 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC + Result sort description: multiply(plus(a, b), c) DESC, sin(divide(a, b)) ASC +select * from tab order by (a + b) * c, sin(a / b) desc; +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b) desc) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) DESC +-- Wrong order, full sort +select * from tab order by sin(a / b), (a + b) * c; +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from tab order by sin(a / b), (a + b) * c) where explain ilike '%sort description%'; + Sort description: sin(divide(a, b)) ASC, multiply(plus(a, b), c) ASC +-- Fixed point +select * from tab where (a + b) * c = 8 order by sin(a / b); +2 2 2 2 +2 2 2 2 +select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; + Prefix sort description: sin(divide(a, b)) ASC + Result sort description: sin(divide(a, b)) ASC +select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0; +1 1 1 1 +1 1 1 1 +select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; + Prefix sort description: multiply(plus(d, 1), 4) ASC, multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(d, 1), 4) ASC, multiply(plus(a, b), c) ASC +select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0; +2 2 2 2 +2 2 2 2 +select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; + Prefix sort description: multiply(plus(d, 1), 4) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(d, 1), 4) ASC, sin(divide(a, b)) ASC +-- Wrong order with fixed point +select * from tab where (a + b) * c = 8 order by sin(b / a); +2 2 2 2 +2 2 2 2 +select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(b / a)) where explain ilike '%sort description%'; + Sort description: sin(divide(b, a)) ASC +-- Monotonicity +select * from tab order by intDiv((a + b) * c, 2); +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2)) where explain like '%sort description%'; + Prefix sort description: intDiv(multiply(plus(a, b), c), 2) ASC + Result sort description: intDiv(multiply(plus(a, b), c), 2) ASC +select * from tab order by intDiv((a + b) * c, 2), sin(a / b); +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2), sin(a / b)) where explain like '%sort description%'; + Prefix sort description: intDiv(multiply(plus(a, b), c), 2) ASC + Result sort description: intDiv(multiply(plus(a, b), c), 2) ASC, sin(divide(a, b)) ASC +-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC, intDiv(sin(divide(a, b)), 2) ASC + Result sort description: multiply(plus(a, b), c) ASC, intDiv(sin(divide(a, b)), 2) ASC +-- select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC + Result sort description: multiply(plus(a, b), c) DESC, intDiv(sin(divide(a, b)), 2) ASC +-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC, intDiv(sin(divide(a, b)), 2) DESC +-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC, intDiv(sin(divide(a, b)), 2) DESC + Result sort description: multiply(plus(a, b), c) DESC, intDiv(sin(divide(a, b)), 2) DESC +-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC, intDiv(sin(divide(a, b)), -2) ASC + Result sort description: multiply(plus(a, b), c) DESC, intDiv(sin(divide(a, b)), -2) ASC +-- select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) DESC + Result sort description: multiply(plus(a, b), c) DESC, intDiv(intDiv(sin(divide(a, b)), -2), -3) ASC +-- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; + Prefix sort description: multiply(plus(a, b), c) ASC, intDiv(intDiv(sin(divide(a, b)), -2), -3) ASC + Result sort description: multiply(plus(a, b), c) ASC, intDiv(intDiv(sin(divide(a, b)), -2), -3) ASC +-- { echoOn } + +select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3); +2020-02-02 00:00:00 0 0 +2020-02-02 00:00:00 0 0 +2020-02-03 00:00:00 1 1 +2020-02-03 00:00:00 1 1 +2020-02-04 00:00:00 2 2 +2020-02-04 00:00:00 2 2 +2020-02-05 00:00:00 3 3 +2020-02-05 00:00:00 3 3 +select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; + Prefix sort description: toTimeZone(toTimezone(x, \'UTC\'), \'CET\') ASC, intDiv(intDiv(y, -2), -3) ASC + Result sort description: toTimeZone(toTimezone(x, \'UTC\'), \'CET\') ASC, intDiv(intDiv(y, -2), -3) ASC +select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); +2020-02-02 00:00:00 0 0 +2020-02-02 00:00:00 0 0 +2020-02-03 00:00:00 1 1 +2020-02-03 00:00:00 1 1 +2020-02-04 00:00:00 2 2 +2020-02-04 00:00:00 2 2 +2020-02-05 00:00:00 3 3 +2020-02-05 00:00:00 3 3 +select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; + Prefix sort description: toStartOfDay(x) ASC + Result sort description: toStartOfDay(x) ASC, intDiv(intDiv(y, -2), -3) ASC +select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); +2020-02-03 00:00:00 1 1 +2020-02-03 00:00:00 1 1 +select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; + Prefix sort description: intDiv(intDiv(y, -2), -3) ASC + Result sort description: intDiv(intDiv(y, -2), -3) ASC diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql new file mode 100644 index 00000000000..0c2b864bf16 --- /dev/null +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql @@ -0,0 +1,91 @@ +SET optimize_read_in_order = 1, query_plan_read_in_order=1; + +create table tab (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by ((a + b) * c, sin(a / b)); +insert into tab select number, number, number, number from numbers(5); +insert into tab select number, number, number, number from numbers(5); + +-- { echoOn } + +-- Exact match, single key +select * from tab order by (a + b) * c; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c) where explain like '%sort description%'; + +select * from tab order by (a + b) * c desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc) where explain like '%sort description%'; + +-- Exact match, full key +select * from tab order by (a + b) * c, sin(a / b); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b)) where explain like '%sort description%'; + +select * from tab order by (a + b) * c desc, sin(a / b) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b) desc) where explain like '%sort description%'; + +-- Exact match, mixed direction +select * from tab order by (a + b) * c desc, sin(a / b); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, sin(a / b)) where explain like '%sort description%'; + +select * from tab order by (a + b) * c, sin(a / b) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, sin(a / b) desc) where explain like '%sort description%'; + +-- Wrong order, full sort +select * from tab order by sin(a / b), (a + b) * c; +select * from (explain plan actions = 1 select * from tab order by sin(a / b), (a + b) * c) where explain ilike '%sort description%'; + +-- Fixed point +select * from tab where (a + b) * c = 8 order by sin(a / b); +select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; + +select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0; +select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; + +select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0; +select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; + +-- Wrong order with fixed point +select * from tab where (a + b) * c = 8 order by sin(b / a); +select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(b / a)) where explain ilike '%sort description%'; + +-- Monotonicity +select * from tab order by intDiv((a + b) * c, 2); +select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2)) where explain like '%sort description%'; + +select * from tab order by intDiv((a + b) * c, 2), sin(a / b); +select * from (explain plan actions = 1 select * from tab order by intDiv((a + b) * c, 2), sin(a / b)) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2)) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc , intDiv(sin(a / b), 2)) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc; +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), 2) desc) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(sin(a / b), -2)) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c desc, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; + +-- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3); +select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; + +-- { echoOff } + +create table tab2 (x DateTime, y UInt32, z UInt32) engine = MergeTree order by (x, y); +insert into tab2 select toDate('2020-02-02') + number, number, number from numbers(4); +insert into tab2 select toDate('2020-02-02') + number, number, number from numbers(4); + +-- { echoOn } + +select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3); +select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; + +select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); +select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; + +select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); +select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; From e1e6dbe4aa92f39b6615990486cd136e4851a712 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 1 Nov 2022 03:00:11 +0100 Subject: [PATCH 113/526] Try to fix test --- tests/queries/0_stateless/02473_infile_progress.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_infile_progress.py b/tests/queries/0_stateless/02473_infile_progress.py index 2d6493fe4a5..c1b31eeecbb 100755 --- a/tests/queries/0_stateless/02473_infile_progress.py +++ b/tests/queries/0_stateless/02473_infile_progress.py @@ -27,7 +27,7 @@ with client(name="client>", log=log) as client1: ) client1.expect(prompt) client1.send(f"INSERT INTO test.infile_progress FROM INFILE '{filename}'") - client1.expect("Progress: 5.00 rows.*\)") + client1.expect("Progress: 0.00 rows, 10.00 B.*\)") client1.expect(prompt) # send Ctrl-C From b689f7ab70fd72473a0c73d3b1de739ff18caa95 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Tue, 1 Nov 2022 10:13:10 +0100 Subject: [PATCH 114/526] Updated registerFunction method to take TableFunctionFactoryData as input , moved table_functions_allowed_in_readonly_mode inside TableFunctionFactory- 42414 Enable table functions in readonly mode --- src/TableFunctions/TableFunctionFactory.cpp | 13 +++---------- src/TableFunctions/TableFunctionFactory.h | 11 ++++++++--- 2 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 79802d2ec77..3d2a72ddc9e 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -16,22 +16,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static const NameSet table_functions_allowed_in_readonly_mode -{ - "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" -}; - void TableFunctionFactory::registerFunction( - const std::string & name, TableFunctionCreator creator, Documentation doc, CaseSensitiveness case_sensitiveness) + const std::string & name, Value value, CaseSensitiveness case_sensitiveness) { - bool allowed_in_readonly_mode = table_functions_allowed_in_readonly_mode.contains(name); - - if (!table_functions.emplace(name, TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) + if (!table_functions.emplace(name, value).second) throw Exception("TableFunctionFactory: the table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); if (case_sensitiveness == CaseInsensitive - && !case_insensitive_table_functions.emplace(Poco::toLower(name), TableFunctionFactoryData{creator, {doc,allowed_in_readonly_mode}}).second) + && !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index 24b46d40de0..e2b8e29bd59 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -47,15 +47,16 @@ public: /// No locking, you must register all functions before usage of get. void registerFunction( const std::string & name, - TableFunctionCreator creator, - Documentation doc = {}, + Value value, CaseSensitiveness case_sensitiveness = CaseSensitive); template void registerFunction(Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; - registerFunction(Function::name, std::move(creator), std::move(doc), case_sensitiveness); + registerFunction(Function::name, + TableFunctionFactoryData{std::move(creator), {std::move(doc), table_functions_allowed_in_readonly_mode.contains(Function::name)}} , + case_sensitiveness); } /// Throws an exception if not found. @@ -81,6 +82,10 @@ private: TableFunctions table_functions; TableFunctions case_insensitive_table_functions; + + inline static const NameSet table_functions_allowed_in_readonly_mode = { + "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" + }; }; } From 97d361bce05f4dadcc5c7abdd56a677793461f49 Mon Sep 17 00:00:00 2001 From: SaltTan <20357526+SaltTan@users.noreply.github.com> Date: Tue, 1 Nov 2022 10:02:04 +0000 Subject: [PATCH 115/526] Update check-table.md --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index f9b428b74a1..9bcda724860 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -8,7 +8,7 @@ title: "CHECK TABLE Statement" Checks if the data in the table is corrupted. ``` sql -CHECK TABLE [db.]name +CHECK TABLE [db.]name PARTITION partition_expr ``` The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. From 9dbb7e335cc4323689e46dc50d049fe528c02728 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 1 Nov 2022 14:02:25 +0100 Subject: [PATCH 116/526] Fix tests --- tests/queries/0_stateless/01825_type_json_schema_inference.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01825_type_json_schema_inference.sh b/tests/queries/0_stateless/01825_type_json_schema_inference.sh index f6917ab18ad..a524eaec20a 100755 --- a/tests/queries/0_stateless/01825_type_json_schema_inference.sh +++ b/tests/queries/0_stateless/01825_type_json_schema_inference.sh @@ -46,7 +46,7 @@ ${CLICKHOUSE_CLIENT} -q "CREATE TABLE t_json_inference (obj JSON, map Map(String echo '{"map": {"k1": 1, "k2": 2}, "obj": {"k1": 1, "k2": 2}}' > $filename -${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 1 +${CLICKHOUSE_CLIENT} -q "INSERT INTO t_json_inference SELECT * FROM file('${CLICKHOUSE_TEST_UNIQUE_NAME}/data.json', 'JSONEachRow')" --allow_experimental_object_type 1 --use_structure_from_insertion_table_in_table_functions 0 ${CLICKHOUSE_CLIENT} -q "SELECT * FROM t_json_inference FORMAT JSONEachRow" --output_format_json_named_tuples_as_objects 1 ${CLICKHOUSE_CLIENT} -q "SELECT toTypeName(obj) FROM t_json_inference LIMIT 1" From 3afc688751a56be3e56cb2d44dd98d91d829ee48 Mon Sep 17 00:00:00 2001 From: SaltTan <20357526+SaltTan@users.noreply.github.com> Date: Tue, 1 Nov 2022 13:04:24 +0000 Subject: [PATCH 117/526] Update docs/en/sql-reference/statements/check-table.md Co-authored-by: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> --- docs/en/sql-reference/statements/check-table.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/check-table.md b/docs/en/sql-reference/statements/check-table.md index 9bcda724860..8c4b8ab90a2 100644 --- a/docs/en/sql-reference/statements/check-table.md +++ b/docs/en/sql-reference/statements/check-table.md @@ -8,7 +8,7 @@ title: "CHECK TABLE Statement" Checks if the data in the table is corrupted. ``` sql -CHECK TABLE [db.]name PARTITION partition_expr +CHECK TABLE [db.]name [PARTITION partition_expr] ``` The `CHECK TABLE` query compares actual file sizes with the expected values which are stored on the server. If the file sizes do not match the stored values, it means the data is corrupted. This can be caused, for example, by a system crash during query execution. From c1e2b2af74910d0fa28c89612c20fd5658bb84ec Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Mon, 31 Oct 2022 08:57:31 +0800 Subject: [PATCH 118/526] Optimize TTL merge, completely expired parts can be removed in time --- src/Interpreters/PartLog.cpp | 2 ++ src/Interpreters/PartLog.h | 2 ++ src/Storages/MergeTree/MergeFromLogEntryTask.cpp | 8 +++++--- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 7 +++++-- src/Storages/MergeTree/MergeType.cpp | 2 +- src/Storages/MergeTree/MergeType.h | 2 ++ src/Storages/StorageMergeTree.cpp | 7 ++++++- 7 files changed, 23 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index b35ee50b98e..6797d6b3a41 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -31,6 +31,8 @@ PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType mer return TTL_DELETE_MERGE; case MergeType::TTLRecompress: return TTL_RECOMPRESS_MERGE; + case MergeType::TTLDrop: + return TTL_DROP_MERGE; } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 2ce0dfd76de..6180963908d 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -41,6 +41,8 @@ struct PartLogElement TTL_DELETE_MERGE = 3, /// Merge with recompression TTL_RECOMPRESS_MERGE = 4, + /// Merge assigned to drop parts (with TTLMergeSelector) + TTL_DROP_MERGE = 5, }; String query_id; diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 9a9b8a4a6bb..0711ed157a5 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -152,7 +152,9 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() } /// Start to make the main work - size_t estimated_space_for_merge = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts); + size_t need_total_size = 0; + if (entry.merge_type != MergeType::TTLDrop) + need_total_size = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts); /// Can throw an exception while reserving space. IMergeTreeDataPart::TTLInfos ttl_infos; @@ -180,7 +182,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() std::optional tagger; ReservationSharedPtr reserved_space = storage.balancedReservation( metadata_snapshot, - estimated_space_for_merge, + need_total_size, max_volume_index, future_merged_part->name, future_merged_part->part_info, @@ -190,7 +192,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() if (!reserved_space) reserved_space = storage.reserveSpacePreferringTTLRules( - metadata_snapshot, estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); + metadata_snapshot, need_total_size, ttl_infos, time(nullptr), max_volume_index); future_merged_part->uuid = entry.new_part_uuid; future_merged_part->updatePath(storage, reserved_space.get()); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0b5c5285d15..879cd2fa3da 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -309,10 +309,13 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( data_settings->merge_with_ttl_timeout, data_settings->ttl_only_drop_parts); - parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); + parts_to_merge = delete_ttl_selector.select( + parts_ranges, + data_settings->ttl_only_drop_parts ? data_settings->max_bytes_to_merge_at_max_space_in_pool : max_total_size_to_merge); + if (!parts_to_merge.empty()) { - future_part->merge_type = MergeType::TTLDelete; + future_part->merge_type = data_settings->ttl_only_drop_parts ? MergeType::TTLDrop : MergeType::TTLDelete; } else if (metadata_snapshot->hasAnyRecompressionTTL()) { diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 045114578d0..96862b6cca1 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -20,7 +20,7 @@ MergeType checkAndGetMergeType(UInt32 merge_type) bool isTTLMergeType(MergeType merge_type) { - return merge_type == MergeType::TTLDelete || merge_type == MergeType::TTLRecompress; + return merge_type == MergeType::TTLDelete || merge_type == MergeType::TTLRecompress || merge_type == MergeType::TTLDrop; } } diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index ce9a40c5931..fa5df87b25a 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -19,6 +19,8 @@ enum class MergeType TTLDelete = 2, /// Merge with recompression TTLRecompress = 3, + /// Merge assigned to drop parts (with TTLMergeSelector) + TTLDrop = 4, }; /// Check parsed merge_type from raw int and get enum value. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a450a9ef3a9..805242fbec9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -929,7 +929,12 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (isTTLMergeType(future_part->merge_type)) getContext()->getMergeList().bookMergeWithTTL(); - merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts), *this, metadata_snapshot, false); + /// If merge_type is TTLDrop, no need to reserve disk space + size_t need_total_size = 0; + if (future_part->merge_type != MergeType::TTLDrop) + need_total_size = MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts); + + merging_tagger = std::make_unique(future_part, need_total_size, *this, metadata_snapshot, false); return std::make_shared(future_part, std::move(merging_tagger), std::make_shared()); } From 34a33d182138a145ed2560ffe9fbc467f69a858e Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Tue, 1 Nov 2022 15:09:05 +0000 Subject: [PATCH 119/526] add documentation for functions --- src/Functions/UTCTimestamp.cpp | 18 +++++++++++++----- src/Functions/dateDiff.cpp | 11 ++++++++++- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 218c7fa5c1c..275e5c278fa 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -5,8 +5,6 @@ #include #include -#include - namespace DB { @@ -86,7 +84,7 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { - if (arguments.size() > 0) + if (!arguments.empty()) { throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } @@ -96,7 +94,7 @@ public: FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override { - if (arguments.size() > 0) + if (!arguments.empty()) { throw Exception("Arguments size of function " + getName() + " should be 0", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } @@ -110,7 +108,17 @@ public: /// UTC_timestamp for MySQL interface support REGISTER_FUNCTION(UTCTimestamp) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({ + R"( +Returns the current date and time at the moment of query analysis. The function is a constant expression. +Same as `now('UTC')`. Was added only for MySQL support. `now` is prefered. + +Example: +[example:typical] +)", + Documentation::Examples{ + {"typical", "SELECT UTCTimestamp();"}}, + Documentation::Categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); factory.registerAlias("UTC_timestamp", UTCTimestampOverloadResolver::name, FunctionFactory::CaseInsensitive); } diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index c653db3ddf2..039d0736a7c 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -364,7 +364,16 @@ REGISTER_FUNCTION(DateDiff) REGISTER_FUNCTION(TimeDiff) { - factory.registerFunction({}, FunctionFactory::CaseInsensitive); + factory.registerFunction({R"( +Returns the difference between two dates or dates with time values. The difference is calculated in seconds units (see toRelativeSecondNum). +It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is prefered. + +Example: +[example:typical] +)", + Documentation::Examples{ + {"typical", "SELECT timeDiff(UTCTimestamp(), now());"}}, + Documentation::Categories{"Dates and Times"}}, FunctionFactory::CaseInsensitive); } } From 52204233e474a58f4e4e18e4c1e86c50f882bfb7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Nov 2022 15:11:26 +0000 Subject: [PATCH 120/526] Fix some more tests. --- src/Functions/negate.cpp | 2 +- .../QueryPlan/Optimizations/Optimizations.h | 6 +++--- .../Optimizations/distinctReadInOrder.cpp | 4 +++- .../optimizePrimaryKeyCondition.cpp | 11 +++++++++-- .../reuseStorageOrderingForWindowFunctions.cpp | 17 ++++++++++++----- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 ++ src/Processors/QueryPlan/ReadFromMergeTree.h | 2 +- ...155_read_in_order_max_rows_to_read.reference | 7 +++++++ .../02155_read_in_order_max_rows_to_read.sql | 5 ++--- ...inct_in_order_optimization_explain.reference | 6 ------ 10 files changed, 40 insertions(+), 22 deletions(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 731c0766e9d..5047ae43652 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -42,7 +42,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity static bool has() { return true; } static IFunction::Monotonicity get(const Field &, const Field &) { - return { .is_monotonic = true, .is_positive = false, .is_strict = true }; + return { .is_monotonic = true, .is_positive = false, .is_always_monotonic = true, .is_strict = true }; } }; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 05266ccdbd2..594ab95031d 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -56,11 +56,11 @@ size_t tryExecuteFunctionsAfterSorting(QueryPlan::Node * parent_node, QueryPlan: size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); /// Reading in order from MergeTree table if DISTINCT columns match or form a prefix of MergeTree sorting key -size_t tryDistinctReadInOrder(QueryPlan::Node * node, QueryPlan::Nodes & nodes); +size_t tryDistinctReadInOrder(QueryPlan::Node * node); inline const auto & getOptimizations() { - static const std::array optimizations = {{ + static const std::array optimizations = {{ {tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::optimize_plan}, {trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::optimize_plan}, @@ -68,7 +68,7 @@ inline const auto & getOptimizations() {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan}, - {tryDistinctReadInOrder, "distinctReadInOrder", &QueryPlanOptimizationSettings::distinct_in_order}, + //{tryDistinctReadInOrder, "distinctReadInOrder", &QueryPlanOptimizationSettings::distinct_in_order}, }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 38cb6f3d3c5..7a5950b29a7 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -7,8 +7,9 @@ namespace DB::QueryPlanOptimizations { -size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &) +size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) { + std::cerr << "&&&&&&&&&&&& tryDistinctReadInOrder\n"; /// check if it is preliminary distinct node DistinctStep * pre_distinct = nullptr; if (auto * distinct = typeid_cast(parent_node->step.get()); distinct) @@ -84,6 +85,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node, QueryPlan::Nodes &) if (output_data_stream.sort_scope != DataStream::SortScope::Chunk && number_of_sorted_distinct_columns <= output_sort_desc.size()) return 0; + std::cerr << "++++++++ tryDistinctReadInOrder\n"; /// update input order info in read_from_merge_tree step const int direction = 0; /// for DISTINCT direction doesn't matter, ReadFromMergeTree will choose proper one read_from_merge_tree->requestReadingInOrder(number_of_sorted_distinct_columns, direction, pre_distinct->getLimitHint()); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index b33ae646535..4c4d515911a 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -24,6 +24,15 @@ void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimizat { auto & frame = stack.back(); + if (frame.next_child == 0) + { + if (optimization_settings.read_in_order) + optimizeReadInOrder(*frame.node); + + if (optimization_settings.distinct_in_order) + tryDistinctReadInOrder(frame.node); + } + /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { @@ -33,8 +42,6 @@ void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimizat continue; } - if (optimization_settings.read_in_order) - optimizeReadInOrder(*frame.node); auto add_filter = [&](auto & storage) { diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 01132e36502..f26f7a10b71 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -36,7 +36,7 @@ ReadFromMergeTree * findReadingStep(QueryPlan::Node * node) if (node->children.size() != 1) return nullptr; - if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) + if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) return findReadingStep(node->children.front()); return nullptr; @@ -495,6 +495,9 @@ InputOrderInfoPtr buildInputOrderInfo( current_direction = descr.direction; + + std::cerr << "====== (no dag) Found direct match" << std::endl; + ++next_descr_column; ++next_sort_key; } @@ -507,11 +510,11 @@ InputOrderInfoPtr buildInputOrderInfo( const auto & match = matches[sort_node]; - //std::cerr << "====== Finding match for " << sort_column_node->result_name << ' ' << static_cast(sort_column_node) << std::endl; + std::cerr << "====== Finding match for " << sort_column_node->result_name << ' ' << static_cast(sort_column_node) << std::endl; if (match.node && match.node == sort_column_node) { - //std::cerr << "====== Found direct match" << std::endl; + std::cerr << "====== Found direct match" << std::endl; /// We try to find the match first even if column is fixed. In this case, potentially more keys will match. /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' @@ -530,13 +533,13 @@ InputOrderInfoPtr buildInputOrderInfo( } else if (fixed_key_columns.contains(sort_column_node)) { - //std::cerr << "+++++++++ Found fixed key by match" << std::endl; + std::cerr << "+++++++++ Found fixed key by match" << std::endl; ++next_sort_key; } else { - //std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; + std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); if (!is_fixed_column) break; @@ -583,6 +586,10 @@ void optimizeReadInOrder(QueryPlan::Node & node) if (!reading) return; + /// Already read-in-order, skip. + if (reading->getQueryInfo().input_order_info) + return; + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); if (sorting_key.column_names.empty()) return; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 2e40ec4246d..2efde7a0148 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1055,6 +1055,8 @@ void ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction, else query_info.input_order_info = order_info; + reader_settings.read_in_order = true; + /// update sort info for output stream SortDescription sort_description; const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 15258eb6c40..04dc61664db 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -163,7 +163,7 @@ private: return 1; } - const MergeTreeReaderSettings reader_settings; + MergeTreeReaderSettings reader_settings; MergeTreeData::DataPartsVector prepared_parts; Names real_column_names; diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference index b73ab43cabb..d21747026dc 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference @@ -4,3 +4,10 @@ 2 3 4 +11 +12 +13 +14 +15 +10 +20 diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql index 9846c1208a1..a893221df0f 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql @@ -13,11 +13,10 @@ SELECT a FROM t_max_rows_to_read WHERE a = 10 SETTINGS max_rows_to_read = 4; SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; --- This should work, but actually it doesn't. Need to investigate. --- SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 20; +SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; +SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 SETTINGS max_rows_to_read = 12; SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 20 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } -SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 FORMAT Null SETTINGS max_rows_to_read = 4; -- { serverError 158 } DROP TABLE t_max_rows_to_read; diff --git a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference index 2511c806e1b..f85b9cd9e86 100644 --- a/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference +++ b/tests/queries/0_stateless/02317_distinct_in_order_optimization_explain.reference @@ -59,8 +59,6 @@ Sorting (Stream): a ASC, b ASC -- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query -- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns Sorting (Stream): a ASC -Sorting (Stream): a ASC -Sorting (Stream): a ASC -- enabled, check that ReadFromMergeTree sorting description is overwritten by DISTINCT optimization i.e. it contains columns from DISTINCT clause Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC @@ -71,12 +69,8 @@ Sorting (Stream): a DESC, b DESC Sorting (Stream): a DESC, b DESC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (1), - it contains columns from ORDER BY clause Sorting (Stream): a ASC, b ASC -Sorting (Stream): a ASC, b ASC -Sorting (Stream): a ASC, b ASC -- enabled, check that ReadFromMergeTree sorting description is NOT overwritten by DISTINCT optimization (2), - direction used from ORDER BY clause Sorting (Stream): a DESC, b DESC -Sorting (Stream): a DESC, b DESC -Sorting (Stream): a DESC, b DESC -- enabled, check that disabling other 'read in order' optimizations do not disable distinct in order optimization Sorting (Stream): a ASC, b ASC Sorting (Stream): a ASC, b ASC From e58008b711acc08ba4f17b611d877ec14be95e00 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 1 Nov 2022 18:10:31 +0100 Subject: [PATCH 121/526] Fix rare possible hung on query cancellation Some queries can hung after cancelling (because enough rows had been read for Distributed queries) or after KILL. Example of such query is a Distributed query that uses DelayedSource that uses ExpandPipeline to add new processors, and in this case it is possible that all already existing processes was cancelled but new had been added and now PipelineExecutor will wait for them undefinitelly since nobody will read from them and nobody will cancel them either. Signed-off-by: Azat Khuzhin --- src/Processors/Executors/ExecutingGraph.cpp | 8 ++++++++ src/Processors/Executors/ExecutingGraph.h | 1 + 2 files changed, 9 insertions(+) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 4ab2c5b3802..cd94ca7ceae 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -109,6 +109,13 @@ bool ExecutingGraph::expandPipeline(std::stack & stack, uint64_t pid) { std::lock_guard guard(processors_mutex); + /// Do not add new processors to existing list, since the query was already cancelled. + if (cancelled) + { + for (auto & processor : new_processors) + processor->cancel(); + return false; + } processors->insert(processors->end(), new_processors.begin(), new_processors.end()); } @@ -388,6 +395,7 @@ void ExecutingGraph::cancel() std::lock_guard guard(processors_mutex); for (auto & processor : *processors) processor->cancel(); + cancelled = true; } } diff --git a/src/Processors/Executors/ExecutingGraph.h b/src/Processors/Executors/ExecutingGraph.h index b374f968122..834ef5d4d9d 100644 --- a/src/Processors/Executors/ExecutingGraph.h +++ b/src/Processors/Executors/ExecutingGraph.h @@ -157,6 +157,7 @@ private: UpgradableMutex nodes_mutex; const bool profile_processors; + bool cancelled = false; }; } From 84c88ddc914ffd77a6d9c296bd35ce3cb753b100 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Tue, 1 Nov 2022 17:29:15 +0000 Subject: [PATCH 122/526] fix typos --- src/Functions/UTCTimestamp.cpp | 2 +- src/Functions/dateDiff.cpp | 2 +- .../02415_all_new_functions_must_be_documented.reference | 2 -- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Functions/UTCTimestamp.cpp b/src/Functions/UTCTimestamp.cpp index 275e5c278fa..27b5b33bc6e 100644 --- a/src/Functions/UTCTimestamp.cpp +++ b/src/Functions/UTCTimestamp.cpp @@ -111,7 +111,7 @@ REGISTER_FUNCTION(UTCTimestamp) factory.registerFunction({ R"( Returns the current date and time at the moment of query analysis. The function is a constant expression. -Same as `now('UTC')`. Was added only for MySQL support. `now` is prefered. +Same as `now('UTC')`. Was added only for MySQL support. `now` is preferred. Example: [example:typical] diff --git a/src/Functions/dateDiff.cpp b/src/Functions/dateDiff.cpp index 039d0736a7c..ec9c9df8e49 100644 --- a/src/Functions/dateDiff.cpp +++ b/src/Functions/dateDiff.cpp @@ -366,7 +366,7 @@ REGISTER_FUNCTION(TimeDiff) { factory.registerFunction({R"( Returns the difference between two dates or dates with time values. The difference is calculated in seconds units (see toRelativeSecondNum). -It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is prefered. +It is same as `dateDiff` and was added only for MySQL support. `dateDiff` is preferred. Example: [example:typical] diff --git a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference index ae701d9ae83..040a8c8d317 100644 --- a/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference +++ b/tests/queries/0_stateless/02415_all_new_functions_must_be_documented.reference @@ -51,11 +51,9 @@ LpNormalize MACNumToString MACStringToNum MACStringToOUI -TimeDiff URLHash URLHierarchy URLPathHierarchy -UTCTimestamp UUIDNumToString UUIDStringToNum _CAST From e76c3c381f449fcec2876796c4eb8e1e33c14be1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 1 Nov 2022 18:47:51 +0000 Subject: [PATCH 123/526] Better handling tables in read-only mode during insert + throw immediately if table has no metadata in zk + stop retries in case of shutdown + check if table is readonly at the begining of every retry --- src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 11 +++++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 11 +++++++++-- 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 9da29d0becf..dd1f750b493 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; + extern const int TABLE_IS_READ_ONLY; } struct ReplicatedMergeTreeSink::DelayedChunk @@ -367,6 +368,16 @@ void ReplicatedMergeTreeSink::commitPart( retries_ctl.retryLoop([&]() { zookeeper->setKeeper(storage.getZooKeeper()); + if (storage.is_readonly) + { + /// stop retries if in shutdown + if (storage.shutdown_called) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + + retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); + return; + } if (retries_ctl.isRetry()) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 860eed43c4f..ed20d60861b 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4462,9 +4462,16 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - const auto storage_settings_ptr = getSettings(); - assertNotReadonly(); + /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it + /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk + /// In case of remote storage like s3, it'll generate unnecessary PUT requests + if (is_readonly && (!has_metadata_in_zookeeper.has_value() || false == has_metadata_in_zookeeper.value())) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, + "Table is in readonly mode since table metadata was not found in zookeeper: replica_path={}", + replica_path); + const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; From 9ffebf4a0c63f7a4a5f38b165f12fd1a8085a703 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Nov 2022 19:51:52 +0000 Subject: [PATCH 124/526] Fixing read-in-order for special storages. --- src/Interpreters/InterpreterSelectQuery.cpp | 52 ++-- src/Planner/Planner.cpp | 34 +-- src/Planner/PlannerJoinTree.cpp | 17 +- .../QueryPlan/Optimizations/Optimizations.h | 10 +- .../QueryPlan/Optimizations/liftUpUnion.cpp | 65 +++++ .../optimizePrimaryKeyCondition.cpp | 4 +- ...reuseStorageOrderingForWindowFunctions.cpp | 230 +++++++++++++++--- src/Processors/QueryPlan/QueryPlan.cpp | 2 +- src/Processors/QueryPlan/SortingStep.cpp | 42 ++-- src/Processors/QueryPlan/SortingStep.h | 28 +-- src/Storages/StorageMerge.cpp | 8 +- src/Storages/StorageMerge.h | 6 + 12 files changed, 376 insertions(+), 122 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d8ac263e3d1..8cfa902ae04 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1447,17 +1447,19 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + auto sorting_step = std::make_unique( plan.getCurrentDataStream(), std::move(order_descr), - settings.max_block_size, - 0 /* LIMIT */, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - this->context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + 0 /* LIMIT */, sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_pos)); plan.addStep(std::move(sorting_step)); @@ -2617,17 +2619,20 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) // happens in case of `over ()`. if (!window.full_sort_description.empty() && (i == 0 || !sortIsPrefix(window, *windows_sorted[i - 1]))) { + SortingStep::Settings sort_settings; + sort_settings.max_block_size = settings.max_block_size; + sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + sort_settings.tmp_data = context->getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window.full_sort_description, - settings.max_block_size, 0 /* LIMIT */, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription("Sorting for window '" + window.window_name + "'"); query_plan.addStep(std::move(sorting_step)); @@ -2675,18 +2680,21 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo const Settings & settings = context->getSettingsRef(); + SortingStep::Settings sort_settings; + sort_settings.max_block_size = settings.max_block_size; + sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + sort_settings.tmp_data = context->getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + /// Merge the sorted blocks. auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), output_order_descr, - settings.max_block_size, limit, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription("Sorting for ORDER BY"); diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 97f82d06463..fc510150fc8 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -571,17 +571,20 @@ void Planner::buildQueryPlanIfNeeded() if (!window_description.full_sort_description.empty() && (i == 0 || !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description))) { + SortingStep::Settings sort_settings; + sort_settings.max_block_size = settings.max_block_size; + sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + sort_settings.tmp_data = query_context->getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), window_description.full_sort_description, - settings.max_block_size, 0 /*limit*/, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - query_context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription("Sorting for window '" + window_description.window_name + "'"); @@ -673,18 +676,21 @@ void Planner::buildQueryPlanIfNeeded() const Settings & settings = query_context->getSettingsRef(); + SortingStep::Settings sort_settings; + sort_settings.max_block_size = settings.max_block_size; + sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + sort_settings.tmp_data = query_context->getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + /// Merge the sorted blocks auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), sort_description, - settings.max_block_size, partial_sorting_limit, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - query_context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription("Sorting for ORDER BY"); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4cb446a65a0..ce8dfb56225 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -528,17 +528,20 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (const auto & key_name : key_names) sort_description.emplace_back(key_name); + SortingStep::Settings sort_settings; + sort_settings.max_block_size = settings.max_block_size; + sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + sort_settings.tmp_data = query_context->getTempDataOnDisk(); + sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + auto sorting_step = std::make_unique( plan.getCurrentDataStream(), std::move(sort_description), - settings.max_block_size, 0 /*limit*/, - SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode), - settings.max_bytes_before_remerge_sort, - settings.remerge_sort_lowered_memory_bytes_ratio, - settings.max_bytes_before_external_sort, - query_context->getTempDataOnDisk(), - settings.min_free_disk_space_for_temporary_data, + sort_settings, settings.optimize_sorting_by_input_stream_properties); sorting_step->setStepDescription(fmt::format("Sort {} before JOIN", join_table_side)); plan.addStep(std::move(sorting_step)); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 594ab95031d..e87d2fdb32b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -12,8 +12,8 @@ namespace QueryPlanOptimizations /// This is the main function which optimizes the whole QueryPlan tree. void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); -void optimizeReadInOrder(QueryPlan::Node & node); -void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root); +void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); +void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. @@ -58,9 +58,12 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, /// Reading in order from MergeTree table if DISTINCT columns match or form a prefix of MergeTree sorting key size_t tryDistinctReadInOrder(QueryPlan::Node * node); +/// Put some steps under union, so that plan optimisation could be applied to union parts separately. +size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); + inline const auto & getOptimizations() { - static const std::array optimizations = {{ + static const std::array optimizations = {{ {tryLiftUpArrayJoin, "liftUpArrayJoin", &QueryPlanOptimizationSettings::optimize_plan}, {tryPushDownLimit, "pushDownLimit", &QueryPlanOptimizationSettings::optimize_plan}, {trySplitFilter, "splitFilter", &QueryPlanOptimizationSettings::optimize_plan}, @@ -68,6 +71,7 @@ inline const auto & getOptimizations() {tryPushDownFilter, "pushDownFilter", &QueryPlanOptimizationSettings::filter_push_down}, {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan}, + {tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::optimize_plan}, //{tryDistinctReadInOrder, "distinctReadInOrder", &QueryPlanOptimizationSettings::distinct_in_order}, }}; diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp new file mode 100644 index 00000000000..35d8b1a35e4 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -0,0 +1,65 @@ +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) +{ + if (parent_node->children.empty()) + return 0; + + QueryPlan::Node * child_node = parent_node->children.front(); + auto & parent = parent_node->step; + auto & child = child_node->step; + + auto * union_step = typeid_cast(child.get()); + if (!union_step) + return 0; + + if (auto * expression = typeid_cast(parent.get())) + { + /// Union does not change header. + /// We can push down expression and update header. + auto union_input_streams = child->getInputStreams(); + for (auto & input_stream : union_input_streams) + input_stream.header = expression->getOutputStream().header; + + /// - Something + /// Expression - Union - Something + /// - Something + + child = std::make_unique(union_input_streams, union_step->getMaxThreads()); + + std::swap(parent, child); + std::swap(parent_node->children, child_node->children); + std::swap(parent_node->children.front(), child_node->children.front()); + + /// - Expression - Something + /// Union - Something + /// - Something + + for (size_t i = 1; i < parent_node->children.size(); ++i) + { + auto & expr_node = nodes.emplace_back(); + expr_node.children.push_back(parent_node->children[i]); + parent_node->children[i] = &expr_node; + + expr_node.step = std::make_unique( + expr_node.children.front()->step->getOutputStream(), + expression->getExpression()->clone()); + } + + /// - Expression - Something + /// Union - Expression - Something + /// - Expression - Something + + return 3; + } + + return 0; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 4c4d515911a..4faf6f680b0 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -9,7 +9,7 @@ namespace DB::QueryPlanOptimizations { -void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root) +void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { struct Frame { @@ -27,7 +27,7 @@ void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimizat if (frame.next_child == 0) { if (optimization_settings.read_in_order) - optimizeReadInOrder(*frame.node); + optimizeReadInOrder(*frame.node, nodes); if (optimization_settings.distinct_in_order) tryDistinctReadInOrder(frame.node); diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index f26f7a10b71..ba289ed4e26 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -27,11 +28,46 @@ namespace DB::QueryPlanOptimizations { -ReadFromMergeTree * findReadingStep(QueryPlan::Node * node) +ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) +{ + if (auto * reading = typeid_cast(step)) + { + /// Already read-in-order, skip. + if (reading->getQueryInfo().input_order_info) + return nullptr; + + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); + if (sorting_key.column_names.empty()) + return nullptr; + + return reading; + } + + if (auto * merge = typeid_cast(step)) + { + const auto & tables = merge->getSelectedTables(); + if (tables.empty()) + return nullptr; + + for (const auto & table : tables) + { + auto storage = std::get(table); + const auto & sorting_key = storage->getInMemoryMetadataPtr()->getSortingKey(); + if (sorting_key.column_names.empty()) + return nullptr; + } + + return merge; + } + + return nullptr; +} + +QueryPlan::Node * findReadingStep(QueryPlan::Node * node) { IQueryPlanStep * step = node->step.get(); - if (auto * reading = typeid_cast(step)) - return reading; + if (auto * reading = checkSupportedReadingStep(step)) + return node; if (node->children.size() != 1) return nullptr; @@ -570,7 +606,112 @@ InputOrderInfoPtr buildInputOrderInfo( return std::make_shared(order_key_prefix_descr, next_sort_key, read_direction, limit); } -void optimizeReadInOrder(QueryPlan::Node & node) +InputOrderInfoPtr buildInputOrderInfo( + ReadFromMergeTree * reading, + const FixedColumns & fixed_columns, + const ActionsDAGPtr & dag, + const SortDescription & description, + size_t limit) +{ + const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); + const auto & sorting_key_columns = sorting_key.column_names; + + return buildInputOrderInfo( + fixed_columns, + dag, description, + sorting_key.expression->getActionsDAG(), sorting_key_columns, + limit); +} + +InputOrderInfoPtr buildInputOrderInfo( + ReadFromMerge * merge, + const FixedColumns & fixed_columns, + const ActionsDAGPtr & dag, + const SortDescription & description, + size_t limit) +{ + const auto & tables = merge->getSelectedTables(); + + InputOrderInfoPtr order_info; + for (const auto & table : tables) + { + auto storage = std::get(table); + const auto & sorting_key = storage->getInMemoryMetadataPtr()->getSortingKey(); + const auto & sorting_key_columns = sorting_key.column_names; + + if (sorting_key_columns.empty()) + return nullptr; + + auto table_order_info = buildInputOrderInfo( + fixed_columns, + dag, description, + sorting_key.expression->getActionsDAG(), sorting_key_columns, + limit); + + if (!table_order_info) + return nullptr; + + if (!order_info) + order_info = table_order_info; + else if (*order_info != *table_order_info) + return nullptr; + } + + return order_info; +} + +InputOrderInfoPtr buildInputOrderInfo( + SortingStep & sorting, + QueryPlan::Node & node) +{ + if (node.children.size() != 1) + return nullptr; + + QueryPlan::Node * reading_node = findReadingStep(node.children.front()); + if (!reading_node) + return nullptr; + + const auto & description = sorting.getSortDescription(); + auto limit = sorting.getLimit(); + + ActionsDAGPtr dag; + FixedColumns fixed_columns; + buildSortingDAG(node.children.front(), dag, fixed_columns); + + if (dag && !fixed_columns.empty()) + enreachFixedColumns(*dag, fixed_columns); + + if (auto * reading = typeid_cast(reading_node->step.get())) + { + auto order_info = buildInputOrderInfo( + reading, + fixed_columns, + dag, description, + limit); + + if (order_info) + reading->requestReadingInOrder(order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); + + return order_info; + } + else if (auto * merge = typeid_cast(reading_node->step.get())) + { + auto order_info = buildInputOrderInfo( + merge, + fixed_columns, + dag, description, + limit); + + if (order_info) + merge->requestReadingInOrder(order_info); + + return order_info; + } + + return nullptr; +} + +void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) return; @@ -582,40 +723,69 @@ void optimizeReadInOrder(QueryPlan::Node & node) if (sorting->getType() != SortingStep::Type::Full) return; - ReadFromMergeTree * reading = findReadingStep(node.children.front()); - if (!reading) - return; + if (typeid_cast(node.children.front()->step.get())) + { + auto & union_node = node.children.front(); - /// Already read-in-order, skip. - if (reading->getQueryInfo().input_order_info) - return; + std::vector infos; + const SortDescription * max_sort_descr = nullptr; + infos.reserve(node.children.size()); + for (auto * child : union_node->children) + { + infos.push_back(buildInputOrderInfo(*sorting, *child)); - const auto & sorting_key = reading->getStorageMetadata()->getSortingKey(); - if (sorting_key.column_names.empty()) - return; + if (infos.back() && (!max_sort_descr || max_sort_descr->size() < infos.back()->sort_description_for_merging.size())) + max_sort_descr = &infos.back()->sort_description_for_merging; + } - ActionsDAGPtr dag; - FixedColumns fixed_columns; - buildSortingDAG(node.children.front(), dag, fixed_columns); + if (!max_sort_descr || max_sort_descr->empty()) + return; - if (dag && !fixed_columns.empty()) - enreachFixedColumns(*dag, fixed_columns); + for (size_t i = 0; i < infos.size(); ++i) + { + const auto & info = infos[i]; + auto & child = union_node->children[i]; - const auto & description = sorting->getSortDescription(); - auto limit = sorting->getLimit(); - const auto & sorting_key_columns = sorting_key.column_names; + QueryPlanStepPtr additional_sorting; - auto order_info = buildInputOrderInfo( - fixed_columns, - dag, description, - sorting_key.expression->getActionsDAG(), sorting_key_columns, - limit); + if (!info) + { + auto limit = sorting->getLimit(); + /// If we have limit, it's better to sort up to full description and apply limit. + /// We cannot sort up to partial read-in-order description with limit cause result set can be wrong. + const auto & descr = limit ? sorting->getSortDescription() : *max_sort_descr; + additional_sorting = std::make_unique( + child->step->getOutputStream(), + descr, + limit, /// TODO: support limit with ties + sorting->getSettings(), + false); + } + else if (info->sort_description_for_merging.size() < max_sort_descr->size()) + { + additional_sorting = std::make_unique( + child->step->getOutputStream(), + info->sort_description_for_merging, + *max_sort_descr, + sorting->getSettings().max_block_size, + 0); /// TODO: support limit with ties + } - if (!order_info) - return; + if (additional_sorting) + { + auto & sort_node = nodes.emplace_back(); + sort_node.step = std::move(additional_sorting); + sort_node.children.push_back(child); + child = &sort_node; + } + } - reading->requestReadingInOrder(order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit); - sorting->convertToFinishSorting(order_info->sort_description_for_merging); + sorting->convertToFinishSorting(*max_sort_descr); + } + else if (auto order_info = buildInputOrderInfo(*sorting, *node.children.front())) + { + sorting->convertToFinishSorting(order_info->sort_description_for_merging); + } } size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index d4e55955899..abbb5daae81 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -448,7 +448,7 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizePrimaryKeyCondition(optimization_settings, *root); + QueryPlanOptimizations::optimizePrimaryKeyCondition(optimization_settings, *root, nodes); } void QueryPlan::explainEstimate(MutableColumns & columns) diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index bb58eff2f13..d9d1f08b661 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -42,29 +42,17 @@ static ITransformingStep::Traits getTraits(size_t limit) SortingStep::SortingStep( const DataStream & input_stream, SortDescription description_, - size_t max_block_size_, UInt64 limit_, - SizeLimits size_limits_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - TemporaryDataOnDiskScopePtr tmp_data_, - size_t min_free_disk_space_, + const Settings & settings_, bool optimize_sorting_by_input_stream_properties_) : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) , type(Type::Full) , result_description(std::move(description_)) - , max_block_size(max_block_size_) , limit(limit_) - , size_limits(size_limits_) - , max_bytes_before_remerge(max_bytes_before_remerge_) - , remerge_lowered_memory_bytes_ratio(remerge_lowered_memory_bytes_ratio_) - , max_bytes_before_external_sort(max_bytes_before_external_sort_) - , tmp_data(tmp_data_) - , min_free_disk_space(min_free_disk_space_) + , sort_settings(settings_) , optimize_sorting_by_input_stream_properties(optimize_sorting_by_input_stream_properties_) { - if (max_bytes_before_external_sort && tmp_data == nullptr) + if (sort_settings.max_bytes_before_external_sort && sort_settings.tmp_data == nullptr) throw Exception("Temporary data storage for external sorting is not provided", ErrorCodes::LOGICAL_ERROR); /// TODO: check input_stream is partially sorted by the same description. @@ -82,9 +70,9 @@ SortingStep::SortingStep( , type(Type::FinishSorting) , prefix_description(std::move(prefix_description_)) , result_description(std::move(result_description_)) - , max_block_size(max_block_size_) , limit(limit_) { + sort_settings.max_block_size = max_block_size_; /// TODO: check input_stream is sorted by prefix_description. output_stream->sort_description = result_description; output_stream->sort_scope = DataStream::SortScope::Global; @@ -98,9 +86,9 @@ SortingStep::SortingStep( : ITransformingStep(input_stream, input_stream.header, getTraits(limit_)) , type(Type::MergingSorted) , result_description(std::move(sort_description_)) - , max_block_size(max_block_size_) , limit(limit_) { + sort_settings.max_block_size = max_block_size_; /// TODO: check input_stream is partially sorted (each port) by the same description. output_stream->sort_description = result_description; output_stream->sort_scope = DataStream::SortScope::Global; @@ -155,7 +143,7 @@ void SortingStep::finishSorting( increase_sort_description_compile_attempts = false; return std::make_shared( - header, input_sort_desc, result_sort_desc, max_block_size, limit_, increase_sort_description_compile_attempts_current); + header, input_sort_desc, result_sort_desc, sort_settings.max_block_size, limit_, increase_sort_description_compile_attempts_current); }); } @@ -168,7 +156,7 @@ void SortingStep::mergingSorted(QueryPipelineBuilder & pipeline, const SortDescr pipeline.getHeader(), pipeline.getNumStreams(), result_sort_desc, - max_block_size, + sort_settings.max_block_size, SortingQueueStrategy::Batch, limit_); @@ -197,14 +185,14 @@ void SortingStep::mergeSorting(QueryPipelineBuilder & pipeline, const SortDescri return std::make_shared( header, result_sort_desc, - max_block_size, + sort_settings.max_block_size, limit_, increase_sort_description_compile_attempts_current, - max_bytes_before_remerge / pipeline.getNumStreams(), - remerge_lowered_memory_bytes_ratio, - max_bytes_before_external_sort, - std::make_unique(tmp_data, CurrentMetrics::TemporaryFilesForSort), - min_free_disk_space); + sort_settings.max_bytes_before_remerge / pipeline.getNumStreams(), + sort_settings.remerge_lowered_memory_bytes_ratio, + sort_settings.max_bytes_before_external_sort, + std::make_unique(sort_settings.tmp_data, CurrentMetrics::TemporaryFilesForSort), + sort_settings.min_free_disk_space); }); } @@ -223,7 +211,7 @@ void SortingStep::fullSort(QueryPipelineBuilder & pipeline, const SortDescriptio StreamLocalLimits limits; limits.mode = LimitsMode::LIMITS_CURRENT; //-V1048 - limits.size_limits = size_limits; + limits.size_limits = sort_settings.size_limits; pipeline.addSimpleTransform( [&](const Block & header, QueryPipelineBuilder::StreamType stream_type) -> ProcessorPtr @@ -241,7 +229,7 @@ void SortingStep::fullSort(QueryPipelineBuilder & pipeline, const SortDescriptio if (pipeline.getNumStreams() > 1) { auto transform = std::make_shared( - pipeline.getHeader(), pipeline.getNumStreams(), result_sort_desc, max_block_size, SortingQueueStrategy::Batch, limit_); + pipeline.getHeader(), pipeline.getNumStreams(), result_sort_desc, sort_settings.max_block_size, SortingQueueStrategy::Batch, limit_); pipeline.addTransform(std::move(transform)); } diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index 000c4955b67..a24df8d48f8 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -18,18 +18,23 @@ public: MergingSorted, }; + struct Settings + { + size_t max_block_size; + SizeLimits size_limits; + size_t max_bytes_before_remerge = 0; + double remerge_lowered_memory_bytes_ratio = 0; + size_t max_bytes_before_external_sort = 0; + TemporaryDataOnDiskScopePtr tmp_data = nullptr; + size_t min_free_disk_space = 0; + }; + /// Full SortingStep( const DataStream & input_stream, SortDescription description_, - size_t max_block_size_, UInt64 limit_, - SizeLimits size_limits_, - size_t max_bytes_before_remerge_, - double remerge_lowered_memory_bytes_ratio_, - size_t max_bytes_before_external_sort_, - TemporaryDataOnDiskScopePtr tmp_data_, - size_t min_free_disk_space_, + const Settings & settings_, bool optimize_sorting_by_input_stream_properties_); /// FinishSorting @@ -63,6 +68,7 @@ public: void convertToFinishSorting(SortDescription prefix_description); Type getType() const { return type; } + const Settings & getSettings() const { return sort_settings; } private: void updateOutputStream() override; @@ -81,16 +87,10 @@ private: SortDescription prefix_description; const SortDescription result_description; - const size_t max_block_size; UInt64 limit; - SizeLimits size_limits; - size_t max_bytes_before_remerge = 0; - double remerge_lowered_memory_bytes_ratio = 0; - size_t max_bytes_before_external_sort = 0; - TemporaryDataOnDiskScopePtr tmp_data = nullptr; + Settings sort_settings; - size_t min_free_disk_space = 0; const bool optimize_sorting_by_input_stream_properties = false; }; diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 7fb21b7e053..3c4fa3674d2 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -364,9 +364,13 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu size_t num_streams = static_cast(requested_num_streams * num_streams_multiplier); size_t remaining_streams = num_streams; - InputOrderInfoPtr input_sorting_info; - if (query_info.order_optimizer) + if (order_info) { + query_info.input_order_info = order_info; + } + else if (query_info.order_optimizer) + { + InputOrderInfoPtr input_sorting_info; for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it) { auto storage_ptr = std::get<1>(*it); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 33406321100..54f3999514d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -144,6 +144,10 @@ public: added_filter_nodes.nodes.push_back(&expression->findInOutputs(column_name)); } + const StorageListWithLocks & getSelectedTables() const { return selected_tables; } + + void requestReadingInOrder(InputOrderInfoPtr order_info_) { order_info = order_info_; } + private: const size_t required_max_block_size; const size_t requested_num_streams; @@ -165,6 +169,8 @@ private: std::string added_filter_column_name; + InputOrderInfoPtr order_info; + struct AliasData { String name; From 30f7c04896f4d787e70c5b4d7d269cc2c8a3ff07 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 1 Nov 2022 19:58:20 +0000 Subject: [PATCH 125/526] Remove some debug output. --- .../QueryPlan/Optimizations/distinctReadInOrder.cpp | 2 -- .../reuseStorageOrderingForWindowFunctions.cpp | 10 +++++----- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index 7a5950b29a7..c01dfd0e6e3 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -9,7 +9,6 @@ namespace DB::QueryPlanOptimizations { size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) { - std::cerr << "&&&&&&&&&&&& tryDistinctReadInOrder\n"; /// check if it is preliminary distinct node DistinctStep * pre_distinct = nullptr; if (auto * distinct = typeid_cast(parent_node->step.get()); distinct) @@ -85,7 +84,6 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) if (output_data_stream.sort_scope != DataStream::SortScope::Chunk && number_of_sorted_distinct_columns <= output_sort_desc.size()) return 0; - std::cerr << "++++++++ tryDistinctReadInOrder\n"; /// update input order info in read_from_merge_tree step const int direction = 0; /// for DISTINCT direction doesn't matter, ReadFromMergeTree will choose proper one read_from_merge_tree->requestReadingInOrder(number_of_sorted_distinct_columns, direction, pre_distinct->getLimitHint()); diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index ba289ed4e26..a877fe7aedf 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -532,7 +532,7 @@ InputOrderInfoPtr buildInputOrderInfo( current_direction = descr.direction; - std::cerr << "====== (no dag) Found direct match" << std::endl; + //std::cerr << "====== (no dag) Found direct match" << std::endl; ++next_descr_column; ++next_sort_key; @@ -546,11 +546,11 @@ InputOrderInfoPtr buildInputOrderInfo( const auto & match = matches[sort_node]; - std::cerr << "====== Finding match for " << sort_column_node->result_name << ' ' << static_cast(sort_column_node) << std::endl; + //std::cerr << "====== Finding match for " << sort_column_node->result_name << ' ' << static_cast(sort_column_node) << std::endl; if (match.node && match.node == sort_column_node) { - std::cerr << "====== Found direct match" << std::endl; + //std::cerr << "====== Found direct match" << std::endl; /// We try to find the match first even if column is fixed. In this case, potentially more keys will match. /// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1' @@ -569,13 +569,13 @@ InputOrderInfoPtr buildInputOrderInfo( } else if (fixed_key_columns.contains(sort_column_node)) { - std::cerr << "+++++++++ Found fixed key by match" << std::endl; + //std::cerr << "+++++++++ Found fixed key by match" << std::endl; ++next_sort_key; } else { - std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; + //std::cerr << "====== Check for fixed const : " << bool(sort_node->column) << " fixed : " << fixed_columns.contains(sort_node) << std::endl; bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node); if (!is_fixed_column) break; From 4f50a999bba357f20656f89c6d7980fadbe2542f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 1 Nov 2022 21:35:25 +0100 Subject: [PATCH 126/526] remove abandonable_lock part 1 --- .../MergeTree/EphemeralLockInZooKeeper.cpp | 60 +++++++------------ .../MergeTree/EphemeralLockInZooKeeper.h | 23 +++---- src/Storages/StorageReplicatedMergeTree.cpp | 17 ++++++ 3 files changed, 49 insertions(+), 51 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 7abed125b7a..600f6d30bb6 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -12,11 +12,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_) - : zookeeper(&zookeeper_), path_prefix(path_prefix_), holder_path(holder_path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_) + : zookeeper(&zookeeper_), path_prefix(path_prefix_), path(path_) { - /// Write the path to the secondary node in the main node. - path = zookeeper->create(path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential); if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); } @@ -24,22 +22,22 @@ EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, std::optional createEphemeralLockInZooKeeper( const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) { - /// The /abandonable_lock- name is for backward compatibility. - String holder_path_prefix = temp_path + "/abandonable_lock-"; - String holder_path; + String path; - /// Let's create an secondary ephemeral node. if (deduplication_path.empty()) { - holder_path = zookeeper_.create(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential); + String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER; + path = zookeeper_.create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); } else { + String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_INSERT; + /// Check for duplicates in advance, to avoid superfluous block numbers allocation Coordination::Requests ops; ops.emplace_back(zkutil::makeCreateRequest(deduplication_path, "", zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); - ops.emplace_back(zkutil::makeCreateRequest(holder_path_prefix, "", zkutil::CreateMode::EphemeralSequential)); + ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; Coordination::Error e = zookeeper_.tryMulti(ops, responses); if (e != Coordination::Error::ZOK) @@ -55,10 +53,10 @@ std::optional createEphemeralLockInZooKeeper( } } - holder_path = dynamic_cast(responses.back().get())->path_created; + path = dynamic_cast(responses.back().get())->path_created; } - return EphemeralLockInZooKeeper{path_prefix_, zookeeper_, holder_path}; + return EphemeralLockInZooKeeper{path_prefix_, zookeeper_, path}; } void EphemeralLockInZooKeeper::unlock() @@ -66,19 +64,18 @@ void EphemeralLockInZooKeeper::unlock() Coordination::Requests ops; getUnlockOps(ops); zookeeper->multi(ops); - holder_path = ""; + zookeeper = nullptr; } void EphemeralLockInZooKeeper::getUnlockOps(Coordination::Requests & ops) { checkCreated(); ops.emplace_back(zkutil::makeRemoveRequest(path, -1)); - ops.emplace_back(zkutil::makeRemoveRequest(holder_path, -1)); } EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { - if (!isCreated()) + if (!isLocked()) return; try @@ -97,33 +94,18 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( zkutil::ZooKeeper & zookeeper_) : zookeeper(&zookeeper_) { - std::vector holders; + String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER; while (true) { Coordination::Stat partitions_stat; Strings partitions = zookeeper->getChildren(block_numbers_path, &partitions_stat); - if (holders.size() < partitions.size()) - { - std::vector> holder_futures; - for (size_t i = 0; i < partitions.size() - holders.size(); ++i) - { - String path = temp_path + "/abandonable_lock-"; - holder_futures.push_back(zookeeper->asyncCreate(path, {}, zkutil::CreateMode::EphemeralSequential)); - } - for (auto & future : holder_futures) - { - auto resp = future.get(); - holders.push_back(resp.path_created); - } - } - Coordination::Requests lock_ops; - for (size_t i = 0; i < partitions.size(); ++i) + for (const auto & partition : partitions) { - String partition_path_prefix = block_numbers_path + "/" + partitions[i] + "/" + path_prefix; + String partition_path_prefix = block_numbers_path + "/" + partition + "/" + path_prefix; lock_ops.push_back(zkutil::makeCreateRequest( - partition_path_prefix, holders[i], zkutil::CreateMode::EphemeralSequential)); + partition_path_prefix, holder_path, zkutil::CreateMode::EphemeralSequential)); } lock_ops.push_back(zkutil::makeCheckRequest(block_numbers_path, partitions_stat.version)); @@ -146,7 +128,7 @@ EphemeralLocksInAllPartitions::EphemeralLocksInAllPartitions( ErrorCodes::LOGICAL_ERROR); UInt64 number = parse(path.c_str() + prefix_size, path.size() - prefix_size); - locks.push_back(LockInfo{path, holders[i], partitions[i], number}); + locks.push_back(LockInfo{path, partitions[i], number}); } return; @@ -158,19 +140,17 @@ void EphemeralLocksInAllPartitions::unlock() if (!zookeeper) return; - std::vector futures; + std::vector futures; for (const auto & lock : locks) { - Coordination::Requests unlock_ops; - unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.path, -1)); - unlock_ops.emplace_back(zkutil::makeRemoveRequest(lock.holder_path, -1)); - futures.push_back(zookeeper->asyncMulti(unlock_ops)); + futures.push_back(zookeeper->asyncRemove(lock.path)); } for (auto & future : futures) future.get(); locks.clear(); + zookeeper = nullptr; } EphemeralLocksInAllPartitions::~EphemeralLocksInAllPartitions() diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 276ffab5254..670f5de2489 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -19,9 +19,9 @@ namespace ErrorCodes } /// A class that is used for locking a block number in a partition. -/// It creates a secondary ephemeral node in `temp_path` and a main ephemeral node with `path_prefix` -/// that references the secondary node. The reasons for this two-level scheme are historical (of course -/// it would be simpler to allocate block numbers for all partitions in one ZK directory). +/// Before 22.11 it used to create a secondary ephemeral node in `temp_path` with "abandonable_lock-" prefix +/// and a main ephemeral node with `path_prefix` that references the secondary node. The reasons for this two-level scheme are historical. +/// Since 22.11 it creates single ephemeral node with `path_prefix` that references persistent fake "secondary node". class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( @@ -29,9 +29,13 @@ class EphemeralLockInZooKeeper : public boost::noncopyable protected: EphemeralLockInZooKeeper() = delete; - EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & holder_path_); + EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_); public: + /// Fake "secondary node" names for blocks with and without "deduplication_path" + static constexpr const char * LEGACY_LOCK_INSERT = "abandonable_lock-insert"; + static constexpr const char * LEGACY_LOCK_OTHER = "abandonable_lock-other"; + EphemeralLockInZooKeeper(EphemeralLockInZooKeeper && rhs) noexcept { *this = std::move(rhs); @@ -43,13 +47,12 @@ public: rhs.zookeeper = nullptr; path_prefix = std::move(rhs.path_prefix); path = std::move(rhs.path); - holder_path = std::move(rhs.holder_path); return *this; } - bool isCreated() const + bool isLocked() const { - return zookeeper && !holder_path.empty() && !path.empty(); + return zookeeper; } String getPath() const @@ -74,12 +77,12 @@ public: /// because the nodes will be already deleted. void assumeUnlocked() { - holder_path.clear(); + zookeeper = nullptr; } void checkCreated() const { - if (!isCreated()) + if (!isLocked()) throw Exception("EphemeralLock is not created", ErrorCodes::LOGICAL_ERROR); } @@ -89,7 +92,6 @@ private: zkutil::ZooKeeper * zookeeper = nullptr; String path_prefix; String path; - String holder_path; }; std::optional createEphemeralLockInZooKeeper( @@ -124,7 +126,6 @@ public: struct LockInfo { String path; - String holder_path; String partition_id; UInt64 number = 0; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8dad5755dab..b2bb616b119 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -623,6 +623,16 @@ void StorageReplicatedMergeTree::createNewZooKeeperNodes() /// For ALTER PARTITION with multi-leaders futures.push_back(zookeeper->asyncTryCreateNoThrow(zookeeper_path + "/alter_partition_version", String(), zkutil::CreateMode::Persistent)); + /// As for now, "/temp" node must exist, but we want to be able to remove it in future + if (zookeeper->exists(zookeeper_path + "/temp")) + { + /// For block numbers allocation (since 22.11) + futures.push_back(zookeeper->asyncTryCreateNoThrow( + zookeeper_path + "/temp/" + EphemeralLockInZooKeeper::LEGACY_LOCK_INSERT, String(), zkutil::CreateMode::Persistent)); + futures.push_back(zookeeper->asyncTryCreateNoThrow( + zookeeper_path + "/temp/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER, String(), zkutil::CreateMode::Persistent)); + } + for (auto & future : futures) { auto res = future.get(); @@ -700,6 +710,13 @@ bool StorageReplicatedMergeTree::createTableIfNotExists(const StorageMetadataPtr zkutil::CreateMode::Persistent)); ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp", "", zkutil::CreateMode::Persistent)); + + /// The following 2 nodes were added in 22.11 + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp/" + EphemeralLockInZooKeeper::LEGACY_LOCK_INSERT, "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER, "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "last added replica: " + replica_name, zkutil::CreateMode::Persistent)); From e06572cc27f22536805fae16cd1cb95bee70e4b6 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 1 Nov 2022 23:03:07 +0000 Subject: [PATCH 127/526] Validate Query Tree in debug --- src/Analyzer/IQueryTreePass.h | 2 +- src/Analyzer/QueryTreePassManager.cpp | 55 ++++++++++++++++++++------- 2 files changed, 43 insertions(+), 14 deletions(-) diff --git a/src/Analyzer/IQueryTreePass.h b/src/Analyzer/IQueryTreePass.h index 39b3d743ed3..4293934c32d 100644 --- a/src/Analyzer/IQueryTreePass.h +++ b/src/Analyzer/IQueryTreePass.h @@ -16,7 +16,7 @@ namespace DB * Dependencies between passes must be avoided. */ class IQueryTreePass; -using QueryTreePassPtr = std::shared_ptr; +using QueryTreePassPtr = std::unique_ptr; using QueryTreePasses = std::vector; class IQueryTreePass diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 853b4a23f38..1c4679e799c 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -18,6 +18,8 @@ #include #include +#include +#include namespace DB { @@ -27,6 +29,28 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +namespace +{ + +#ifndef NDEBUG + +// This visitor checks if Query Tree structure is valid after each pass +// in debug build. +class ValidationChecker : public InDepthQueryTreeVisitor +{ +public: + void visitImpl(QueryTreeNodePtr & node) const + { + auto * column = node->as(); + if (!column) + return; + column->getColumnSource(); + } +}; +#endif + +} + /** ClickHouse query tree pass manager. * * TODO: Support _shard_num into shardNum() rewriting. @@ -61,7 +85,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) size_t passes_size = passes.size(); for (size_t i = 0; i < passes_size; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker().visit(query_tree_node); +#endif + } } void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pass_index) @@ -114,38 +143,38 @@ void addQueryTreePasses(QueryTreePassManager & manager) auto context = manager.getContext(); const auto & settings = context->getSettingsRef(); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_functions_to_subcolumns) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.count_distinct_optimization) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_rewrite_sum_if_to_count_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_normalize_count_variants) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_arithmetic_operations_in_aggregate_functions) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_injective_functions_inside_uniq) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_multiif_to_if) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); if (settings.optimize_if_chain_to_multiif) - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); - manager.addPass(std::make_shared()); - manager.addPass(std::make_shared()); + manager.addPass(std::make_unique()); + manager.addPass(std::make_unique()); } } From ec382d7903fb64b5d8c85ce93792c52071299000 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 2 Nov 2022 09:50:41 +0800 Subject: [PATCH 128/526] add comment --- src/Storages/MergeTree/KeyCondition.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 15283eeb435..3bea261fd52 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -56,6 +56,7 @@ String Range::toString() const /// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. +/// If perfect_prefix_match == true, only consider pattern in the format `prefix%_` String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) { String fixed_prefix; From 2766c550f6a0ae5d84d218eadeee3b8e25f21c87 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Nov 2022 10:38:32 +0000 Subject: [PATCH 129/526] Fix typos. --- .../reuseStorageOrderingForWindowFunctions.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index a877fe7aedf..2e7e51644e0 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -395,13 +395,13 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG //std::cerr << ".. candidate " << parent->result_name << std::endl; if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName()) { - const auto & childern = parent->children; - size_t num_children = childern.size(); + const auto & children = parent->children; + size_t num_children = children.size(); if (frame.mapped_children.size() == num_children) { bool all_children_matched = true; for (size_t i = 0; all_children_matched && i < num_children; ++i) - all_children_matched = frame.mapped_children[i] == childern[i]; + all_children_matched = frame.mapped_children[i] == children[i]; if (all_children_matched) { From 550c776ffe493b1915d5c436734b6ff44ca492bf Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 2 Nov 2022 13:16:27 +0100 Subject: [PATCH 130/526] Wait for all files are in sync before archiving them --- tests/integration/ci-runner.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4eab305358b..551466cf583 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -353,6 +353,11 @@ class ClickhouseIntegrationTestsRunner: ) def _compress_logs(self, dir, relpaths, result_path): + # We execute sync in advance to have all files written after containers + # are finished or killed + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + "sync", shell=True + ) subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL "tar czf {} -C {} {}".format(result_path, dir, " ".join(relpaths)), shell=True, From a1c028f20c156a7326b21969f433aec54f6eaf17 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Nov 2022 13:43:04 +0100 Subject: [PATCH 131/526] fix --- .../0_stateless/01700_system_zookeeper_path_in.reference | 2 ++ .../0_stateless/02221_system_zookeeper_unrestricted.reference | 4 ++++ .../02221_system_zookeeper_unrestricted_like.reference | 4 ++++ 3 files changed, 10 insertions(+) diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index dcee18b33e0..664d8e84f27 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -9,6 +9,8 @@ r1 block_numbers blocks ======== +abandonable_lock-insert +abandonable_lock-other failed_parts last_part parallel diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference index bd0c9cee464..d250d1c9140 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted.reference @@ -1,5 +1,9 @@ 1 1 +abandonable_lock-insert +abandonable_lock-insert +abandonable_lock-other +abandonable_lock-other alter_partition_version alter_partition_version block_numbers diff --git a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference index f95d60dc07b..67920b13d71 100644 --- a/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference +++ b/tests/queries/0_stateless/02221_system_zookeeper_unrestricted_like.reference @@ -1,4 +1,6 @@ 1 +abandonable_lock-insert +abandonable_lock-other alter_partition_version block_numbers blocks @@ -37,6 +39,8 @@ zero_copy_hdfs zero_copy_s3 ------------------------- 1 +abandonable_lock-insert +abandonable_lock-other alter_partition_version block_numbers blocks From acbad99b24757449d36ba468221d5331a53d1f3d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Nov 2022 14:28:49 +0100 Subject: [PATCH 132/526] remove abandonable_lock part 2 --- docker/test/stress/run.sh | 1 + .../MergeTree/ReplicatedMergeTreeQueue.cpp | 62 +++++-------------- .../01158_zookeeper_log_long.reference | 22 +++---- 3 files changed, 27 insertions(+), 58 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 7058853b43e..78f627bf45e 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -481,6 +481,7 @@ else -e "The set of parts restored in place of" \ -e "(ReplicatedMergeTreeAttachThread): Initialization failed. Error" \ -e "Code: 269. DB::Exception: Destination table is myself" \ + -e "Coordination::Exception: Connection loss" \ /var/log/clickhouse-server/clickhouse-server.backward.clean.log | zgrep -Fa "" > /test_output/bc_check_error_messages.txt \ && echo -e 'Backward compatibility check: Error message in clickhouse-server.log (see bc_check_error_messages.txt)\tFAIL' >> /test_output/test_results.tsv \ || echo -e 'Backward compatibility check: No Error messages in clickhouse-server.log\tOK' >> /test_output/test_results.tsv diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d6d937ce66f..99946e9d938 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1972,54 +1972,24 @@ ReplicatedMergeTreeMergePredicate::ReplicatedMergeTreeMergePredicate( auto quorum_status_future = zookeeper->asyncTryGet(fs::path(queue.zookeeper_path) / "quorum" / "status"); /// Load current inserts - std::unordered_set lock_holder_paths; - for (const String & entry : zookeeper->getChildren(fs::path(queue.zookeeper_path) / "temp")) + Strings partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); + std::vector paths; + paths.reserve(partitions.size()); + for (const String & partition : partitions) + paths.push_back(fs::path(queue.zookeeper_path) / "block_numbers" / partition); + + auto locks_children = zookeeper->getChildren(paths); + + for (size_t i = 0; i < partitions.size(); ++i) { - if (startsWith(entry, "abandonable_lock-")) - lock_holder_paths.insert(fs::path(queue.zookeeper_path) / "temp" / entry); - } - - if (!lock_holder_paths.empty()) - { - Strings partitions = zookeeper->getChildren(fs::path(queue.zookeeper_path) / "block_numbers"); - std::vector paths; - paths.reserve(partitions.size()); - for (const String & partition : partitions) - paths.push_back(fs::path(queue.zookeeper_path) / "block_numbers" / partition); - - auto locks_children = zookeeper->getChildren(paths); - - struct BlockInfoInZooKeeper + Strings partition_block_numbers = locks_children[i].names; + for (const String & entry : partition_block_numbers) { - String partition; - Int64 number; - String zk_path; - std::future contents_future; - }; - - std::vector block_infos; - for (size_t i = 0; i < partitions.size(); ++i) - { - Strings partition_block_numbers = locks_children[i].names; - for (const String & entry : partition_block_numbers) - { - /// TODO: cache block numbers that are abandoned. - /// We won't need to check them on the next iteration. - if (startsWith(entry, "block-")) - { - Int64 block_number = parse(entry.substr(strlen("block-"))); - String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; - block_infos.emplace_back( - BlockInfoInZooKeeper{partitions[i], block_number, zk_path, zookeeper->asyncTryGet(zk_path)}); - } - } - } - - for (auto & block : block_infos) - { - Coordination::GetResponse resp = block.contents_future.get(); - if (resp.error == Coordination::Error::ZOK && lock_holder_paths.contains(resp.data)) - committing_blocks[block.partition].insert(block.number); + if (!startsWith(entry, "block-")) + continue; + Int64 block_number = parse(entry.substr(strlen("block-"))); + String zk_path = fs::path(queue.zookeeper_path) / "block_numbers" / partitions[i] / entry; + committing_blocks[partitions[i]].insert(block_number); } } diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.reference b/tests/queries/0_stateless/01158_zookeeper_log_long.reference index ee2b465b3d3..a0088610c9d 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.reference +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.reference @@ -5,37 +5,35 @@ log ::1 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 ::1 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 parts -Request 0 Multi 0 0 \N 5 0 \N \N \N 0 0 0 0 +Request 0 Multi 0 0 \N 4 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 \N \N \N 0 0 0 0 Request 0 Remove /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 -1 0 2 \N \N \N 0 0 0 0 -Request 0 Remove /test/01158/default/rmt/temp/abandonable_lock-0000000000 0 0 -1 0 3 \N \N \N 0 0 0 0 -Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 4 \N \N \N 0 0 0 0 -Request 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 5 \N \N \N 0 0 0 0 -Response 0 Multi 0 0 \N 5 0 ZOK \N \N 0 0 0 0 +Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 3 \N \N \N 0 0 0 0 +Request 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 4 \N \N \N 0 0 0 0 +Response 0 Multi 0 0 \N 4 0 ZOK \N \N 0 0 0 0 Response 0 Create /test/01158/default/rmt/log/log- 0 1 \N 0 1 ZOK \N \N /test/01158/default/rmt/log/log-0000000000 0 0 0 0 Response 0 Remove /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 -1 0 2 ZOK \N \N 0 0 0 0 -Response 0 Remove /test/01158/default/rmt/temp/abandonable_lock-0000000000 0 0 -1 0 3 ZOK \N \N 0 0 0 0 -Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0 -Response 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 5 ZOK \N \N /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 0 0 +Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 3 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0 +Response 0 Create /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 4 ZOK \N \N /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 0 0 Request 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 \N \N \N 0 0 0 0 Response 0 Exists /test/01158/default/rmt/replicas/1/parts/all_0_0_0 0 0 \N 0 0 ZOK \N \N 0 0 96 0 blocks Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0 Request 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 \N \N \N 0 0 0 0 -Request 0 Create /test/01158/default/rmt/temp/abandonable_lock- 1 1 \N 0 3 \N \N \N 0 0 0 0 +Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 \N \N \N 0 0 0 0 Response 0 Multi 0 0 \N 3 0 ZOK \N \N 0 0 0 0 Response 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 ZOK \N \N /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 0 0 Response 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 ZOK \N \N 0 0 0 0 -Response 0 Create /test/01158/default/rmt/temp/abandonable_lock- 1 1 \N 0 3 ZOK \N \N /test/01158/default/rmt/temp/abandonable_lock-0000000000 0 0 0 0 +Response 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 ZOK \N \N /test/01158/default/rmt/block_numbers/all/block-0000000000 0 0 0 0 Request 0 Multi 0 0 \N 3 0 \N \N \N 0 0 0 0 Request 0 Create /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 \N \N \N 0 0 0 0 Request 0 Remove /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 \N \N \N 0 0 0 0 -Request 0 Create /test/01158/default/rmt/temp/abandonable_lock- 1 1 \N 0 3 \N \N \N 0 0 0 0 +Request 0 Create /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 \N \N \N 0 0 0 0 Response 0 Multi 0 0 \N 3 0 ZNODEEXISTS \N \N 0 0 0 0 Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 1 ZNODEEXISTS \N \N 0 0 0 0 Response 0 Error /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 -1 0 2 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0 -Response 0 Error /test/01158/default/rmt/temp/abandonable_lock- 1 1 \N 0 3 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0 +Response 0 Error /test/01158/default/rmt/block_numbers/all/block- 1 1 \N 0 3 ZRUNTIMEINCONSISTENCY \N \N 0 0 0 0 Request 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 \N \N \N 0 0 0 0 Response 0 Get /test/01158/default/rmt/blocks/all_6308706741995381342_2495791770474910886 0 0 \N 0 0 ZOK \N \N 0 0 9 0 duration_ms From 204dac3d5d9d423192e171773952c2983574b29f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Nov 2022 17:14:20 +0300 Subject: [PATCH 133/526] Suggest using https://fiddle.clickhouse.com/ --- .github/ISSUE_TEMPLATE/85_bug-report.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/85_bug-report.md b/.github/ISSUE_TEMPLATE/85_bug-report.md index 3d2ed6148e3..08d03c284ca 100644 --- a/.github/ISSUE_TEMPLATE/85_bug-report.md +++ b/.github/ISSUE_TEMPLATE/85_bug-report.md @@ -13,6 +13,8 @@ assignees: '' > A clear and concise description of what works not as it is supposed to. +> A link to reproducer in [https://fiddle.clickhouse.com/](https://fiddle.clickhouse.com/). + **Does it reproduce on recent release?** [The list of releases](https://github.com/ClickHouse/ClickHouse/blob/master/utils/list-versions/version_date.tsv) From a860b252492cc7840f23bf374e930a4048dd4c61 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Wed, 2 Nov 2022 15:24:09 +0000 Subject: [PATCH 134/526] Improve ValidationChecker readability --- src/Analyzer/QueryTreePassManager.cpp | 23 +++++++++++++++++++---- 1 file changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index 1c4679e799c..d59e7457ed2 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -20,6 +20,7 @@ #include #include #include +#include namespace DB { @@ -34,17 +35,26 @@ namespace #ifndef NDEBUG -// This visitor checks if Query Tree structure is valid after each pass -// in debug build. +/** This visitor checks if Query Tree structure is valid after each pass + * in debug build. + */ class ValidationChecker : public InDepthQueryTreeVisitor { + String pass_name; public: + explicit ValidationChecker(String pass_name_) + : pass_name(std::move(pass_name_)) + {} + void visitImpl(QueryTreeNodePtr & node) const { auto * column = node->as(); if (!column) return; - column->getColumnSource(); + if (column->getColumnSourceOrNull() == nullptr) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Column {} {} query tree node does not have valid source node after running {} pass", + column->getColumnName(), column->getColumnType(), pass_name); } }; #endif @@ -88,7 +98,7 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node) { passes[i]->run(query_tree_node, current_context); #ifndef NDEBUG - ValidationChecker().visit(query_tree_node); + ValidationChecker(passes[i]->getName()).visit(query_tree_node); #endif } } @@ -104,7 +114,12 @@ void QueryTreePassManager::run(QueryTreeNodePtr query_tree_node, size_t up_to_pa auto current_context = getContext(); for (size_t i = 0; i < up_to_pass_index; ++i) + { passes[i]->run(query_tree_node, current_context); +#ifndef NDEBUG + ValidationChecker(passes[i]->getName()).visit(query_tree_node); +#endif + } } void QueryTreePassManager::dump(WriteBuffer & buffer) From 00c9e50ee37f631026ed119a7fdcdfa755b5d9c0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Nov 2022 16:24:18 +0100 Subject: [PATCH 135/526] fix race between INSERT and ALTER PARTITION --- src/Common/ThreadFuzzer.cpp | 10 +- src/Common/ThreadFuzzer.h | 1 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 4 +- .../MergeTree/EphemeralLockInZooKeeper.h | 3 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 16 ++- src/Storages/StorageReplicatedMergeTree.cpp | 110 +++++++++++++++--- src/Storages/StorageReplicatedMergeTree.h | 3 + tests/queries/0_stateless/replication.lib | 1 + 8 files changed, 124 insertions(+), 24 deletions(-) diff --git a/src/Common/ThreadFuzzer.cpp b/src/Common/ThreadFuzzer.cpp index 16b51c8bbce..ee6dc222600 100644 --- a/src/Common/ThreadFuzzer.cpp +++ b/src/Common/ThreadFuzzer.cpp @@ -243,15 +243,17 @@ static void injection( } } +void ThreadFuzzer::maybeInjectSleep() +{ + auto & fuzzer = ThreadFuzzer::instance(); + injection(fuzzer.yield_probability, fuzzer.migrate_probability, fuzzer.sleep_probability, fuzzer.sleep_time_us); +} void ThreadFuzzer::signalHandler(int) { DENY_ALLOCATIONS_IN_SCOPE; auto saved_errno = errno; - - auto & fuzzer = ThreadFuzzer::instance(); - injection(fuzzer.yield_probability, fuzzer.migrate_probability, fuzzer.sleep_probability, fuzzer.sleep_time_us); - + maybeInjectSleep(); errno = saved_errno; } diff --git a/src/Common/ThreadFuzzer.h b/src/Common/ThreadFuzzer.h index 743b8c75dc0..ff391dfcd8f 100644 --- a/src/Common/ThreadFuzzer.h +++ b/src/Common/ThreadFuzzer.h @@ -58,6 +58,7 @@ public: static void start(); static bool isStarted(); + static void maybeInjectSleep(); private: uint64_t cpu_time_period_us = 0; double yield_probability = 0; diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 600f6d30bb6..802d648655f 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -62,12 +62,12 @@ std::optional createEphemeralLockInZooKeeper( void EphemeralLockInZooKeeper::unlock() { Coordination::Requests ops; - getUnlockOps(ops); + getUnlockOp(ops); zookeeper->multi(ops); zookeeper = nullptr; } -void EphemeralLockInZooKeeper::getUnlockOps(Coordination::Requests & ops) +void EphemeralLockInZooKeeper::getUnlockOp(Coordination::Requests & ops) { checkCreated(); ops.emplace_back(zkutil::makeRemoveRequest(path, -1)); diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 670f5de2489..aaa1fc2af24 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -71,7 +71,8 @@ public: void unlock(); /// Adds actions equivalent to `unlock()` to the list. - void getUnlockOps(Coordination::Requests & ops); + /// Returns index of the action that removes + void getUnlockOp(Coordination::Requests & ops); /// Do not delete nodes in destructor. You may call this method after 'getUnlockOps' and successful execution of these ops, /// because the nodes will be already deleted. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 082228d7ebf..50afd8a8eb3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -31,6 +32,7 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; + extern const int QUERY_WAS_CANCELLED; } struct ReplicatedMergeTreeSink::DelayedChunk @@ -344,12 +346,14 @@ void ReplicatedMergeTreeSink::commitPart( bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); + ThreadFuzzer::maybeInjectSleep(); /// Prepare transaction to ZooKeeper /// It will simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. Coordination::Requests ops; Int64 block_number = 0; + size_t block_unlock_op_idx = std::numeric_limits::max(); String existing_part_name; if (block_number_lock) { @@ -393,7 +397,8 @@ void ReplicatedMergeTreeSink::commitPart( zkutil::CreateMode::PersistentSequential)); /// Deletes the information that the block number is used for writing. - block_number_lock->getUnlockOps(ops); + block_unlock_op_idx = ops.size(); + block_number_lock->getUnlockOp(ops); /** If we need a quorum - create a node in which the quorum is monitored. * (If such a node already exists, then someone has managed to make another quorum record at the same time, @@ -520,8 +525,12 @@ void ReplicatedMergeTreeSink::commitPart( part->name); } + ThreadFuzzer::maybeInjectSleep(); + storage.lockSharedData(*part, false, {}); + ThreadFuzzer::maybeInjectSleep(); + Coordination::Responses responses; Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT @@ -534,6 +543,11 @@ void ReplicatedMergeTreeSink::commitPart( if (block_number_lock) block_number_lock->assumeUnlocked(); } + else if (multi_code == Coordination::Error::ZNONODE && zkutil::getFailedOpIndex(multi_code, responses) == block_unlock_op_idx) + { + throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, + "Insert query (for block {}) was cancelled by concurrent ALTER PARTITION", block_number_lock->getPath()); + } else if (multi_code == Coordination::Error::ZCONNECTIONLOSS || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b2bb616b119..57dd4feb77f 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6394,6 +6394,76 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper( } } +void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( + zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) +{ + /// Imagine that some INSERT query has allocated block number 42, but it's still in progress. + /// Some DROP PARTITION query gets block number 43 and commits DROP_RANGE all_0_42_999_999. + /// And after that INSERT commits GET_PART all_42_42_0. Oops, intersecting parts. + /// So we have to either wait for unfinished INSERTs or cancel them. + /// It's totally fine to cancel since we are going to remove data anyway. + /// We can safely cancel INSERT query by removing its ephemeral block number. + /// Usually it's bad idea to remove ephemeral nodes owned by someone else, + /// but INSERTs remove such nodes atomically with part commit, so INSERT will fail if node does not exist. + + fs::path partition_path = fs::path(zookeeper_path) / "block_numbers" / partition_id; + Strings queries_in_progress = zookeeper.getChildren(partition_path); + if (queries_in_progress.empty()) + return; + + Strings paths_to_get; + for (const auto & block : queries_in_progress) + { + if (!startsWith(block, "block-")) + continue; + Int64 block_number = parse(block.substr(strlen("block-"))); + if (min_block_num <= block_number && block_number <= max_block_num) + paths_to_get.push_back(partition_path / block); + } + + auto results = zookeeper.get(paths_to_get); + for (size_t i = 0; i < paths_to_get.size(); ++i) + { + auto & result = results[i]; + + /// The query already finished + if (result.error == Coordination::Error::ZNONODE) + continue; + + /// The query is not an insert (it does not have block_id) + if (result.data.ends_with(EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER)) + continue; + + if (result.data.ends_with(EphemeralLockInZooKeeper::LEGACY_LOCK_INSERT)) + { + /// Remove block number, so insert will fail to commit (it will try to remove this node too) + LOG_WARNING(log, "Some query is trying to concurrently insert block {}, will cancel it", paths_to_get[i]); + zookeeper.tryRemove(paths_to_get[i]); + } + else + { + constexpr const char * old_version_warning = "Ephemeral lock {} (referencing {}) is created by a replica " + "that running old version of ClickHouse (< 22.11). Cannot remove it, will wait for this lock to disappear. " + "Upgrade remaining hosts in the cluster to address this warning."; + constexpr const char * new_version_warning = "Ephemeral lock {} has unexpected content ({}), " + "probably it is created by a replica that running newer version of ClickHouse. " + "Cannot remove it, will wait for this lock to disappear. Upgrade remaining hosts in the cluster to address this warning."; + + if (result.data.starts_with(zookeeper_path + "/temp/abandonable_lock-")) + LOG_WARNING(log, old_version_warning, paths_to_get[i], result.data); + else + LOG_WARNING(log, new_version_warning, paths_to_get[i], result.data); + + Stopwatch time_waiting; + const auto & stop_waiting = [this, &time_waiting]() + { + auto timeout = getContext()->getSettingsRef().lock_acquire_timeout.value.seconds(); + return partial_shutdown_called || (timeout < time_waiting.elapsedSeconds()); + }; + zookeeper.waitForDisappear(paths_to_get[i], stop_waiting); + } + } +} void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) @@ -6403,21 +6473,18 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); String partition_prefix = partition_id + "_"; - zkutil::AsyncResponses get_futures; + Strings paths_to_get; for (const String & block_id : blocks) - { if (startsWith(block_id, partition_prefix)) - { - String path = fs::path(zookeeper_path) / "blocks" / block_id; - get_futures.emplace_back(path, zookeeper.asyncTryGet(path)); - } - } + paths_to_get.push_back(fs::path(zookeeper_path) / "blocks" / block_id); - for (auto & pair : get_futures) + auto results = zookeeper.get(paths_to_get); + + for (size_t i = 0; i < paths_to_get.size(); ++i) { - const String & path = pair.first; - auto result = pair.second.get(); + const String & path = paths_to_get[i]; + auto & result = results[i]; if (result.error == Coordination::Error::ZNONODE) continue; @@ -6574,9 +6641,13 @@ void StorageReplicatedMergeTree::replacePartitionFrom( entry_replace.columns_version = -1; } - /// Remove deduplication block_ids of replacing parts if (replace) + { + /// Cancel concurrent inserts in range + clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); + /// Remove deduplication block_ids of replacing parts clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); + } DataPartsVector parts_to_remove; Coordination::Responses op_results; @@ -6587,13 +6658,13 @@ void StorageReplicatedMergeTree::replacePartitionFrom( for (size_t i = 0; i < dst_parts.size(); ++i) { getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); - ephemeral_locks[i].getUnlockOps(ops); + ephemeral_locks[i].getUnlockOp(ops); } if (auto txn = query_context->getZooKeeperMetadataTransaction()) txn->moveOpsTo(ops); - delimiting_block_lock->getUnlockOps(ops); + delimiting_block_lock->getUnlockOp(ops); /// Check and update version to avoid race with DROP_RANGE ops.emplace_back(zkutil::makeSetRequest(alter_partition_version_path, "", alter_partition_version_stat.version)); /// Just update version, because merges assignment relies on it @@ -6812,6 +6883,9 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta entry_replace.columns_version = -1; } + /// Cancel concurrent inserts in range + clearLockedBlockNumbersInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); + clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); DataPartsVector parts_to_remove; @@ -6823,7 +6897,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta for (size_t i = 0; i < dst_parts.size(); ++i) { dest_table_storage->getCommitPartOps(ops, dst_parts[i], block_id_paths[i]); - ephemeral_locks[i].getUnlockOps(ops); + ephemeral_locks[i].getUnlockOp(ops); } /// Check and update version to avoid race with DROP_RANGE @@ -6887,7 +6961,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta fs::path(zookeeper_path) / "log/log-", entry_delete.toString(), zkutil::CreateMode::PersistentSequential)); /// Just update version, because merges assignment relies on it ops_src.emplace_back(zkutil::makeSetRequest(fs::path(zookeeper_path) / "log", "", -1)); - delimiting_block_lock->getUnlockOps(ops_src); + delimiting_block_lock->getUnlockOp(ops_src); op_results = zookeeper->multi(ops_src); @@ -7189,6 +7263,7 @@ bool StorageReplicatedMergeTree::dropPartImpl( } Coordination::Requests ops; + /// NOTE Don't need to remove block numbers too, because no in-progress inserts in the range are possible getClearBlocksInPartitionOps(ops, *zookeeper, part_info.partition_id, part_info.min_block, part_info.max_block); size_t clear_block_ops_size = ops.size(); @@ -7249,6 +7324,9 @@ bool StorageReplicatedMergeTree::addOpsToDropAllPartsInPartition( return false; } + /// Cancel concurrent inserts in range + clearLockedBlockNumbersInPartition(zookeeper, partition_id, drop_range_info.min_block, drop_range_info.max_block); + clearBlocksInPartition(zookeeper, partition_id, drop_range_info.min_block, drop_range_info.max_block); String drop_range_fake_part_name = getPartNamePossiblyFake(format_version, drop_range_info); @@ -7266,7 +7344,7 @@ bool StorageReplicatedMergeTree::addOpsToDropAllPartsInPartition( log_entry_ops_idx.push_back(ops.size()); ops.emplace_back(zkutil::makeCreateRequest(fs::path(zookeeper_path) / "log/log-", entry->toString(), zkutil::CreateMode::PersistentSequential)); - delimiting_block_lock->getUnlockOps(ops); + delimiting_block_lock->getUnlockOp(ops); delimiting_block_locks.push_back(std::move(*delimiting_block_lock)); entries.push_back(std::move(entry)); return true; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5213f963fdf..0d8c6b47dd5 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -750,6 +750,9 @@ private: mutable std::mutex existing_nodes_cache_mutex; bool existsNodeCached(const std::string & path) const; + /// Cancels INSERTs in the block range by removing ephemeral block numbers + void clearLockedBlockNumbersInPartition(zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); + void getClearBlocksInPartitionOps(Coordination::Requests & ops, zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); /// Remove block IDs from `blocks/` in ZooKeeper for the given partition ID in the given block number range. void clearBlocksInPartition( diff --git a/tests/queries/0_stateless/replication.lib b/tests/queries/0_stateless/replication.lib index 37c82ec7239..7c7db48e078 100755 --- a/tests/queries/0_stateless/replication.lib +++ b/tests/queries/0_stateless/replication.lib @@ -108,6 +108,7 @@ function check_replication_consistency() $CLICKHOUSE_CLIENT -q "select 'mutations', * from system.mutations where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, mutation_id" $CLICKHOUSE_CLIENT -q "select 'parts', * from system.parts where database=currentDatabase() and table like '$table_name_prefix%' order by database, table, name" echo "Good luck with debugging..." + exit 1 fi } From 3ed18ab1624aa3e714f554225a7d9a6817b2c448 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 2 Nov 2022 15:31:58 +0000 Subject: [PATCH 136/526] Mark a test as long --- .../queries/0_stateless/02369_lost_part_intersecting_merges.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index d61052392c7..88ff1f5b7c6 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: long, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From d551161a2999c5702717e47382e00b3406022692 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Nov 2022 17:11:17 +0000 Subject: [PATCH 137/526] Another one try. --- src/Functions/negate.cpp | 2 +- ...reuseStorageOrderingForWindowFunctions.cpp | 28 ++++++----- ...mizations_optimize_read_in_window_order.sh | 6 +-- ...istributed_group_by_sharding_key.reference | 5 +- ..._input_stream_properties_explain.reference | 15 +++--- ...ting_by_input_stream_properties_explain.sh | 2 +- .../02381_join_dup_columns_in_plan.reference | 46 ++++++++----------- 7 files changed, 50 insertions(+), 54 deletions(-) diff --git a/src/Functions/negate.cpp b/src/Functions/negate.cpp index 5047ae43652..731c0766e9d 100644 --- a/src/Functions/negate.cpp +++ b/src/Functions/negate.cpp @@ -42,7 +42,7 @@ template <> struct FunctionUnaryArithmeticMonotonicity static bool has() { return true; } static IFunction::Monotonicity get(const Field &, const Field &) { - return { .is_monotonic = true, .is_positive = false, .is_always_monotonic = true, .is_strict = true }; + return { .is_monotonic = true, .is_positive = false, .is_strict = true }; } }; diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 2e7e51644e0..67f117bbfd3 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -63,17 +63,17 @@ ISourceStep * checkSupportedReadingStep(IQueryPlanStep * step) return nullptr; } -QueryPlan::Node * findReadingStep(QueryPlan::Node * node) +QueryPlan::Node * findReadingStep(QueryPlan::Node & node) { - IQueryPlanStep * step = node->step.get(); + IQueryPlanStep * step = node.step.get(); if (auto * reading = checkSupportedReadingStep(step)) - return node; + return &node; - if (node->children.size() != 1) + if (node.children.size() != 1) return nullptr; if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) - return findReadingStep(node->children.front()); + return findReadingStep(*node.children.front()); return nullptr; } @@ -134,16 +134,16 @@ void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) dag = expression->clone(); } -void buildSortingDAG(QueryPlan::Node * node, ActionsDAGPtr & dag, FixedColumns & fixed_columns) +void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns) { - IQueryPlanStep * step = node->step.get(); + IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) return; - if (node->children.size() != 1) + if (node.children.size() != 1) return; - buildSortingDAG(node->children.front(), dag, fixed_columns); + buildSortingDAG(*node.children.front(), dag, fixed_columns); if (auto * expression = typeid_cast(step)) appendExpression(dag, expression->getExpression()); @@ -432,7 +432,7 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (child_match.node) { auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {}); - if (info.is_always_monotonic) + if (info.is_monotonic) { MatchedTrees::Monotonicity monotonicity; monotonicity.direction *= info.is_positive ? 1 : -1; @@ -468,6 +468,7 @@ InputOrderInfoPtr buildInputOrderInfo( const Names & sorting_key_columns, size_t limit) { + //std::cerr << "------- buildInputOrderInfo " << std::endl; SortDescription order_key_prefix_descr; order_key_prefix_descr.reserve(description.size()); @@ -664,10 +665,7 @@ InputOrderInfoPtr buildInputOrderInfo( SortingStep & sorting, QueryPlan::Node & node) { - if (node.children.size() != 1) - return nullptr; - - QueryPlan::Node * reading_node = findReadingStep(node.children.front()); + QueryPlan::Node * reading_node = findReadingStep(node); if (!reading_node) return nullptr; @@ -676,7 +674,7 @@ InputOrderInfoPtr buildInputOrderInfo( ActionsDAGPtr dag; FixedColumns fixed_columns; - buildSortingDAG(node.children.front(), dag, fixed_columns); + buildSortingDAG(node, dag, fixed_columns); if (dag && !fixed_columns.empty()) enreachFixedColumns(*dag, fixed_columns); diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 328d181fadd..8fca30f27db 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -19,14 +19,14 @@ $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" echo 'Partial sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" echo 'No sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" @@ -35,7 +35,7 @@ echo 'Complex ORDER BY' $CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" $CLICKHOUSE_CLIENT -q "INSERT INTO ${name}_complex VALUES (1, 2, 3), (2, 3, 4), (3, 4, 5)" echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0" +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=1" diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index 9592ed5691d..9016e731106 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -18,11 +18,12 @@ Union explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- not optimized Expression (Projection) LimitBy - Expression (Before LIMIT BY) - Union + Union + Expression (Before LIMIT BY) LimitBy Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) ReadFromStorage (SystemNumbers) + Expression ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index a7498e68bc0..2484001af11 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -6,13 +6,13 @@ Sorting (Global): a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort --- QUERY: set optimize_sorting_by_input_stream_properties=0;set max_threads=1;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a +-- QUERY: set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=1;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergeSortingTransform LimitsCheckingTransform PartialSortingTransform -- enable optimization -> sorting order is propagated from subquery -> merge sort -- QUERY: set optimize_sorting_by_input_stream_properties=1;set max_threads=1;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a -MergeSortingTransform +MergingSortedTransform 3 → 1 -- enable optimization -> there is no sorting order to propagate from subquery -> full sort -- QUERY: set optimize_sorting_by_input_stream_properties=1;set max_threads=1;EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c MergeSortingTransform @@ -21,8 +21,9 @@ PartialSortingTransform -- ExpressionStep preserves sort mode -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) Sorting (Global): a ASC -Sorting (Stream): a ASC +Sorting (Chunk): a ASC Sorting (Stream): a ASC -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) @@ -65,8 +66,9 @@ Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (None) +Sorting (Sorting for ORDER BY) Sorting (Global): a ASC -Sorting (Stream): a ASC +Sorting (Chunk): a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) @@ -74,7 +76,7 @@ Sorting (Global): x ASC, y ASC Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC Sorting (Chunk): a ASC, b ASC -Sorting (Chunk): a ASC, b ASC +Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) @@ -88,6 +90,7 @@ Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query -- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC +Sorting (Sorting for ORDER BY) Sorting (Global): a ASC -Sorting (Stream): a ASC +Sorting (Chunk): a ASC, b ASC Sorting (Stream): a ASC diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index a308d9bcbc1..683c073f06d 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -4,7 +4,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -DISABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=0;set max_threads=1" +DISABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=1" ENABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=1;set max_threads=1" MAKE_OUTPUT_STABLE="set optimize_read_in_order=1" GREP_SORTING="grep 'PartialSortingTransform\|LimitsCheckingTransform\|MergeSortingTransform\|MergingSortedTransform'" diff --git a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference index d2f1b4efdd5..bbf288c45d7 100644 --- a/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference +++ b/tests/queries/0_stateless/02381_join_dup_columns_in_plan.reference @@ -8,22 +8,19 @@ Header: key String Header: key String ReadFromStorage Header: dummy UInt8 - Expression + Union Header: s2.key String value String - Union - Header: key String + Expression + Header: s2.key String value String - Expression - Header: key String - value String - ReadFromStorage - Header: dummy UInt8 - Expression - Header: key String - value String - ReadFromStorage - Header: dummy UInt8 + ReadFromStorage + Header: dummy UInt8 + Expression + Header: s2.key String + value String + ReadFromStorage + Header: dummy UInt8 Expression Header: key String value String @@ -40,19 +37,16 @@ Header: key String Sorting Header: s2.key String value String - Expression + Union Header: s2.key String value String - Union - Header: key String + Expression + Header: s2.key String value String - Expression - Header: key String - value String - ReadFromStorage - Header: dummy UInt8 - Expression - Header: key String - value String - ReadFromStorage - Header: dummy UInt8 + ReadFromStorage + Header: dummy UInt8 + Expression + Header: s2.key String + value String + ReadFromStorage + Header: dummy UInt8 From fc38ddd5dabc3b34a8b4e10619d2d901c7656492 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Nov 2022 17:54:42 +0000 Subject: [PATCH 138/526] Another try. --- .../00940_order_by_read_in_order_query_plan.reference | 4 +--- .../0_stateless/00940_order_by_read_in_order_query_plan.sql | 2 +- ...1655_plan_optimizations_optimize_read_in_window_order.sh | 6 +++--- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference index 99a844b287e..841b8e5b06a 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference @@ -209,9 +209,7 @@ select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; Prefix sort description: toStartOfDay(x) ASC Result sort description: toStartOfDay(x) ASC, intDiv(intDiv(y, -2), -3) ASC -select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); -2020-02-03 00:00:00 1 1 -2020-02-03 00:00:00 1 1 +-- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; Prefix sort description: intDiv(intDiv(y, -2), -3) ASC Result sort description: intDiv(intDiv(y, -2), -3) ASC diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql index 0c2b864bf16..624bf26f0a6 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql @@ -87,5 +87,5 @@ select * from (explain plan actions = 1 select * from tab2 order by toTimeZone(t select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; -select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); +-- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 8fca30f27db..52be8904091 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -19,14 +19,14 @@ $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" echo 'Partial sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" echo 'No sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" @@ -35,7 +35,7 @@ echo 'Complex ORDER BY' $CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" $CLICKHOUSE_CLIENT -q "INSERT INTO ${name}_complex VALUES (1, 2, 3), (2, 3, 4), (3, 4, 5)" echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0,query_plan_read_in_order=0" +$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0" echo ' optimize_read_in_window_order=1' $CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=1" From d8b3a2af8490c3940811b7b080f36bb5e4b756d9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 2 Nov 2022 19:26:50 +0100 Subject: [PATCH 139/526] fix race between INSERT and DROP --- src/Storages/StorageReplicatedMergeTree.cpp | 2 ++ utils/keeper-data-dumper/main.cpp | 3 +++ 2 files changed, 5 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 57dd4feb77f..76f606aaeb7 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -5311,6 +5311,8 @@ StorageReplicatedMergeTree::allocateBlockNumber( if (!existsNodeCached(partition_path)) { Coordination::Requests ops; + /// Check that table is not being dropped ("host" is the first node that is removed on replica drop) + ops.push_back(zkutil::makeCheckRequest(fs::path(replica_path) / "host", -1)); ops.push_back(zkutil::makeCreateRequest(partition_path, "", zkutil::CreateMode::Persistent)); /// We increment data version of the block_numbers node so that it becomes possible /// to check in a ZK transaction that the set of partitions didn't change diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index dd3c3a4e2ad..0ea6371b49f 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -79,7 +79,10 @@ int main(int argc, char *argv[]) for (size_t i = last_commited_index + 1; i < changelog.next_slot(); ++i) { if (changelog.entry_at(i)->get_val_type() == nuraft::log_val_type::app_log) + { + state_machine->pre_commit(i, changelog.entry_at(i)->get_buf()); state_machine->commit(i, changelog.entry_at(i)->get_buf()); + } } dumpMachine(state_machine); From 4641f12964abdeb46d609b441b761353c1e0e59b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Nov 2022 18:33:48 +0000 Subject: [PATCH 140/526] Another try. --- .../Optimizations/reuseStorageOrderingForWindowFunctions.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index 67f117bbfd3..b5a7dcc299e 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -72,7 +72,10 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) if (node.children.size() != 1) return nullptr; - if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) + if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) + return findReadingStep(*node.children.front()); + + if (auto * distinct = typeid_cast(step); distinct->isPreliminary()) return findReadingStep(*node.children.front()); return nullptr; From 816f2319f7bffc07885f6b925cb92d0847dc7572 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Wed, 2 Nov 2022 19:35:22 +0100 Subject: [PATCH 141/526] Updated finished_asts to fix fast test fails with Normalise AST is too deep and updated ConvertFunctionOrLikeData to merge conditions by aliasorColumnName instead of ASTPtr - 42452 Bug fix for parent node corrupted --- src/Interpreters/ConvertFunctionOrLikeVisitor.cpp | 7 ++++--- src/Interpreters/QueryNormalizer.cpp | 4 ++++ 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp index 257bbda68eb..5d48391d56d 100644 --- a/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp +++ b/src/Interpreters/ConvertFunctionOrLikeVisitor.cpp @@ -15,7 +15,7 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) if (function.name != "or") return; - std::unordered_map> identifier_to_literals; + std::unordered_map> identifier_to_literals; for (auto & child : function.children) { if (auto * expr_list_fn = child->as()) @@ -51,10 +51,11 @@ void ConvertFunctionOrLikeData::visit(ASTFunction & function, ASTPtr &) regexp = "(?i)" + regexp; unique_elems.pop_back(); - auto it = identifier_to_literals.find(identifier); + auto it = identifier_to_literals.find(identifier->getAliasOrColumnName()); + if (it == identifier_to_literals.end()) { - it = identifier_to_literals.insert({identifier, std::make_shared(Field{Array{}})}).first; + it = identifier_to_literals.insert({identifier->getAliasOrColumnName(), std::make_shared(Field{Array{}})}).first; auto match = makeASTFunction("multiMatchAny"); match->arguments->children.push_back(arguments[0]); match->arguments->children.push_back(it->second); diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 0f57a8f549c..176a00c4ef1 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -118,7 +118,11 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) } } else + { ast = alias_node->clone(); + if (data.finished_asts.contains(alias_node)) + data.finished_asts[ast] = ast; + } } } From 1f11c73dabae2206a911224f636955160aca5105 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Nov 2022 18:46:56 +0000 Subject: [PATCH 142/526] Another try. --- .../Optimizations/reuseStorageOrderingForWindowFunctions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index b5a7dcc299e..f4f5fa8b201 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -75,7 +75,7 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) if (typeid_cast(step) || typeid_cast(step) || typeid_cast(step)) return findReadingStep(*node.children.front()); - if (auto * distinct = typeid_cast(step); distinct->isPreliminary()) + if (auto * distinct = typeid_cast(step); distinct && distinct->isPreliminary()) return findReadingStep(*node.children.front()); return nullptr; From 774a86021f0cd020eb0c1765fe859a46fbc5df83 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 2 Nov 2022 19:16:51 +0000 Subject: [PATCH 143/526] Fix datetime schema inference in case of empty string --- src/Formats/EscapingRuleUtils.cpp | 3 +++ .../0_stateless/02475_date_time_schema_inference_bug.reference | 1 + .../0_stateless/02475_date_time_schema_inference_bug.sql | 1 + 3 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference create mode 100644 tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index e80ab50968d..bba94e98e49 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -464,6 +464,9 @@ bool tryInferDate(const std::string_view & field) bool tryInferDateTime(const std::string_view & field, const FormatSettings & settings) { + if (field.empty()) + return false; + ReadBufferFromString buf(field); Float64 tmp_float; /// Check if it's just a number, and if so, don't try to infer DateTime from it, diff --git a/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference new file mode 100644 index 00000000000..8b137891791 --- /dev/null +++ b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.reference @@ -0,0 +1 @@ + diff --git a/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql new file mode 100644 index 00000000000..1aea4a8024c --- /dev/null +++ b/tests/queries/0_stateless/02475_date_time_schema_inference_bug.sql @@ -0,0 +1 @@ +select * from format(CSV, '""'); From c8f9af1afa52a0d75500b70dfb7508b42e55a889 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 2 Nov 2022 15:47:11 -0400 Subject: [PATCH 144/526] start renaming --- .../operations/system-tables/dictionaries.md | 4 +- .../external-dictionaries/_category_.yml | 4 +- .../external-dicts-dict-sources.md | 6 +- .../external-dicts-dict-structure.md | 2 +- .../external-dicts-dict.md | 4 +- .../external-dictionaries/external-dicts.md | 27 +++++--- docs/en/sql-reference/dictionaries/index.md | 4 +- .../dictionaries/internal-dicts.md | 4 +- .../statements/create/dictionary.md | 65 +++++++++++++++---- 9 files changed, 84 insertions(+), 36 deletions(-) diff --git a/docs/en/operations/system-tables/dictionaries.md b/docs/en/operations/system-tables/dictionaries.md index 112e2cc2cdf..4b256f0de97 100644 --- a/docs/en/operations/system-tables/dictionaries.md +++ b/docs/en/operations/system-tables/dictionaries.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/dictionaries --- # dictionaries -Contains information about [external dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +Contains information about [dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). Columns: @@ -33,7 +33,7 @@ Columns: - `lifetime_min` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Minimum [lifetime](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. - `lifetime_max` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Maximum [lifetime](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) of the dictionary in memory, after which ClickHouse tries to reload the dictionary (if `invalidate_query` is set, then only if it has changed). Set in seconds. - `loading_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Start time for loading the dictionary. -- `last_successful_update_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with external sources and investigate causes. +- `last_successful_update_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — End time for loading or updating the dictionary. Helps to monitor some troubles with dictionary sources and investigate the causes. - `loading_duration` ([Float32](../../sql-reference/data-types/float.md)) — Duration of a dictionary loading. - `last_exception` ([String](../../sql-reference/data-types/string.md)) — Text of the error that occurs when creating or reloading the dictionary if the dictionary couldn’t be created. - `comment` ([String](../../sql-reference/data-types/string.md)) — Text of the comment to dictionary. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml b/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml index 1f98223c54c..af79ff9af23 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/_category_.yml @@ -1,8 +1,8 @@ position: 37 -label: 'External Dictionaries' +label: 'Dictionaries' collapsible: true collapsed: true link: type: generated-index - title: External Dictionaries + title: Dictionaries slug: /en/sql-reference/dictionaries/external-dictionaries diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index d457f327e7a..b13295a84e7 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -1,12 +1,12 @@ --- slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources sidebar_position: 43 -sidebar_label: Sources of External Dictionaries +sidebar_label: Dictionary Sources --- -# Sources of External Dictionaries +# Dictionary Sources -An external dictionary can be connected to ClickHouse from many different sources. +A dictionary can be connected to ClickHouse from many different sources. If the dictionary is configured using an xml-file, the configuration looks like this: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 895743c3b50..40664e0ced3 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -171,5 +171,5 @@ Configuration fields: **See Also** -- [Functions for working with external dictionaries](../../../sql-reference/functions/ext-dict-functions.md). +- [Functions for working with dictionaries](../../../sql-reference/functions/ext-dict-functions.md). diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md index 5c237eea8c7..90e447385f8 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict sidebar_position: 40 -sidebar_label: Configuring an External Dictionary +sidebar_label: Configuring a Dictionary --- -# Configuring an External Dictionary +# Configuring a Dictionary If dictionary is configured using xml file, than dictionary configuration has the following structure: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 095fb6360cd..1d97a12f452 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -4,17 +4,17 @@ sidebar_position: 39 sidebar_label: General Description --- -# External Dictionaries +# Dictionaries -You can add your own dictionaries from various data sources. The data source for a dictionary can be a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Sources for external dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. +You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a DDL query, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. ClickHouse: - Fully or partially stores dictionaries in RAM. - Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. -- Allows to create external dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create/dictionary.md). +- Allows creating dictionaries with xml files or [DDL queries](../../../sql-reference/statements/create/dictionary.md). -The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. +The configuration of dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_config) parameter. Dictionaries can be loaded at server startup or at first use, depending on the [dictionaries_lazy_load](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-dictionaries_lazy_load) setting. @@ -24,6 +24,16 @@ The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tabl - Configuration parameters. - Metrics like amount of RAM allocated for the dictionary or a number of queries since the dictionary was successfully loaded. +## Creating a dictionary with a DDL query + +Dictionaries can be created with [DDL queries](../../../sql-reference/statements/create/dictionary.md). This does not require any additional records in a server configuration file. This allows dictionaries to be worked with as first-class entities, like tables or views. + +## Creating a dictionary with a configuration file + +:::note +Creating a dictionary with a configuration file is not applicable to ClickHouse Cloud. Please use DDL (see above). +::: + The dictionary configuration file has the following format: ``` xml @@ -44,18 +54,17 @@ The dictionary configuration file has the following format: You can [configure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) any number of dictionaries in the same file. -[DDL queries for dictionaries](../../../sql-reference/statements/create/dictionary.md) does not require any additional records in server configuration. They allow to work with dictionaries as first-class entities, like tables or views. :::note -You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../../sql-reference/functions/other-functions.md) function). This functionality is not related to external dictionaries. +You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../../../sql-reference/functions/other-functions.md) function). This functionality is not related to dictionaries. ::: ## See Also -- [Configuring an External Dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) +- [Configuring a Dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md) - [Storing Dictionaries in Memory](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) - [Dictionary Updates](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md) -- [Sources of External Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) +- [Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) - [Dictionary Key and Fields](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md) -- [Functions for Working with External Dictionaries](../../../sql-reference/functions/ext-dict-functions.md) +- [Functions for Working with Dictionaries](../../../sql-reference/functions/ext-dict-functions.md) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index eccd1215e30..b6aa62bdb47 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -12,6 +12,6 @@ ClickHouse supports special functions for working with dictionaries that can be ClickHouse supports: -- [Built-in dictionaries](../../sql-reference/dictionaries/internal-dicts.md#internal_dicts) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). -- [Plug-in (external) dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md#dicts-external-dicts) with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). +- [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md#dicts-external-dicts) with a [set of functions](../../sql-reference/functions/ext-dict-functions.md). +- [Embedded dictionaries](../../sql-reference/dictionaries/internal-dicts.md#internal_dicts) with a specific [set of functions](../../sql-reference/functions/ym-dict-functions.md). diff --git a/docs/en/sql-reference/dictionaries/internal-dicts.md b/docs/en/sql-reference/dictionaries/internal-dicts.md index dbc12a576f7..252bac5f5e8 100644 --- a/docs/en/sql-reference/dictionaries/internal-dicts.md +++ b/docs/en/sql-reference/dictionaries/internal-dicts.md @@ -1,10 +1,10 @@ --- slug: /en/sql-reference/dictionaries/internal-dicts sidebar_position: 39 -sidebar_label: Internal Dictionaries +sidebar_label: Embedded Dictionaries --- -# Internal Dictionaries +# Embedded Dictionaries ClickHouse contains a built-in feature for working with a geobase. diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index b24ff480c2d..cb07f627018 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -5,9 +5,9 @@ sidebar_label: DICTIONARY title: "CREATE DICTIONARY" --- -Creates a new [external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) with given [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) and [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). +Creates a new [dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) with given [structure](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md), [source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md), [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) and [lifetime](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md). -**Syntax** +## Syntax ``` sql CREATE [OR REPLACE] DICTIONARY [IF NOT EXISTS] [db.]dictionary_name [ON CLUSTER cluster] @@ -25,17 +25,25 @@ SETTINGS(setting_name = setting_value, setting_name = setting_value, ...) COMMENT 'Comment' ``` -External dictionary structure consists of attributes. Dictionary attributes are specified similarly to table columns. The only required attribute property is its type, all other properties may have default values. +The dictionary structure consists of attributes. Dictionary attributes are specified similarly to table columns. The only required attribute property is its type, all other properties may have default values. `ON CLUSTER` clause allows creating dictionary on a cluster, see [Distributed DDL](../../../sql-reference/distributed-ddl.md). Depending on dictionary [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) one or more attributes can be specified as dictionary keys. -For more information, see [External Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. +### SOURCE + +The source for a dictionary can be a: +- table in the current ClickHouse service +- table in a remote ClickHouse service +- file available by HTTP(S) +- another database + You can add a comment to the dictionary when you creating it using `COMMENT` clause. -**Example** +#### Create a dictionary from a table in the current ClickHouse service + Input table `source_table`: @@ -49,27 +57,26 @@ Input table `source_table`: Creating the dictionary: ``` sql -CREATE DICTIONARY dictionary_with_comment +CREATE DICTIONARY id_value_dictionary ( id UInt64, value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) +SOURCE(CLICKHOUSE(TABLE 'source_table')) LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000) -COMMENT 'The temporary dictionary'; ``` Output the dictionary: ``` sql -SHOW CREATE DICTIONARY dictionary_with_comment; +SHOW CREATE DICTIONARY id_value_dictionary; ``` ```text ┌─statement───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ CREATE DICTIONARY default.dictionary_with_comment +│ CREATE DICTIONARY default.id_value_dictionary ( `id` UInt64, `value` String @@ -77,10 +84,41 @@ SHOW CREATE DICTIONARY dictionary_with_comment; PRIMARY KEY id SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) LIFETIME(MIN 0 MAX 1000) -LAYOUT(FLAT()) -COMMENT 'The temporary dictionary' │ +LAYOUT(FLAT()) | └─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` +#### Create a dictionary from a table in a remote ClickHouse service +Creating the dictionary: + +``` sql +CREATE DICTIONARY id_value_dictionary +( + id UInt64, + value String +) +PRIMARY KEY id +SOURCE(CLICKHOUSE(HOST 'HOSTNAME' PORT '8443' TABLE 'source_table')) +LAYOUT(FLAT()) +LIFETIME(MIN 0 MAX 1000) +``` +#### Create a dictionary from a file available by HTTP(S) +```sql +statement: CREATE DICTIONARY default.taxi_zone_dictionary +( + `LocationID` UInt16 DEFAULT 0, + `Borough` String, + `Zone` String, + `service_zone` String +) +PRIMARY KEY LocationID +SOURCE(HTTP(URL 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/nyc-taxi/taxi_zone_lookup.csv' FORMAT 'CSVWithNames')) +LIFETIME(MIN 0 MAX 0) +LAYOUT(HASHED()) +``` + +#### Create a dictionary from another database + + Output the comment to dictionary: @@ -96,4 +134,5 @@ SELECT comment FROM system.dictionaries WHERE name == 'dictionary_with_comment' **See Also** -- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [external dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +- For more information, see the [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. +- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). From 3dd5df46159057dfbaf36bd9b29ebc22e05b2680 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 2 Nov 2022 16:58:07 -0400 Subject: [PATCH 145/526] update examples --- .../external-dicts-dict-sources.md | 19 ++++--- .../statements/create/dictionary.md | 53 +++++++++---------- 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index b13295a84e7..8218c066530 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -6,6 +6,11 @@ sidebar_label: Dictionary Sources # Dictionary Sources +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + A dictionary can be connected to ClickHouse from many different sources. If the dictionary is configured using an xml-file, the configuration looks like this: @@ -65,13 +70,13 @@ Types of sources (`source_type`): - [Executable Pool](#dicts-external_dicts_dict_sources-executable_pool) - [HTTP(s)](#dicts-external_dicts_dict_sources-http) - DBMS - - [ODBC](#dicts-external_dicts_dict_sources-odbc) - - [MySQL](#dicts-external_dicts_dict_sources-mysql) - - [ClickHouse](#dicts-external_dicts_dict_sources-clickhouse) - - [MongoDB](#dicts-external_dicts_dict_sources-mongodb) - - [Redis](#dicts-external_dicts_dict_sources-redis) - - [Cassandra](#dicts-external_dicts_dict_sources-cassandra) - - [PostgreSQL](#dicts-external_dicts_dict_sources-postgresql) + - [ODBC](#odbc) + - [MySQL](#mysql) + - [ClickHouse](#clickhouse) + - [MongoDB](#mongodb) + - [Redis](#redis) + - [Cassandra](#cassandra) + - [PostgreSQL](#postgresql) ## Local File diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index cb07f627018..37051f8031a 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -31,7 +31,7 @@ The dictionary structure consists of attributes. Dictionary attributes are speci Depending on dictionary [layout](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md) one or more attributes can be specified as dictionary keys. -### SOURCE +## SOURCE The source for a dictionary can be a: - table in the current ClickHouse service @@ -39,11 +39,7 @@ The source for a dictionary can be a: - file available by HTTP(S) - another database - -You can add a comment to the dictionary when you creating it using `COMMENT` clause. - -#### Create a dictionary from a table in the current ClickHouse service - +### Create a dictionary from a table in the current ClickHouse service Input table `source_table`: @@ -74,20 +70,29 @@ Output the dictionary: SHOW CREATE DICTIONARY id_value_dictionary; ``` -```text -┌─statement───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ CREATE DICTIONARY default.id_value_dictionary +```response +CREATE DICTIONARY default.id_value_dictionary ( `id` UInt64, `value` String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'source_table')) +SOURCE(CLICKHOUSE(TABLE 'source_table')) LIFETIME(MIN 0 MAX 1000) -LAYOUT(FLAT()) | -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +LAYOUT(FLAT()) ``` -#### Create a dictionary from a table in a remote ClickHouse service + +### Create a dictionary from a table in a remote ClickHouse service + +Input table (in the remote ClickHouse service) `source_table`: + +``` text +┌─id─┬─value──┐ +│ 1 │ First │ +│ 2 │ Second │ +└────┴────────┘ +``` + Creating the dictionary: ``` sql @@ -97,11 +102,13 @@ CREATE DICTIONARY id_value_dictionary value String ) PRIMARY KEY id -SOURCE(CLICKHOUSE(HOST 'HOSTNAME' PORT '8443' TABLE 'source_table')) +SOURCE(CLICKHOUSE(HOST 'HOSTNAME' PORT 9000 USER 'default' PASSWORD 'PASSWORD' TABLE 'source_table' DB 'default')) LAYOUT(FLAT()) LIFETIME(MIN 0 MAX 1000) ``` -#### Create a dictionary from a file available by HTTP(S) + +### Create a dictionary from a file available by HTTP(S) + ```sql statement: CREATE DICTIONARY default.taxi_zone_dictionary ( @@ -116,21 +123,9 @@ LIFETIME(MIN 0 MAX 0) LAYOUT(HASHED()) ``` -#### Create a dictionary from another database +### Create a dictionary from another database - - -Output the comment to dictionary: - -``` sql -SELECT comment FROM system.dictionaries WHERE name == 'dictionary_with_comment' AND database == currentDatabase(); -``` - -```text -┌─comment──────────────────┐ -│ The temporary dictionary │ -└──────────────────────────┘ -``` +Please see the details in [Dictionary sources](/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md/#dbms). **See Also** From df4868587b4002b33fa22c053438865fe8bac775 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 2 Nov 2022 17:40:55 -0400 Subject: [PATCH 146/526] add note to use DDL --- .../external-dictionaries/external-dicts-dict-layout.md | 5 +++++ .../external-dictionaries/external-dicts-dict-lifetime.md | 5 +++++ .../external-dictionaries/external-dicts-dict-polygon.md | 5 +++++ .../external-dictionaries/external-dicts-dict-structure.md | 5 +++++ .../external-dictionaries/external-dicts-dict.md | 5 +++++ .../dictionaries/external-dictionaries/external-dicts.md | 5 +++++ 6 files changed, 30 insertions(+) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 02a4ad57a3b..19884e7676d 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -24,6 +24,11 @@ ClickHouse generates an exception for errors with dictionaries. Examples of erro You can view the list of external dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + The configuration looks like this: ``` xml diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md index 6e4c8c4b94e..f0d6cb64a0b 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md @@ -12,6 +12,11 @@ Dictionary updates (other than loading for first use) do not block queries. Duri Example of settings: +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + ``` xml ... diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index e5ee48c9166..2124f816883 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -10,6 +10,11 @@ For example: defining a city area by geographical coordinates. Example of a polygon dictionary configuration: +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + ``` xml diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 40664e0ced3..20056c6ac9c 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -6,6 +6,11 @@ sidebar_label: Dictionary Key and Fields # Dictionary Key and Fields +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + The `structure` clause describes the dictionary key and fields available for queries. XML description: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md index 90e447385f8..f95a3593ae4 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md @@ -6,6 +6,11 @@ sidebar_label: Configuring a Dictionary # Configuring a Dictionary +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + If dictionary is configured using xml file, than dictionary configuration has the following structure: ``` xml diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 1d97a12f452..19bb7474d40 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -6,6 +6,11 @@ sidebar_label: General Description # Dictionaries +:::tip +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: + You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a DDL query, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. ClickHouse: From 7271d4d122e01249e6995011c51421285d3ab8d0 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 2 Nov 2022 17:52:35 -0400 Subject: [PATCH 147/526] add note to embedded dict --- docs/en/sql-reference/dictionaries/internal-dicts.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/sql-reference/dictionaries/internal-dicts.md b/docs/en/sql-reference/dictionaries/internal-dicts.md index 252bac5f5e8..f26c60880a4 100644 --- a/docs/en/sql-reference/dictionaries/internal-dicts.md +++ b/docs/en/sql-reference/dictionaries/internal-dicts.md @@ -3,9 +3,12 @@ slug: /en/sql-reference/dictionaries/internal-dicts sidebar_position: 39 sidebar_label: Embedded Dictionaries --- +import SelfManaged from '@site/docs/en/_snippets/_self_managed_only_no_roadmap.md'; # Embedded Dictionaries + + ClickHouse contains a built-in feature for working with a geobase. This allows you to: From 5fa0968bd5f07808df5942b27b54019d9a8143ff Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 3 Nov 2022 15:05:23 +0800 Subject: [PATCH 148/526] reset to original solution --- .../functions/splitting-merging-functions.md | 2 +- src/Functions/FunctionsStringArray.h | 99 +++++++++---------- src/Functions/URL/URLHierarchy.cpp | 4 +- src/Functions/URL/URLPathHierarchy.cpp | 4 +- .../URL/extractURLParameterNames.cpp | 4 +- src/Functions/URL/extractURLParameters.cpp | 4 +- .../01866_split_by_regexp.reference | 3 + .../0_stateless/01866_split_by_regexp.sql | 13 ++- 8 files changed, 69 insertions(+), 64 deletions(-) diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 70a1f10083b..1bf5839bf9f 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -8,7 +8,7 @@ sidebar_label: Splitting and Merging Strings and Arrays ## splitByChar(separator, s) -Splits a string into substrings separated by a specified character. It uses a constant string `separator` which consisting of exactly one character. +Splits a string into substrings separated by a specified character. It uses a constant string `separator` which consists of exactly one character. Returns an array of selected substrings. Empty substrings may be selected if the separator occurs at the beginning or end of the string, or if there are multiple consecutive separators. **Syntax** diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index 42334b68251..db62e108a2f 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -70,9 +70,9 @@ public: static constexpr auto name = "alphaTokens"; static String getName() { return name; } - static bool isVariadic() { return false; } + static bool isVariadic() { return true; } - static size_t getNumberOfArguments() { return 1; } + static size_t getNumberOfArguments() { return 0; } /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) @@ -98,10 +98,10 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { - return {}; + return 1; } /// Get the next token, if any, or return false. @@ -136,8 +136,8 @@ public: static constexpr auto name = "splitByNonAlpha"; static String getName() { return name; } - static bool isVariadic() { return false; } - static size_t getNumberOfArguments() { return 1; } + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) @@ -163,10 +163,10 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { - return {}; + return 1; } /// Get the next token, if any, or return false. @@ -201,8 +201,8 @@ public: static constexpr auto name = "splitByWhitespace"; static String getName() { return name; } - static bool isVariadic() { return false; } - static size_t getNumberOfArguments() { return 1; } + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) @@ -228,10 +228,10 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { - return {}; + return 1; } /// Get the next token, if any, or return false. @@ -308,8 +308,8 @@ public: return 1; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return 2; } @@ -318,7 +318,6 @@ public: { pos = pos_; end = end_; - curr_split = 0; } bool get(Pos & token_begin, Pos & token_end) @@ -353,8 +352,8 @@ private: public: static constexpr auto name = "splitByString"; static String getName() { return name; } - static bool isVariadic() { return false; } - static size_t getNumberOfArguments() { return 2; } + static bool isVariadic() { return true; } + static size_t getNumberOfArguments() { return 0; } static void checkArguments(const DataTypes & arguments) { @@ -379,10 +378,10 @@ public: return 1; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { - return {}; + return 2; } /// Called for each next string. @@ -468,8 +467,8 @@ public: return 1; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return 2; } @@ -559,10 +558,10 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { - return {}; + return std::nullopt; } /// Called for each next string. @@ -622,15 +621,15 @@ public: { Generator::checkArguments(arguments); - const auto max_split_pos = Generator::getMaxSplitArgumentPosition(); - if (max_split_pos) - if (arguments.size() > *max_split_pos && !isNativeInteger(arguments[*max_split_pos])) + const auto max_substrings_pos = Generator::getMaxSubstringsArgumentPosition(); + if (max_substrings_pos) + if (arguments.size() > *max_substrings_pos && !isNativeInteger(arguments[*max_substrings_pos])) throw Exception( ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "{}-th argument for function '{}' must be integer, got '{}' instead", - *max_split_pos + 1, + *max_substrings_pos + 1, getName(), - arguments[*max_split_pos]->getName()); + arguments[*max_substrings_pos]->getName()); return std::make_shared(std::make_shared()); } @@ -642,8 +641,8 @@ public: const auto & array_argument = arguments[generator.getStringsArgumentPosition()]; /// Whether we need to limit max tokens returned by Generator::get - /// If max_split is std::nullopt, no limit is applied. - auto max_split = getMaxSplit(arguments); + /// If max_substrings is std::nullopt, no limit is applied. + auto max_substrings = getMaxSubstrings(arguments); const ColumnString * col_str = checkAndGetColumn(array_argument.column.get()); const ColumnConst * col_const_str = @@ -679,7 +678,7 @@ public: generator.set(pos, end); size_t j = 0; - while (generator.get(token_begin, token_end) && !(max_split && j >= *max_split)) + while (generator.get(token_begin, token_end) && !(max_substrings && j >= *max_substrings)) { size_t token_size = token_end - token_begin; @@ -707,7 +706,7 @@ public: Pos token_begin = nullptr; Pos token_end = nullptr; - while (generator.get(token_begin, token_end) && !(max_split && dst.size() >= *max_split)) + while (generator.get(token_begin, token_end) && !(max_substrings && dst.size() >= *max_substrings)) dst.push_back(String(token_begin, token_end - token_begin)); return result_type->createColumnConst(col_const_str->size(), dst); @@ -721,7 +720,7 @@ public: private: template - std::optional getMaxSplitImpl(const ColumnWithTypeAndName & argument) const + std::optional getMaxSubstringsImpl(const ColumnWithTypeAndName & argument) const { const auto * col = checkAndGetColumnConst>(argument.column.get()); if (!col) @@ -731,20 +730,20 @@ private: return static_cast(value); } - std::optional getMaxSplit(const ColumnsWithTypeAndName & arguments) const + std::optional getMaxSubstrings(const ColumnsWithTypeAndName & arguments) const { - const auto pos = Generator::getMaxSplitArgumentPosition(); + const auto pos = Generator::getMaxSubstringsArgumentPosition(); if (!pos) return {}; if (arguments.size() <= *pos) return {}; - std::optional max_split; - if (!((max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) - || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) - || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])) - || (max_split = getMaxSplitImpl(arguments[2])) || (max_split = getMaxSplitImpl(arguments[2])))) + std::optional max_substrings; + if (!((max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) + || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) + || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) + || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])))) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}, which is {}-th argument of function {}", @@ -752,12 +751,12 @@ private: *pos + 1, getName()); - /// If max_split is negative or zero, tokenize will be applied as many times as possible, which is equivalent to - /// no max_split argument in function - if (max_split && *max_split <= 0) + /// If max_substrings is negative or zero, tokenize will be applied as many times as possible, which is equivalent to + /// no max_substrings argument in function + if (max_substrings && *max_substrings <= 0) return {}; - - return *max_split; + + return *max_substrings; } }; diff --git a/src/Functions/URL/URLHierarchy.cpp b/src/Functions/URL/URLHierarchy.cpp index f8b7cbb3858..80c775ea32d 100644 --- a/src/Functions/URL/URLHierarchy.cpp +++ b/src/Functions/URL/URLHierarchy.cpp @@ -38,8 +38,8 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return std::nullopt; } diff --git a/src/Functions/URL/URLPathHierarchy.cpp b/src/Functions/URL/URLPathHierarchy.cpp index 9af592b8db7..b8795e91244 100644 --- a/src/Functions/URL/URLPathHierarchy.cpp +++ b/src/Functions/URL/URLPathHierarchy.cpp @@ -37,8 +37,8 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return std::nullopt; } diff --git a/src/Functions/URL/extractURLParameterNames.cpp b/src/Functions/URL/extractURLParameterNames.cpp index 2cb6af7ae12..aaedc6eaac1 100644 --- a/src/Functions/URL/extractURLParameterNames.cpp +++ b/src/Functions/URL/extractURLParameterNames.cpp @@ -35,8 +35,8 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return std::nullopt; } diff --git a/src/Functions/URL/extractURLParameters.cpp b/src/Functions/URL/extractURLParameters.cpp index f7a2fd10b77..f83b11c6cc7 100644 --- a/src/Functions/URL/extractURLParameters.cpp +++ b/src/Functions/URL/extractURLParameters.cpp @@ -37,8 +37,8 @@ public: return 0; } - /// Returns the position of the possible max_split argument. std::nullopt means max_split argument is disabled in current function. - static std::optional getMaxSplitArgumentPosition() + /// Returns the position of the possible max_substrings argument. std::nullopt means max_substrings argument is disabled in current function. + static std::optional getMaxSubstringsArgumentPosition() { return std::nullopt; } diff --git a/tests/queries/0_stateless/01866_split_by_regexp.reference b/tests/queries/0_stateless/01866_split_by_regexp.reference index 1b873bf5621..ce33c1dd63a 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.reference +++ b/tests/queries/0_stateless/01866_split_by_regexp.reference @@ -10,3 +10,6 @@ ['one','two','three',''] ['one'] ['one','two'] +['one','two','three'] +['one','two','three',''] +['one','two','three',''] diff --git a/tests/queries/0_stateless/01866_split_by_regexp.sql b/tests/queries/0_stateless/01866_split_by_regexp.sql index c6d4b12380e..8553c962038 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.sql +++ b/tests/queries/0_stateless/01866_split_by_regexp.sql @@ -4,8 +4,11 @@ select splitByRegexp('<[^<>]*>', x) from (select arrayJoin(['

hello

world< select splitByRegexp('ab', ''); select splitByRegexp('', ''); -SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC'); -SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0); -SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1); -SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1); -SELECT splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC'); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 3); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 4); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 5); \ No newline at end of file From a51a1b439418b95aa61d745bdc77493457e68d0d Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 3 Nov 2022 15:06:58 +0800 Subject: [PATCH 149/526] rename alphaTokens to SplitByAlphaImpl --- src/Functions/FunctionsStringArray.cpp | 2 +- src/Functions/FunctionsStringArray.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsStringArray.cpp b/src/Functions/FunctionsStringArray.cpp index 660de15973f..fce87ed9cb3 100644 --- a/src/Functions/FunctionsStringArray.cpp +++ b/src/Functions/FunctionsStringArray.cpp @@ -30,7 +30,7 @@ DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const DataTypes & argum REGISTER_FUNCTION(StringArray) { factory.registerFunction(); - factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index db62e108a2f..f1805a76586 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -59,7 +59,7 @@ using Pos = const char *; /// Substring generators. All of them have a common interface. -class AlphaTokensImpl +class SplitByAlphaImpl { private: Pos pos; @@ -929,7 +929,7 @@ public: }; -using FunctionAlphaTokens = FunctionTokens; +using FunctionSplitByAlpha = FunctionTokens; using FunctionSplitByNonAlpha = FunctionTokens; using FunctionSplitByWhitespace = FunctionTokens; using FunctionSplitByChar = FunctionTokens; From 0df7e958455b76f763a8225c5831a222d530a2d2 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 3 Nov 2022 16:12:19 +0800 Subject: [PATCH 150/526] improve doc and uts --- .../functions/splitting-merging-functions.md | 49 +++++++++++---- src/Functions/FunctionsStringArray.cpp | 2 + src/Functions/FunctionsStringArray.h | 43 +++++++------- .../01866_split_by_regexp.reference | 8 --- .../0_stateless/01866_split_by_regexp.sql | 9 --- .../0_stateless/02185_split_by_char.reference | 9 --- .../0_stateless/02185_split_by_char.sql | 11 ---- .../02475_split_with_max_substrings.reference | 44 ++++++++++++++ .../02475_split_with_max_substrings.sql | 59 +++++++++++++++++++ 9 files changed, 164 insertions(+), 70 deletions(-) create mode 100644 tests/queries/0_stateless/02475_split_with_max_substrings.reference create mode 100644 tests/queries/0_stateless/02475_split_with_max_substrings.sql diff --git a/docs/en/sql-reference/functions/splitting-merging-functions.md b/docs/en/sql-reference/functions/splitting-merging-functions.md index 1bf5839bf9f..7cad6b2fbbf 100644 --- a/docs/en/sql-reference/functions/splitting-merging-functions.md +++ b/docs/en/sql-reference/functions/splitting-merging-functions.md @@ -6,7 +6,7 @@ sidebar_label: Splitting and Merging Strings and Arrays # Functions for Splitting and Merging Strings and Arrays -## splitByChar(separator, s) +## splitByChar(separator, s[, max_substrings]) Splits a string into substrings separated by a specified character. It uses a constant string `separator` which consists of exactly one character. Returns an array of selected substrings. Empty substrings may be selected if the separator occurs at the beginning or end of the string, or if there are multiple consecutive separators. @@ -14,13 +14,14 @@ Returns an array of selected substrings. Empty substrings may be selected if the **Syntax** ``` sql -splitByChar(separator, s) +splitByChar(separator, s[, max_substrings])) ``` **Arguments** - `separator` — The separator which should contain exactly one character. [String](../../sql-reference/data-types/string.md). - `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. **Returned value(s)** @@ -44,20 +45,22 @@ SELECT splitByChar(',', '1,2,3,abcde'); └─────────────────────────────────┘ ``` -## splitByString(separator, s) +## splitByString(separator, s[, max_substrings]) Splits a string into substrings separated by a string. It uses a constant string `separator` of multiple characters as the separator. If the string `separator` is empty, it will split the string `s` into an array of single characters. **Syntax** ``` sql -splitByString(separator, s) +splitByString(separator, s[, max_substrings])) ``` **Arguments** - `separator` — The separator. [String](../../sql-reference/data-types/string.md). - `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. + **Returned value(s)** @@ -91,20 +94,22 @@ SELECT splitByString('', 'abcde'); └────────────────────────────┘ ``` -## splitByRegexp(regexp, s) +## splitByRegexp(regexp, s[, max_substrings]) Splits a string into substrings separated by a regular expression. It uses a regular expression string `regexp` as the separator. If the `regexp` is empty, it will split the string `s` into an array of single characters. If no match is found for this regular expression, the string `s` won't be split. **Syntax** ``` sql -splitByRegexp(regexp, s) +splitByRegexp(regexp, s[, max_substrings])) ``` **Arguments** - `regexp` — Regular expression. Constant. [String](../data-types/string.md) or [FixedString](../data-types/fixedstring.md). - `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. + **Returned value(s)** @@ -146,7 +151,7 @@ Result: └────────────────────────────┘ ``` -## splitByWhitespace(s) +## splitByWhitespace(s[, max_substrings]) Splits a string into substrings separated by whitespace characters. Returns an array of selected substrings. @@ -154,12 +159,14 @@ Returns an array of selected substrings. **Syntax** ``` sql -splitByWhitespace(s) +splitByWhitespace(s[, max_substrings])) ``` **Arguments** - `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. + **Returned value(s)** @@ -179,7 +186,7 @@ SELECT splitByWhitespace(' 1! a, b. '); └─────────────────────────────────────┘ ``` -## splitByNonAlpha(s) +## splitByNonAlpha(s[, max_substrings]) Splits a string into substrings separated by whitespace and punctuation characters. Returns an array of selected substrings. @@ -187,12 +194,14 @@ Returns an array of selected substrings. **Syntax** ``` sql -splitByNonAlpha(s) +splitByNonAlpha(s[, max_substrings])) ``` **Arguments** - `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. + **Returned value(s)** @@ -217,10 +226,28 @@ SELECT splitByNonAlpha(' 1! a, b. '); Concatenates string representations of values listed in the array with the separator. `separator` is an optional parameter: a constant string, set to an empty string by default. Returns the string. -## alphaTokens(s) +## alphaTokens(s[, max_substrings]), splitByAlpha(s[, max_substrings]) Selects substrings of consecutive bytes from the ranges a-z and A-Z.Returns an array of substrings. +**Syntax** + +``` sql +alphaTokens(s[, max_substrings])) +splitByAlpha(s[, max_substrings]) +``` + +**Arguments** + +- `s` — The string to split. [String](../../sql-reference/data-types/string.md). +- `max_substrings` — An optional `Int64` defaulting to 0. When `max_substrings` > 0, the returned substrings will be no more than `max_substrings`, otherwise the function will return as many substrings as possible. + +**Returned value(s)** + +Returns an array of selected substrings. + +Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). + **Example** ``` sql diff --git a/src/Functions/FunctionsStringArray.cpp b/src/Functions/FunctionsStringArray.cpp index fce87ed9cb3..06f58730e41 100644 --- a/src/Functions/FunctionsStringArray.cpp +++ b/src/Functions/FunctionsStringArray.cpp @@ -30,7 +30,9 @@ DataTypePtr FunctionArrayStringConcat::getReturnTypeImpl(const DataTypes & argum REGISTER_FUNCTION(StringArray) { factory.registerFunction(); + factory.registerFunction(); + factory.registerAlias("splitByAlpha", FunctionSplitByAlpha::name); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index f1805a76586..ef0d9ac2896 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -32,12 +32,12 @@ namespace ErrorCodes /** Functions that split strings into an array of strings or vice versa. * - * splitByChar(sep, s) - * splitByString(sep, s) - * splitByRegexp(regexp, s) + * splitByChar(sep, s[, max_substrings]) + * splitByString(sep, s[, max_substrings]) + * splitByRegexp(regexp, s[, max_substrings]) * - * splitByWhitespace(s) - split the string by whitespace characters - * splitByNonAlpha(s) - split the string by whitespace and punctuation characters + * splitByWhitespace(s[, max_substrings]) - split the string by whitespace characters + * splitByNonAlpha(s[, max_substrings]) - split the string by whitespace and punctuation characters * * extractAll(s, regexp) - select from the string the subsequences corresponding to the regexp. * - first subpattern, if regexp has subpattern; @@ -48,7 +48,7 @@ namespace ErrorCodes * arrayStringConcat(arr, delimiter) * - join an array of strings into one string via a separator. * - * alphaTokens(s) - select from the string subsequence `[a-zA-Z]+`. + * alphaTokens(s[, max_substrings]) - select from the string subsequence `[a-zA-Z]+`. * * URL functions are located separately. */ @@ -622,14 +622,13 @@ public: Generator::checkArguments(arguments); const auto max_substrings_pos = Generator::getMaxSubstringsArgumentPosition(); - if (max_substrings_pos) - if (arguments.size() > *max_substrings_pos && !isNativeInteger(arguments[*max_substrings_pos])) - throw Exception( - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "{}-th argument for function '{}' must be integer, got '{}' instead", - *max_substrings_pos + 1, - getName(), - arguments[*max_substrings_pos]->getName()); + if (max_substrings_pos && *max_substrings_pos < arguments.size() && !isNativeInteger(arguments[*max_substrings_pos])) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "{}-th argument for function '{}' must be integer, got '{}' instead", + *max_substrings_pos + 1, + getName(), + arguments[*max_substrings_pos]->getName()); return std::make_shared(std::make_shared()); } @@ -734,16 +733,16 @@ private: { const auto pos = Generator::getMaxSubstringsArgumentPosition(); if (!pos) - return {}; + return std::nullopt; - if (arguments.size() <= *pos) - return {}; + if (*pos >= arguments.size()) + return std::nullopt; std::optional max_substrings; - if (!((max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) - || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) - || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])) - || (max_substrings = getMaxSubstringsImpl(arguments[2])) || (max_substrings = getMaxSubstringsImpl(arguments[2])))) + if (!((max_substrings = getMaxSubstringsImpl(arguments[*pos])) || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) + || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) + || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) + || (max_substrings = getMaxSubstringsImpl(arguments[*pos])) || (max_substrings = getMaxSubstringsImpl(arguments[*pos])))) throw Exception( ErrorCodes::ILLEGAL_COLUMN, "Illegal column {}, which is {}-th argument of function {}", @@ -754,7 +753,7 @@ private: /// If max_substrings is negative or zero, tokenize will be applied as many times as possible, which is equivalent to /// no max_substrings argument in function if (max_substrings && *max_substrings <= 0) - return {}; + return std::nullopt; return *max_substrings; } diff --git a/tests/queries/0_stateless/01866_split_by_regexp.reference b/tests/queries/0_stateless/01866_split_by_regexp.reference index ce33c1dd63a..a3ae2f35a5f 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.reference +++ b/tests/queries/0_stateless/01866_split_by_regexp.reference @@ -5,11 +5,3 @@ ['gbye','bug'] [''] [] -['one','two','three',''] -['one','two','three',''] -['one','two','three',''] -['one'] -['one','two'] -['one','two','three'] -['one','two','three',''] -['one','two','three',''] diff --git a/tests/queries/0_stateless/01866_split_by_regexp.sql b/tests/queries/0_stateless/01866_split_by_regexp.sql index 8553c962038..e472fb68d94 100644 --- a/tests/queries/0_stateless/01866_split_by_regexp.sql +++ b/tests/queries/0_stateless/01866_split_by_regexp.sql @@ -3,12 +3,3 @@ select splitByRegexp('', 'abcde'); select splitByRegexp('<[^<>]*>', x) from (select arrayJoin(['

hello

world

', 'gbyebug']) x); select splitByRegexp('ab', ''); select splitByRegexp('', ''); - -select splitByRegexp('[ABC]', 'oneAtwoBthreeC'); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 3); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 4); -select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 5); \ No newline at end of file diff --git a/tests/queries/0_stateless/02185_split_by_char.reference b/tests/queries/0_stateless/02185_split_by_char.reference index d8dd65ed63d..e831ed938fe 100644 --- a/tests/queries/0_stateless/02185_split_by_char.reference +++ b/tests/queries/0_stateless/02185_split_by_char.reference @@ -1,10 +1 @@ ['1','2','3'] -['1'] -['1','2'] -['1','2','3'] -['1','2','3'] -['1','2','3'] -['1','2','3'] -['1','2','3'] -['expr1'] -['expr2'] diff --git a/tests/queries/0_stateless/02185_split_by_char.sql b/tests/queries/0_stateless/02185_split_by_char.sql index 6ab1fc19079..127e2f631e5 100644 --- a/tests/queries/0_stateless/02185_split_by_char.sql +++ b/tests/queries/0_stateless/02185_split_by_char.sql @@ -1,12 +1 @@ select splitByChar(',', '1,2,3'); -select splitByChar(',', '1,2,3', 1); -select splitByChar(',', '1,2,3', 2); -select splitByChar(',', '1,2,3', 3); -select splitByChar(',', '1,2,3', 4); -select splitByChar(',', '1,2,3', 0); -select splitByChar(',', '1,2,3', -1); -select splitByChar(',', '1,2,3', -2); -SELECT splitByChar('=', s, 1) FROM values('s String', 'expr1=1+1=2', 'expr2=2+2=4=1+3'); - -select splitByChar(',', '1,2,3', ''); -- { serverError 43 } -select splitByChar(',', '1,2,3', ''); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02475_split_with_max_substrings.reference b/tests/queries/0_stateless/02475_split_with_max_substrings.reference new file mode 100644 index 00000000000..d55ef45a5e0 --- /dev/null +++ b/tests/queries/0_stateless/02475_split_with_max_substrings.reference @@ -0,0 +1,44 @@ +['1','2','3'] +['1','2','3'] +['1','2','3'] +['1'] +['1','2'] +['1','2','3'] +['1','2','3'] +['one','two','three',''] +['one','two','three',''] +['one','two','three',''] +['one'] +['one','two'] +['one','two','three'] +['one','two','three',''] +['one','two','three',''] +['abca','abc'] +['abca','abc'] +['abca','abc'] +['abca'] +['abca','abc'] +['abca','abc'] +['abca','abc'] +['1','a','b'] +['1','a','b'] +['1','a','b'] +['1'] +['1','a'] +['1','a','b'] +['1','a','b'] +['1!','a,','b.'] +['1!','a,','b.'] +['1!','a,','b.'] +['1!'] +['1!','a,'] +['1!','a,','b.'] +['1!','a,','b.'] +['1','2 3','4,5','abcde'] +['1','2 3','4,5','abcde'] +['1','2 3','4,5','abcde'] +['1'] +['1','2 3'] +['1','2 3','4,5'] +['1','2 3','4,5','abcde'] +['1','2 3','4,5','abcde'] diff --git a/tests/queries/0_stateless/02475_split_with_max_substrings.sql b/tests/queries/0_stateless/02475_split_with_max_substrings.sql new file mode 100644 index 00000000000..c51133c604e --- /dev/null +++ b/tests/queries/0_stateless/02475_split_with_max_substrings.sql @@ -0,0 +1,59 @@ +select splitByChar(',', '1,2,3'); +select splitByChar(',', '1,2,3', -1); +select splitByChar(',', '1,2,3', 0); +select splitByChar(',', '1,2,3', 1); +select splitByChar(',', '1,2,3', 2); +select splitByChar(',', '1,2,3', 3); +select splitByChar(',', '1,2,3', 4); + +select splitByRegexp('[ABC]', 'oneAtwoBthreeC'); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', -1); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 0); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 1); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 2); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 3); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 4); +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', 5); + +SELECT alphaTokens('abca1abc'); +SELECT alphaTokens('abca1abc', -1); +SELECT alphaTokens('abca1abc', 0); +SELECT alphaTokens('abca1abc', 1); +SELECT alphaTokens('abca1abc', 2); +SELECT alphaTokens('abca1abc', 3); + +SELECT splitByAlpha('abca1abc'); + +SELECT splitByNonAlpha(' 1! a, b. '); +SELECT splitByNonAlpha(' 1! a, b. ', -1); +SELECT splitByNonAlpha(' 1! a, b. ', 0); +SELECT splitByNonAlpha(' 1! a, b. ', 1); +SELECT splitByNonAlpha(' 1! a, b. ', 2); +SELECT splitByNonAlpha(' 1! a, b. ', 3); +SELECT splitByNonAlpha(' 1! a, b. ', 4); + +SELECT splitByWhitespace(' 1! a, b. '); +SELECT splitByWhitespace(' 1! a, b. ', -1); +SELECT splitByWhitespace(' 1! a, b. ', 0); +SELECT splitByWhitespace(' 1! a, b. ', 1); +SELECT splitByWhitespace(' 1! a, b. ', 2); +SELECT splitByWhitespace(' 1! a, b. ', 3); +SELECT splitByWhitespace(' 1! a, b. ', 4); + +SELECT splitByString(', ', '1, 2 3, 4,5, abcde'); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', -1); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 0); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 1); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 2); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 3); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 4); +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', 5); + + +select splitByChar(',', '1,2,3', ''); -- { serverError 43 } +select splitByRegexp('[ABC]', 'oneAtwoBthreeC', ''); -- { serverError 43 } +SELECT alphaTokens('abca1abc', ''); -- { serverError 43 } +SELECT splitByAlpha('abca1abc', ''); -- { serverError 43 } +SELECT splitByNonAlpha(' 1! a, b. ', ''); -- { serverError 43 } +SELECT splitByWhitespace(' 1! a, b. ', ''); -- { serverError 43 } +SELECT splitByString(', ', '1, 2 3, 4,5, abcde', ''); -- { serverError 43 } \ No newline at end of file From 9a002bd3783ee8cf64d88244c08b8606ad6720b3 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Nov 2022 11:57:32 +0100 Subject: [PATCH 151/526] =?UTF-8?q?Removed=20finished=5Fasts=20updation=20?= =?UTF-8?q?when=20alias=20is=20replaced=20and=20updated=20alias=20of=20the?= =?UTF-8?q?=20replaced=20ast=20it=20its=20initial=20name=20so=20that=20we?= =?UTF-8?q?=20don=E2=80=99t=20replace=20it=20further=20-=2042452=20Bug=20f?= =?UTF-8?q?ix=20for=20parent=20node=20corrupted?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- src/Interpreters/QueryNormalizer.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 176a00c4ef1..6c8bdb8628d 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -119,9 +119,9 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) } else { + auto alias_name = ast->getAliasOrColumnName(); ast = alias_node->clone(); - if (data.finished_asts.contains(alias_node)) - data.finished_asts[ast] = ast; + ast->setAlias(alias_name); } } } From e6a387e7b2fb753f335ab130f5de2ea59633c8d5 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Nov 2022 12:31:07 +0100 Subject: [PATCH 152/526] Updated registerFunction factory method and removed table_functions_allowed_in_readonly_mode - 42414 Enable table functions in readonly mode --- src/TableFunctions/TableFunctionExplain.cpp | 4 ++-- src/TableFunctions/TableFunctionFactory.h | 8 ++------ src/TableFunctions/TableFunctionGenerateRandom.cpp | 2 +- src/TableFunctions/TableFunctionNull.cpp | 2 +- src/TableFunctions/TableFunctionNumbers.cpp | 4 ++-- src/TableFunctions/TableFunctionRemote.cpp | 4 ++-- src/TableFunctions/TableFunctionValues.cpp | 2 +- src/TableFunctions/TableFunctionView.cpp | 2 +- src/TableFunctions/TableFunctionViewIfPermitted.cpp | 2 +- src/TableFunctions/TableFunctionZeros.cpp | 8 ++++---- 10 files changed, 17 insertions(+), 21 deletions(-) diff --git a/src/TableFunctions/TableFunctionExplain.cpp b/src/TableFunctions/TableFunctionExplain.cpp index 02493aa9b0e..02b9308ed22 100644 --- a/src/TableFunctions/TableFunctionExplain.cpp +++ b/src/TableFunctions/TableFunctionExplain.cpp @@ -91,7 +91,7 @@ InterpreterExplainQuery TableFunctionExplain::getInterpreter(ContextPtr context) void registerTableFunctionExplain(TableFunctionFactory & factory) { - factory.registerFunction({R"( + factory.registerFunction({.documentation = {R"( Returns result of EXPLAIN query. The function should not be called directly but can be invoked via `SELECT * FROM (EXPLAIN )`. @@ -103,7 +103,7 @@ Example: )", {{"1", "SELECT explain FROM (EXPLAIN AST SELECT * FROM system.numbers) WHERE explain LIKE '%Asterisk%'"}} -}); +}}); } diff --git a/src/TableFunctions/TableFunctionFactory.h b/src/TableFunctions/TableFunctionFactory.h index e2b8e29bd59..2cc648ba181 100644 --- a/src/TableFunctions/TableFunctionFactory.h +++ b/src/TableFunctions/TableFunctionFactory.h @@ -51,11 +51,11 @@ public: CaseSensitiveness case_sensitiveness = CaseSensitive); template - void registerFunction(Documentation doc = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) + void registerFunction(TableFunctionProperties properties = {}, CaseSensitiveness case_sensitiveness = CaseSensitive) { auto creator = []() -> TableFunctionPtr { return std::make_shared(); }; registerFunction(Function::name, - TableFunctionFactoryData{std::move(creator), {std::move(doc), table_functions_allowed_in_readonly_mode.contains(Function::name)}} , + TableFunctionFactoryData{std::move(creator), {std::move(properties)}} , case_sensitiveness); } @@ -82,10 +82,6 @@ private: TableFunctions table_functions; TableFunctions case_insensitive_table_functions; - - inline static const NameSet table_functions_allowed_in_readonly_mode = { - "null", "view", "viewIfPermitted", "numbers", "numbers_mt", "generateRandom", "values", "cluster", "clusterAllReplicas" - }; }; } diff --git a/src/TableFunctions/TableFunctionGenerateRandom.cpp b/src/TableFunctions/TableFunctionGenerateRandom.cpp index 8d1c06c7c4d..1ddbb48962d 100644 --- a/src/TableFunctions/TableFunctionGenerateRandom.cpp +++ b/src/TableFunctions/TableFunctionGenerateRandom.cpp @@ -91,7 +91,7 @@ StoragePtr TableFunctionGenerateRandom::executeImpl(const ASTPtr & /*ast_functio void registerTableFunctionGenerate(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNull.cpp b/src/TableFunctions/TableFunctionNull.cpp index 1a35830b06c..9ff07cc1946 100644 --- a/src/TableFunctions/TableFunctionNull.cpp +++ b/src/TableFunctions/TableFunctionNull.cpp @@ -52,6 +52,6 @@ StoragePtr TableFunctionNull::executeImpl(const ASTPtr & /*ast_function*/, Conte void registerTableFunctionNull(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionNumbers.cpp b/src/TableFunctions/TableFunctionNumbers.cpp index 0ef1d3815d9..2056cd838f5 100644 --- a/src/TableFunctions/TableFunctionNumbers.cpp +++ b/src/TableFunctions/TableFunctionNumbers.cpp @@ -51,8 +51,8 @@ StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_f void registerTableFunctionNumbers(TableFunctionFactory & factory) { - factory.registerFunction>(); - factory.registerFunction>(); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); + factory.registerFunction>({.documentation = {}, .allow_readonly = true}); } template diff --git a/src/TableFunctions/TableFunctionRemote.cpp b/src/TableFunctions/TableFunctionRemote.cpp index 097a239ccae..ff19485274c 100644 --- a/src/TableFunctions/TableFunctionRemote.cpp +++ b/src/TableFunctions/TableFunctionRemote.cpp @@ -323,8 +323,8 @@ void registerTableFunctionRemote(TableFunctionFactory & factory) { factory.registerFunction("remote", [] () -> TableFunctionPtr { return std::make_shared("remote"); }); factory.registerFunction("remoteSecure", [] () -> TableFunctionPtr { return std::make_shared("remote", /* secure = */ true); }); - factory.registerFunction("cluster", [] () -> TableFunctionPtr { return std::make_shared("cluster"); }); - factory.registerFunction("clusterAllReplicas", [] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }); + factory.registerFunction("cluster", {[] () -> TableFunctionPtr { return std::make_shared("cluster"); }, {.documentation = {}, .allow_readonly = true}}); + factory.registerFunction("clusterAllReplicas", {[] () -> TableFunctionPtr { return std::make_shared("clusterAllReplicas"); }, {.documentation = {}, .allow_readonly = true}}); } } diff --git a/src/TableFunctions/TableFunctionValues.cpp b/src/TableFunctions/TableFunctionValues.cpp index 7b8de69a4f8..05574825275 100644 --- a/src/TableFunctions/TableFunctionValues.cpp +++ b/src/TableFunctions/TableFunctionValues.cpp @@ -147,7 +147,7 @@ StoragePtr TableFunctionValues::executeImpl(const ASTPtr & ast_function, Context void registerTableFunctionValues(TableFunctionFactory & factory) { - factory.registerFunction({}, TableFunctionFactory::CaseInsensitive); + factory.registerFunction({.documentation = {}, .allow_readonly = true}, TableFunctionFactory::CaseInsensitive); } } diff --git a/src/TableFunctions/TableFunctionView.cpp b/src/TableFunctions/TableFunctionView.cpp index c53d26a794b..c999cba08e9 100644 --- a/src/TableFunctions/TableFunctionView.cpp +++ b/src/TableFunctions/TableFunctionView.cpp @@ -55,7 +55,7 @@ StoragePtr TableFunctionView::executeImpl( void registerTableFunctionView(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionViewIfPermitted.cpp b/src/TableFunctions/TableFunctionViewIfPermitted.cpp index ba3d2cb9d16..6128fe0a36f 100644 --- a/src/TableFunctions/TableFunctionViewIfPermitted.cpp +++ b/src/TableFunctions/TableFunctionViewIfPermitted.cpp @@ -107,7 +107,7 @@ bool TableFunctionViewIfPermitted::isPermitted(const ContextPtr & context, const void registerTableFunctionViewIfPermitted(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionZeros.cpp b/src/TableFunctions/TableFunctionZeros.cpp index 126166b547b..5874fca67e6 100644 --- a/src/TableFunctions/TableFunctionZeros.cpp +++ b/src/TableFunctions/TableFunctionZeros.cpp @@ -48,7 +48,7 @@ StoragePtr TableFunctionZeros::executeImpl(const ASTPtr & ast_fun void registerTableFunctionZeros(TableFunctionFactory & factory) { - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -62,9 +62,9 @@ This query will test the speed of `randomPrintableASCII` function using single t See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros(100000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); - factory.registerFunction>({R"( + factory.registerFunction>({.documentation = {R"( Generates a stream of zeros (a table with one column 'zero' of type 'UInt8') of specified size. This table function is used in performance tests, where you want to spend as little time as possible to data generation while testing some other parts of queries. @@ -78,7 +78,7 @@ This query will test the speed of `randomPrintableASCII` function using multiple See also the `system.zeros` table. )", {{"1", "SELECT count() FROM zeros_mt(1000000000) WHERE NOT ignore(randomPrintableASCII(10))"}} -}); +}}); } template From eeb9c57c5995d255610a71fe70aa3f79af97214e Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Thu, 3 Nov 2022 20:21:02 +0800 Subject: [PATCH 153/526] to fix issue 42856 and MaterializedMySQL improvement --- src/Core/MySQL/MySQLReplication.cpp | 7 +++++-- src/Core/MySQL/MySQLReplication.h | 3 ++- src/DataTypes/convertMySQLDataType.cpp | 2 +- src/Parsers/MySQL/ASTAlterCommand.cpp | 8 +++++++- src/Parsers/MySQL/ASTDeclareColumn.cpp | 2 ++ .../materialize_with_ddl.py | 10 ++++++++++ 6 files changed, 27 insertions(+), 5 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 6f3ac1b40e9..45437ea5843 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -116,9 +116,10 @@ namespace MySQLReplication if (!query.starts_with("XA COMMIT")) transaction_complete = false; } - else if (query.starts_with("SAVEPOINT")) + else if (query.starts_with("SAVEPOINT") || query.starts_with("ROLLBACK") + || query.starts_with("RELEASE SAVEPOINT")) { - throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); + typ = QUERY_EVENT_OTHER; } } @@ -941,6 +942,8 @@ namespace MySQLReplication { case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: + /// Ignore queries that have no impact on the data + case QUERY_EVENT_OTHER: { event = std::make_shared(std::move(query->header)); break; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 8900eee0102..d4bb3fb86f9 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -368,7 +368,8 @@ namespace MySQLReplication { QUERY_EVENT_DDL = 0, QUERY_EVENT_MULTI_TXN_FLAG = 1, - QUERY_EVENT_XA = 2 + QUERY_EVENT_XA = 2, + QUERY_EVENT_OTHER = 3 }; class QueryEvent : public EventBase diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index 64633c6fd7b..307ff317204 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -55,7 +55,7 @@ DataTypePtr convertMySQLDataType(MultiEnum type_support, else res = std::make_shared(); } - else if (type_name == "int" || type_name == "mediumint") + else if (type_name == "int" || type_name == "mediumint" || type_name == "integer") { if (is_unsigned) res = std::make_shared(); diff --git a/src/Parsers/MySQL/ASTAlterCommand.cpp b/src/Parsers/MySQL/ASTAlterCommand.cpp index b5b36ff3c74..10446d40172 100644 --- a/src/Parsers/MySQL/ASTAlterCommand.cpp +++ b/src/Parsers/MySQL/ASTAlterCommand.cpp @@ -267,7 +267,12 @@ static inline bool parseRenameCommand(IParser::Pos & pos, ASTPtr & node, Expecte } else { - return false; + if (!ParserCompoundIdentifier(true).parse(pos, new_name, expected)) + return false; + auto new_table_id = new_name->as()->getTableId(); + alter_command->type = ASTAlterCommand::RENAME_TABLE; + alter_command->new_table_name = new_table_id.table_name; + alter_command->new_database_name = new_table_id.database_name; } node = alter_command; @@ -306,6 +311,7 @@ static inline bool parseOtherCommand(IParser::Pos & pos, ASTPtr & node, Expected OptionDescribe("CONVERT TO CHARACTER SET", "charset", std::make_shared()), OptionDescribe("CHARACTER SET", "charset", std::make_shared()), OptionDescribe("DEFAULT CHARACTER SET", "charset", std::make_shared()), + OptionDescribe("COMMENT", "", std::make_shared()), OptionDescribe("LOCK", "lock", std::make_shared()) } }; diff --git a/src/Parsers/MySQL/ASTDeclareColumn.cpp b/src/Parsers/MySQL/ASTDeclareColumn.cpp index 89085ef989d..e585dcb670c 100644 --- a/src/Parsers/MySQL/ASTDeclareColumn.cpp +++ b/src/Parsers/MySQL/ASTDeclareColumn.cpp @@ -52,6 +52,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("KEY", "primary_key", std::make_unique()), OptionDescribe("COMMENT", "comment", std::make_unique()), OptionDescribe("CHARACTER SET", "charset_name", std::make_unique()), + OptionDescribe("CHARSET", "charset", std::make_unique()), OptionDescribe("COLLATE", "collate", std::make_unique()), OptionDescribe("COLUMN_FORMAT", "column_format", std::make_unique()), OptionDescribe("STORAGE", "storage", std::make_unique()), @@ -59,6 +60,7 @@ static inline bool parseColumnDeclareOptions(IParser::Pos & pos, ASTPtr & node, OptionDescribe("GENERATED ALWAYS AS", "generated", std::make_unique()), OptionDescribe("STORED", "is_stored", std::make_unique()), OptionDescribe("VIRTUAL", "is_stored", std::make_unique()), + OptionDescribe("INVISIBLE", "", std::make_unique()), OptionDescribe("", "reference", std::make_unique()), OptionDescribe("", "constraint", std::make_unique()), } diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 22d4633685e..bfb6d2bc6dd 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -875,6 +875,16 @@ def alter_rename_table_with_materialized_mysql_database( "1\n2\n3\n4\n5\n", ) + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5") + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6") + mysql_node.query("ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7") + + check_query( + clickhouse_node, + "SELECT * FROM test_database_rename_table.test_table_7 ORDER BY id FORMAT TSV", + "1\n2\n3\n4\n5\n", + ) + clickhouse_node.query("DROP DATABASE test_database_rename_table") mysql_node.query("DROP DATABASE test_database_rename_table") From 4c70622202411991e64543a4b588ebfec684e828 Mon Sep 17 00:00:00 2001 From: Smita Kulkarni Date: Thu, 3 Nov 2022 14:42:34 +0100 Subject: [PATCH 154/526] Updated to check size of alias before cloning, so that we throw error early - 42452 Bug fix for parent node corrupted --- src/Interpreters/QueryNormalizer.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/QueryNormalizer.cpp b/src/Interpreters/QueryNormalizer.cpp index 6c8bdb8628d..6a128d37e5d 100644 --- a/src/Interpreters/QueryNormalizer.cpp +++ b/src/Interpreters/QueryNormalizer.cpp @@ -113,12 +113,16 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) if (!is_cycle) { /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. + /// Check size of the alias before cloning too large alias AST + alias_node->checkSize(data.settings.max_expanded_ast_elements); ast = alias_node->clone(); ast->setAlias(node_alias); } } else { + /// Check size of the alias before cloning too large alias AST + alias_node->checkSize(data.settings.max_expanded_ast_elements); auto alias_name = ast->getAliasOrColumnName(); ast = alias_node->clone(); ast->setAlias(alias_name); From e68a3b0c7addd3bd2fb49d163c1599e91f322a7f Mon Sep 17 00:00:00 2001 From: Yakko Majuri <38760734+yakkomajuri@users.noreply.github.com> Date: Thu, 3 Nov 2022 11:25:07 -0300 Subject: [PATCH 155/526] fix(typo): Passible -> Possible --- src/Storages/Kafka/KafkaSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/Kafka/KafkaSettings.h b/src/Storages/Kafka/KafkaSettings.h index 51e5ee47077..e0da8031dd8 100644 --- a/src/Storages/Kafka/KafkaSettings.h +++ b/src/Storages/Kafka/KafkaSettings.h @@ -30,7 +30,7 @@ class ASTStorage; M(String, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ M(UInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ M(Bool, kafka_thread_per_consumer, false, "Provide independent thread for each consumer", 0) \ - M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Passible values: default, stream.", 0) \ + M(HandleKafkaErrorMode, kafka_handle_error_mode, HandleKafkaErrorMode::DEFAULT, "How to handle errors for Kafka engine. Possible values: default, stream.", 0) \ M(Bool, kafka_commit_on_select, false, "Commit messages when select query is made", 0) \ /** TODO: */ From 280e609744ac915cfbd583a668beee6a5fb2d371 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Nov 2022 16:42:26 +0000 Subject: [PATCH 156/526] More fixes. --- ...reuseStorageOrderingForWindowFunctions.cpp | 25 ++++++++++ .../QueryPlan/ReadFromMergeTree.cpp | 6 +-- src/Processors/QueryPlan/ReadFromMergeTree.h | 1 + ...rder_by_read_in_order_query_plan.reference | 10 ++-- ...0940_order_by_read_in_order_query_plan.sql | 10 ++-- ..._input_stream_properties_explain.reference | 46 ++++++++++--------- ...ting_by_input_stream_properties_explain.sh | 46 +++++++++---------- 7 files changed, 86 insertions(+), 58 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp index f4f5fa8b201..0ee4c353a3b 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp @@ -123,6 +123,15 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr { //std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; fiexd_columns.insert(maybe_fixed_column); + + const ActionsDAG::Node * maybe_injective = maybe_fixed_column; + while (maybe_injective->type == ActionsDAG::ActionType::FUNCTION + && maybe_injective->children.size() == 1 + && maybe_injective->function_base->isInjective({})) + { + maybe_injective = maybe_injective->children.front(); + fiexd_columns.insert(maybe_injective); + } } } } @@ -141,7 +150,19 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) + { + if (const auto * prewhere_info = reading->getPrewhereInfo()) + { + if (prewhere_info->prewhere_actions) + { + //std::cerr << "====== Adding prewhere " << std::endl; + appendExpression(dag, prewhere_info->prewhere_actions); + if (const auto * filter_expression = dag->tryFindInOutputs(prewhere_info->prewhere_column_name)) + appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); + } + } return; + } if (node.children.size() != 1) return; @@ -684,6 +705,8 @@ InputOrderInfoPtr buildInputOrderInfo( if (auto * reading = typeid_cast(reading_node->step.get())) { + + //std::cerr << "---- optimizeReadInOrder found mt" << std::endl; auto order_info = buildInputOrderInfo( reading, fixed_columns, @@ -721,6 +744,8 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) if (!sorting) return; + //std::cerr << "---- optimizeReadInOrder found sorting" << std::endl; + if (sorting->getType() != SortingStep::Type::Full) return; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f158b80a37e..20a9ddaa763 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -67,7 +67,7 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings( }; } -static const PrewhereInfoPtr & getPrewhereInfo(const SelectQueryInfo & query_info) +static const PrewhereInfoPtr & getPrewhereInfoFromQueryInfo(const SelectQueryInfo & query_info) { return query_info.projection ? query_info.projection->prewhere_info : query_info.prewhere_info; @@ -90,7 +90,7 @@ ReadFromMergeTree::ReadFromMergeTree( bool enable_parallel_reading) : ISourceStep(DataStream{.header = MergeTreeBaseSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(real_column_names_), - getPrewhereInfo(query_info_), + getPrewhereInfoFromQueryInfo(query_info_), data_.getPartitionValueType(), virt_column_names_)}) , reader_settings(getMergeTreeReaderSettings(context_, query_info_)) @@ -99,7 +99,7 @@ ReadFromMergeTree::ReadFromMergeTree( , virt_column_names(std::move(virt_column_names_)) , data(data_) , query_info(query_info_) - , prewhere_info(getPrewhereInfo(query_info)) + , prewhere_info(getPrewhereInfoFromQueryInfo(query_info)) , actions_settings(ExpressionActionsSettings::fromContext(context_)) , storage_snapshot(std::move(storage_snapshot_)) , metadata_for_reading(storage_snapshot->getMetadataForQuery()) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 04dc61664db..808e849fe03 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -150,6 +150,7 @@ public: ContextPtr getContext() const { return context; } const SelectQueryInfo & getQueryInfo() const { return query_info; } StorageMetadataPtr getStorageMetadata() const { return metadata_for_reading; } + const PrewhereInfo * getPrewhereInfo() const { return prewhere_info.get(); } void requestReadingInOrder(size_t prefix_size, int direction, size_t limit); diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference index 841b8e5b06a..f6a41f5dfff 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference @@ -108,16 +108,16 @@ select * from tab where (a + b) * c = 8 order by sin(a / b); select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; Prefix sort description: sin(divide(a, b)) ASC Result sort description: sin(divide(a, b)) ASC -select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0; +select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c; 1 1 1 1 1 1 1 1 -select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; +select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c) where explain ilike '%sort description%'; Prefix sort description: multiply(plus(d, 1), 4) ASC, multiply(plus(a, b), c) ASC Result sort description: multiply(plus(d, 1), 4) ASC, multiply(plus(a, b), c) ASC -select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0; +select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b); 2 2 2 2 2 2 2 2 -select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; +select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b)) where explain ilike '%sort description%'; Prefix sort description: multiply(plus(d, 1), 4) ASC, sin(divide(a, b)) ASC Result sort description: multiply(plus(d, 1), 4) ASC, sin(divide(a, b)) ASC -- Wrong order with fixed point @@ -210,6 +210,6 @@ select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay Prefix sort description: toStartOfDay(x) ASC Result sort description: toStartOfDay(x) ASC, intDiv(intDiv(y, -2), -3) ASC -- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); -select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; +select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; Prefix sort description: intDiv(intDiv(y, -2), -3) ASC Result sort description: intDiv(intDiv(y, -2), -3) ASC diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql index 624bf26f0a6..977bc79dbeb 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql @@ -35,11 +35,11 @@ select * from (explain plan actions = 1 select * from tab order by sin(a / b), ( select * from tab where (a + b) * c = 8 order by sin(a / b); select * from (explain plan actions = 1 select * from tab where (a + b) * c = 8 order by sin(a / b)) where explain ilike '%sort description%'; -select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0; -select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; +select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c; +select * from (explain plan actions = 1 select * from tab where d + 1 = 2 order by (d + 1) * 4, (a + b) * c) where explain ilike '%sort description%'; -select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0; -select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b) settings optimize_move_to_prewhere = 0) where explain ilike '%sort description%'; +select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b); +select * from (explain plan actions = 1 select * from tab where d + 1 = 3 and (a + b) = 4 and c = 2 order by (d + 1) * 4, sin(a / b)) where explain ilike '%sort description%'; -- Wrong order with fixed point select * from tab where (a + b) * c = 8 order by sin(b / a); @@ -88,4 +88,4 @@ select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay(x), intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; -- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); -select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3) settings optimize_move_to_prewhere=0) where explain like '%sort description%'; +select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index 2484001af11..ab4f8f320ec 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -1,67 +1,69 @@ -- EXPLAIN PLAN sorting for MergeTree w/o sorting key --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort --- QUERY: set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=1;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a -MergeSortingTransform -LimitsCheckingTransform -PartialSortingTransform +-- QUERY: EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a +MergingSortedTransform 3 → 1 +MergeSortingTransform × 3 +LimitsCheckingTransform × 3 +PartialSortingTransform × 3 -- enable optimization -> sorting order is propagated from subquery -> merge sort --- QUERY: set optimize_sorting_by_input_stream_properties=1;set max_threads=1;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a +-- QUERY: EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergingSortedTransform 3 → 1 -- enable optimization -> there is no sorting order to propagate from subquery -> full sort --- QUERY: set optimize_sorting_by_input_stream_properties=1;set max_threads=1;EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c -MergeSortingTransform -LimitsCheckingTransform -PartialSortingTransform +-- QUERY: EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c +MergingSortedTransform 3 → 1 +MergeSortingTransform × 3 +LimitsCheckingTransform × 3 +PartialSortingTransform × 3 -- ExpressionStep preserves sort mode --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Chunk): a ASC Sorting (Stream): a ASC --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC -- aliases break sorting order --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC @@ -71,14 +73,14 @@ Sorting (Global): a ASC Sorting (Chunk): a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): x ASC, y ASC Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC Sorting (Chunk): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 +-- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC @@ -88,7 +90,7 @@ Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query --- QUERY: set optimize_read_in_order=1;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a +-- QUERY: EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index 683c073f06d..edcb73adcfd 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -4,9 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -DISABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=1" -ENABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=1;set max_threads=1" -MAKE_OUTPUT_STABLE="set optimize_read_in_order=1" +DISABLE_OPTIMIZATION="--optimize_sorting_by_input_stream_properties=0 --query_plan_read_in_order=0 --max_threads=1" +ENABLE_OPTIMIZATION="--optimize_sorting_by_input_stream_properties=1 --optimize_read_in_order=1 --query_plan_read_in_order=1 --max_threads=1" +MAKE_OUTPUT_STABLE="--optimize_read_in_order=1 --max_threads=3" GREP_SORTING="grep 'PartialSortingTransform\|LimitsCheckingTransform\|MergeSortingTransform\|MergingSortedTransform'" GREP_SORTMODE="grep 'Sorting ('" TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'" @@ -14,19 +14,19 @@ FIND_SORTING="$GREP_SORTING | $TRIM_LEADING_SPACES" FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES" function explain_sorting { - echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTING + echo "-- QUERY:" "${@: -1}" + $CLICKHOUSE_CLIENT "${@:1}" -q "${@: -1}" | eval $FIND_SORTING } function explain_sortmode { - echo "-- QUERY: "$1 - $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTMODE + echo "-- QUERY: " "${@: -1}" + $CLICKHOUSE_CLIENT "${@:1}" -q "${@: -1}" | eval $FIND_SORTMODE } $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" $CLICKHOUSE_CLIENT -q "create table optimize_sorting (a UInt64, b UInt64, c UInt64) engine=MergeTree() order by tuple()" $CLICKHOUSE_CLIENT -q "insert into optimize_sorting values (0, 0, 0) (1, 1, 1)" echo "-- EXPLAIN PLAN sorting for MergeTree w/o sorting key" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" @@ -36,43 +36,43 @@ $CLICKHOUSE_CLIENT -q "insert into optimize_sorting select number, number % 5, n $CLICKHOUSE_CLIENT -q "insert into optimize_sorting SELECT number, number % 5, number % 2 from numbers(20,10)" echo "-- disable optimization -> sorting order is NOT propagated from subquery -> full sort" -explain_sorting "$DISABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" +explain_sorting $DISABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" echo "-- enable optimization -> sorting order is propagated from subquery -> merge sort" -explain_sorting "$ENABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" +explain_sorting $ENABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" echo "-- enable optimization -> there is no sorting order to propagate from subquery -> full sort" -explain_sorting "$ENABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c" +explain_sorting $ENABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c" echo "-- ExpressionStep preserves sort mode" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1" echo "-- ExpressionStep breaks sort mode" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1" echo "-- FilterStep preserves sort mode" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0" echo "-- FilterStep breaks sort mode" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0" echo "-- aliases break sorting order" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a" # FIXME: we still do full sort here, - it's because, for most inner subqueury, sorting description contains original column names but header contains only aliases on those columns: #| Header: x Int32 │ #│ y Int32 │ #│ Sort Mode: Chunk: a ASC, b ASC │ echo "-- aliases DONT break sorting order" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)" echo "-- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1)" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1" echo "-- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query" -explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a" +explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" From a9f8948c8defc7775929b32bed349435ff279e46 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 17:28:41 +0000 Subject: [PATCH 157/526] Fix build --- src/Storages/StorageDelta.cpp | 82 +++++++++++++++++++---------------- src/Storages/StorageDelta.h | 9 ++-- 2 files changed, 49 insertions(+), 42 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 027312385bb..222cbaa5377 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -12,6 +12,9 @@ # include # include +#include + +#include # include # include @@ -168,19 +171,18 @@ void JsonMetadataGetter::handleJSON(const JSON & json) } StorageDelta::StorageDelta( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_) + ContextPtr context_, + std::optional format_settings_) : IStorage(table_id_) - , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) - , table_path(uri_.key) + , table_path(base_configuration.uri.key) + , format_name(configuration_.format) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); @@ -193,32 +195,38 @@ StorageDelta::StorageDelta( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - auto s3_uri = S3::URI(Poco::URI(new_uri)); + + // set new url in configuration + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(format_name_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - s3_uri, - access_key_, - secret_access_key_, - table_id_, - format_name_, - base_configuration.rw_settings, - columns_, - constraints_, - comment, - context_, - std::nullopt); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */false, + nullptr); } Pipe StorageDelta::read( @@ -253,29 +261,29 @@ void registerStorageDelta(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); - String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - String format = "Parquet"; - if (engine_args.size() == 4) - { - format = checkAndGetLiteralArgument(engine_args[3], "format"); - } - - auto s3_uri = S3::URI(Poco::URI(table_url)); + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + + if (configuration.format == "auto") + configuration.format = "Parquet"; + + //auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - s3_uri, - access_key_id, - secret_access_key, + configuration, args.table_id, - format, args.columns, args.constraints, args.comment, - args.getContext()); + args.getContext(), + std::nullopt); + }, { .supports_settings = true, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index c5dc0b2fd07..ff5986ef505 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -64,15 +64,13 @@ class StorageDelta : public IStorage { public: StorageDelta( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_name_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_); + ContextPtr context_, + std::optional format_settings_); String getName() const override { return "DeltaLake"; } @@ -93,6 +91,7 @@ private: std::shared_ptr s3engine; Poco::Logger * log; String table_path; + String format_name; }; } From 8bd6607a8eaca6d8eb55009463f4524d07cbee55 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Nov 2022 17:46:19 +0000 Subject: [PATCH 158/526] Fixing test again. --- ..._input_stream_properties_explain.reference | 32 ++++++------- ...ting_by_input_stream_properties_explain.sh | 46 +++++++++---------- 2 files changed, 39 insertions(+), 39 deletions(-) diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference index ab4f8f320ec..ba924f5daa2 100644 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.reference @@ -1,69 +1,69 @@ -- EXPLAIN PLAN sorting for MergeTree w/o sorting key --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (None) Sorting (None) -- disable optimization -> sorting order is NOT propagated from subquery -> full sort --- QUERY: EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a +-- QUERY: set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=3;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergingSortedTransform 3 → 1 MergeSortingTransform × 3 LimitsCheckingTransform × 3 PartialSortingTransform × 3 -- enable optimization -> sorting order is propagated from subquery -> merge sort --- QUERY: EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a +-- QUERY: set optimize_sorting_by_input_stream_properties=1;set query_plan_read_in_order=1;set optimize_read_in_order=1;set max_threads=3;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a MergingSortedTransform 3 → 1 -- enable optimization -> there is no sorting order to propagate from subquery -> full sort --- QUERY: EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c +-- QUERY: set optimize_sorting_by_input_stream_properties=1;set query_plan_read_in_order=1;set optimize_read_in_order=1;set max_threads=3;EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c MergingSortedTransform 3 → 1 MergeSortingTransform × 3 LimitsCheckingTransform × 3 PartialSortingTransform × 3 -- ExpressionStep preserves sort mode --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC Sorting (Chunk): a ASC Sorting (Stream): a ASC --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- ExpressionStep breaks sort mode --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1 Sorting (Global): plus(a, 1) ASC Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC Sorting (None) Sorting (Chunk): a ASC -- FilterStep preserves sort mode --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (Chunk): a ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- FilterStep breaks sort mode --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0 Sorting (None) Sorting (None) Sorting (Chunk): a ASC -- aliases break sorting order --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC @@ -73,14 +73,14 @@ Sorting (Global): a ASC Sorting (Chunk): a ASC Sorting (Stream): a ASC -- aliases DONT break sorting order --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y) Sorting (Global): x ASC, y ASC Sorting (Sorting for ORDER BY) Sorting (Global): x ASC, y ASC Sorting (Chunk): a ASC, b ASC Sorting (Stream): a ASC, b ASC -- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1) --- QUERY: EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1 Sorting (None) Sorting (Sorting for ORDER BY) Sorting (Global): plus(a, 1) ASC @@ -90,7 +90,7 @@ Sorting (Global): plus(a, 1) ASC Sorting (Chunk): a ASC Sorting (Chunk): a ASC -- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query --- QUERY: EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a +-- QUERY: set optimize_read_in_order=1;set max_threads=3;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a Sorting (Global): a ASC Sorting (Sorting for ORDER BY) Sorting (Global): a ASC diff --git a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh index edcb73adcfd..62051701cb6 100755 --- a/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh +++ b/tests/queries/0_stateless/02377_optimize_sorting_by_input_stream_properties_explain.sh @@ -4,9 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -DISABLE_OPTIMIZATION="--optimize_sorting_by_input_stream_properties=0 --query_plan_read_in_order=0 --max_threads=1" -ENABLE_OPTIMIZATION="--optimize_sorting_by_input_stream_properties=1 --optimize_read_in_order=1 --query_plan_read_in_order=1 --max_threads=1" -MAKE_OUTPUT_STABLE="--optimize_read_in_order=1 --max_threads=3" +DISABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=0;set query_plan_read_in_order=0;set max_threads=3" +ENABLE_OPTIMIZATION="set optimize_sorting_by_input_stream_properties=1;set query_plan_read_in_order=1;set optimize_read_in_order=1;set max_threads=3" +MAKE_OUTPUT_STABLE="set optimize_read_in_order=1;set max_threads=3" GREP_SORTING="grep 'PartialSortingTransform\|LimitsCheckingTransform\|MergeSortingTransform\|MergingSortedTransform'" GREP_SORTMODE="grep 'Sorting ('" TRIM_LEADING_SPACES="sed -e 's/^[ \t]*//'" @@ -14,19 +14,19 @@ FIND_SORTING="$GREP_SORTING | $TRIM_LEADING_SPACES" FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES" function explain_sorting { - echo "-- QUERY:" "${@: -1}" - $CLICKHOUSE_CLIENT "${@:1}" -q "${@: -1}" | eval $FIND_SORTING + echo "-- QUERY: "$1 + $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTING } function explain_sortmode { - echo "-- QUERY: " "${@: -1}" - $CLICKHOUSE_CLIENT "${@:1}" -q "${@: -1}" | eval $FIND_SORTMODE + echo "-- QUERY: "$1 + $CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTMODE } $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" $CLICKHOUSE_CLIENT -q "create table optimize_sorting (a UInt64, b UInt64, c UInt64) engine=MergeTree() order by tuple()" $CLICKHOUSE_CLIENT -q "insert into optimize_sorting values (0, 0, 0) (1, 1, 1)" echo "-- EXPLAIN PLAN sorting for MergeTree w/o sorting key" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" @@ -36,43 +36,43 @@ $CLICKHOUSE_CLIENT -q "insert into optimize_sorting select number, number % 5, n $CLICKHOUSE_CLIENT -q "insert into optimize_sorting SELECT number, number % 5, number % 2 from numbers(20,10)" echo "-- disable optimization -> sorting order is NOT propagated from subquery -> full sort" -explain_sorting $DISABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" +explain_sorting "$DISABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" echo "-- enable optimization -> sorting order is propagated from subquery -> merge sort" -explain_sorting $ENABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" +explain_sorting "$ENABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT a FROM (SELECT a FROM optimize_sorting) ORDER BY a" echo "-- enable optimization -> there is no sorting order to propagate from subquery -> full sort" -explain_sorting $ENABLE_OPTIMIZATION "EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c" +explain_sorting "$ENABLE_OPTIMIZATION;EXPLAIN PIPELINE SELECT c FROM (SELECT c FROM optimize_sorting) ORDER BY c" echo "-- ExpressionStep preserves sort mode" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting ORDER BY a+1" echo "-- ExpressionStep breaks sort mode" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting ORDER BY a+1" echo "-- FilterStep preserves sort mode" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a > 0" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, a+1 FROM optimize_sorting WHERE a+1 > 0" echo "-- FilterStep breaks sort mode" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a > 0 FROM optimize_sorting WHERE a > 0" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a+1 FROM optimize_sorting WHERE a+1 > 0" echo "-- aliases break sorting order" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a FROM (SELECT sipHash64(a) AS a FROM (SELECT a FROM optimize_sorting ORDER BY a)) ORDER BY a" # FIXME: we still do full sort here, - it's because, for most inner subqueury, sorting description contains original column names but header contains only aliases on those columns: #| Header: x Int32 │ #│ y Int32 │ #│ Sort Mode: Chunk: a ASC, b ASC │ echo "-- aliases DONT break sorting order" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, b FROM (SELECT x AS a, y AS b FROM (SELECT a AS x, b AS y FROM optimize_sorting) ORDER BY x, y)" echo "-- actions chain breaks sorting order: input(column a)->sipHash64(column a)->alias(sipHash64(column a), a)->plus(alias a, 1)" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN actions=1, header=1, sorting=1 SELECT a, z FROM (SELECT sipHash64(a) AS a, a + 1 AS z FROM (SELECT a FROM optimize_sorting ORDER BY a + 1)) ORDER BY a + 1" echo "-- check that correct sorting info is provided in case of only prefix of sorting key is in ORDER BY clause but all sorting key columns returned by query" -explain_sortmode $MAKE_OUTPUT_STABLE "EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a" +explain_sortmode "$MAKE_OUTPUT_STABLE;EXPLAIN PLAN sorting=1 SELECT a, b FROM optimize_sorting ORDER BY a" $CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync" From 7b57e94d9f5687bc544d6ad2b0e1d3e692f85425 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 18:14:10 +0000 Subject: [PATCH 159/526] Fix StorageHudi --- src/Storages/StorageHudi.cpp | 79 ++++++++++++++++++------------------ src/Storages/StorageHudi.h | 8 ++-- 2 files changed, 43 insertions(+), 44 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 4aff4ff3a43..d36f0c3c737 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -8,7 +8,7 @@ # include # include # include - +#include # include # include # include @@ -25,34 +25,37 @@ namespace ErrorCodes } StorageHudi::StorageHudi( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_) + ContextPtr context_, + std::optional format_settings_) : IStorage(table_id_) - , base_configuration({uri_, access_key_, secret_access_key_, {}, {}, {}}) + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) - , table_path(uri_.key) + , table_path(base_configuration.uri.key) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), format_); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - auto s3_uri = S3::URI(Poco::URI(new_uri)); - + + StorageS3Configuration new_configuration; + new_configuration.url = new_uri; + new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; + new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.format = configuration_.format; + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(format_, s3_uri, access_key_, secret_access_key_, "", false, {}, context_); + columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else @@ -63,17 +66,15 @@ StorageHudi::StorageHudi( setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - s3_uri, - access_key_, - secret_access_key_, - table_id_, - format_, - base_configuration.rw_settings, - columns_, - constraints_, - comment, - context_, - std::nullopt); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */false, + nullptr); } Pipe StorageHudi::read( @@ -207,28 +208,28 @@ void registerStorageHudi(StorageFactory & factory) "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - String table_url = checkAndGetLiteralArgument(engine_args[0], "url"); - String access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); - String secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - String format = "Parquet"; - if (engine_args.size() == 4) - { - format = checkAndGetLiteralArgument(engine_args[3], "format"); - } - - auto s3_uri = S3::URI(Poco::URI(table_url)); + StorageS3Configuration configuration; + + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); + configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); + configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); + + if (engine_args.size() == 4) + configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + + if (configuration.format == "auto") + configuration.format = "Parquet"; + + auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - s3_uri, - access_key_id, - secret_access_key, + configuration, args.table_id, - format, args.columns, args.constraints, args.comment, - args.getContext()); + args.getContext(), + format_settings); }, { .supports_settings = true, diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 7b647345441..0ed1935a36c 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -24,15 +24,13 @@ class StorageHudi : public IStorage { public: StorageHudi( - const S3::URI & uri_, - const String & access_key_, - const String & secret_access_key_, + const StorageS3Configuration & configuration_, const StorageID & table_id_, - const String & format_, ColumnsDescription columns_, const ConstraintsDescription & constraints_, const String & comment, - ContextPtr context_); + ContextPtr context_, + std::optional format_settings_); String getName() const override { return "Hudi"; } From a573d8aef76e6761d97d0befef9c3563adc08ff0 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Thu, 3 Nov 2022 18:17:39 +0000 Subject: [PATCH 160/526] Apply clang-format --- src/Storages/StorageDelta.cpp | 51 +++++++++++++++-------------------- src/Storages/StorageDelta.h | 1 - src/Storages/StorageHudi.cpp | 47 +++++++++++++++----------------- 3 files changed, 43 insertions(+), 56 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 222cbaa5377..01598996161 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -10,11 +10,11 @@ # include # include +# include # include # include -#include -#include +# include # include # include @@ -182,7 +182,6 @@ StorageDelta::StorageDelta( , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) - , format_name(configuration_.format) { StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); @@ -195,38 +194,39 @@ StorageDelta::StorageDelta( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - + // set new url in configuration StorageS3Configuration new_configuration; new_configuration.url = new_uri; new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; - new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; new_configuration.format = configuration_.format; if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else storage_metadata.setColumns(columns_); - + storage_metadata.setConstraints(constraints_); storage_metadata.setComment(comment); setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - new_configuration, - table_id_, - columns_, - constraints_, - comment, - context_, - format_settings_, - /* distributed_processing_ */false, - nullptr); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); } Pipe StorageDelta::read( @@ -262,28 +262,21 @@ void registerStorageDelta(StorageFactory & factory) "Storage DeltaLake requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); StorageS3Configuration configuration; - + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - if (engine_args.size() == 4) + + if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - + if (configuration.format == "auto") configuration.format = "Parquet"; - + //auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - std::nullopt); - + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); }, { .supports_settings = true, diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index ff5986ef505..3f20a850526 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -91,7 +91,6 @@ private: std::shared_ptr s3engine; Poco::Logger * log; String table_path; - String format_name; }; } diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index d36f0c3c737..57e6461fb72 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -5,10 +5,10 @@ # include # include +# include # include # include # include -#include # include # include # include @@ -46,16 +46,17 @@ StorageHudi::StorageHudi( LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); - + StorageS3Configuration new_configuration; new_configuration.url = new_uri; new_configuration.auth_settings.access_key_id = configuration_.auth_settings.access_key_id; - new_configuration.auth_settings.secret_access_key= configuration_.auth_settings.secret_access_key; + new_configuration.auth_settings.secret_access_key = configuration_.auth_settings.secret_access_key; new_configuration.format = configuration_.format; - + if (columns_.empty()) { - columns_ = StorageS3::getTableStructureFromData(new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); + columns_ = StorageS3::getTableStructureFromData( + new_configuration, /*distributed processing*/ false, format_settings_, context_, nullptr); storage_metadata.setColumns(columns_); } else @@ -66,15 +67,15 @@ StorageHudi::StorageHudi( setInMemoryMetadata(storage_metadata); s3engine = std::make_shared( - new_configuration, - table_id_, - columns_, - constraints_, - comment, - context_, - format_settings_, - /* distributed_processing_ */false, - nullptr); + new_configuration, + table_id_, + columns_, + constraints_, + comment, + context_, + format_settings_, + /* distributed_processing_ */ false, + nullptr); } Pipe StorageHudi::read( @@ -209,27 +210,21 @@ void registerStorageHudi(StorageFactory & factory) StorageS3Configuration configuration; - + configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); configuration.auth_settings.access_key_id = checkAndGetLiteralArgument(engine_args[1], "access_key_id"); configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(engine_args[2], "secret_access_key"); - - if (engine_args.size() == 4) + + if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - + if (configuration.format == "auto") configuration.format = "Parquet"; - + auto format_settings = getFormatSettings(args.getContext()); return std::make_shared( - configuration, - args.table_id, - args.columns, - args.constraints, - args.comment, - args.getContext(), - format_settings); + configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), format_settings); }, { .supports_settings = true, From 51ec95eb82c3e3ae23b4a5ff0d8d7e5284e65c71 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Nov 2022 20:32:32 +0000 Subject: [PATCH 161/526] Try to fix #39157 in a different way. --- src/Interpreters/ExpressionAnalyzer.cpp | 12 ++++----- .../QueryPlan/ReadFromMergeTree.cpp | 26 ++++++++++++++++--- 2 files changed, 29 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 9ac0ea4f5d4..3761fdc6f0c 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1952,12 +1952,12 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( && !query.final() && join_allow_read_in_order; - if (storage && optimize_read_in_order) - { - Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); - additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), - columns_for_sorting_key.begin(), columns_for_sorting_key.end()); - } + // if (storage && optimize_read_in_order) + // { + // Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); + // additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), + // columns_for_sorting_key.begin(), columns_for_sorting_key.end()); + // } /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 20a9ddaa763..d16e0a8904d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -441,6 +441,21 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (info.sum_marks == 0) return {}; + bool added_prewhere_output = false; + if (prewhere_info && prewhere_info->prewhere_actions) + { + auto & outputs = prewhere_info->prewhere_actions->getOutputs(); + std::unordered_set outputs_set(outputs.begin(), outputs.end()); + for (const auto * input : prewhere_info->prewhere_actions->getInputs()) + { + if (!outputs_set.contains(input)) + { + outputs.push_back(input); + added_prewhere_output = true; + } + } + } + /// Let's split ranges to avoid reading much data. auto split_ranges = [rows_granularity = data_settings->index_granularity, max_block_size = max_block_size] (const auto & ranges, int direction) @@ -567,6 +582,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( info.use_uncompressed_cache, input_order_info->limit)); } + Block pipe_header; + if (!pipes.empty()) + pipe_header = pipes.front().getHeader(); + if (need_preliminary_merge) { size_t prefix_size = input_order_info->used_prefix_of_sorting_key_size; @@ -588,9 +607,6 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( for (auto & pipe : pipes) { - /// Drop temporary columns, added by 'sorting_key_prefix_expr' - out_projection = createProjection(pipe.getHeader()); - pipe.addSimpleTransform([sorting_key_expr](const Block & header) { return std::make_shared(header, sorting_key_expr); @@ -610,6 +626,10 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } } + if (!pipes.empty() && (need_preliminary_merge || added_prewhere_output)) + /// Drop temporary columns, added by 'sorting_key_prefix_expr' + out_projection = createProjection(pipe_header); + return Pipe::unitePipes(std::move(pipes)); } From cdd09fd72a4378cade1586394faa3a8465e72809 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 19:23:19 +0100 Subject: [PATCH 162/526] Improve messages for loading data parts (add part type) Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 83e87a0e462..bb12f3a62a6 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1019,13 +1019,14 @@ void MergeTreeData::loadDataPartsFromDisk( if (!part_opt) return; - LOG_TRACE(log, "Loading part {} from disk {}", part_name, part_disk_ptr->getName()); const auto & part_info = *part_opt; auto single_disk_volume = std::make_shared("volume_" + part_name, part_disk_ptr, 0); auto data_part_storage = std::make_shared(single_disk_volume, relative_data_path, part_name); auto part = createPart(part_name, part_info, data_part_storage); bool broken = false; + LOG_TRACE(log, "Loading part {} ({}) from disk {}", part_name, part->getType().toString(), part_disk_ptr->getName()); + String part_path = fs::path(relative_data_path) / part_name; String marker_path = fs::path(part_path) / IMergeTreeDataPart::DELETE_ON_DESTROY_MARKER_FILE_NAME; if (part_disk_ptr->exists(marker_path)) From 7032b2b98a48a876d7d8af1f465fc7331b4af293 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 19:35:16 +0100 Subject: [PATCH 163/526] Attach thread pool for loading parts to the query Detach should be done via CurrentThread::defaultThreadDeleter() Signed-off-by: Azat Khuzhin --- src/Storages/MergeTree/MergeTreeData.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb12f3a62a6..51d8097c1db 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1131,8 +1131,11 @@ void MergeTreeData::loadDataPartsFromDisk( { for (size_t thread = 0; thread < num_threads; ++thread) { - pool.scheduleOrThrowOnError([&, thread] + pool.scheduleOrThrowOnError([&, thread, thread_group = CurrentThread::getGroup()] { + if (thread_group) + CurrentThread::attachToIfDetached(thread_group); + while (true) { std::pair thread_part; From e773eb2fd17c84620738ef9cd16561e2a1764d80 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Fri, 4 Nov 2022 17:19:34 +0800 Subject: [PATCH 164/526] batter --- .../MergeTree/MergeTreeDataMergerMutator.cpp | 23 +++++++++++++------ 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 879cd2fa3da..c7d1786135a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -303,19 +303,28 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) { /// TTL delete is preferred to recompression - TTLDeleteMergeSelector delete_ttl_selector( + TTLDeleteMergeSelector drop_ttl_selector( next_delete_ttl_merge_times_by_partition, current_time, data_settings->merge_with_ttl_timeout, - data_settings->ttl_only_drop_parts); - - parts_to_merge = delete_ttl_selector.select( - parts_ranges, - data_settings->ttl_only_drop_parts ? data_settings->max_bytes_to_merge_at_max_space_in_pool : max_total_size_to_merge); + true); + parts_to_merge = drop_ttl_selector.select(parts_ranges,data_settings->max_bytes_to_merge_at_max_space_in_pool); if (!parts_to_merge.empty()) { - future_part->merge_type = data_settings->ttl_only_drop_parts ? MergeType::TTLDrop : MergeType::TTLDelete; + future_part->merge_type = MergeType::TTLDrop; + } + else if (!data_settings->ttl_only_drop_parts) + { + TTLDeleteMergeSelector delete_ttl_selector( + next_delete_ttl_merge_times_by_partition, + current_time, + data_settings->merge_with_ttl_timeout, + false); + + parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); + if (!parts_to_merge.empty()) + future_part->merge_type = MergeType::TTLDelete; } else if (metadata_snapshot->hasAnyRecompressionTTL()) { From 4db8389f7d24f2cab9a8090199f8288d5b7b81be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 Nov 2022 09:40:51 +0000 Subject: [PATCH 165/526] Fixnig test. --- ...plan_optimizations_optimize_read_in_window_order_long.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh index 297688a29c3..af053fec052 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh @@ -19,13 +19,13 @@ $CLICKHOUSE_CLIENT -q "create table ${name}_n_x engine=MergeTree order by (n, x) $CLICKHOUSE_CLIENT -q "optimize table ${name}_n final" $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n+x%2 ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' From cf05ac84dc74adf3daa4370625d3082ce6a6ff91 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 4 Nov 2022 11:12:45 +0100 Subject: [PATCH 166/526] Add no-fasttest tag --- .../0_stateless/02473_extract_low_cardinality_from_json.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql index 0a0ae398227..664c52e772f 100644 --- a/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql +++ b/tests/queries/0_stateless/02473_extract_low_cardinality_from_json.sql @@ -1,2 +1,3 @@ +-- Tags: no-fasttest SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a LowCardinality(String), b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); -SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a String, b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); \ No newline at end of file +SELECT JSONExtract('{"a" : {"b" : {"c" : 1, "d" : "str"}}}', 'Tuple( a String, b LowCardinality(String), c LowCardinality(String), d LowCardinality(String))'); From ecf45bb2165af6e9691a7c57f1993b2b60b393a5 Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Fri, 4 Nov 2022 20:52:21 +0800 Subject: [PATCH 167/526] fix test --- src/Interpreters/PartLog.cpp | 1 + .../0_stateless/02293_part_log_has_merge_reason.reference | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index 6797d6b3a41..f1b29a8a9b2 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -74,6 +74,7 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() {"RegularMerge", static_cast(REGULAR_MERGE)}, {"TTLDeleteMerge", static_cast(TTL_DELETE_MERGE)}, {"TTLRecompressMerge", static_cast(TTL_RECOMPRESS_MERGE)}, + {"TTLDropMerge", static_cast(TTL_DROP_MERGE)}, } ); diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference index 220107cf15b..37156ee37e7 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -1 +1 @@ -MergeParts TTLDeleteMerge +MergeParts TTLDropMerge From dd709faa143f6d568ce689a4e26a1dc4a76dddd7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 13:04:29 +0000 Subject: [PATCH 168/526] Create Keeper on additional node --- .../resources/users.xml | 110 ----------- .../src/jepsen/clickhouse/nemesis.clj | 23 --- .../src/jepsen/clickhouse/server/db.clj | 56 ------ .../src/jepsen/clickhouse/server/nemesis.clj | 8 - .../.gitignore | 0 .../LICENSE | 0 .../README.md | 0 .../doc/intro.md | 0 .../project.clj | 0 .../resources/config.xml | 2 - .../resources/keeper_config.xml | 0 .../resources/keeper_config_solo.xml | 34 ++++ .../resources/replicated_merge_tree.xml | 45 +---- tests/jepsen.clickhouse/resources/users.xml | 16 ++ .../resources/zoo.cfg | 0 .../src/jepsen/clickhouse/constants.clj | 0 .../src/jepsen/clickhouse/keeper/bench.clj | 0 .../src/jepsen/clickhouse/keeper/counter.clj | 18 +- .../src/jepsen/clickhouse/keeper/db.clj | 0 .../src/jepsen/clickhouse/keeper/main.clj | 0 .../src/jepsen/clickhouse/keeper/nemesis.clj | 46 +---- .../src/jepsen/clickhouse/keeper/queue.clj | 0 .../src/jepsen/clickhouse/keeper/register.clj | 3 +- .../src/jepsen/clickhouse/keeper/set.clj | 0 .../src/jepsen/clickhouse/keeper/unique.clj | 0 .../src/jepsen/clickhouse/keeper/utils.clj | 4 +- .../jepsen/clickhouse/keeper/zookeeperdb.clj | 0 .../src/jepsen/clickhouse/main.clj | 0 .../src/jepsen/clickhouse/nemesis.clj | 59 ++++++ .../src/jepsen/clickhouse/server/client.clj | 0 .../src/jepsen/clickhouse/server/db.clj | 185 ++++++++++++++++++ .../src/jepsen/clickhouse/server/main.clj | 9 +- .../src/jepsen/clickhouse/server/nemesis.clj | 22 +++ .../src/jepsen/clickhouse/server/set.clj | 5 +- .../src/jepsen/clickhouse/server/utils.clj | 0 .../src/jepsen/clickhouse/utils.clj | 8 +- .../test/jepsen/keeper_test.clj | 0 37 files changed, 349 insertions(+), 304 deletions(-) delete mode 100644 tests/jepsen.clickhouse-keeper/resources/users.xml delete mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj delete mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj delete mode 100644 tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/.gitignore (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/LICENSE (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/README.md (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/doc/intro.md (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/project.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/resources/config.xml (73%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/resources/keeper_config.xml (100%) create mode 100644 tests/jepsen.clickhouse/resources/keeper_config_solo.xml rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/resources/replicated_merge_tree.xml (50%) create mode 100644 tests/jepsen.clickhouse/resources/users.xml rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/resources/zoo.cfg (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/constants.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/bench.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/counter.clj (72%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/db.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/main.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/nemesis.clj (77%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/queue.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/register.clj (96%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/set.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/unique.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/utils.clj (97%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/keeper/zookeeperdb.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/main.clj (100%) create mode 100644 tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/server/client.clj (100%) create mode 100644 tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/server/main.clj (92%) create mode 100644 tests/jepsen.clickhouse/src/jepsen/clickhouse/server/nemesis.clj rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/server/set.clj (90%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/server/utils.clj (100%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/src/jepsen/clickhouse/utils.clj (97%) rename tests/{jepsen.clickhouse-keeper => jepsen.clickhouse}/test/jepsen/keeper_test.clj (100%) diff --git a/tests/jepsen.clickhouse-keeper/resources/users.xml b/tests/jepsen.clickhouse-keeper/resources/users.xml deleted file mode 100644 index 5e2ff51bf4d..00000000000 --- a/tests/jepsen.clickhouse-keeper/resources/users.xml +++ /dev/null @@ -1,110 +0,0 @@ - - - - - - - - - - - - 1 - - - - - - - - - - - - - ::/0 - - - - default - - - default - - - - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj deleted file mode 100644 index 116fd1648b2..00000000000 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/nemesis.clj +++ /dev/null @@ -1,23 +0,0 @@ -(ns jepsen.clickhouse.nemesis - (:require - [clojure.tools.logging :refer :all] - [jepsen - [nemesis :as nemesis] - [generator :as gen]])) - -(defn random-node-hammer-time-nemesis - [] - (nemesis/hammer-time "clickhouse")) - -(defn all-nodes-hammer-time-nemesis - [] - (nemesis/hammer-time identity "clickhouse")) - -(defn start-stop-generator - [time-corrupt time-ok] - (->> - (cycle [(gen/sleep time-ok) - {:type :info, :f :start} - (gen/sleep time-corrupt) - {:type :info, :f :stop}]))) - diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj deleted file mode 100644 index e38c3c93759..00000000000 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/db.clj +++ /dev/null @@ -1,56 +0,0 @@ -(ns jepsen.control.scp) - -;; We need to overwrite Jepsen's implementation of scp! because it -;; doesn't use strict-host-key-checking - -(defn scp! - "Runs an SCP command by shelling out. Takes a conn-spec (used for port, key, - etc), a seq of sources, and a single destination, all as strings." - [conn-spec sources dest] - (apply util/sh "scp" "-rpC" - "-P" (str (:port conn-spec)) - (concat (when-let [k (:private-key-path conn-spec)] - ["-i" k]) - (if-not (:strict-host-key-checking conn-spec) - ["-o StrictHostKeyChecking=no"]) - sources - [dest])) - nil) - -(ns jepsen.clickhouse.keeper.db) - -(ns jepsen.clickhouse.server.db - (:require [clojure.tools.logging :refer :all] - [clojure.java.io :as io] - [jepsen - [control :as c] - [db :as db]] - [jepsen.clickhouse.constants :refer :all] - [jepsen.clickhouse.server.utils :refer :all] - [jepsen.clickhouse.utils :as chu])) - -(defn replicated-merge-tree-config - [test node config-template] - (let [nodes (:nodes test) - replacement-map {#"\{server1\}" (get nodes 0) - #"\{server2\}" (get nodes 1) - #"\{server3\}" (get nodes 2) - #"\{server_id\}" (str (inc (.indexOf nodes node))) - #"\{replica_name\}" node}] - (reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map))) - -(defn install-configs - [test node] - (c/exec :echo (slurp (io/resource "config.xml")) :> (str configs-dir "/config.xml")) - (c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml")) - (c/exec :echo (replicated-merge-tree-config test node (slurp (io/resource "replicated_merge_tree.xml"))) :> (str sub-configs-dir "/replicated_merge_tree.xml"))) - -(defn extra-setup - [test node] - (do - (info "Installing configs") - (install-configs test node))) - -(defn db - [version reuse-binary] - (chu/db version reuse-binary start-clickhouse! extra-setup)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj b/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj deleted file mode 100644 index faf16386526..00000000000 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/nemesis.clj +++ /dev/null @@ -1,8 +0,0 @@ -(ns jepsen.clickhouse.server.nemesis - (:require [jepsen.clickhouse.nemesis :as chnem])) - -(def custom-nemeses - {"random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) - :generator (chnem/start-stop-generator 5 5)} - "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) - :generator (chnem/start-stop-generator 1 10)}}) diff --git a/tests/jepsen.clickhouse-keeper/.gitignore b/tests/jepsen.clickhouse/.gitignore similarity index 100% rename from tests/jepsen.clickhouse-keeper/.gitignore rename to tests/jepsen.clickhouse/.gitignore diff --git a/tests/jepsen.clickhouse-keeper/LICENSE b/tests/jepsen.clickhouse/LICENSE similarity index 100% rename from tests/jepsen.clickhouse-keeper/LICENSE rename to tests/jepsen.clickhouse/LICENSE diff --git a/tests/jepsen.clickhouse-keeper/README.md b/tests/jepsen.clickhouse/README.md similarity index 100% rename from tests/jepsen.clickhouse-keeper/README.md rename to tests/jepsen.clickhouse/README.md diff --git a/tests/jepsen.clickhouse-keeper/doc/intro.md b/tests/jepsen.clickhouse/doc/intro.md similarity index 100% rename from tests/jepsen.clickhouse-keeper/doc/intro.md rename to tests/jepsen.clickhouse/doc/intro.md diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse/project.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/project.clj rename to tests/jepsen.clickhouse/project.clj diff --git a/tests/jepsen.clickhouse-keeper/resources/config.xml b/tests/jepsen.clickhouse/resources/config.xml similarity index 73% rename from tests/jepsen.clickhouse-keeper/resources/config.xml rename to tests/jepsen.clickhouse/resources/config.xml index a0b1b33b26d..25e38a58e63 100644 --- a/tests/jepsen.clickhouse-keeper/resources/config.xml +++ b/tests/jepsen.clickhouse/resources/config.xml @@ -5,8 +5,6 @@ trace - /var/log/clickhouse-keeper/clickhouse-keeper.log - /var/log/clickhouse-keeper/clickhouse-keeper.err.log never diff --git a/tests/jepsen.clickhouse-keeper/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml similarity index 100% rename from tests/jepsen.clickhouse-keeper/resources/keeper_config.xml rename to tests/jepsen.clickhouse/resources/keeper_config.xml diff --git a/tests/jepsen.clickhouse/resources/keeper_config_solo.xml b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml new file mode 100644 index 00000000000..dfbc098b3dd --- /dev/null +++ b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml @@ -0,0 +1,34 @@ + + 0.0.0.0 + + + trace + /var/log/clickhouse-keeper/clickhouse-keeper.log + /var/log/clickhouse-keeper/clickhouse-keeper.err.log + never + + + + 9181 + 1 + + + 10000 + 30000 + false + 120000 + trace + 1000 + 2000 + 4000 + + + + + 1 + {srv1} + 9444 + + + + diff --git a/tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml b/tests/jepsen.clickhouse/resources/replicated_merge_tree.xml similarity index 50% rename from tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml rename to tests/jepsen.clickhouse/resources/replicated_merge_tree.xml index 43392c8da5b..d089c019f4a 100644 --- a/tests/jepsen.clickhouse-keeper/resources/replicated_merge_tree.xml +++ b/tests/jepsen.clickhouse/resources/replicated_merge_tree.xml @@ -13,52 +13,9 @@ never - - 9181 - {server_id} - - - 10000 - 30000 - false - 120000 - trace - 1000 - 2000 - 4000 - false - - - - - 1 - {server1} - 9444 - - - 2 - {server2} - 9444 - - - 3 - {server3} - 9444 - - - - - {server1} - 9181 - - - {server2} - 9181 - - - {server3} + {keeper} 9181 diff --git a/tests/jepsen.clickhouse/resources/users.xml b/tests/jepsen.clickhouse/resources/users.xml new file mode 100644 index 00000000000..098dc6ac388 --- /dev/null +++ b/tests/jepsen.clickhouse/resources/users.xml @@ -0,0 +1,16 @@ + + + + + + + + + + + ::/0 + + default + + + diff --git a/tests/jepsen.clickhouse-keeper/resources/zoo.cfg b/tests/jepsen.clickhouse/resources/zoo.cfg similarity index 100% rename from tests/jepsen.clickhouse-keeper/resources/zoo.cfg rename to tests/jepsen.clickhouse/resources/zoo.cfg diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/constants.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/constants.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/bench.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/bench.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/bench.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/bench.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/counter.clj similarity index 72% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/counter.clj index 0e2e97670f9..bd497c2f0a6 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/counter.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/counter.clj @@ -28,10 +28,14 @@ (invoke! [this test op] (case (:f op) - :read (chu/exec-with-retries 30 (fn [] - (assoc op - :type :ok - :value (count (zk-list conn root-path))))) + :read (try + (assoc op + :type :ok + :value (count (zk-list conn root-path))) + (catch Exception _ (assoc op :type :info, :error :connect-error))) + :final-read (chu/exec-with-retries 30 (fn [] (assoc op + :type :ok + :value (count (zk-list conn root-path))))) :add (try (do (zk-multi-create-many-seq-nodes conn (concat-path root-path "seq-") (:value op)) @@ -50,7 +54,5 @@ :checker (checker/compose {:counter (checker/counter) :perf (checker/perf)}) - :generator (->> (range) - (map (fn [x] - (->> (gen/mix [r add]))))) - :final-generator (gen/once {:type :invoke, :f :read, :value nil})}) + :generator (gen/mix [r add]) + :final-generator (gen/once {:type :invoke, :f :final-read, :value nil})}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/db.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/db.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/db.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/main.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/main.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/main.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/nemesis.clj similarity index 77% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/nemesis.clj index 6e54d1eee84..6f0350ac2b5 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/nemesis.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/nemesis.clj @@ -10,20 +10,6 @@ [jepsen.clickhouse.utils :as chu] [jepsen.clickhouse.keeper.utils :refer :all])) -(defn random-node-killer-nemesis - [] - (nemesis/node-start-stopper - rand-nth - (fn start [test node] (chu/kill-clickhouse! node test)) - (fn stop [test node] (start-clickhouse! node test)))) - -(defn all-nodes-killer-nemesis - [] - (nemesis/node-start-stopper - identity - (fn start [test node] (chu/kill-clickhouse! node test)) - (fn stop [test node] (start-clickhouse! node test)))) - (defn select-last-file [path] (last (clojure.string/split @@ -87,28 +73,6 @@ (corruptor-nemesis coordination-data-dir (fn [path] (c/exec :rm :-fr path)))) -(defn partition-bridge-nemesis - [] - (nemesis/partitioner nemesis/bridge)) - -(defn blind-node - [nodes] - (let [[[victim] others] (nemesis/split-one nodes)] - {victim (into #{} others)})) - -(defn blind-node-partition-nemesis - [] - (nemesis/partitioner blind-node)) - -(defn blind-others - [nodes] - (let [[[victim] others] (nemesis/split-one nodes)] - (into {} (map (fn [node] [node #{victim}])) others))) - -(defn blind-others-partition-nemesis - [] - (nemesis/partitioner blind-others)) - (defn network-non-symmetric-nemesis [] (nemesis/partitioner nemesis/bridge)) @@ -120,9 +84,9 @@ {:type :info, :f :corrupt}]))) (def custom-nemesises - {"random-node-killer" {:nemesis (random-node-killer-nemesis) + {"random-node-killer" {:nemesis (chnem/random-node-killer-nemesis start-clickhouse!) :generator (chnem/start-stop-generator 5 5)} - "all-nodes-killer" {:nemesis (all-nodes-killer-nemesis) + "all-nodes-killer" {:nemesis (chnem/all-nodes-killer-nemesis start-clickhouse!) :generator (chnem/start-stop-generator 1 10)} "simple-partitioner" {:nemesis (nemesis/partition-random-halves) :generator (chnem/start-stop-generator 5 5)} @@ -138,9 +102,9 @@ :generator (corruption-generator)} "drop-data-corruptor" {:nemesis (drop-all-corruption-nemesis) :generator (corruption-generator)} - "bridge-partitioner" {:nemesis (partition-bridge-nemesis) + "bridge-partitioner" {:nemesis (chnem/partition-bridge-nemesis) :generator (chnem/start-stop-generator 5 5)} - "blind-node-partitioner" {:nemesis (blind-node-partition-nemesis) + "blind-node-partitioner" {:nemesis (chnem/blind-node-partition-nemesis) :generator (chnem/start-stop-generator 5 5)} - "blind-others-partitioner" {:nemesis (blind-others-partition-nemesis) + "blind-others-partitioner" {:nemesis (chnem/blind-others-partition-nemesis) :generator (chnem/start-stop-generator 5 5)}}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/queue.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/queue.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/queue.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/register.clj similarity index 96% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/register.clj index 72cf113a9c3..65e51a67382 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/register.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/register.clj @@ -21,7 +21,8 @@ (assoc this :conn (zk-connect node 9181 30000))) (setup! [this test] - (zk-create-range conn 300)) ; 300 nodes to be sure + (chu/exec-with-retries 30 (fn [] + (zk-create-range conn 300)))) (invoke! [_ test op] (let [[k v] (:value op) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/set.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/set.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/set.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/unique.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/unique.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/unique.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/unique.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj similarity index 97% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj index 6767c144e01..b882af77758 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/utils.clj @@ -25,7 +25,7 @@ (defn zk-connect [host port timeout] - (chu/exec-with-retries 30 (fn [] (zk/connect (str host ":" port) :timeout-msec timeout)))) + (zk/connect (str host ":" port) :timeout-msec timeout)) (defn zk-create-range [conn n] @@ -129,7 +129,7 @@ (defn clickhouse-alive? [node test] - (info "Checking server alive on" node) + (info "Checking Keeper alive on" node) (try (zk-connect (name node) 9181 30000) (catch Exception _ false))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/zookeeperdb.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/zookeeperdb.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/keeper/zookeeperdb.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/keeper/zookeeperdb.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/main.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/main.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/main.clj diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj new file mode 100644 index 00000000000..4f912aa1b7c --- /dev/null +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/nemesis.clj @@ -0,0 +1,59 @@ +(ns jepsen.clickhouse.nemesis + (:require + [clojure.tools.logging :refer :all] + [jepsen.clickhouse.utils :as chu] + [jepsen + [nemesis :as nemesis] + [generator :as gen]])) + +(defn random-node-hammer-time-nemesis + [] + (nemesis/hammer-time "clickhouse")) + +(defn all-nodes-hammer-time-nemesis + [] + (nemesis/hammer-time identity "clickhouse")) + +(defn start-stop-generator + [time-corrupt time-ok] + (->> + (cycle [(gen/sleep time-ok) + {:type :info, :f :start} + (gen/sleep time-corrupt) + {:type :info, :f :stop}]))) + +(defn random-node-killer-nemesis + [start-clickhouse!] + (nemesis/node-start-stopper + rand-nth + (fn start [test node] (chu/kill-clickhouse! node test)) + (fn stop [test node] (start-clickhouse! node test)))) + +(defn all-nodes-killer-nemesis + [start-clickhouse!] + (nemesis/node-start-stopper + identity + (fn start [test node] (chu/kill-clickhouse! node test)) + (fn stop [test node] (start-clickhouse! node test)))) + +(defn partition-bridge-nemesis + [] + (nemesis/partitioner nemesis/bridge)) + +(defn blind-node + [nodes] + (let [[[victim] others] (nemesis/split-one nodes)] + {victim (into #{} others)})) + +(defn blind-node-partition-nemesis + [] + (nemesis/partitioner blind-node)) + +(defn blind-others + [nodes] + (let [[[victim] others] (nemesis/split-one nodes)] + (into {} (map (fn [node] [node #{victim}])) others))) + +(defn blind-others-partition-nemesis + [] + (nemesis/partitioner blind-others)) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/client.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj new file mode 100644 index 00000000000..1c35831d80b --- /dev/null +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/db.clj @@ -0,0 +1,185 @@ +(ns jepsen.control.scp) + +;; We need to overwrite Jepsen's implementation of scp! because it +;; doesn't use strict-host-key-checking + +(defn scp! + "Runs an SCP command by shelling out. Takes a conn-spec (used for port, key, + etc), a seq of sources, and a single destination, all as strings." + [conn-spec sources dest] + (apply util/sh "scp" "-rpC" + "-P" (str (:port conn-spec)) + (concat (when-let [k (:private-key-path conn-spec)] + ["-i" k]) + (if-not (:strict-host-key-checking conn-spec) + ["-o StrictHostKeyChecking=no"]) + sources + [dest])) + nil) + +(ns jepsen.clickhouse.keeper.db) + +(ns jepsen.clickhouse.server.db + (:require [clojure.tools.logging :refer :all] + [clojure.java.io :as io] + [clojure.string :as str] + [zookeeper :as zk] + [jepsen + [control :as c] + [store :as store] + [core :as core] + [os :as os] + [db :as db]] + [jepsen.control.util :as cu] + [jepsen.clickhouse.constants :refer :all] + [jepsen.clickhouse.server.utils :refer :all] + [jepsen.clickhouse.keeper.utils :as keeperutils] + [jepsen.clickhouse.utils :as chu])) + +(defn replicated-merge-tree-config + [test node config-template] + (let [nodes (:nodes test) + replacement-map {#"\{server1\}" (get nodes 0) + #"\{server2\}" (get nodes 1) + #"\{server3\}" (get nodes 2) + #"\{keeper\}" (:keeper test) + #"\{replica_name\}" node}] + (reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map))) + +(defn install-configs + [test node] + (c/exec :echo (slurp (io/resource "config.xml")) :> (str configs-dir "/config.xml")) + (c/exec :echo (slurp (io/resource "users.xml")) :> (str configs-dir "/users.xml")) + (c/exec :echo (replicated-merge-tree-config test node (slurp (io/resource "replicated_merge_tree.xml"))) :> (str sub-configs-dir "/replicated_merge_tree.xml"))) + +(defn extra-setup + [test node] + (do + (info "Installing configs") + (install-configs test node))) + +(defn keeper-config + [test node config-template] + (let [replacement-map {#"\{srv1\}" node}] + (reduce #(clojure.string/replace %1 (get %2 0) (get %2 1)) config-template replacement-map))) + +(defn install-keeper-configs + [test node] + (c/exec :echo (keeper-config test node (slurp (io/resource "keeper_config_solo.xml"))) :> (str configs-dir "/keeper_config.xml"))) + +(defn keeper + [version reuse-binary] + (chu/db version reuse-binary keeperutils/start-clickhouse! install-keeper-configs)) + +(defn snarf-keeper-logs! + "Downloads Keeper logs" + [test] + ; Download logs + (let [keeper-node (:keeper test)] + (info "Snarfing Keeper log files") + (c/on keeper-node + (doseq [[remote local] (db/log-files-map (:db test) test keeper-node)] + (when (cu/exists? remote) + (info "downloading" remote "to" local) + (try + (c/download + remote + (.getCanonicalPath + (store/path! test (name keeper-node) + ; strip leading / + (str/replace local #"^/" "")))) + (catch java.io.IOException e + (if (= "Pipe closed" (.getMessage e)) + (info remote "pipe closed") + (throw e))) + (catch java.lang.IllegalArgumentException e + ; This is a jsch bug where the file is just being + ; created + (info remote "doesn't exist")))))))) + +(defn is-primary + "Is node primary" + [test node] + (= 0 (.indexOf (:nodes test) node))) + +(defn zk-connect + [host port timeout] + (let [conn (zk/connect (str host ":" port) :timeout-msec timeout) + sessionId (.getSessionId conn)] + (when (= -1 sessionId) + (throw (RuntimeException. + (str "Connection to " host " failed")))) + conn)) + +(defn keeper-alive? + [node test] + (info "Checking Keeper alive on" node) + (try + (zk-connect (name node) 9181 30000) + (catch Exception _ false))) + +(defn db + [version reuse-binary] + (reify db/DB + (setup! [this test node] + (let [keeper-node (:keeper test)] + (when (is-primary test node) + (info (str "Starting Keeper on " keeper-node)) + (c/on keeper-node + (os/setup! (:os test) test keeper-node) + (db/setup! (keeper version reuse-binary) test keeper-node))) + (c/su + (do + (info "Preparing directories") + (chu/prepare-dirs) + (if (or (not (cu/exists? binary-path)) (not reuse-binary)) + (do (info "Downloading clickhouse") + (let [clickhouse-path (chu/download-clickhouse version)] + (chu/install-downloaded-clickhouse clickhouse-path))) + (info "Binary already exsist on path" binary-path "skipping download")) + (extra-setup test node) + (info "Waiting for Keeper") + (chu/wait-clickhouse-alive! keeper-node test keeper-alive?) + (info "Starting server") + (start-clickhouse! node test) + (info "ClickHouse started"))))) + + (teardown! [_ test node] + (let [keeper-node (:keeper test)] + (when (is-primary test node) + (info (str "Tearing down Keeper on " keeper-node)) + (c/on keeper-node + (db/teardown! (keeper version reuse-binary) test keeper-node)) + (os/teardown! (:os test) test keeper-node))) + (info node "Tearing down clickhouse") + (c/su + (chu/kill-clickhouse! node test) + (if (not reuse-binary) + (c/exec :rm :-rf binary-path)) + (c/exec :rm :-rf pid-file-path) + (c/exec :rm :-rf data-dir) + (c/exec :rm :-rf logs-dir) + (c/exec :rm :-rf configs-dir))) + + db/LogFiles + (log-files [_ test node] + (when (is-primary test node) + (info "Downloading Keeper logs") + (snarf-keeper-logs! test)) + (c/su + (chu/kill-clickhouse! node test) + (if (cu/exists? data-dir) + (do + (info node "Data folder exists, going to compress") + (c/cd root-folder + (c/exec :tar :czf "data.tar.gz" "db")))) + (if (cu/exists? (str logs-dir)) + (do + (info node "Logs exist, going to compress") + (c/cd root-folder + (c/exec :tar :czf "logs.tar.gz" "logs"))) (info node "Logs are missing"))) + (let [common-logs [(str root-folder "/logs.tar.gz") (str root-folder "/data.tar.gz")] + gdb-log (str logs-dir "/gdb.log")] + (if (cu/exists? (str logs-dir "/gdb.log")) + (conj common-logs gdb-log) + common-logs))))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/main.clj similarity index 92% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/server/main.clj index 09e395ce183..4b5a9babe7d 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/main.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/main.clj @@ -17,7 +17,9 @@ [jepsen.control.util :as cu] [jepsen.os.ubuntu :as ubuntu] [jepsen.checker.timeline :as timeline] - [clojure.java.io :as io])) + [clojure.java.io :as io]) + (:import (ch.qos.logback.classic Level) + (org.slf4j Logger LoggerFactory))) (def workloads "A map of workload names to functions that construct workloads, given opts." @@ -28,6 +30,9 @@ [["-w" "--workload NAME" "What workload should we run?" :default "set" :validate [workloads (cli/one-of workloads)]] + [nil "--keeper ADDRESS", "Address of a Keeper instance" + :default "" + :validate [#(not-empty %) "Address for Keeper cannot be empty"]] [nil "--nemesis NAME" "Which nemesis will poison our lives?" :default "random-node-killer" :validate [ch-nemesis/custom-nemeses (cli/one-of ch-nemesis/custom-nemeses)]] @@ -99,6 +104,8 @@ "Handles command line arguments. Can either run a test, or a web server for browsing results." [& args] + (.setLevel + (LoggerFactory/getLogger "org.apache.zookeeper") Level/OFF) (cli/run! (merge (cli/single-test-cmd {:test-fn clickhouse-server-test :opt-spec cli-opts}) (cli/test-all-cmd {:tests-fn (partial all-tests clickhouse-server-test) diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/nemesis.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/nemesis.clj new file mode 100644 index 00000000000..542be7d7a0a --- /dev/null +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/nemesis.clj @@ -0,0 +1,22 @@ +(ns jepsen.clickhouse.server.nemesis + (:require [jepsen.clickhouse.nemesis :as chnem] + [jepsen.clickhouse.server.utils :refer :all] + [jepsen.nemesis :as nemesis])) + +(def custom-nemeses + {"random-node-killer" {:nemesis (chnem/random-node-killer-nemesis start-clickhouse!) + :generator (chnem/start-stop-generator 5 5)} + "all-nodes-killer" {:nemesis (chnem/all-nodes-killer-nemesis start-clickhouse!) + :generator (chnem/start-stop-generator 1 10)} + "simple-partitioner" {:nemesis (nemesis/partition-random-halves) + :generator (chnem/start-stop-generator 5 5)} + "random-node-hammer-time" {:nemesis (chnem/random-node-hammer-time-nemesis) + :generator (chnem/start-stop-generator 5 5)} + "all-nodes-hammer-time" {:nemesis (chnem/all-nodes-hammer-time-nemesis) + :generator (chnem/start-stop-generator 1 10)} + "bridge-partitioner" {:nemesis (chnem/partition-bridge-nemesis) + :generator (chnem/start-stop-generator 5 5)} + "blind-node-partitioner" {:nemesis (chnem/blind-node-partition-nemesis) + :generator (chnem/start-stop-generator 5 5)} + "blind-others-partitioner" {:nemesis (chnem/blind-others-partition-nemesis) + :generator (chnem/start-stop-generator 5 5)}}) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/set.clj similarity index 90% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/server/set.clj index e624db1e5c5..05543a8f343 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/set.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/set.clj @@ -34,10 +34,7 @@ (mapv :value) (assoc op :type :ok, :value)))))) - (teardown! [_ test] - (util/timeout chc/operation-timeout - (chc/with-connection [c conn] false - (j/query c ["DROP TABLE set ON CLUSTER test_cluster"])))) + (teardown! [_ test]) (close! [_ test] (rc/close! conn))) diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/server/utils.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj diff --git a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj similarity index 97% rename from tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj rename to tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj index 9e42a0b8323..cbf18129d88 100644 --- a/tests/jepsen.clickhouse-keeper/src/jepsen/clickhouse/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj @@ -42,7 +42,7 @@ (let [encoded-url (md5 url) expected-file-name (.getName (io/file url)) dest-folder (str binaries-cache-dir "/" encoded-url) - dest-file (str root-folder "/clickhouse") + dest-file (str dest-folder "/clickhouse") dest-symlink (str root-folder "/" expected-file-name) wget-opts (concat cu/std-wget-opts [:-O dest-file])] (when-not (cu/exists? dest-file) @@ -56,7 +56,7 @@ (defn get-clickhouse-url [url] - (non-precise-cached-wget! url root-folder binaries-cache-dir)) + (non-precise-cached-wget! url)) (defn get-clickhouse-scp [path] @@ -66,7 +66,7 @@ [source] (info "Downloading clickhouse from" source) (cond - (clojure.string/starts-with? source "http") (get-clickhouse-url source root-folder binaries-cache-dir) + (clojure.string/starts-with? source "http") (get-clickhouse-url source) (.exists (io/file source)) (get-clickhouse-scp source root-folder) :else (throw (Exception. (str "Don't know how to download clickhouse from" source))))) @@ -156,7 +156,7 @@ (prepare-dirs) (if (or (not (cu/exists? binary-path)) (not reuse-binary)) (do (info "Downloading clickhouse") - (let [clickhouse-path (download-clickhouse version root-folder binaries-cache-dir)] + (let [clickhouse-path (download-clickhouse version)] (install-downloaded-clickhouse clickhouse-path))) (info "Binary already exsist on path" binary-path "skipping download")) (extra-setup test node) diff --git a/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj b/tests/jepsen.clickhouse/test/jepsen/keeper_test.clj similarity index 100% rename from tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj rename to tests/jepsen.clickhouse/test/jepsen/keeper_test.clj From 98d3b4a835f805cd3b30d78794cf467ea0969e8c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 13:40:13 +0000 Subject: [PATCH 169/526] Support Jepsen for server in CI --- docker/test/keeper-jepsen/run.sh | 2 +- docker/test/server-jepsen/Dockerfile | 43 ++++++++++++++++++ docker/test/server-jepsen/run.sh | 22 ++++++++++ ...keeper_jepsen_check.py => jepsen_check.py} | 44 ++++++++++++++----- 4 files changed, 99 insertions(+), 12 deletions(-) create mode 100644 docker/test/server-jepsen/Dockerfile create mode 100644 docker/test/server-jepsen/run.sh rename tests/ci/{keeper_jepsen_check.py => jepsen_check.py} (86%) diff --git a/docker/test/keeper-jepsen/run.sh b/docker/test/keeper-jepsen/run.sh index 220f1cf7459..5e321b7c347 100644 --- a/docker/test/keeper-jepsen/run.sh +++ b/docker/test/keeper-jepsen/run.sh @@ -15,7 +15,7 @@ if [ -z "$CLICKHOUSE_REPO_PATH" ]; then ls -lath ||: fi -cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse-keeper" +cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse" (lein run keeper test-all --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --snapshot-distance 100 --stale-log-gap 100 --reserved-log-items 10 --lightweight-run --clickhouse-source "$CLICKHOUSE_PACKAGE" -q --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log" diff --git a/docker/test/server-jepsen/Dockerfile b/docker/test/server-jepsen/Dockerfile new file mode 100644 index 00000000000..0335186cf1d --- /dev/null +++ b/docker/test/server-jepsen/Dockerfile @@ -0,0 +1,43 @@ +# rebuild in #33610 +# docker build -t clickhouse/server-jepsen-test . +ARG FROM_TAG=latest +FROM clickhouse/test-base:$FROM_TAG + +ENV DEBIAN_FRONTEND=noninteractive +ENV CLOJURE_VERSION=1.10.3.814 + +# arguments +ENV PR_TO_TEST="" +ENV SHA_TO_TEST="" + +ENV NODES_USERNAME="root" +ENV NODES_PASSWORD="" +ENV TESTS_TO_RUN="30" +ENV TIME_LIMIT="30" + +ENV KEEPER_NODE="" + + +# volumes +ENV NODES_FILE_PATH="/nodes.txt" +ENV TEST_OUTPUT="/test_output" + +RUN mkdir "/root/.ssh" +RUN touch "/root/.ssh/known_hosts" + +# install java +RUN apt-get update && apt-get install default-jre default-jdk libjna-java libjna-jni ssh gnuplot graphviz --yes --no-install-recommends + +# install clojure +RUN curl -O "https://download.clojure.org/install/linux-install-${CLOJURE_VERSION}.sh" && \ + chmod +x "linux-install-${CLOJURE_VERSION}.sh" && \ + bash "./linux-install-${CLOJURE_VERSION}.sh" + +# install leiningen +RUN curl -O "https://raw.githubusercontent.com/technomancy/leiningen/stable/bin/lein" && \ + chmod +x ./lein && \ + mv ./lein /usr/bin + +COPY run.sh / + +CMD ["/bin/bash", "/run.sh"] diff --git a/docker/test/server-jepsen/run.sh b/docker/test/server-jepsen/run.sh new file mode 100644 index 00000000000..4a966d50f74 --- /dev/null +++ b/docker/test/server-jepsen/run.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash +set -euo pipefail + + +CLICKHOUSE_PACKAGE=${CLICKHOUSE_PACKAGE:="https://clickhouse-builds.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/clickhouse_build_check/clang-15_relwithdebuginfo_none_unsplitted_disable_False_binary/clickhouse"} +CLICKHOUSE_REPO_PATH=${CLICKHOUSE_REPO_PATH:=""} + + +if [ -z "$CLICKHOUSE_REPO_PATH" ]; then + CLICKHOUSE_REPO_PATH=ch + rm -rf ch ||: + mkdir ch ||: + wget -nv -nd -c "https://clickhouse-test-reports.s3.amazonaws.com/$PR_TO_TEST/$SHA_TO_TEST/repo/clickhouse_no_subs.tar.gz" + tar -C ch --strip-components=1 -xf clickhouse_no_subs.tar.gz + ls -lath ||: +fi + +cd "$CLICKHOUSE_REPO_PATH/tests/jepsen.clickhouse" + +(lein run server test-all --keeper "$KEEPER_NODE" --nodes-file "$NODES_FILE_PATH" --username "$NODES_USERNAME" --logging-json --password "$NODES_PASSWORD" --time-limit "$TIME_LIMIT" --concurrency 50 -r 50 --clickhouse-source "$CLICKHOUSE_PACKAGE" --test-count "$TESTS_TO_RUN" || true) | tee "$TEST_OUTPUT/jepsen_run_all_tests.log" + +mv store "$TEST_OUTPUT/" diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/jepsen_check.py similarity index 86% rename from tests/ci/keeper_jepsen_check.py rename to tests/ci/jepsen_check.py index a0695d3283a..5a2e710e87f 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -5,6 +5,8 @@ import logging import os import sys +import argparse + import boto3 from github import Github import requests @@ -26,8 +28,11 @@ from rerun_helper import RerunHelper JEPSEN_GROUP_NAME = "jepsen_group" DESIRED_INSTANCE_COUNT = 3 -IMAGE_NAME = "clickhouse/keeper-jepsen-test" -CHECK_NAME = "ClickHouse Keeper Jepsen" +KEEPER_IMAGE_NAME = "clickhouse/keeper-jepsen-test" +KEEPER_CHECK_NAME = "ClickHouse Keeper Jepsen" + +SERVER_IMAGE_NAME = "clickhouse/server-jepsen-test" +SERVER_CHECK_NAME = "ClickHouse Server Jepsen" SUCCESSFUL_TESTS_ANCHOR = "# Successful tests" @@ -49,8 +54,8 @@ def _parse_jepsen_output(path): current_type = "FAIL" if ( - line.startswith("store/clickhouse-keeper") - or line.startswith("clickhouse-keeper") + line.startswith("store/clickhouse") + or line.startswith("clickhouse") ) and current_type: test_results.append((line.strip(), current_type)) @@ -132,17 +137,27 @@ def get_run_command( repo_path, build_url, result_path, + extra_args, docker_image, ): return ( f"docker run --network=host -v '{ssh_sock_dir}:{ssh_sock_dir}' -e SSH_AUTH_SOCK={ssh_auth_sock} " f"-e PR_TO_TEST={pr_info.number} -e SHA_TO_TEST={pr_info.sha} -v '{nodes_path}:/nodes.txt' -v {result_path}:/test_output " - f"-e 'CLICKHOUSE_PACKAGE={build_url}' -v '{repo_path}:/ch' -e 'CLICKHOUSE_REPO_PATH=/ch' -e NODES_USERNAME=ubuntu {docker_image}" + f"-e 'CLICKHOUSE_PACKAGE={build_url}' -v '{repo_path}:/ch' -e 'CLICKHOUSE_REPO_PATH=/ch' -e NODES_USERNAME=ubuntu {extra_args} {docker_image}" ) if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + parser = argparse.ArgumentParser( + prog = 'Jepsen Check', + description = 'Check that uses Jepsen. Both Keeper and Server can be tested.') + parser.add_argument('program', help='What should be tested. Valid values "keeper", "server"') + args = parser.parse_args() + + if args.program != 'server' and args.program != 'keeper': + logging.warning(f"Invalid argument '{args.program}'") + sys.exit(0) stopwatch = Stopwatch() @@ -161,7 +176,9 @@ if __name__ == "__main__": gh = Github(get_best_robot_token(), per_page=100) - rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) + check_name = KEEPER_CHECK_NAME if args.program == 'keeper' else SERVER_CHECK_NAME + + rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") sys.exit(0) @@ -177,9 +194,9 @@ if __name__ == "__main__": nodes_path = save_nodes_to_file(instances, TEMP_PATH) # always use latest - docker_image = IMAGE_NAME + docker_image = KEEPER_IMAGE_NAME if args.program == 'keeper' else SERVER_IMAGE_NAME - build_name = get_build_name_for_check(CHECK_NAME) + build_name = get_build_name_for_check(check_name) if pr_info.number == 0: version = get_version_from_repo() @@ -203,6 +220,10 @@ if __name__ == "__main__": logging.warning("Cannot fetch build in 30 minutes, exiting") sys.exit(0) + extra_args = '' + if args.program == 'server': + extra_args = f'-e KEEPER_NODE test' + with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + "\n"): ssh_auth_sock = os.environ["SSH_AUTH_SOCK"] auth_sock_dir = os.path.dirname(ssh_auth_sock) @@ -214,6 +235,7 @@ if __name__ == "__main__": REPO_COPY, build_url, result_path, + extra_args, docker_image, ) logging.info("Going to run jepsen: %s", cmd) @@ -255,11 +277,11 @@ if __name__ == "__main__": pr_info.sha, test_result, [run_log_path] + additional_data, - CHECK_NAME, + check_name, ) print(f"::notice ::Report url: {report_url}") - post_commit_status(gh, pr_info.sha, CHECK_NAME, description, status, report_url) + post_commit_status(gh, pr_info.sha, check_name, description, status, report_url) ch_helper = ClickHouseHelper() prepared_events = prepare_tests_results_for_clickhouse( @@ -269,7 +291,7 @@ if __name__ == "__main__": stopwatch.duration_seconds, stopwatch.start_time_str, report_url, - CHECK_NAME, + check_name, ) ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) clear_autoscaling_group() From 5f7b7a8a330f409e59e1a86cdf00565b61213275 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 13:48:46 +0000 Subject: [PATCH 170/526] Add server --- .github/workflows/jepsen.yml | 33 +++++++++++++++++++++++++++++++-- 1 file changed, 31 insertions(+), 2 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index a8b04af5773..49a04008a51 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -10,7 +10,36 @@ on: # yamllint disable-line rule:truthy workflow_dispatch: workflow_call: jobs: - KeeperJepsenRelease: + #KeeperJepsenRelease: + # runs-on: [self-hosted, style-checker] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/keeper_jepsen + # REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse + # EOF + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # with: + # fetch-depth: 0 + # - name: Jepsen Test + # run: | + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" + # python3 jepsen_check.py keeper + # - name: Cleanup + # if: always() + # run: | + # docker ps --quiet | xargs --no-run-if-empty docker kill ||: + # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + # sudo rm -fr "$TEMP_PATH" + ServerJepsenRelease: runs-on: [self-hosted, style-checker] steps: - name: Set envs @@ -32,7 +61,7 @@ jobs: mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" cd "$REPO_COPY/tests/ci" - python3 keeper_jepsen_check.py + python3 jepsen_check.py keeper - name: Cleanup if: always() run: | From 0f86cbaf540b5d0876c7c9d8e4b12ce2ffe07776 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 13:49:03 +0000 Subject: [PATCH 171/526] Enable only jepsen --- .github/workflows/pull_request.yml | 3397 +--------------------------- 1 file changed, 1 insertion(+), 3396 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 23245c16374..13dae70c853 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -32,19 +32,6 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py - PythonUnitTests: - needs: CheckLabels - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Python unit tests - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: needs: CheckLabels runs-on: [self-hosted, style-checker-aarch64] @@ -109,3388 +96,6 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json - StyleCheck: - needs: DockerHubPush - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() || always() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{ runner.temp }}/style_check - ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/fasttest - REPO_COPY=${{runner.temp}}/fasttest/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" - mkdir "$GITHUB_WORKSPACE" - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.TEMP_PATH }} - - name: Fast Test - run: | - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - CompatibilityCheck: - needs: [BuilderDebRelease] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/compatibility_check - REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: CompatibilityCheck - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - SharedBuildSmokeTest: - needs: [BuilderDebShared] - runs-on: [self-hosted, style-checker] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/split_build_check - REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH=${{runner.temp}}/reports_dir - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Shared build check - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -######################################################################################### -#################################### ORDINARY BUILDS #################################### -######################################################################################### - BuilderDebRelease: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_release - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - BuilderBinRelease: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_release - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAarch64: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # for performance artifact - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebAsan: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_asan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebUBsan: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_ubsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebTsan: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_tsan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebMsan: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_msan - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderDebDebug: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=package_debug - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -########################################################################################## -##################################### SPECIAL BUILDS ##################################### -########################################################################################## - BuilderDebShared: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_shared - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinClangTidy: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_tidy - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwin: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinAarch64: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinFreeBSD: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_freebsd - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinDarwinAarch64: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_darwin_aarch64 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinPPC64: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_ppc64le - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinAmd64SSE2: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_amd64sse2 - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" - BuilderBinAarch64V80Compat: - needs: [DockerHubPush, FastTest, StyleCheck] - runs-on: [self-hosted, builder] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/build_check - IMAGES_PATH=${{runner.temp}}/images_path - REPO_COPY=${{runner.temp}}/build_check/ClickHouse - CACHES_PATH=${{runner.temp}}/../ccaches - BUILD_NAME=binary_aarch64_v80compat - EOF - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ env.IMAGES_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Build - run: | - git -C "$GITHUB_WORKSPACE" submodule sync --recursive - git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" - - name: Upload build URLs to artifacts - if: ${{ success() || failure() }} - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_URLS }} - path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" -############################################################################################ -##################################### Docker images ####################################### -############################################################################################ - DockerServerImages: - needs: - - BuilderDebRelease - - BuilderDebAarch64 - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself - - name: Check docker clickhouse/clickhouse-server building - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_server.py --release-type head --no-push - python3 docker_server.py --release-type head --no-push --no-ubuntu \ - --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################ -##################################### BUILD REPORTER ####################################### -############################################################################################ - BuilderReport: - needs: - - BuilderBinRelease - - BuilderDebAarch64 - - BuilderDebAsan - - BuilderDebDebug - - BuilderDebMsan - - BuilderDebRelease - - BuilderDebTsan - - BuilderDebUBsan - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - CHECK_NAME=ClickHouse build check - REPORTS_PATH=${{runner.temp}}/reports_dir - TEMP_PATH=${{runner.temp}}/report_check - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - BuilderSpecialReport: - needs: - - BuilderBinAarch64 - - BuilderBinDarwin - - BuilderBinDarwinAarch64 - - BuilderBinFreeBSD - # - BuilderBinGCC - - BuilderBinPPC64 - - BuilderBinAmd64SSE2 - - BuilderBinAarch64V80Compat - - BuilderBinClangTidy - - BuilderDebShared - runs-on: [self-hosted, style-checker] - if: ${{ success() || failure() }} - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/report_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=ClickHouse special build check - NEEDS_DATA_PATH=${{runner.temp}}/needs.json - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cat > "$NEEDS_DATA_PATH" << 'EOF' - ${{ toJSON(needs) }} - EOF - cd "$GITHUB_WORKSPACE/tests/ci" - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -########################### FUNCTIONAl STATELESS TESTS ####################################### -############################################################################################## - FunctionalStatelessTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseDatabaseReplicated1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_database_replicated - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, DatabaseReplicated) - REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseWideParts: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_wide_parts - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, wide parts enabled) - REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestReleaseS3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (release, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Debug0: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Debug1: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Debug2: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Tsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Tsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestS3Tsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan, s3 storage) - REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (aarch64) - REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (asan) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (asan) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (tsan) - REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (ubsan) - REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT=10800 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan0: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (msan) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan1: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (msan) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestMsan2: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (msan) - REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug0: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug1: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestDebug2: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests (debug) - REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT=10800 - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatelessTestFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateless_flaky_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateless tests flaky check (asan) - REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - TestsBugfixCheck: - needs: [CheckLabels, StyleCheck] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/tests_bugfix_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=tests bugfix validate check - KILL_TIMEOUT=3600 - REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Bugfix test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - - TEMP_PATH="${TEMP_PATH}/integration" \ - REPORTS_PATH="${REPORTS_PATH}/integration" \ - python3 integration_test_check.py "Integration $CHECK_NAME" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - TEMP_PATH="${TEMP_PATH}/stateless" \ - REPORTS_PATH="${REPORTS_PATH}/stateless" \ - python3 functional_test_check.py "Stateless $CHECK_NAME" "$KILL_TIMEOUT" \ - --validate-bugfix --post-commit-status=file || echo 'ignore exit code' - - python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -############################ FUNCTIONAl STATEFUL TESTS ####################################### -############################################################################################## - FunctionalStatefulTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (release) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAarch64: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (aarch64) - REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (asan) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (tsan) - REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (msan) - REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (ubsan) - REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - FunctionalStatefulTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stateful_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stateful tests (debug) - REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT=3600 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -######################################### STRESS TESTS ####################################### -############################################################################################## - StressTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (asan) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - StressTestTsan: - needs: [BuilderDebTsan] - # func testers have 16 cores + 128 GB memory - # while stress testers have 36 cores + 72 memory - # It would be better to have something like 32 + 128, - # but such servers almost unavailable as spot instances. - runs-on: [self-hosted, func-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_thread - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (tsan) - REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - StressTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_memory - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (msan) - REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - StressTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_undefined - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (ubsan) - REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - StressTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/stress_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Stress test (debug) - REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################## -##################################### AST FUZZERS ############################################ -############################################################################################## - ASTFuzzerTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (asan) - REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (tsan) - REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestUBSan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (ubsan) - REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestMSan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (msan) - REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - ASTFuzzerTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=AST fuzzer (debug) - REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -############################# INTEGRATION TESTS ############################################# -############################################################################################# - IntegrationTestsAsan0: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan1: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsAsan2: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (asan) - REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=3 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan0: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (tsan) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan1: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (tsan) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan2: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (tsan) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsTsan3: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (tsan) - REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsRelease1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_release - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests (release) - REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=2 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - IntegrationTestsFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Integration tests flaky check (asan) - REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Integration test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 integration_test_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### UNIT TESTS ############################################# -############################################################################################# - UnitTestsAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (asan) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsReleaseClang: - needs: [BuilderBinRelease] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_asan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (release-clang) - REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_tsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (tsan) - REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_msan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (msan) - REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - UnitTestsUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, fuzzer-unit-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Unit tests (ubsan) - REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" -############################################################################################# -#################################### PERFORMANCE TESTS ###################################### -############################################################################################# - PerformanceComparisonX86-0: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonX86-1: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonX86-2: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonX86-3: - needs: [BuilderDebRelease] - runs-on: [self-hosted, stress-tester] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch-0: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=0 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch-1: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=1 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch-2: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=2 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" - PerformanceComparisonAarch-3: - needs: [BuilderDebAarch64] - runs-on: [self-hosted, func-tester-aarch64] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/performance_comparison - REPORTS_PATH=${{runner.temp}}/reports_dir - CHECK_NAME=Performance Comparison Aarch64 - REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse - RUN_BY_HASH_NUM=3 - RUN_BY_HASH_TOTAL=4 - EOF - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{ env.REPORTS_PATH }} - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Performance Comparison - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - cd "$REPO_COPY/tests/ci" - python3 performance_comparison_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# @@ -3499,7 +104,7 @@ jobs: # When it's skipped, all dependent tasks will be skipped too. # DO NOT add it there if: contains(github.event.pull_request.labels.*.name, 'jepsen-test') - needs: [BuilderBinRelease] + needs: [DockerHubPush] uses: ./.github/workflows/jepsen.yml FinishCheck: From fee53a415ae0c33745a84e16295bc019694df3e3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 13:49:12 +0000 Subject: [PATCH 172/526] Reuse same PR --- tests/ci/jepsen_check.py | 32 ++++++++++++++++---------------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 5a2e710e87f..33a786c6a9e 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -163,16 +163,16 @@ if __name__ == "__main__": pr_info = PRInfo() - logging.info( - "Start at PR number %s, commit sha %s labels %s", - pr_info.number, - pr_info.sha, - pr_info.labels, - ) + #logging.info( + # "Start at PR number %s, commit sha %s labels %s", + # pr_info.number, + # pr_info.sha, + # pr_info.labels, + #) - if pr_info.number != 0 and "jepsen-test" not in pr_info.labels: - logging.info("Not jepsen test label in labels list, skipping") - sys.exit(0) + #if pr_info.number != 0 and "jepsen-test" not in pr_info.labels: + # logging.info("Not jepsen test label in labels list, skipping") + # sys.exit(0) gh = Github(get_best_robot_token(), per_page=100) @@ -198,18 +198,18 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - if pr_info.number == 0: - version = get_version_from_repo() - release_or_pr = f"{version.major}.{version.minor}" - else: - # PR number for anything else - release_or_pr = str(pr_info.number) + #if pr_info.number == 0: + # version = get_version_from_repo() + # release_or_pr = f"{version.major}.{version.minor}" + #else: + # # PR number for anything else + # release_or_pr = str(pr_info.number) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit # dependency on a build job and using busy loop on it's results. For the # same reason we are using latest docker image. - build_url = f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/{release_or_pr}/{pr_info.sha}/{build_name}/clickhouse" + build_url = f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/22.11/4012fcbf0191e9e64a34430db1468a2ae0744616/{build_name}/clickhouse" head = requests.head(build_url) counter = 0 while head.status_code != 200: From 51ee6ec436833a1fc58a681e533a7c793775efe0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Fri, 4 Nov 2022 14:12:30 +0000 Subject: [PATCH 173/526] Use autoscaling for keeper --- tests/ci/jepsen_check.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 33a786c6a9e..87963a158c2 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -27,7 +27,10 @@ from build_download_helper import get_build_name_for_check from rerun_helper import RerunHelper JEPSEN_GROUP_NAME = "jepsen_group" -DESIRED_INSTANCE_COUNT = 3 + +KEEPER_DESIRED_INSTANCE_COUNT = 3 +SERVER_DESIRED_INSTANCE_COUNT = 4 + KEEPER_IMAGE_NAME = "clickhouse/keeper-jepsen-test" KEEPER_CHECK_NAME = "ClickHouse Keeper Jepsen" @@ -87,15 +90,15 @@ def get_instances_addresses(ec2_client, instance_ids): return instance_ips -def prepare_autoscaling_group_and_get_hostnames(): +def prepare_autoscaling_group_and_get_hostnames(count): asg_client = boto3.client("autoscaling", region_name="us-east-1") asg_client.set_desired_capacity( - AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=DESIRED_INSTANCE_COUNT + AutoScalingGroupName=JEPSEN_GROUP_NAME, DesiredCapacity=count ) instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) counter = 0 - while len(instances) < DESIRED_INSTANCE_COUNT: + while len(instances) < count: time.sleep(5) instances = get_autoscaling_group_instances_ids(asg_client, JEPSEN_GROUP_NAME) counter += 1 @@ -190,8 +193,8 @@ if __name__ == "__main__": if not os.path.exists(result_path): os.makedirs(result_path) - instances = prepare_autoscaling_group_and_get_hostnames() - nodes_path = save_nodes_to_file(instances, TEMP_PATH) + instances = prepare_autoscaling_group_and_get_hostnames(KEEPER_DESIRED_INSTANCE_COUNT if args.program == 'keeper' else SERVER_DESIRED_INSTANCE_COUNT) + nodes_path = save_nodes_to_file(instances[:SERVER_DESIRED_INSTANCE_COUNT - 1], TEMP_PATH) # always use latest docker_image = KEEPER_IMAGE_NAME if args.program == 'keeper' else SERVER_IMAGE_NAME @@ -222,7 +225,7 @@ if __name__ == "__main__": extra_args = '' if args.program == 'server': - extra_args = f'-e KEEPER_NODE test' + extra_args = f'-e KEEPER_NODE={instances[-1]}' with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + "\n"): ssh_auth_sock = os.environ["SSH_AUTH_SOCK"] From 79facdb4f286c32ba06cb40de59f2581a53b3b56 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 Nov 2022 15:44:31 +0000 Subject: [PATCH 174/526] Make query plan optimisation respect query settings in EXPLAIN --- src/Interpreters/ExpressionAnalyzer.cpp | 7 ------- .../IInterpreterUnionOrSelectQuery.h | 2 ++ src/Interpreters/InterpreterExplainQuery.cpp | 21 +++++++++++++------ 3 files changed, 17 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3761fdc6f0c..d40d886de62 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1952,13 +1952,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( && !query.final() && join_allow_read_in_order; - // if (storage && optimize_read_in_order) - // { - // Names columns_for_sorting_key = metadata_snapshot->getColumnsRequiredForSortingKey(); - // additional_required_columns_after_prewhere.insert(additional_required_columns_after_prewhere.end(), - // columns_for_sorting_key.begin(), columns_for_sorting_key.end()); - // } - /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/src/Interpreters/IInterpreterUnionOrSelectQuery.h b/src/Interpreters/IInterpreterUnionOrSelectQuery.h index a1c86f9de85..6f893d4703e 100644 --- a/src/Interpreters/IInterpreterUnionOrSelectQuery.h +++ b/src/Interpreters/IInterpreterUnionOrSelectQuery.h @@ -58,6 +58,8 @@ public: /// Add limits from external query. void addStorageLimits(const StorageLimitsList & limits); + ContextPtr getContext() const { return context; } + protected: ASTPtr query_ptr; ContextMutablePtr context; diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index fb8d3c6049f..2864b433e00 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -419,19 +419,23 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; + ContextPtr context; + if (getContext()->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + context = interpreter.getContext(); plan = std::move(interpreter).extractQueryPlan(); } else { InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); interpreter.buildQueryPlan(plan); + context = interpreter.getContext(); } if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); + plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); if (settings.json) { @@ -461,21 +465,24 @@ QueryPipeline InterpreterExplainQuery::executeImpl() { auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; + ContextPtr context; if (getContext()->getSettingsRef().allow_experimental_analyzer) { InterpreterSelectQueryAnalyzer interpreter(ast.getExplainedQuery(), options, getContext()); + context = interpreter.getContext(); plan = std::move(interpreter).extractQueryPlan(); } else { InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), options); interpreter.buildQueryPlan(plan); + context = interpreter.getContext(); } auto pipeline = plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(getContext()), - BuildQueryPipelineSettings::fromContext(getContext())); + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); if (settings.graph) { @@ -511,16 +518,18 @@ QueryPipeline InterpreterExplainQuery::executeImpl() auto settings = checkAndGetSettings(ast.getSettings()); QueryPlan plan; + ContextPtr context; InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), getContext(), SelectQueryOptions()); interpreter.buildQueryPlan(plan); + context = interpreter.getContext(); // collect the selected marks, rows, parts during build query pipeline. plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(getContext()), - BuildQueryPipelineSettings::fromContext(getContext())); + QueryPlanOptimizationSettings::fromContext(context), + BuildQueryPipelineSettings::fromContext(context)); if (settings.optimize) - plan.optimize(QueryPlanOptimizationSettings::fromContext(getContext())); + plan.optimize(QueryPlanOptimizationSettings::fromContext(context)); plan.explainEstimate(res_columns); insert_buf = false; break; From 9c066e964d2d3a2abd07e99d1406e8b2d975d909 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 4 Nov 2022 15:52:48 +0000 Subject: [PATCH 175/526] Less use of CH-specific bit_cast() Converted usage of CH-custom bit_cast to std::bit_cast if possible, i.e. when sizeof(From) == sizeof(To). (The CH-custom bit_cast is able to deal with sizeof(From) != sizeof(To).) Motivation for this came from #42847 where it is not clear how the internal bit_cast should behave on big endian systems, so we better avoid that situation as much as possible. --- programs/library-bridge/ExternalDictionaryLibraryHandler.cpp | 3 +-- programs/library-bridge/ExternalDictionaryLibraryUtils.h | 1 - programs/obfuscator/Obfuscator.cpp | 5 ++--- src/AggregateFunctions/QuantileBFloat16Histogram.h | 5 ++--- src/Databases/MySQL/MaterializedMySQLSyncThread.cpp | 1 - src/Functions/FunctionsRound.h | 1 - src/Functions/isFinite.cpp | 5 ++--- src/Functions/isInfinite.cpp | 5 ++--- src/Functions/padString.cpp | 5 ++--- src/Interpreters/BloomFilterHash.h | 1 - .../MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp | 1 - src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp | 1 - src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp | 1 - 13 files changed, 11 insertions(+), 24 deletions(-) diff --git a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp index 14850da2ebf..c60d4a4e5cc 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp +++ b/programs/library-bridge/ExternalDictionaryLibraryHandler.cpp @@ -1,7 +1,6 @@ #include "ExternalDictionaryLibraryHandler.h" #include -#include #include #include @@ -113,7 +112,7 @@ Block ExternalDictionaryLibraryHandler::loadAll() Block ExternalDictionaryLibraryHandler::loadIds(const std::vector & ids) { - const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{bit_cast(ids.data()), ids.size()}; + const ExternalDictionaryLibraryAPI::VectorUInt64 ids_data{std::bit_cast(ids.data()), ids.size()}; auto columns_holder = std::make_unique(attributes_names.size()); ExternalDictionaryLibraryAPI::CStrings columns_pass{static_cast(columns_holder.get()), attributes_names.size()}; diff --git a/programs/library-bridge/ExternalDictionaryLibraryUtils.h b/programs/library-bridge/ExternalDictionaryLibraryUtils.h index e813efab2a6..c9d03d27f75 100644 --- a/programs/library-bridge/ExternalDictionaryLibraryUtils.h +++ b/programs/library-bridge/ExternalDictionaryLibraryUtils.h @@ -2,7 +2,6 @@ #include #include -#include #include #include "ExternalDictionaryLibraryAPI.h" diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 7fdc5a54d8a..b6952ad6cb0 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -32,7 +32,6 @@ #include #include #include -#include #include #include #include @@ -278,9 +277,9 @@ Float transformFloatMantissa(Float x, UInt64 seed) using UInt = std::conditional_t, UInt32, UInt64>; constexpr size_t mantissa_num_bits = std::is_same_v ? 23 : 52; - UInt x_uint = bit_cast(x); + UInt x_uint = std::bit_cast(x); x_uint = static_cast(feistelNetwork(x_uint, mantissa_num_bits, seed)); - return bit_cast(x_uint); + return std::bit_cast(x_uint); } diff --git a/src/AggregateFunctions/QuantileBFloat16Histogram.h b/src/AggregateFunctions/QuantileBFloat16Histogram.h index 8ec325e238d..de9f61e01a2 100644 --- a/src/AggregateFunctions/QuantileBFloat16Histogram.h +++ b/src/AggregateFunctions/QuantileBFloat16Histogram.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include @@ -104,13 +103,13 @@ private: /// Take the most significant 16 bits of the floating point number. BFloat16 toBFloat16(const Value & x) const { - return bit_cast(static_cast(x)) >> 16; + return std::bit_cast(static_cast(x)) >> 16; } /// Put the bits into most significant 16 bits of the floating point number and fill other bits with zeros. Float32 toFloat32(const BFloat16 & x) const { - return bit_cast(x << 16); + return std::bit_cast(x << 16); } using Pair = PairNoInit; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index ed9199a359f..29f5719e3ed 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Functions/FunctionsRound.h b/src/Functions/FunctionsRound.h index 283f1ea5a43..2c7883cf471 100644 --- a/src/Functions/FunctionsRound.h +++ b/src/Functions/FunctionsRound.h @@ -16,7 +16,6 @@ #include #include #include -#include #include #include diff --git a/src/Functions/isFinite.cpp b/src/Functions/isFinite.cpp index 90185b64fff..612e7067bf5 100644 --- a/src/Functions/isFinite.cpp +++ b/src/Functions/isFinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -20,11 +19,11 @@ struct IsFiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111100000000000000000000000) != 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111110000000000000000000000000000000000000000000000000000) != 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/isInfinite.cpp b/src/Functions/isInfinite.cpp index e923e1461bc..ace2c334873 100644 --- a/src/Functions/isInfinite.cpp +++ b/src/Functions/isInfinite.cpp @@ -1,6 +1,5 @@ #include #include -#include #include @@ -16,11 +15,11 @@ struct IsInfiniteImpl static bool execute(const T t) { if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b01111111111111111111111111111111) == 0b01111111100000000000000000000000; else if constexpr (std::is_same_v) - return (bit_cast(t) + return (std::bit_cast(t) & 0b0111111111111111111111111111111111111111111111111111111111111111) == 0b0111111111110000000000000000000000000000000000000000000000000000; else diff --git a/src/Functions/padString.cpp b/src/Functions/padString.cpp index c8ed920755c..486fa328fa0 100644 --- a/src/Functions/padString.cpp +++ b/src/Functions/padString.cpp @@ -5,7 +5,6 @@ #include #include #include -#include namespace DB { @@ -59,10 +58,10 @@ namespace { if (num_chars <= step) { - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(num_chars)}, res_sink); break; } - writeSlice(StringSource::Slice{bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); + writeSlice(StringSource::Slice{std::bit_cast(pad_string.data()), numCharsToNumBytes(step)}, res_sink); num_chars -= step; } } diff --git a/src/Interpreters/BloomFilterHash.h b/src/Interpreters/BloomFilterHash.h index 31532cc888a..b95abbfd770 100644 --- a/src/Interpreters/BloomFilterHash.h +++ b/src/Interpreters/BloomFilterHash.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp index b19080b5097..9bc0e4e6dc0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAggregatorBloomFilter.cpp @@ -1,6 +1,5 @@ #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp index 33668b96a60..c2ed081ac00 100644 --- a/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexBloomFilter.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp index f80d40d2fa8..7efaf0866db 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #include #include From d6bbd65ddea49d67ccea3dd187f5437bcf5ddafc Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 16:03:12 +0000 Subject: [PATCH 176/526] Fix broken include --- src/Storages/StorageDelta.cpp | 4 ++-- src/Storages/StorageDelta.h | 4 ++-- src/Storages/StorageHudi.cpp | 4 ++-- src/Storages/StorageHudi.h | 4 ++-- 4 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 01598996161..7ab522860e8 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,4 +1,4 @@ -#include +#include "config.h" #if USE_AWS_S3 @@ -236,7 +236,7 @@ Pipe StorageDelta::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { StorageS3::updateS3Configuration(context, base_configuration); diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index 3f20a850526..d0992dd4cb7 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -1,6 +1,6 @@ #pragma once -#include "config_core.h" +#include "config.h" #if USE_AWS_S3 @@ -81,7 +81,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: void Init(); diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 57e6461fb72..e2a46fad2d0 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -1,4 +1,4 @@ -#include +#include "config.h" #if USE_AWS_S3 @@ -85,7 +85,7 @@ Pipe StorageHudi::read( ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) + size_t num_streams) { StorageS3::updateS3Configuration(context, base_configuration); return s3engine->read(column_names, storage_snapshot, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 0ed1935a36c..91a77ec83ff 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -1,6 +1,6 @@ #pragma once -#include "config_core.h" +#include "config.h" #if USE_AWS_S3 @@ -41,7 +41,7 @@ public: ContextPtr context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, - unsigned num_streams) override; + size_t num_streams) override; private: std::vector getKeysFromS3(); From 2db1638b41efcafebcce536f264cf8004434aa92 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 Nov 2022 16:18:35 +0000 Subject: [PATCH 177/526] Disable optimize_in_window_order in case if read-in-order for query plan is enabled. --- .../QueryPlan/Optimizations/Optimizations.h | 25 +++++-- .../QueryPlanOptimizationSettings.h | 2 +- .../optimizePrimaryKeyCondition.cpp | 68 ++++++------------- ...wFunctions.cpp => optimizeReadInOrder.cpp} | 5 +- .../QueryPlan/Optimizations/optimizeTree.cpp | 35 +++++++++- src/Processors/QueryPlan/QueryPlan.cpp | 4 +- ...mizations_optimize_read_in_window_order.sh | 12 ++-- ...ions_optimize_read_in_window_order_long.sh | 14 ++-- 8 files changed, 93 insertions(+), 72 deletions(-) rename src/Processors/QueryPlan/Optimizations/{reuseStorageOrderingForWindowFunctions.cpp => optimizeReadInOrder.cpp} (99%) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index e87d2fdb32b..b4b2f0c8c9c 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -9,13 +9,13 @@ namespace DB namespace QueryPlanOptimizations { -/// This is the main function which optimizes the whole QueryPlan tree. -void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); +/// Main functions which optimize QueryPlan tree. +/// First pass (ideally) apply local idempotent operations on top of Plan. +void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); +/// Second pass is used to apply read-in-order and attach a predicate to PK. +void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); -void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); -void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); - -/// Optimization is a function applied to QueryPlan::Node. +/// Optimization (first pass) is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. /// It return the number of updated layers of subtree if some change happened. /// It must guarantee that the structure of tree is correct. @@ -72,12 +72,23 @@ inline const auto & getOptimizations() {tryExecuteFunctionsAfterSorting, "liftUpFunctions", &QueryPlanOptimizationSettings::optimize_plan}, {tryReuseStorageOrderingForWindowFunctions, "reuseStorageOrderingForWindowFunctions", &QueryPlanOptimizationSettings::optimize_plan}, {tryLiftUpUnion, "liftUpUnion", &QueryPlanOptimizationSettings::optimize_plan}, - //{tryDistinctReadInOrder, "distinctReadInOrder", &QueryPlanOptimizationSettings::distinct_in_order}, }}; return optimizations; } +struct Frame +{ + QueryPlan::Node * node = nullptr; + size_t next_child = 0; +}; + +using Stack = std::vector; + +/// Second pass optimizations. +void optimizePrimaryKeyCondition(const Stack & stack); +void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); + } } diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h index 56dd39fdf0a..7185d2fe869 100644 --- a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -25,7 +25,7 @@ struct QueryPlanOptimizationSettings bool distinct_in_order = false; /// If read-in-order optimisation is enabled - bool read_in_order = false; + bool read_in_order = true; static QueryPlanOptimizationSettings fromSettings(const Settings & from); static QueryPlanOptimizationSettings fromContext(ContextPtr from); diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 143bfd3d2f3..cac4944bad7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -9,58 +9,32 @@ namespace DB::QueryPlanOptimizations { -void optimizePrimaryKeyCondition(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) +void optimizePrimaryKeyCondition(const Stack & stack) { - struct Frame + const auto & frame = stack.back(); + + auto * read_from_merge_tree = typeid_cast(frame.node->step.get()); + auto * read_from_merge = typeid_cast(frame.node->step.get()); + + if (!read_from_merge && !read_from_merge_tree) + return; + + for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { - QueryPlan::Node * node = nullptr; - size_t next_child = 0; - }; - - std::vector stack; - stack.push_back({.node = &root}); - - while (!stack.empty()) - { - auto & frame = stack.back(); - - if (frame.next_child == 0) + if (auto * filter_step = typeid_cast(iter->node->step.get())) { - if (optimization_settings.read_in_order) - optimizeReadInOrder(*frame.node, nodes); - - if (optimization_settings.distinct_in_order) - tryDistinctReadInOrder(frame.node); + if (read_from_merge_tree) + read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + if (read_from_merge) + read_from_merge->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); } - - /// Traverse all children first. - if (frame.next_child < frame.node->children.size()) - { - auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; - ++frame.next_child; - stack.push_back(next_frame); + /// Note: actually, plan optimizations merge Filter and Expression steps. + /// Ideally, chain should look like (Expression -> ...) -> (Filter -> ...) -> ReadFromStorage, + /// So this is likely not needed. + else if (typeid_cast(iter->node->step.get())) continue; - } - - auto add_read_from_storage_filter = [&](auto & storage) - { - for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) - { - if (auto * filter_step = typeid_cast(iter->node->step.get())) - storage.addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); - else if (typeid_cast(iter->node->step.get())) - continue; - else - break; - } - }; - - if (auto * read_from_merge_tree = typeid_cast(frame.node->step.get())) - add_read_from_storage_filter(*read_from_merge_tree); - else if (auto * read_from_merge = typeid_cast(frame.node->step.get())) - add_read_from_storage_filter(*read_from_merge); - - stack.pop_back(); + else + break; } } diff --git a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp similarity index 99% rename from src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp rename to src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 0ee4c353a3b..8af05fbc7d4 100644 --- a/src/Processors/QueryPlan/Optimizations/reuseStorageOrderingForWindowFunctions.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -814,6 +814,8 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) } } +/// This optimisation is obsolete and will be removed. +/// optimizeReadInOrder covers it. size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/) { /// Find the following sequence of steps, add InputOrderInfo and apply prefix sort description to @@ -851,7 +853,8 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, } auto context = read_from_merge_tree->getContext(); - if (!context->getSettings().optimize_read_in_window_order || context->getSettingsRef().allow_experimental_analyzer) + const auto & settings = context->getSettings(); + if (!settings.optimize_read_in_window_order || (settings.optimize_read_in_order && settings.query_plan_read_in_order) || context->getSettingsRef().allow_experimental_analyzer) { return 0; } diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index ff30cfd8cf3..f514e145a92 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes namespace QueryPlanOptimizations { -void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) +void optimizeTreeFirstPass(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { if (!settings.optimize_plan) return; @@ -98,5 +98,38 @@ void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Nod } } +void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) +{ + Stack stack; + stack.push_back({.node = &root}); + + while (!stack.empty()) + { + auto & frame = stack.back(); + + if (frame.next_child == 0) + { + if (optimization_settings.read_in_order) + optimizeReadInOrder(*frame.node, nodes); + + if (optimization_settings.distinct_in_order) + tryDistinctReadInOrder(frame.node); + } + + /// Traverse all children first. + if (frame.next_child < frame.node->children.size()) + { + auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; + ++frame.next_child; + stack.push_back(next_frame); + continue; + } + + optimizePrimaryKeyCondition(stack); + + stack.pop_back(); + } +} + } } diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index abbb5daae81..e1662d43015 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -447,8 +447,8 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { - QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes); - QueryPlanOptimizations::optimizePrimaryKeyCondition(optimization_settings, *root, nodes); + QueryPlanOptimizations::optimizeTreeFirstPass(optimization_settings, *root, nodes); + QueryPlanOptimizations::optimizeTreeSecondPass(optimization_settings, *root, nodes); } void QueryPlan::explainEstimate(MutableColumns & columns) diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 52be8904091..2952944aa8e 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -19,25 +19,25 @@ $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" echo 'Partial sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" | grep -i "sort description" echo 'No sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" | grep -i "sort description" echo 'Complex ORDER BY' $CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" $CLICKHOUSE_CLIENT -q "INSERT INTO ${name}_complex VALUES (1, 2, 3), (2, 3, 4), (3, 4, 5)" echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT --query_plan_read_in_order=0 -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=0" +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_window_order=1" +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" $CLICKHOUSE_CLIENT -q "drop table ${name}" $CLICKHOUSE_CLIENT -q "drop table ${name}_n" diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh index af053fec052..50dab5219aa 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh @@ -19,16 +19,16 @@ $CLICKHOUSE_CLIENT -q "create table ${name}_n_x engine=MergeTree order by (n, x) $CLICKHOUSE_CLIENT -q "optimize table ${name}_n final" $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=0, query_plan_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n+x%2 ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_window_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n+x%2 ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "drop table ${name}" $CLICKHOUSE_CLIENT -q "drop table ${name}_n" From 5cb69d8a22a9e4b70fc574eaea416041483e48e5 Mon Sep 17 00:00:00 2001 From: Aleksandr Musorin Date: Wed, 2 Nov 2022 11:04:58 +0100 Subject: [PATCH 178/526] changed type name for S3_Plain storage renamed a disk for S3PlainObjectStorage in system.disks table from s3 to s3_plain --- src/Disks/DiskType.h | 3 +++ src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 4 +++- .../test_backup_restore_s3/test.py | 19 +++++++++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 037b65f8e07..4d099e33a7a 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -11,6 +11,7 @@ enum class DataSourceType Local, RAM, S3, + S3_Plain, HDFS, WebServer, AzureBlobStorage, @@ -26,6 +27,8 @@ inline String toString(DataSourceType data_source_type) return "memory"; case DataSourceType::S3: return "s3"; + case DataSourceType::S3_Plain: + return "s3_plain"; case DataSourceType::HDFS: return "hdfs"; case DataSourceType::WebServer: diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 6b1e8289b15..56f1c895924 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -213,7 +213,9 @@ public: template S3PlainObjectStorage(Args && ...args) : S3ObjectStorage("S3PlainObjectStorage", std::forward(args)...) - {} + { + data_source_description.type = DataSourceType::S3_Plain; + } }; } diff --git a/tests/integration/test_backup_restore_s3/test.py b/tests/integration/test_backup_restore_s3/test.py index 7ddb1459ab9..ee9c458c044 100644 --- a/tests/integration/test_backup_restore_s3/test.py +++ b/tests/integration/test_backup_restore_s3/test.py @@ -50,6 +50,24 @@ def check_backup_and_restore(storage_policy, backup_destination, size=1000): ) +def check_system_tables(): + disks = [ + tuple(disk.split("\t")) + for disk in node.query("SELECT name, type FROM system.disks").split("\n") + if disk + ] + expected_disks = ( + ("default", "local"), + ("disk_s3", "s3"), + ("disk_s3_other_bucket", "s3"), + ("disk_s3_plain", "s3_plain"), + ) + assert len(expected_disks) == len(disks) + for expected_disk in expected_disks: + if expected_disk not in disks: + raise AssertionError(f"Missed {expected_disk} in {disks}") + + @pytest.mark.parametrize( "storage_policy, to_disk", [ @@ -93,6 +111,7 @@ def test_backup_to_s3(): f"S3('http://minio1:9001/root/data/backups/{backup_name}', 'minio', 'minio123')" ) check_backup_and_restore(storage_policy, backup_destination) + check_system_tables() def test_backup_to_s3_named_collection(): From 9043df56a80d9b913dbba63e020e9beccacfe9e6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 4 Nov 2022 17:31:20 +0000 Subject: [PATCH 179/526] Add comments, fix tests. --- .../Optimizations/optimizeReadInOrder.cpp | 19 +++++++++++++++++++ ...mizations_optimize_read_in_window_order.sh | 12 ++++++------ ...ions_optimize_read_in_window_order_long.sh | 6 +++--- 3 files changed, 28 insertions(+), 9 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 8af05fbc7d4..a1184607979 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -196,6 +196,8 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & } } +/// Add more functions to fixed columns. +/// Functions result is fixed if all arguments are fixed or constants. void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) { struct Frame @@ -267,8 +269,25 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) } } +/// This structure stores a node mapping from one DAG to another. +/// The rule is following: +/// * Input nodes are mapped by name. +/// * Function is mapped to function if all children are mapped and function names are same. +/// * Alias is mapped to it's children mapping. +/// * Monotonic function can be mapped to it's children mapping if dirrect mapping does not exist. +/// In this case, information about monotonicity is filled. +/// * Mapped node is nullptr if there is no mapping found. +/// +/// Overall, directly mapped nodes represent equal calculations. +/// Notes: +/// * Mapped DAG can contain many nodes which represent the same calculation. +/// In this case mapping is ambiguous and only one node is mapped. +/// * Aliases for mapped DAG are not supported. +/// DAG for PK does not contain aliases and ambiguous nodes. struct MatchedTrees { + /// Monotonicity is calculated for monotonic functions chain. + /// Chain is not strict if there is any non-strict monotonic function. struct Monotonicity { int direction = 1; diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh index 2952944aa8e..a606f1a2f9e 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order.sh @@ -19,25 +19,25 @@ $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" echo 'Partial sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1" | grep -i "sort description" echo 'No sorting plan' echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0" | grep -i "sort description" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" | grep -i "sort description" +$CLICKHOUSE_CLIENT -q "explain plan actions=1, description=1 select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1" | grep -i "sort description" echo 'Complex ORDER BY' $CLICKHOUSE_CLIENT -q "CREATE TABLE ${name}_complex (unique1 Int32, unique2 Int32, ten Int32) ENGINE=MergeTree ORDER BY tuple() SETTINGS index_granularity = 8192" $CLICKHOUSE_CLIENT -q "INSERT INTO ${name}_complex VALUES (1, 2, 3), (2, 3, 4), (3, 4, 5)" echo ' optimize_read_in_window_order=0' -$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=0,query_plan_read_in_order=0" +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=0,optimize_read_in_window_order=0" echo ' optimize_read_in_window_order=1' -$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=1,query_plan_read_in_order=1" +$CLICKHOUSE_CLIENT -q "SELECT ten, sum(unique1) + sum(unique2) AS res, rank() OVER (ORDER BY sum(unique1) + sum(unique2) ASC) AS rank FROM ${name}_complex GROUP BY ten ORDER BY ten ASC SETTINGS optimize_read_in_order=1" $CLICKHOUSE_CLIENT -q "drop table ${name}" $CLICKHOUSE_CLIENT -q "drop table ${name}_n" diff --git a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh index 50dab5219aa..fc79725aebe 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations_optimize_read_in_window_order_long.sh @@ -19,13 +19,13 @@ $CLICKHOUSE_CLIENT -q "create table ${name}_n_x engine=MergeTree order by (n, x) $CLICKHOUSE_CLIENT -q "optimize table ${name}_n final" $CLICKHOUSE_CLIENT -q "optimize table ${name}_n_x final" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=0, optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" -$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' +$CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=0, optimize_read_in_window_order=0, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n ORDER BY x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" $CLICKHOUSE_CLIENT -q "select n, sum(x) OVER (PARTITION BY n+x%2 ORDER BY n, x ROWS BETWEEN 100 PRECEDING AND CURRENT ROW) from ${name}_n_x SETTINGS optimize_read_in_order=1, max_memory_usage=$max_memory_usage, max_threads=1 format Null" 2>&1 | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo 'OK' || echo 'FAIL' From 37048e986a303b8563cc9882d29384e3e9ec7123 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 20:49:24 +0000 Subject: [PATCH 180/526] Add comments, fix review issues --- src/Storages/StorageDelta.cpp | 27 ++++++++++++++++++++------- src/Storages/StorageDelta.h | 15 ++++++++++++--- src/Storages/StorageHudi.cpp | 12 ++++++------ src/Storages/StorageHudi.h | 11 +++++++++++ 4 files changed, 49 insertions(+), 16 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 7ab522860e8..59e7c3d3187 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -60,20 +60,20 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && return keys; } -JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_) +JsonMetadataGetter::JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context) : base_configuration(configuration_), table_path(table_path_) { - Init(); + Init(context); } -void JsonMetadataGetter::Init() +void JsonMetadataGetter::Init(ContextPtr context) { auto keys = getJsonLogFiles(); // read data from every json log file for (const String & key : keys) { - auto buf = createS3ReadBuffer(key); + auto buf = createS3ReadBuffer(key, context); while (!buf->eof()) { @@ -110,6 +110,8 @@ std::vector JsonMetadataGetter::getJsonLogFiles() const auto bucket{base_configuration.uri.bucket}; request.SetBucket(bucket); + + // DeltaLake format stores all metadata json files in _delta_log directory request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) @@ -129,18 +131,25 @@ std::vector JsonMetadataGetter::getJsonLogFiles() { const auto & filename = obj.GetKey(); + // DeltaLake metadata files have json extension if (std::filesystem::path(filename).extension() == ".json") keys.push_back(filename); } + // Needed in case any more results are available + // if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + + /// Set to false if all of the results were returned. Set to true if more keys + /// are available to return. If the number of results exceeds that specified by + /// MaxKeys, all of the results might not be returned is_finished = !outcome.GetResult().GetIsTruncated(); } return keys; } -std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key) +std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { // TBD: add parallel downloads return std::make_shared( @@ -149,7 +158,7 @@ std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String key, base_configuration.uri.version_id, /* max single read retries */ 10, - ReadSettings{}); + context->getReadSettings()); } void JsonMetadataGetter::handleJSON(const JSON & json) @@ -186,7 +195,7 @@ StorageDelta::StorageDelta( StorageInMemoryMetadata storage_metadata; StorageS3::updateS3Configuration(context_, base_configuration); - JsonMetadataGetter getter{base_configuration, table_path}; + JsonMetadataGetter getter{base_configuration, table_path, context_}; auto keys = getter.getFiles(); @@ -245,6 +254,9 @@ Pipe StorageDelta::read( String StorageDelta::generateQueryFromKeys(std::vector && keys) { + // DeltaLake store data parts in different files + // keys are filenames of parts + // for StorageS3 to read all parts we need format {key1,key2,key3,...keyn} std::string new_query = fmt::format("{{{}}}", fmt::join(keys, ",")); return new_query; } @@ -270,6 +282,7 @@ void registerStorageDelta(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + // DeltaLake uses Parquet by default if (configuration.format == "auto") configuration.format = "Parquet"; diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index d0992dd4cb7..f759d0086a2 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -42,16 +42,16 @@ private: class JsonMetadataGetter { public: - JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_); + JsonMetadataGetter(StorageS3::S3Configuration & configuration_, const String & table_path_, ContextPtr context); std::vector getFiles() { return std::move(metadata).ListCurrentFiles(); } private: - void Init(); + void Init(ContextPtr context); std::vector getJsonLogFiles(); - std::shared_ptr createS3ReadBuffer(const String & key); + std::shared_ptr createS3ReadBuffer(const String & key, ContextPtr context); void handleJSON(const JSON & json); @@ -63,6 +63,9 @@ private: class StorageDelta : public IStorage { public: + // 1. Parses internal file structure of table + // 2. Finds out parts with latest version + // 3. Creates url for underlying StorageS3 enigne to handle reads StorageDelta( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -74,6 +77,7 @@ public: String getName() const override { return "DeltaLake"; } + // Reads latest version of DeltaLake table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -85,6 +89,11 @@ public: private: void Init(); + + // DeltaLake stores data in parts in different files + // keys is vector of parts with latest version + // generateQueryFromKeys constructs query from parts filenames for + // underlying StorageS3 engine static String generateQueryFromKeys(std::vector && keys); StorageS3::S3Configuration base_configuration; diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index e2a46fad2d0..46016771f49 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -142,8 +142,7 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); // for each partition path take only latest file - - std::unordered_map> latest_parquets; + std::unordered_map> latest_parts; for (const auto & key : keys) { @@ -161,9 +160,9 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, // every filename contains metadata split by "_", timestamp is after last "_" uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); - auto it = latest_parquets.find(path); + auto it = latest_parts.find(path); - if (it != latest_parquets.end()) + if (it != latest_parts.end()) { if (it->second.second < timestamp) { @@ -172,13 +171,13 @@ std::string StorageHudi::generateQueryFromKeys(std::vector && keys, } else { - latest_parquets[path] = {key, timestamp}; + latest_parts[path] = {key, timestamp}; } } std::vector filtered_keys; std::transform( - latest_parquets.begin(), latest_parquets.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); + latest_parts.begin(), latest_parts.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); std::string new_query; @@ -218,6 +217,7 @@ void registerStorageHudi(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); + // Apache Hudi uses Parquet by default if (configuration.format == "auto") configuration.format = "Parquet"; diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index 91a77ec83ff..c25c347f4c2 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -23,6 +23,9 @@ namespace DB class StorageHudi : public IStorage { public: + // 1. Parses internal file structure of table + // 2. Finds out parts with latest version + // 3. Creates url for underlying StorageS3 enigne to handle reads StorageHudi( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -34,6 +37,8 @@ public: String getName() const override { return "Hudi"; } + + // Reads latest version of Apache Hudi table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -45,6 +50,12 @@ public: private: std::vector getKeysFromS3(); + + // Apache Hudi store parts of data in different files + // Every part file has timestamp in it + // Every partition(directory) in Apache Hudi has different versions of part + // To find needed parts we need to find out latest part file for every partition + // Part format is usually parquet, but can differ static std::string generateQueryFromKeys(std::vector && keys, String format); StorageS3::S3Configuration base_configuration; From 7500cc38c0e47de381a323ab25c90b10f013f468 Mon Sep 17 00:00:00 2001 From: Daniil Rubin Date: Fri, 4 Nov 2022 20:51:44 +0000 Subject: [PATCH 181/526] Apply clang-format --- src/Storages/StorageDelta.cpp | 4 ++-- src/Storages/StorageDelta.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index 59e7c3d3187..e14fa2401a4 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -111,7 +111,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() request.SetBucket(bucket); - // DeltaLake format stores all metadata json files in _delta_log directory + // DeltaLake format stores all metadata json files in _delta_log directory request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); while (!is_finished) @@ -139,7 +139,7 @@ std::vector JsonMetadataGetter::getJsonLogFiles() // Needed in case any more results are available // if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); - + /// Set to false if all of the results were returned. Set to true if more keys /// are available to return. If the number of results exceeds that specified by /// MaxKeys, all of the results might not be returned diff --git a/src/Storages/StorageDelta.h b/src/Storages/StorageDelta.h index f759d0086a2..e3bb4c0b416 100644 --- a/src/Storages/StorageDelta.h +++ b/src/Storages/StorageDelta.h @@ -89,7 +89,7 @@ public: private: void Init(); - + // DeltaLake stores data in parts in different files // keys is vector of parts with latest version // generateQueryFromKeys constructs query from parts filenames for From d5de68733af125e892d9772fe9fd6b1371c2c6d0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 4 Nov 2022 23:29:54 +0000 Subject: [PATCH 182/526] Disable keeper retries during INSERT by default --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index abd3a046a6b..89cb22ca9e2 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -658,7 +658,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ - M(UInt64, insert_keeper_max_retries, 20, "Max retries for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_max_retries, 0, "Max retries for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ From ea4235d4ed29ca35c43c51f4d9cfbf1fb1ff4a53 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sat, 5 Nov 2022 01:30:07 +0000 Subject: [PATCH 183/526] Try to fix lambda parsing --- src/Parsers/ExpressionListParsers.cpp | 10 +++++----- .../0_stateless/02476_fix_lambda_parsing.reference | 4 ++++ .../0_stateless/02476_fix_lambda_parsing.sh | 14 ++++++++++++++ 3 files changed, 23 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02476_fix_lambda_parsing.reference create mode 100755 tests/queries/0_stateless/02476_fix_lambda_parsing.sh diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index c362340d013..0f103690e37 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -682,7 +682,7 @@ public: bool parseLambda() { - // 0. If empty - create function tuple with 0 args + // 1. If empty - create function tuple with 0 args if (isCurrentElementEmpty()) { auto function = makeASTFunction("tuple"); @@ -693,16 +693,16 @@ public: if (operands.size() != 1 || !operators.empty() || !mergeElement()) return false; - /// 1. If there is already tuple do nothing - if (tryGetFunctionName(elements.back()) == "tuple") + /// 2. If there is already tuple do nothing + if (elements.size() == 1 && tryGetFunctionName(elements.back()) == "tuple") { pushOperand(elements.back()); elements.pop_back(); } - /// 2. Put all elements in a single tuple + /// 3. Put all elements in a single tuple else { - auto function = makeASTFunction("tuple", elements); + auto function = makeASTFunction("tuple", std::move(elements)); elements.clear(); pushOperand(function); } diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.reference b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference new file mode 100644 index 00000000000..fce203907eb --- /dev/null +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference @@ -0,0 +1,4 @@ +SELECT f((x, tuple(y)) -> z) +SELECT f((x, tuple(y)) -> z) +SELECT f((x, y) -> z) +SELECT f((x, y) -> z) diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.sh b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh new file mode 100755 index 00000000000..2f134010991 --- /dev/null +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh @@ -0,0 +1,14 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -e + +format="$CLICKHOUSE_FORMAT" + +echo "SELECT f((x, tuple(y)) -> z)" | $format +echo "select f(x, tuple(y) -> z)" | $format +echo "select f(x, y -> z)" | $format +echo "select f((x, y) -> z)" | $format From dd598fdd4a1b7fbbe502f586b0b2d4b8025af4d7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 5 Nov 2022 17:57:59 +0000 Subject: [PATCH 184/526] Added missed insert_keeper_max_retries setting in test config --- tests/config/users.d/insert_keeper_retries.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml index e4420d53fb0..462c9df5248 100644 --- a/tests/config/users.d/insert_keeper_retries.xml +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -1,7 +1,8 @@ - 0.01 + 20 + 0.01 From e94b9cd0039bafb1e2fbbc3fe88ef076014d3488 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 5 Nov 2022 22:28:32 +0000 Subject: [PATCH 185/526] Fixes + enable retries in integration test + typo in fault injection traces --- .../MergeTree/ZooKeeperWithFaultInjection.h | 16 ++++++++-------- .../test_inserts_with_keeper_retries/test.py | 8 ++++---- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h index c2e689b0307..491f97b52bd 100644 --- a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -36,7 +36,7 @@ private: }; /// -/// ZooKeeperWithFailtInjection mimics ZooKeeper interface and inject failures according to failure policy if set +/// ZooKeeperWithFaultInjection mimics ZooKeeper interface and inject failures according to failure policy if set /// class ZooKeeperWithFaultInjection { @@ -66,7 +66,7 @@ class ZooKeeperWithFaultInjection if (unlikely(logger)) LOG_TRACE( logger, - "ZooKeeperWithFailtInjection created: name={} seed={} fault_probability={}", + "ZooKeeperWithFaultInjection created: name={} seed={} fault_probability={}", name, seed, fault_injection_probability); @@ -103,7 +103,7 @@ public: if (unlikely(logger)) LOG_TRACE( logger, - "ZooKeeperWithFailtInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", + "ZooKeeperWithFaultInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", name, seed, calls_total, @@ -251,7 +251,7 @@ public: { keeper->remove(result_path); if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFailtInjection cleanup: seed={} func={} path={}", seed, "create", result_path); + LOG_TRACE(logger, "ZooKeeperWithFaultInjection cleanup: seed={} func={} path={}", seed, "create", result_path); } } catch (const zkutil::KeeperException & e) @@ -259,7 +259,7 @@ public: if (unlikely(logger)) LOG_TRACE( logger, - "ZooKeeperWithFailtInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", + "ZooKeeperWithFaultInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", seed, "create", result_path, @@ -462,7 +462,7 @@ private: ++calls_without_fault_injection; if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFailtInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); return res; } @@ -483,7 +483,7 @@ private: ++calls_without_fault_injection; if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFailtInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); } } catch (const zkutil::KeeperException & e) @@ -491,7 +491,7 @@ private: if (unlikely(logger)) LOG_TRACE( logger, - "ZooKeeperWithFailtInjection call FAILED: seed={} func={} path={} code={} message={} ", + "ZooKeeperWithFaultInjection call FAILED: seed={} func={} path={} code={} message={} ", seed, func_name, path, diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py index 09849998185..ffce2a09afc 100644 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -43,9 +43,9 @@ def test_replica_inserts_with_keeper_restart(started_cluster): zk_stopped_event.wait(90) - node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10)") + node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20") node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)" + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" ) job.wait() @@ -81,9 +81,9 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): ) disconnect_event.wait(90) - node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10)") + node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20") node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10)" + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" ) job.wait() From f44e4fe7c6b413a6f092651e7f5732e70bc747f3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sat, 5 Nov 2022 22:36:32 +0000 Subject: [PATCH 186/526] Automatic style fix --- .../integration/test_inserts_with_keeper_retries/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py index ffce2a09afc..dbf18365053 100644 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -43,7 +43,9 @@ def test_replica_inserts_with_keeper_restart(started_cluster): zk_stopped_event.wait(90) - node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20") + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) node1.query( "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" ) @@ -81,7 +83,9 @@ def test_replica_inserts_with_keeper_disconnect(started_cluster): ) disconnect_event.wait(90) - node1.query("INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20") + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) node1.query( "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" ) From ef6065b331c19a59ea1e7409e169afa206d697ba Mon Sep 17 00:00:00 2001 From: zhongyuankai <872237106@qq.com> Date: Sun, 6 Nov 2022 13:40:26 +0800 Subject: [PATCH 187/526] fix test --- src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c7d1786135a..c926b30781a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -326,7 +326,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (!parts_to_merge.empty()) future_part->merge_type = MergeType::TTLDelete; } - else if (metadata_snapshot->hasAnyRecompressionTTL()) + + if (parts_to_merge.empty() && metadata_snapshot->hasAnyRecompressionTTL()) { TTLRecompressMergeSelector recompress_ttl_selector( next_recompress_ttl_merge_times_by_partition, From 69d6d4244334b454b02271ec6b3d7bdb356a023f Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Sun, 6 Nov 2022 15:26:19 +0800 Subject: [PATCH 188/526] NOT LIKE only work for 'prefix%' --- src/Storages/MergeTree/KeyCondition.cpp | 29 ++++++++++++------- src/Storages/MergeTree/KeyCondition.h | 2 +- .../StorageSystemMergeTreeMetadataCache.cpp | 2 +- ...58_key_condition_not_like_prefix.reference | 1 + .../02458_key_condition_not_like_prefix.sql | 9 +++++- 5 files changed, 29 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3bea261fd52..a4ac9bf8c25 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -29,6 +29,7 @@ #include #include +#include #include #include #include @@ -55,11 +56,15 @@ String Range::toString() const } -/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. -/// If perfect_prefix_match == true, only consider pattern in the format `prefix%_` -String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match) +/// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' +/// We call a prefix "perfect" if: +/// - (1) the prefix must have willcard +/// - (2) the first wildcard is '%' and is only followed by nothing or other '%' +/// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) { String fixed_prefix; + fixed_prefix.reserve(like_pattern.size()); const char * pos = like_pattern.data(); const char * end = pos + like_pattern.size(); @@ -68,12 +73,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfe switch (*pos) { case '%': - [[fallthrough]]; case '_': - if (perfect_prefix_match && std::find_if(pos+1, end, [](const char c) { return c != '%' && c != '_'; }) != end) - return ""; + if (require_perfect_prefix) + { + bool is_prefect_prefix = std::all_of(pos, end, [](auto c) { return c == '%'; }); + return is_prefect_prefix ? fixed_prefix : ""; + } return fixed_prefix; - case '\\': ++pos; if (pos == end) @@ -81,12 +87,13 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfe [[fallthrough]]; default: fixed_prefix += *pos; - break; } ++pos; } - + /// If we can reach this code, it means there was no wildcard found in the pattern, so it is not a perfect prefix + if (require_perfect_prefix) + return ""; return fixed_prefix; } @@ -349,7 +356,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get()); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -370,7 +377,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), true); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index e7eb855b1cc..81cfea3d359 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -485,6 +485,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(const String & like_pattern, bool perfect_prefix_match = false); +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix); } diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index 275365648f3..776b85cbffd 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -116,7 +116,7 @@ void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, } else { - String target = extractFixedPrefixFromLikePattern(key); + String target = extractFixedPrefixFromLikePattern(key, /*required_perfect_prefix*/ false); if (target.empty()) throw Exception( "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference index 87766d889a3..02357561e02 100644 --- a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.reference @@ -1 +1,2 @@ 200000 +200000 diff --git a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql index 2c1402df27e..e821b16ed5c 100644 --- a/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql +++ b/tests/queries/0_stateless/02458_key_condition_not_like_prefix.sql @@ -2,4 +2,11 @@ CREATE TABLE data (str String) ENGINE=MergeTree ORDER BY str; INSERT INTO data (str) SELECT 'aa' FROM numbers(100000); INSERT INTO data (str) SELECT 'ba' FROM numbers(100000); INSERT INTO data (str) SELECT 'ca' FROM numbers(100000); -SELECT count()FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a%%' SETTINGS force_primary_key=1; +SELECT count() FROM data WHERE str NOT LIKE 'a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '%a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%_' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE '_a' SETTINGS force_primary_key=1; -- { serverError 277 } +SELECT count() FROM data WHERE str NOT LIKE 'a%\_' SETTINGS force_primary_key=1; -- { serverError 277 } From 30b3864636de4b5f05b21d1d27a7154504c445b5 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 08:12:05 +0000 Subject: [PATCH 189/526] Fix From f7e65ebd72a749c79cdaaa03635811ab5539baa2 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Nov 2022 16:25:36 +0800 Subject: [PATCH 190/526] make comments more clear --- src/Storages/MergeTree/KeyCondition.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index a4ac9bf8c25..a74a3e58dff 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -57,8 +57,8 @@ String Range::toString() const /// Returns the prefix of like_pattern before the first wildcard, e.g. 'Hello\_World% ...' --> 'Hello\_World' -/// We call a prefix "perfect" if: -/// - (1) the prefix must have willcard +/// We call a pattern "perfect prefix" if: +/// - (1) the pattern has a wildcard /// - (2) the first wildcard is '%' and is only followed by nothing or other '%' /// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) From da97e321106df336a95ad2760ee931cb46889135 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 08:26:01 +0000 Subject: [PATCH 191/526] Fix workflow --- .github/workflows/pull_request.yml | 75 +----------------------------- 1 file changed, 1 insertion(+), 74 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 0f774791c26..5358d476636 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -69,7 +69,7 @@ jobs: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json DockerHubPush: - needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] + needs: [DockerHubPushAmd64, DockerHubPushAarch64] runs-on: [self-hosted, style-checker] steps: - name: Clear repository @@ -108,80 +108,7 @@ jobs: uses: ./.github/workflows/jepsen.yml FinishCheck: needs: - - StyleCheck - DockerHubPush - - DockerServerImages - - CheckLabels - - BuilderReport - - FastTest - - FunctionalStatelessTestDebug0 - - FunctionalStatelessTestDebug1 - - FunctionalStatelessTestDebug2 - - FunctionalStatelessTestRelease - - FunctionalStatelessTestReleaseDatabaseReplicated0 - - FunctionalStatelessTestReleaseDatabaseReplicated1 - - FunctionalStatelessTestReleaseWideParts - - FunctionalStatelessTestAarch64 - - FunctionalStatelessTestAsan0 - - FunctionalStatelessTestAsan1 - - FunctionalStatelessTestTsan0 - - FunctionalStatelessTestTsan1 - - FunctionalStatelessTestTsan2 - - FunctionalStatelessTestMsan0 - - FunctionalStatelessTestMsan1 - - FunctionalStatelessTestMsan2 - - FunctionalStatelessTestUBsan - - FunctionalStatefulTestDebug - - FunctionalStatefulTestRelease - - FunctionalStatefulTestAarch64 - - FunctionalStatefulTestAsan - - FunctionalStatefulTestTsan - - FunctionalStatefulTestMsan - - FunctionalStatefulTestUBsan - - FunctionalStatelessTestReleaseS3 - - FunctionalStatelessTestS3Debug0 - - FunctionalStatelessTestS3Debug1 - - FunctionalStatelessTestS3Debug2 - - FunctionalStatelessTestS3Tsan0 - - FunctionalStatelessTestS3Tsan1 - - FunctionalStatelessTestS3Tsan2 - - StressTestDebug - - StressTestAsan - - StressTestTsan - - StressTestMsan - - StressTestUBsan - - ASTFuzzerTestDebug - - ASTFuzzerTestAsan - - ASTFuzzerTestTsan - - ASTFuzzerTestMSan - - ASTFuzzerTestUBSan - - IntegrationTestsAsan0 - - IntegrationTestsAsan1 - - IntegrationTestsAsan2 - - IntegrationTestsRelease0 - - IntegrationTestsRelease1 - - IntegrationTestsTsan0 - - IntegrationTestsTsan1 - - IntegrationTestsTsan2 - - IntegrationTestsTsan3 - - PerformanceComparisonX86-0 - - PerformanceComparisonX86-1 - - PerformanceComparisonX86-2 - - PerformanceComparisonX86-3 - - PerformanceComparisonAarch-0 - - PerformanceComparisonAarch-1 - - PerformanceComparisonAarch-2 - - PerformanceComparisonAarch-3 - - UnitTestsAsan - - UnitTestsTsan - - UnitTestsMsan - - UnitTestsUBsan - - UnitTestsReleaseClang - - SharedBuildSmokeTest - - CompatibilityCheck - - IntegrationTestsFlakyCheck - - SQLancerTestRelease - - SQLancerTestDebug runs-on: [self-hosted, style-checker] steps: - name: Clear repository From 291e8931c1b9d3fd6edcb09ffae484e4215d4e0d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 08:42:24 +0000 Subject: [PATCH 192/526] Build docker image --- .github/workflows/jepsen.yml | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 49a04008a51..3057502fa55 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -45,8 +45,8 @@ jobs: - name: Set envs run: | cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/keeper_jepsen - REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse + TEMP_PATH=${{runner.temp}}/server_jepsen + REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse EOF - name: Clear repository run: | @@ -60,8 +60,9 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + docker build -f "$REPO_COPY/docker/test/server-jepsen/Dockerfile" -t clickhouse/server-jepsen-test "$REPO_COPY/docker/test/server-jepsen" cd "$REPO_COPY/tests/ci" - python3 jepsen_check.py keeper + python3 jepsen_check.py server - name: Cleanup if: always() run: | From 8e1a4bcf9a6376a36509e09efd9df7486346f791 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 08:49:26 +0000 Subject: [PATCH 193/526] Update CI config --- .github/workflows/pull_request.yml | 68 +------------------ docker/test/server-jepsen/Dockerfile | 2 +- tests/ci/ci_config.py | 3 + tests/ci/jepsen_check.py | 2 +- tests/jepsen.clickhouse/resources/config.xml | 2 +- .../resources/keeper_config.xml | 2 +- .../resources/keeper_config_solo.xml | 2 +- .../src/jepsen/clickhouse/server/client.clj | 4 +- .../src/jepsen/clickhouse/server/utils.clj | 2 +- .../src/jepsen/clickhouse/utils.clj | 1 + 10 files changed, 14 insertions(+), 74 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 5358d476636..199a5d71f92 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -32,70 +32,6 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py - DockerHubPushAarch64: - needs: CheckLabels - runs-on: [self-hosted, style-checker-aarch64] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_images_check.py --suffix aarch64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 - with: - name: changed_images_aarch64 - path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json - DockerHubPushAmd64: - needs: CheckLabels - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_images_check.py --suffix amd64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 - with: - name: changed_images_amd64 - path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json - DockerHubPush: - needs: [DockerHubPushAmd64, DockerHubPushAarch64] - runs-on: [self-hosted, style-checker] - steps: - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download changed aarch64 images - uses: actions/download-artifact@v2 - with: - name: changed_images_aarch64 - path: ${{ runner.temp }} - - name: Download changed amd64 images - uses: actions/download-artifact@v2 - with: - name: changed_images_amd64 - path: ${{ runner.temp }} - - name: Images check - run: | - cd "$GITHUB_WORKSPACE/tests/ci" - python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 - - name: Upload images files to artifacts - uses: actions/upload-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/changed_images.json ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# @@ -104,11 +40,11 @@ jobs: # When it's skipped, all dependent tasks will be skipped too. # DO NOT add it there if: contains(github.event.pull_request.labels.*.name, 'jepsen-test') - needs: [DockerHubPush] + needs: [CheckLabels] uses: ./.github/workflows/jepsen.yml FinishCheck: needs: - - DockerHubPush + - CheckLabels runs-on: [self-hosted, style-checker] steps: - name: Clear repository diff --git a/docker/test/server-jepsen/Dockerfile b/docker/test/server-jepsen/Dockerfile index 0335186cf1d..974da67e853 100644 --- a/docker/test/server-jepsen/Dockerfile +++ b/docker/test/server-jepsen/Dockerfile @@ -12,7 +12,7 @@ ENV SHA_TO_TEST="" ENV NODES_USERNAME="root" ENV NODES_PASSWORD="" -ENV TESTS_TO_RUN="30" +ENV TESTS_TO_RUN="1" ENV TIME_LIMIT="30" ENV KEEPER_NODE="" diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 93322b69669..bfc7e45812b 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -347,6 +347,9 @@ CI_CONFIG = { "ClickHouse Keeper Jepsen": { "required_build": "binary_release", }, + "ClickHouse Server Jepsen": { + "required_build": "binary_release", + }, "Performance Comparison": { "required_build": "package_release", "test_grep_exclude_filter": "", diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 87963a158c2..8b898a34e67 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -194,7 +194,7 @@ if __name__ == "__main__": os.makedirs(result_path) instances = prepare_autoscaling_group_and_get_hostnames(KEEPER_DESIRED_INSTANCE_COUNT if args.program == 'keeper' else SERVER_DESIRED_INSTANCE_COUNT) - nodes_path = save_nodes_to_file(instances[:SERVER_DESIRED_INSTANCE_COUNT - 1], TEMP_PATH) + nodes_path = save_nodes_to_file(instances[:KEEPER_DESIRED_INSTANCE_COUNT], TEMP_PATH) # always use latest docker_image = KEEPER_IMAGE_NAME if args.program == 'keeper' else SERVER_IMAGE_NAME diff --git a/tests/jepsen.clickhouse/resources/config.xml b/tests/jepsen.clickhouse/resources/config.xml index 25e38a58e63..ae0570a3a5f 100644 --- a/tests/jepsen.clickhouse/resources/config.xml +++ b/tests/jepsen.clickhouse/resources/config.xml @@ -1,5 +1,5 @@ - 0.0.0.0 + :: 9000 8123 diff --git a/tests/jepsen.clickhouse/resources/keeper_config.xml b/tests/jepsen.clickhouse/resources/keeper_config.xml index 20ca74c1cf5..2ab747fbd71 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config.xml @@ -1,5 +1,5 @@ - 0.0.0.0 + :: trace diff --git a/tests/jepsen.clickhouse/resources/keeper_config_solo.xml b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml index dfbc098b3dd..b20592545cd 100644 --- a/tests/jepsen.clickhouse/resources/keeper_config_solo.xml +++ b/tests/jepsen.clickhouse/resources/keeper_config_solo.xml @@ -1,5 +1,5 @@ - 0.0.0.0 + :: trace diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj index e051d361c73..9335a005149 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj @@ -22,12 +22,12 @@ (util/timeout 30000 (throw (RuntimeException. (str "Connection to " node " timed out"))) - (util/retry 0.1 + ;(util/retry 0.1 (let [spec (db-spec node) connection (j/get-connection spec) added-connection (j/add-connection spec connection)] (assert added-connection) - added-connection)))) + added-connection))) (defn close-connection "Close connection" diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj index 6e99da769a4..a61c825681a 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj @@ -11,7 +11,7 @@ (let [c (chc/open-connection node)] (jdbc/query c "SELECT 1") (chc/close-connection c)) - (catch Exception _ false))) + (catch Exception e (do (info e) false)))) (defn start-clickhouse! [node test] diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj index cbf18129d88..c29e11daadb 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/utils.clj @@ -132,6 +132,7 @@ :chdir data-dir} binary-path binary-args) + (info "Waiting for server") (wait-clickhouse-alive! node test clickhouse-alive?))) (defn prepare-dirs From bd2bd7149db533dc19b9a81877ea4d357f3be62b Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Mon, 7 Nov 2022 18:22:50 +0800 Subject: [PATCH 194/526] fix typo --- src/Storages/MergeTree/KeyCondition.cpp | 10 +++++----- src/Storages/MergeTree/KeyCondition.h | 2 +- .../System/StorageSystemMergeTreeMetadataCache.cpp | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index a74a3e58dff..8ffd1059a58 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -61,7 +61,7 @@ String Range::toString() const /// - (1) the pattern has a wildcard /// - (2) the first wildcard is '%' and is only followed by nothing or other '%' /// e.g. 'test%' or 'test%% has perfect prefix 'test', 'test%x', 'test%_' or 'test_' has no perfect prefix. -String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix) +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix) { String fixed_prefix; fixed_prefix.reserve(like_pattern.size()); @@ -74,7 +74,7 @@ String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool req { case '%': case '_': - if (require_perfect_prefix) + if (requires_perfect_prefix) { bool is_prefect_prefix = std::all_of(pos, end, [](auto c) { return c == '%'; }); return is_prefect_prefix ? fixed_prefix : ""; @@ -92,7 +92,7 @@ String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool req ++pos; } /// If we can reach this code, it means there was no wildcard found in the pattern, so it is not a perfect prefix - if (require_perfect_prefix) + if (requires_perfect_prefix) return ""; return fixed_prefix; } @@ -356,7 +356,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ false); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ false); if (prefix.empty()) return false; @@ -377,7 +377,7 @@ const KeyCondition::AtomMap KeyCondition::atom_map if (value.getType() != Field::Types::String) return false; - String prefix = extractFixedPrefixFromLikePattern(value.get(), /*required_perfect_prefix*/ true); + String prefix = extractFixedPrefixFromLikePattern(value.get(), /*requires_perfect_prefix*/ true); if (prefix.empty()) return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 81cfea3d359..138dce83db9 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -485,6 +485,6 @@ private: bool strict; }; -String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool require_perfect_prefix); +String extractFixedPrefixFromLikePattern(std::string_view like_pattern, bool requires_perfect_prefix); } diff --git a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp index 776b85cbffd..731cdf94157 100644 --- a/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp +++ b/src/Storages/System/StorageSystemMergeTreeMetadataCache.cpp @@ -116,7 +116,7 @@ void StorageSystemMergeTreeMetadataCache::fillData(MutableColumns & res_columns, } else { - String target = extractFixedPrefixFromLikePattern(key, /*required_perfect_prefix*/ false); + String target = extractFixedPrefixFromLikePattern(key, /*requires_perfect_prefix*/ false); if (target.empty()) throw Exception( "SELECT from system.merge_tree_metadata_cache table must contain condition like key = 'key' or key LIKE 'prefix%' in WHERE clause.", ErrorCodes::BAD_ARGUMENTS); From 7467490861c9da7b55c950686a9f5540a482ea3d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 09:50:39 +0000 Subject: [PATCH 195/526] Increase test count --- .github/workflows/jepsen.yml | 61 ++++++++++--------- docker/test/server-jepsen/Dockerfile | 2 +- .../src/jepsen/clickhouse/server/client.clj | 4 +- .../src/jepsen/clickhouse/server/utils.clj | 2 +- 4 files changed, 36 insertions(+), 33 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 3057502fa55..16433e2baed 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -10,37 +10,40 @@ on: # yamllint disable-line rule:truthy workflow_dispatch: workflow_call: jobs: - #KeeperJepsenRelease: - # runs-on: [self-hosted, style-checker] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/keeper_jepsen - # REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse - # EOF - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # with: - # fetch-depth: 0 - # - name: Jepsen Test - # run: | - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" - # python3 jepsen_check.py keeper - # - name: Cleanup - # if: always() - # run: | - # docker ps --quiet | xargs --no-run-if-empty docker kill ||: - # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - # sudo rm -fr "$TEMP_PATH" + KeeperJepsenRelease: + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/keeper_jepsen + REPO_COPY=${{runner.temp}}/keeper_jepsen/ClickHouse + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 + - name: Jepsen Test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + docker build -f "$REPO_COPY/docker/test/keeper-jepsen/Dockerfile" -t clickhouse/keeper-jepsen-test "$REPO_COPY/docker/test/keeper-jepsen" + cd "$REPO_COPY/tests/ci" + python3 jepsen_check.py keeper + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ServerJepsenRelease: runs-on: [self-hosted, style-checker] + if: ${{ always() }} + needs: [KeeperJepsenRelease] steps: - name: Set envs run: | diff --git a/docker/test/server-jepsen/Dockerfile b/docker/test/server-jepsen/Dockerfile index 974da67e853..958dbfa066a 100644 --- a/docker/test/server-jepsen/Dockerfile +++ b/docker/test/server-jepsen/Dockerfile @@ -12,7 +12,7 @@ ENV SHA_TO_TEST="" ENV NODES_USERNAME="root" ENV NODES_PASSWORD="" -ENV TESTS_TO_RUN="1" +ENV TESTS_TO_RUN="8" ENV TIME_LIMIT="30" ENV KEEPER_NODE="" diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj index 9335a005149..e051d361c73 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/client.clj @@ -22,12 +22,12 @@ (util/timeout 30000 (throw (RuntimeException. (str "Connection to " node " timed out"))) - ;(util/retry 0.1 + (util/retry 0.1 (let [spec (db-spec node) connection (j/get-connection spec) added-connection (j/add-connection spec connection)] (assert added-connection) - added-connection))) + added-connection)))) (defn close-connection "Close connection" diff --git a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj index a61c825681a..993f27f922f 100644 --- a/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj +++ b/tests/jepsen.clickhouse/src/jepsen/clickhouse/server/utils.clj @@ -11,7 +11,7 @@ (let [c (chc/open-connection node)] (jdbc/query c "SELECT 1") (chc/close-connection c)) - (catch Exception e (do (info e) false)))) + (catch Exception e false))) (defn start-clickhouse! [node test] From 6603df8e136393ee49866d016d767c74a185ba80 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Mon, 7 Nov 2022 11:28:21 +0000 Subject: [PATCH 196/526] fix tests --- .../02474_timeDiff_UTCTimestamp.reference | 18 +++++++++--------- .../02474_timeDiff_UTCTimestamp.sql | 18 +++++++++--------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference index bc6ff2b709c..bb0850568bb 100644 --- a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.reference @@ -1,9 +1,9 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql index eac5edae483..231bebc1503 100644 --- a/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql +++ b/tests/queries/0_stateless/02474_timeDiff_UTCTimestamp.sql @@ -1,12 +1,12 @@ -- all tests should be equal to zero as timediff is same as dateDiff('second', ... ) -SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')) - timeDiff(toDate32('1927-01-01'), toDate32('1927-01-02')) <= 2; +SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) - timeDiff(toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)) <= 2; +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) - timeDiff(toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')) <= 2; +SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) - timeDiff(toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')) <= 2; +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) - timeDiff(toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')) <= 2; +SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')) - timeDiff(toDate32('2015-08-18'), toDate('2015-08-19')) <= 2; +SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')) - timeDiff(toDate('2015-08-18'), toDate32('2015-08-19')) <= 2; -- UTCTimestamp equals to now('UTC') -SELECT dateDiff('s', UTCTimestamp(), now('UTC')); -SELECT timeDiff(UTCTimestamp(), now('UTC')); +SELECT dateDiff('s', UTCTimestamp(), now('UTC')) <= 2; +SELECT timeDiff(UTCTimestamp(), now('UTC')) <= 2; From 897811371dc47314e5224a96cd40ecd485a21770 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 7 Nov 2022 12:45:26 +0100 Subject: [PATCH 197/526] Minor changes --- src/Storages/StorageDelta.cpp | 65 +++++++++---------- src/Storages/StorageHudi.cpp | 114 ++++++++++++++++------------------ src/Storages/StorageHudi.h | 20 +++--- 3 files changed, 94 insertions(+), 105 deletions(-) diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index e14fa2401a4..e8287a2fd61 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -1,29 +1,29 @@ #include "config.h" - #if USE_AWS_S3 -# include -# include +#include +#include -# include -# include -# include -# include +#include +#include +#include +#include -# include -# include -# include +#include +#include +#include -# include +#include -# include -# include -# include +#include +#include +#include -# include +#include -# include -# include +#include +#include +#include namespace DB { @@ -53,9 +53,7 @@ std::vector DeltaLakeMetadata::ListCurrentFiles() && keys.reserve(file_update_time.size()); for (auto && [k, _] : file_update_time) - { keys.push_back(k); - } return keys; } @@ -75,12 +73,13 @@ void JsonMetadataGetter::Init(ContextPtr context) { auto buf = createS3ReadBuffer(key, context); + char c; while (!buf->eof()) { - // may be some invalid characters before json - char c; + /// May be some invalid characters before json. while (buf->peek(c) && c != '{') buf->ignore(); + if (buf->eof()) break; @@ -91,7 +90,6 @@ void JsonMetadataGetter::Init(ContextPtr context) continue; const JSON json(json_str); - handleJSON(json); } } @@ -111,8 +109,9 @@ std::vector JsonMetadataGetter::getJsonLogFiles() request.SetBucket(bucket); - // DeltaLake format stores all metadata json files in _delta_log directory - request.SetPrefix(std::filesystem::path(table_path) / "_delta_log"); + /// DeltaLake format stores all metadata json files in _delta_log directory + static constexpr auto deltalake_metadata_directory = "_delta_log"; + request.SetPrefix(std::filesystem::path(table_path) / deltalake_metadata_directory); while (!is_finished) { @@ -136,8 +135,8 @@ std::vector JsonMetadataGetter::getJsonLogFiles() keys.push_back(filename); } - // Needed in case any more results are available - // if so, we will continue reading, and not read keys that were already read + /// Needed in case any more results are available + /// if so, we will continue reading, and not read keys that were already read request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); /// Set to false if all of the results were returned. Set to true if more keys @@ -151,13 +150,13 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { - // TBD: add parallel downloads + /// TODO: add parallel downloads return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, - /* max single read retries */ 10, + /* max single read retries */10, context->getReadSettings()); } @@ -198,7 +197,6 @@ StorageDelta::StorageDelta( JsonMetadataGetter getter{base_configuration, table_path, context_}; auto keys = getter.getFiles(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys)); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -281,12 +279,11 @@ void registerStorageDelta(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - - // DeltaLake uses Parquet by default - if (configuration.format == "auto") + else + { + /// DeltaLake uses Parquet by default. configuration.format = "Parquet"; - - //auto format_settings = getFormatSettings(args.getContext()); + } return std::make_shared( configuration, args.table_id, args.columns, args.constraints, args.comment, args.getContext(), std::nullopt); diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 46016771f49..d31d04beef5 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -2,18 +2,21 @@ #if USE_AWS_S3 -# include -# include +#include +#include -# include -# include -# include -# include -# include -# include -# include +#include +#include +#include +#include +#include +#include +#include +#include -# include +#include + +#include namespace DB { @@ -41,7 +44,6 @@ StorageHudi::StorageHudi( StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); @@ -121,7 +123,7 @@ std::vector StorageHudi::getKeysFromS3() const auto & result_batch = outcome.GetResult().GetContents(); for (const auto & obj : result_batch) { - const auto & filename = obj.GetKey().substr(table_path.size()); // object name without tablepath prefix + const auto & filename = obj.GetKey().substr(table_path.size()); /// Object name without tablepath prefix. keys.push_back(filename); LOG_DEBUG(log, "Found file: {}", filename); } @@ -133,65 +135,55 @@ std::vector StorageHudi::getKeysFromS3() return keys; } -std::string StorageHudi::generateQueryFromKeys(std::vector && keys, String format) +String StorageHudi::generateQueryFromKeys(const std::vector & keys, const String & format) { - // make format lowercase - std::transform(format.begin(), format.end(), format.begin(), [](unsigned char c) { return std::tolower(c); }); - - // filter only files with specific format - std::erase_if(keys, [&format](const std::string & s) { return std::filesystem::path(s).extension() != "." + format; }); - - // for each partition path take only latest file - std::unordered_map> latest_parts; - - for (const auto & key : keys) + /// For each partition path take only latest file. + struct FileInfo { - auto slash = key.find_last_of("/"); - std::string path; - if (slash == std::string::npos) - { - path = ""; - } - else - { - path = key.substr(0, slash); - } + String filename; + UInt64 timestamp; + }; + std::unordered_map latest_parts; /// Partition path (directory) -> latest part file info. - // every filename contains metadata split by "_", timestamp is after last "_" - uint64_t timestamp = std::stoul(key.substr(key.find_last_of("_") + 1)); + /// Make format lowercase. + const auto expected_extension= "." + Poco::toLower(format); + /// Filter only files with specific format. + auto keys_filter = [&](const String & key) { return std::filesystem::path(key).extension() == expected_extension; }; - auto it = latest_parts.find(path); + for (const auto & key : keys | std::views::filter(keys_filter)) + { + const auto key_path = fs::path(key); + const String filename = key_path.filename(); + const String partition_path = key_path.parent_path(); - if (it != latest_parts.end()) + /// Every filename contains metadata split by "_", timestamp is after last "_". + const auto delim = key.find_last_of("_") + 1; + if (delim == std::string::npos) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format of metadata files"); + const auto timestamp = parse(key.substr(delim + 1)); + + auto it = latest_parts.find(partition_path); + if (it == latest_parts.end()) { - if (it->second.second < timestamp) - { - it->second = {key, timestamp}; - } + latest_parts.emplace(partition_path, FileInfo{filename, timestamp}); } - else + else if (it->second.timestamp < timestamp) { - latest_parts[path] = {key, timestamp}; + it->second = {filename, timestamp}; } } - std::vector filtered_keys; - std::transform( - latest_parts.begin(), latest_parts.end(), std::back_inserter(filtered_keys), [](const auto & kv) { return kv.second.first; }); + std::string list_of_keys; - std::string new_query; - - for (auto && key : filtered_keys) + for (const auto & [directory, file_info] : latest_parts) { - if (!new_query.empty()) - { - new_query += ","; - } - new_query += key; - } - new_query = "{" + new_query + "}"; + if (!list_of_keys.empty()) + list_of_keys += ","; - return new_query; + list_of_keys += std::filesystem::path(directory) / file_info.filename; + } + + return "{" + list_of_keys + "}"; } @@ -207,7 +199,6 @@ void registerStorageHudi(StorageFactory & factory) ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage Hudi requires 3 to 4 arguments: table_url, access_key, secret_access_key, [format]"); - StorageS3Configuration configuration; configuration.url = checkAndGetLiteralArgument(engine_args[0], "url"); @@ -216,10 +207,11 @@ void registerStorageHudi(StorageFactory & factory) if (engine_args.size() == 4) configuration.format = checkAndGetLiteralArgument(engine_args[3], "format"); - - // Apache Hudi uses Parquet by default - if (configuration.format == "auto") + else + { + // Apache Hudi uses Parquet by default configuration.format = "Parquet"; + } auto format_settings = getFormatSettings(args.getContext()); diff --git a/src/Storages/StorageHudi.h b/src/Storages/StorageHudi.h index c25c347f4c2..bebda4cd4f6 100644 --- a/src/Storages/StorageHudi.h +++ b/src/Storages/StorageHudi.h @@ -23,9 +23,9 @@ namespace DB class StorageHudi : public IStorage { public: - // 1. Parses internal file structure of table - // 2. Finds out parts with latest version - // 3. Creates url for underlying StorageS3 enigne to handle reads + /// 1. Parses internal file structure of table. + /// 2. Finds out parts with latest version. + /// 3. Creates url for underlying StorageS3 enigne to handle reads. StorageHudi( const StorageS3Configuration & configuration_, const StorageID & table_id_, @@ -38,7 +38,7 @@ public: String getName() const override { return "Hudi"; } - // Reads latest version of Apache Hudi table + /// Reads latest version of Apache Hudi table Pipe read( const Names & column_names, const StorageSnapshotPtr & storage_snapshot, @@ -51,12 +51,12 @@ public: private: std::vector getKeysFromS3(); - // Apache Hudi store parts of data in different files - // Every part file has timestamp in it - // Every partition(directory) in Apache Hudi has different versions of part - // To find needed parts we need to find out latest part file for every partition - // Part format is usually parquet, but can differ - static std::string generateQueryFromKeys(std::vector && keys, String format); + /// Apache Hudi store parts of data in different files. + /// Every part file has timestamp in it. + /// Every partition(directory) in Apache Hudi has different versions of part. + /// To find needed parts we need to find out latest part file for every partition. + /// Part format is usually parquet, but can differ. + static String generateQueryFromKeys(const std::vector & keys, const String & format); StorageS3::S3Configuration base_configuration; std::shared_ptr s3engine; From af09ef53516d9a942d0150c796c2b1aaae184e1d Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 11:46:47 +0000 Subject: [PATCH 198/526] Enable release build --- .github/workflows/pull_request.yml | 122 ++++++++++++++++++++++++++++- tests/ci/jepsen_check.py | 32 ++++---- 2 files changed, 137 insertions(+), 17 deletions(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 199a5d71f92..3b7df032cd3 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -32,6 +32,126 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py + PythonUnitTests: + needs: CheckLabels + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Python unit tests + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 -m unittest discover -s . -p '*_test.py' + DockerHubPushAarch64: + needs: CheckLabels + runs-on: [self-hosted, style-checker-aarch64] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_images_check.py --suffix aarch64 + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images_aarch64 + path: ${{ runner.temp }}/docker_images_check/changed_images_aarch64.json + DockerHubPushAmd64: + needs: CheckLabels + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_images_check.py --suffix amd64 + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images_amd64 + path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json + DockerHubPush: + needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed aarch64 images + uses: actions/download-artifact@v2 + with: + name: changed_images_aarch64 + path: ${{ runner.temp }} + - name: Download changed amd64 images + uses: actions/download-artifact@v2 + with: + name: changed_images_amd64 + path: ${{ runner.temp }} + - name: Images check + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_manifests_merge.py --suffix amd64 --suffix aarch64 + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/changed_images.json + BuilderBinRelease: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_release + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# @@ -40,7 +160,7 @@ jobs: # When it's skipped, all dependent tasks will be skipped too. # DO NOT add it there if: contains(github.event.pull_request.labels.*.name, 'jepsen-test') - needs: [CheckLabels] + needs: [BuilderBinRelease] uses: ./.github/workflows/jepsen.yml FinishCheck: needs: diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 8b898a34e67..6f9f77e2cdd 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -166,16 +166,16 @@ if __name__ == "__main__": pr_info = PRInfo() - #logging.info( - # "Start at PR number %s, commit sha %s labels %s", - # pr_info.number, - # pr_info.sha, - # pr_info.labels, - #) + logging.info( + "Start at PR number %s, commit sha %s labels %s", + pr_info.number, + pr_info.sha, + pr_info.labels, + ) - #if pr_info.number != 0 and "jepsen-test" not in pr_info.labels: - # logging.info("Not jepsen test label in labels list, skipping") - # sys.exit(0) + if pr_info.number != 0 and "jepsen-test" not in pr_info.labels: + logging.info("Not jepsen test label in labels list, skipping") + sys.exit(0) gh = Github(get_best_robot_token(), per_page=100) @@ -201,18 +201,18 @@ if __name__ == "__main__": build_name = get_build_name_for_check(check_name) - #if pr_info.number == 0: - # version = get_version_from_repo() - # release_or_pr = f"{version.major}.{version.minor}" - #else: - # # PR number for anything else - # release_or_pr = str(pr_info.number) + if pr_info.number == 0: + version = get_version_from_repo() + release_or_pr = f"{version.major}.{version.minor}" + else: + # PR number for anything else + release_or_pr = str(pr_info.number) # This check run separately from other checks because it requires exclusive # run (see .github/workflows/jepsen.yml) So we cannot add explicit # dependency on a build job and using busy loop on it's results. For the # same reason we are using latest docker image. - build_url = f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/22.11/4012fcbf0191e9e64a34430db1468a2ae0744616/{build_name}/clickhouse" + build_url = f"{S3_DOWNLOAD}/{S3_BUILDS_BUCKET}/{release_or_pr}/{pr_info.sha}/{build_name}/clickhouse" head = requests.head(build_url) counter = 0 while head.status_code != 200: From 4861edb14b3a5758efae50a479a96bb9fd21eb7b Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 13:15:23 +0000 Subject: [PATCH 199/526] bring back all tests --- .github/workflows/pull_request.yml | 3485 +++++++++++++++++++++++++++- 1 file changed, 3484 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 3b7df032cd3..09ca64977f0 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -109,8 +109,189 @@ jobs: with: name: changed_images path: ${{ runner.temp }}/changed_images.json + StyleCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() || always() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{ runner.temp }}/style_check + ROBOT_CLICKHOUSE_SSH_KEY<> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/fasttest + REPO_COPY=${{runner.temp}}/fasttest/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" + mkdir "$GITHUB_WORKSPACE" + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.TEMP_PATH }} + - name: Fast Test + run: | + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 fast_test_check.py + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + CompatibilityCheck: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/compatibility_check + REPO_COPY=${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: CompatibilityCheck + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 compatibility_check.py + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + SharedBuildSmokeTest: + needs: [BuilderDebShared] + runs-on: [self-hosted, style-checker] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/split_build_check + REPO_COPY=${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH=${{runner.temp}}/reports_dir + EOF + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Shared build check + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 split_build_smoke_check.py + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +######################################################################################### +#################################### ORDINARY BUILDS #################################### +######################################################################################### + BuilderDebRelease: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_release + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" BuilderBinRelease: - needs: [DockerHubPush] + needs: [DockerHubPush, FastTest, StyleCheck] runs-on: [self-hosted, builder] steps: - name: Set envs @@ -152,6 +333,3235 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAarch64: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # for performance artifact + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebAsan: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_asan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebUBsan: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_ubsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebTsan: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_tsan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebMsan: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_msan + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderDebDebug: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=package_debug + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +########################################################################################## +##################################### SPECIAL BUILDS ##################################### +########################################################################################## + BuilderDebShared: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_shared + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinClangTidy: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_tidy + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwin: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAarch64: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinFreeBSD: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_freebsd + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinDarwinAarch64: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_darwin_aarch64 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinPPC64: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_ppc64le + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAmd64SSE2: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_amd64sse2 + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + BuilderBinAarch64V80Compat: + needs: [DockerHubPush, FastTest, StyleCheck] + runs-on: [self-hosted, builder] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/build_check + IMAGES_PATH=${{runner.temp}}/images_path + REPO_COPY=${{runner.temp}}/build_check/ClickHouse + CACHES_PATH=${{runner.temp}}/../ccaches + BUILD_NAME=binary_aarch64_v80compat + EOF + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ env.IMAGES_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Build + run: | + git -C "$GITHUB_WORKSPACE" submodule sync --recursive + git -C "$GITHUB_WORKSPACE" submodule update --depth=1 --recursive --init --jobs=10 + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" && python3 build_check.py "$BUILD_NAME" + - name: Upload build URLs to artifacts + if: ${{ success() || failure() }} + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_URLS }} + path: ${{ env.TEMP_PATH }}/${{ env.BUILD_URLS }}.json + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" +############################################################################################ +##################################### Docker images ####################################### +############################################################################################ + DockerServerImages: + needs: + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + with: + fetch-depth: 0 # It MUST BE THE SAME for all dependencies and the job itself + - name: Check docker clickhouse/clickhouse-server building + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 docker_server.py --release-type head --no-push + python3 docker_server.py --release-type head --no-push --no-ubuntu \ + --image-repo clickhouse/clickhouse-keeper --image-path docker/keeper + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderBinRelease + - BuilderDebAarch64 + - BuilderDebAsan + - BuilderDebDebug + - BuilderDebMsan + - BuilderDebRelease + - BuilderDebTsan + - BuilderDebUBsan + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + CHECK_NAME=ClickHouse build check + REPORTS_PATH=${{runner.temp}}/reports_dir + TEMP_PATH=${{runner.temp}}/report_check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + BuilderSpecialReport: + needs: + - BuilderBinAarch64 + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderBinFreeBSD + # - BuilderBinGCC + - BuilderBinPPC64 + - BuilderBinAmd64SSE2 + - BuilderBinAarch64V80Compat + - BuilderBinClangTidy + - BuilderDebShared + runs-on: [self-hosted, style-checker] + if: ${{ success() || failure() }} + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/report_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=ClickHouse special build check + NEEDS_DATA_PATH=${{runner.temp}}/needs.json + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cat > "$NEEDS_DATA_PATH" << 'EOF' + ${{ toJSON(needs) }} + EOF + cd "$GITHUB_WORKSPACE/tests/ci" + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +########################### FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseDatabaseReplicated1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_database_replicated + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, DatabaseReplicated) + REPO_COPY=${{runner.temp}}/stateless_database_replicated/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseWideParts: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_wide_parts + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, wide parts enabled) + REPO_COPY=${{runner.temp}}/stateless_wide_parts/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestReleaseS3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (release, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Debug0: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Debug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Debug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestS3Tsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_s3_storage_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan, s3 storage) + REPO_COPY=${{runner.temp}}/stateless_s3_storage_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (aarch64) + REPO_COPY=${{runner.temp}}/stateless_release/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (asan) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (asan) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (tsan) + REPO_COPY=${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (ubsan) + REPO_COPY=${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT=10800 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan0: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan1: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestMsan2: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (msan) + REPO_COPY=${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug0: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug1: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestDebug2: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests (debug) + REPO_COPY=${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT=10800 + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatelessTestFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateless_flaky_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateless tests flaky check (asan) + REPO_COPY=${{runner.temp}}/stateless_flaky_asan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + TestsBugfixCheck: + needs: [CheckLabels, StyleCheck] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/tests_bugfix_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=tests bugfix validate check + KILL_TIMEOUT=3600 + REPO_COPY=${{runner.temp}}/tests_bugfix_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Bugfix test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + + TEMP_PATH="${TEMP_PATH}/integration" \ + REPORTS_PATH="${REPORTS_PATH}/integration" \ + python3 integration_test_check.py "Integration $CHECK_NAME" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + TEMP_PATH="${TEMP_PATH}/stateless" \ + REPORTS_PATH="${REPORTS_PATH}/stateless" \ + python3 functional_test_check.py "Stateless $CHECK_NAME" "$KILL_TIMEOUT" \ + --validate-bugfix --post-commit-status=file || echo 'ignore exit code' + + python3 bugfix_validate_check.py "${TEMP_PATH}/stateless/post_commit_status.tsv" "${TEMP_PATH}/integration/post_commit_status.tsv" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (release) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAarch64: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (aarch64) + REPO_COPY=${{runner.temp}}/stateful_release/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (asan) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (tsan) + REPO_COPY=${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (msan) + REPO_COPY=${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (ubsan) + REPO_COPY=${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stateful_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stateful tests (debug) + REPO_COPY=${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT=3600 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 functional_test_check.py "$CHECK_NAME" "$KILL_TIMEOUT" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (asan) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + StressTestTsan: + needs: [BuilderDebTsan] + # func testers have 16 cores + 128 GB memory + # while stress testers have 36 cores + 72 memory + # It would be better to have something like 32 + 128, + # but such servers almost unavailable as spot instances. + runs-on: [self-hosted, func-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_thread + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (tsan) + REPO_COPY=${{runner.temp}}/stress_thread/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_memory + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (msan) + REPO_COPY=${{runner.temp}}/stress_memory/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_undefined + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (ubsan) + REPO_COPY=${{runner.temp}}/stress_undefined/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/stress_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Stress test (debug) + REPO_COPY=${{runner.temp}}/stress_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +##################################### AST FUZZERS ############################################ +############################################################################################## + ASTFuzzerTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (asan) + REPO_COPY=${{runner.temp}}/ast_fuzzer_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (tsan) + REPO_COPY=${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestUBSan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (ubsan) + REPO_COPY=${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestMSan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (msan) + REPO_COPY=${{runner.temp}}/ast_fuzzer_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + ASTFuzzerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=AST fuzzer (debug) + REPO_COPY=${{runner.temp}}/ast_fuzzer_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# + IntegrationTestsAsan0: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan1: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsAsan2: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=3 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan0: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan1: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan2: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsTsan3: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (tsan) + REPO_COPY=${{runner.temp}}/integration_tests_tsan/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsRelease1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests (release) + REPO_COPY=${{runner.temp}}/integration_tests_release/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=2 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + IntegrationTestsFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/integration_tests_asan_flaky_check + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Integration tests flaky check (asan) + REPO_COPY=${{runner.temp}}/integration_tests_asan_flaky_check/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### UNIT TESTS ############################################# +############################################################################################# + UnitTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (asan) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsReleaseClang: + needs: [BuilderBinRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_asan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (release-clang) + REPO_COPY=${{runner.temp}}/unit_tests_asan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_tsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (tsan) + REPO_COPY=${{runner.temp}}/unit_tests_tsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_msan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (msan) + REPO_COPY=${{runner.temp}}/unit_tests_msan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + UnitTestsUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Unit tests (ubsan) + REPO_COPY=${{runner.temp}}/unit_tests_ubsan/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################# +#################################### PERFORMANCE TESTS ###################################### +############################################################################################# + PerformanceComparisonX86-0: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonX86-1: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonX86-2: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonX86-3: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch-0: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=0 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch-1: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=1 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch-2: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=2 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + PerformanceComparisonAarch-3: + needs: [BuilderDebAarch64] + runs-on: [self-hosted, func-tester-aarch64] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/performance_comparison + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=Performance Comparison Aarch64 + REPO_COPY=${{runner.temp}}/performance_comparison/ClickHouse + RUN_BY_HASH_NUM=3 + RUN_BY_HASH_TOTAL=4 + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Performance Comparison + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 performance_comparison_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" +############################################################################################## +###################################### SQLANCER FUZZERS ###################################### +############################################################################################## + SQLancerTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqlancer_release + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=SQLancer (release) + REPO_COPY=${{runner.temp}}/sqlancer_release/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: SQLancer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqlancer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" + SQLancerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, fuzzer-unit-tester] + steps: + - name: Set envs + run: | + cat >> "$GITHUB_ENV" << 'EOF' + TEMP_PATH=${{runner.temp}}/sqlancer_debug + REPORTS_PATH=${{runner.temp}}/reports_dir + CHECK_NAME=SQLancer (debug) + REPO_COPY=${{runner.temp}}/sqlancer_debug/ClickHouse + EOF + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{ env.REPORTS_PATH }} + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: SQLancer + run: | + sudo rm -fr "$TEMP_PATH" + mkdir -p "$TEMP_PATH" + cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + cd "$REPO_COPY/tests/ci" + python3 sqlancer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker ps --quiet | xargs --no-run-if-empty docker kill ||: + docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + sudo rm -fr "$TEMP_PATH" ############################################################################################# ###################################### JEPSEN TESTS ######################################### ############################################################################################# @@ -164,7 +3574,80 @@ jobs: uses: ./.github/workflows/jepsen.yml FinishCheck: needs: + - StyleCheck + - DockerHubPush + - DockerServerImages - CheckLabels + - BuilderReport + - FastTest + - FunctionalStatelessTestDebug0 + - FunctionalStatelessTestDebug1 + - FunctionalStatelessTestDebug2 + - FunctionalStatelessTestRelease + - FunctionalStatelessTestReleaseDatabaseReplicated0 + - FunctionalStatelessTestReleaseDatabaseReplicated1 + - FunctionalStatelessTestReleaseWideParts + - FunctionalStatelessTestAarch64 + - FunctionalStatelessTestAsan0 + - FunctionalStatelessTestAsan1 + - FunctionalStatelessTestTsan0 + - FunctionalStatelessTestTsan1 + - FunctionalStatelessTestTsan2 + - FunctionalStatelessTestMsan0 + - FunctionalStatelessTestMsan1 + - FunctionalStatelessTestMsan2 + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAarch64 + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - FunctionalStatelessTestReleaseS3 + - FunctionalStatelessTestS3Debug0 + - FunctionalStatelessTestS3Debug1 + - FunctionalStatelessTestS3Debug2 + - FunctionalStatelessTestS3Tsan0 + - FunctionalStatelessTestS3Tsan1 + - FunctionalStatelessTestS3Tsan2 + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - ASTFuzzerTestDebug + - ASTFuzzerTestAsan + - ASTFuzzerTestTsan + - ASTFuzzerTestMSan + - ASTFuzzerTestUBSan + - IntegrationTestsAsan0 + - IntegrationTestsAsan1 + - IntegrationTestsAsan2 + - IntegrationTestsRelease0 + - IntegrationTestsRelease1 + - IntegrationTestsTsan0 + - IntegrationTestsTsan1 + - IntegrationTestsTsan2 + - IntegrationTestsTsan3 + - PerformanceComparisonX86-0 + - PerformanceComparisonX86-1 + - PerformanceComparisonX86-2 + - PerformanceComparisonX86-3 + - PerformanceComparisonAarch-0 + - PerformanceComparisonAarch-1 + - PerformanceComparisonAarch-2 + - PerformanceComparisonAarch-3 + - UnitTestsAsan + - UnitTestsTsan + - UnitTestsMsan + - UnitTestsUBsan + - UnitTestsReleaseClang + - SharedBuildSmokeTest + - CompatibilityCheck + - IntegrationTestsFlakyCheck + - SQLancerTestRelease + - SQLancerTestDebug runs-on: [self-hosted, style-checker] steps: - name: Clear repository From 7ac258c2a77fa813052ac67cc67977b05368fa1d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 7 Nov 2022 13:31:07 +0000 Subject: [PATCH 200/526] Improve test. --- ...rder_by_read_in_order_query_plan.reference | 190 ++++++++++++++++++ ...0940_order_by_read_in_order_query_plan.sql | 53 +++++ 2 files changed, 243 insertions(+) diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference index f6a41f5dfff..f08c4cfd3e5 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.reference @@ -183,6 +183,49 @@ select * from (explain plan actions = 1 select * from tab order by (a + b) * c d select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; Prefix sort description: multiply(plus(a, b), c) ASC, intDiv(intDiv(sin(divide(a, b)), -2), -3) ASC Result sort description: multiply(plus(a, b), c) ASC, intDiv(intDiv(sin(divide(a, b)), -2), -3) ASC +-- Aliases +select * from (select *, a + b as x from tab) order by x * c; +0 0 0 0 0 +0 0 0 0 0 +1 1 1 1 2 +1 1 1 1 2 +2 2 2 2 4 +2 2 2 2 4 +3 3 3 3 6 +3 3 3 3 6 +4 4 4 4 8 +4 4 4 4 8 +select * from (explain plan actions = 1 select * from (select *, a + b as x from tab) order by x * c) where explain like '%sort description%'; + Prefix sort description: multiply(x, c) ASC + Result sort description: multiply(x, c) ASC +select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y); +0 0 0 0 0 nan +0 0 0 0 0 nan +1 1 1 1 2 1 +1 1 1 1 2 1 +2 2 2 2 4 1 +2 2 2 2 4 1 +3 3 3 3 6 1 +3 3 3 3 6 1 +4 4 4 4 8 1 +4 4 4 4 8 1 +select * from (explain plan actions = 1 select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)) where explain like '%sort description%'; + Prefix sort description: multiply(x, c) ASC, sin(y) ASC + Result sort description: multiply(x, c) ASC, sin(y) ASC +select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y); +0 0 0 0 0 nan +0 0 0 0 0 nan +1 1 1 1 2 1 +1 1 1 1 2 1 +2 2 2 2 4 1 +2 2 2 2 4 1 +3 3 3 3 6 1 +3 3 3 3 6 1 +4 4 4 4 8 1 +4 4 4 4 8 1 +select * from (explain plan actions = 1 select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y)) where explain like '%sort description%'; + Prefix sort description: multiply(x, c) ASC, sin(y) ASC + Result sort description: multiply(x, c) ASC, sin(y) ASC -- { echoOn } select * from tab2 order by toTimeZone(toTimezone(x, 'UTC'), 'CET'), intDiv(intDiv(y, -2), -3); @@ -213,3 +256,150 @@ select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; Prefix sort description: intDiv(intDiv(y, -2), -3) ASC Result sort description: intDiv(intDiv(y, -2), -3) ASC +-- { echoOn } + +-- Union (not fully supported) +select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b); +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + Prefix sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + ReadType: InOrder + ReadType: InOrder +select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b); +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + Prefix sort description: sin(divide(a, b)) ASC + Result sort description: sin(divide(a, b)) ASC + ReadType: InOrder + ReadType: InOrder +select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b); +2 2 2 2 +2 2 2 2 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 4 4 4 +1 1 1 1 +2 2 2 2 +3 3 3 3 +4 4 4 4 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + Prefix sort description: sin(divide(a, b)) ASC + Result sort description: sin(divide(a, b)) ASC + ReadType: InOrder + ReadType: InOrder +select * from (select * from tab union all select * from tab5) order by (a + b) * c; +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c) where explain like '%sort description%' or explain like '%ReadType%'; + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC + ReadType: InOrder + ReadType: InOrder +select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b); +0 0 0 0 +0 0 0 0 +0 0 0 0 +0 0 0 0 +1 1 1 1 +1 1 1 1 +1 1 1 1 +1 1 1 1 +2 2 2 2 +2 2 2 2 +2 2 2 2 +2 2 2 2 +3 3 3 3 +3 3 3 3 +3 3 3 3 +3 3 3 3 +4 4 4 4 +4 4 4 4 +4 4 4 4 +4 4 4 4 +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + Prefix sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + ReadType: InOrder + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + ReadType: InOrder +-- Union with limit +select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3; +0 0 0 0 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; + Limit (preliminary LIMIT (without OFFSET)) + Limit 3 + Prefix sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Limit 3 + ReadType: InOrder + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + ReadType: InOrder +-- In this example, we read-in-order from tab up to ((a + b) * c, sin(a / b)) and from tab5 up to ((a + b) * c). +-- In case of tab5, there would be two finish sorting transforms: ((a + b) * c) -> ((a + b) * c, sin(a / b)) -> ((a + b) * c, sin(a / b), d). +-- It's important that ((a + b) * c) -> ((a + b) * c does not have LIMIT. We can add LIMIT WITH TIES later, when sorting alog support it. +-- In case of tab4, we do full sorting by ((a + b) * c, sin(a / b), d) with LIMIT. We can replace it to sorting by ((a + b) * c, sin(a / b)) and LIMIT WITH TIES, when sorting alog support it. +select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3; +0 0 0 0 +0 0 0 0 +0 0 0 0 +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; + Limit (preliminary LIMIT (without OFFSET)) + Limit 3 + Prefix sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC, d ASC + Limit 3 + ReadType: InOrder + Prefix sort description: multiply(plus(a, b), c) ASC + Result sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC + ReadType: InOrder + Sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC, d ASC + Limit 3 + ReadType: Default diff --git a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql index 977bc79dbeb..e694ccf84ee 100644 --- a/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql +++ b/tests/queries/0_stateless/00940_order_by_read_in_order_query_plan.sql @@ -73,6 +73,16 @@ select * from (explain plan actions = 1 select * from tab order by (a + b) * c d -- select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3); select * from (explain plan actions = 1 select * from tab order by (a + b) * c, intDiv(intDiv(sin(a / b), -2), -3)) where explain like '%sort description%'; +-- Aliases +select * from (select *, a + b as x from tab) order by x * c; +select * from (explain plan actions = 1 select * from (select *, a + b as x from tab) order by x * c) where explain like '%sort description%'; + +select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y); +select * from (explain plan actions = 1 select * from (select *, a + b as x, a / b as y from tab) order by x * c, sin(y)) where explain like '%sort description%'; + +select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y); +select * from (explain plan actions = 1 select * from (select *, a / b as y from (select *, a + b as x from tab)) order by x * c, sin(y)) where explain like '%sort description%'; + -- { echoOff } create table tab2 (x DateTime, y UInt32, z UInt32) engine = MergeTree order by (x, y); @@ -89,3 +99,46 @@ select * from (explain plan actions = 1 select * from tab2 order by toStartOfDay -- select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3); select * from (explain plan actions = 1 select * from tab2 where toTimezone(x, 'CET') = '2020-02-03 01:00:00' order by intDiv(intDiv(y, -2), -3)) where explain like '%sort description%'; + +-- { echoOff } + +create table tab3 (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by ((a + b) * c, sin(a / b)); +insert into tab3 select number, number, number, number from numbers(5); +insert into tab3 select number, number, number, number from numbers(5); + +create table tab4 (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by sin(a / b); +insert into tab4 select number, number, number, number from numbers(5); +insert into tab4 select number, number, number, number from numbers(5); + +create table tab5 (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by (a + b) * c; +insert into tab5 select number, number, number, number from numbers(5); +insert into tab5 select number, number, number, number from numbers(5); + +-- { echoOn } + +-- Union (not fully supported) +select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b); +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab3) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + +select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b); +select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + +select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b); +select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + +select * from (select * from tab union all select * from tab5) order by (a + b) * c; +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c) where explain like '%sort description%' or explain like '%ReadType%'; + +select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b); +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%'; + +-- Union with limit +select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3; +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5) order by (a + b) * c, sin(a / b) limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; + +-- In this example, we read-in-order from tab up to ((a + b) * c, sin(a / b)) and from tab5 up to ((a + b) * c). +-- In case of tab5, there would be two finish sorting transforms: ((a + b) * c) -> ((a + b) * c, sin(a / b)) -> ((a + b) * c, sin(a / b), d). +-- It's important that ((a + b) * c) -> ((a + b) * c does not have LIMIT. We can add LIMIT WITH TIES later, when sorting alog support it. +-- In case of tab4, we do full sorting by ((a + b) * c, sin(a / b), d) with LIMIT. We can replace it to sorting by ((a + b) * c, sin(a / b)) and LIMIT WITH TIES, when sorting alog support it. +select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3; +select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%'; From 500609ccd31b5f199a7841d95b24ffd519e3fc66 Mon Sep 17 00:00:00 2001 From: xlwh <1079039435@qq.com> Date: Tue, 31 May 2022 17:07:27 +0800 Subject: [PATCH 201/526] Add miscellaneous function displayName() --- src/Functions/serverConstants.cpp | 15 +++++++++++++++ .../0_stateless/02313_displayname_test.reference | 2 ++ .../0_stateless/02313_displayname_test.sql | 2 ++ 3 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02313_displayname_test.reference create mode 100644 tests/queries/0_stateless/02313_displayname_test.sql diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index ccefd74b85b..3a213adb02d 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -1,4 +1,6 @@ #include +#include +#include #include #include #include @@ -115,6 +117,13 @@ namespace static FunctionPtr create(ContextPtr context) { return std::make_shared(context); } }; + class FunctionDisplayName : public FunctionConstantBase + { + public: + static constexpr auto name = "displayName"; + static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } + explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + }; } #if defined(__ELF__) && !defined(OS_FREEBSD) @@ -173,4 +182,10 @@ REGISTER_FUNCTION(GetOSKernelVersion) } +REGISTER_FUNCTION(DisplayName) +{ + factory.registerFunction(); +} + + } diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference new file mode 100644 index 00000000000..20030bef031 --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -0,0 +1,2 @@ +localhost +localhost \ No newline at end of file diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname_test.sql new file mode 100644 index 00000000000..c0c91287e3d --- /dev/null +++ b/tests/queries/0_stateless/02313_displayname_test.sql @@ -0,0 +1,2 @@ +select displayname(); +select displayName(); \ No newline at end of file From b3435a7b9ca0cdfbcb3774f050988b7270145597 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 31 May 2022 15:41:47 +0200 Subject: [PATCH 202/526] =?UTF-8?q?Add=20newlines=20to=2002313=5Fdisplayna?= =?UTF-8?q?me=5Ftes=D0=B5?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tests/queries/0_stateless/02313_displayname_test.reference | 2 +- tests/queries/0_stateless/02313_displayname_test.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference index 20030bef031..c3fdafec5eb 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -1,2 +1,2 @@ localhost -localhost \ No newline at end of file +localhost diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname_test.sql index c0c91287e3d..f6db937f1b8 100644 --- a/tests/queries/0_stateless/02313_displayname_test.sql +++ b/tests/queries/0_stateless/02313_displayname_test.sql @@ -1,2 +1,2 @@ select displayname(); -select displayName(); \ No newline at end of file +select displayName(); From 136f589343b4ce985bf33b662f64f4a80fb67672 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 24 Aug 2022 18:38:15 +0200 Subject: [PATCH 203/526] Set display_name in functional tests --- tests/config/config.d/display_name.xml | 3 +++ tests/queries/0_stateless/02313_displayname_test.reference | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) create mode 100644 tests/config/config.d/display_name.xml diff --git a/tests/config/config.d/display_name.xml b/tests/config/config.d/display_name.xml new file mode 100644 index 00000000000..ba4d49c4838 --- /dev/null +++ b/tests/config/config.d/display_name.xml @@ -0,0 +1,3 @@ + + functional-tests + \ No newline at end of file diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname_test.reference index c3fdafec5eb..0c0e11ecf8e 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname_test.reference @@ -1,2 +1,2 @@ -localhost -localhost +functional-tests +functional-tests From 0481999e3558a61565f88f7aba01eb64eb3f070c Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Mon, 19 Sep 2022 10:21:35 +0200 Subject: [PATCH 204/526] Fix displayName test, add doc --- src/Functions/serverConstants.cpp | 14 ++++++++++++-- tests/config/config.d/display_name.xml | 2 +- ..._test.reference => 02313_displayname.reference} | 1 - ..._displayname_test.sql => 02313_displayname.sql} | 1 - 4 files changed, 13 insertions(+), 5 deletions(-) rename tests/queries/0_stateless/{02313_displayname_test.reference => 02313_displayname.reference} (50%) rename tests/queries/0_stateless/{02313_displayname_test.sql => 02313_displayname.sql} (50%) diff --git a/src/Functions/serverConstants.cpp b/src/Functions/serverConstants.cpp index 3a213adb02d..96615d0a4c9 100644 --- a/src/Functions/serverConstants.cpp +++ b/src/Functions/serverConstants.cpp @@ -121,8 +121,8 @@ namespace { public: static constexpr auto name = "displayName"; - static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } explicit FunctionDisplayName(ContextPtr context) : FunctionConstantBase(context->getConfigRef().getString("display_name", getFQDNOrHostName()), context->isDistributed()) {} + static FunctionPtr create(ContextPtr context) {return std::make_shared(context); } }; } @@ -184,7 +184,17 @@ REGISTER_FUNCTION(GetOSKernelVersion) REGISTER_FUNCTION(DisplayName) { - factory.registerFunction(); + factory.registerFunction( + { + R"( +Returns the value of `display_name` from config or server FQDN if not set. + +[example:displayName] +)", + Documentation::Examples{{"displayName", "SELECT displayName();"}}, + Documentation::Categories{"Constant", "Miscellaneous"} + }, + FunctionFactory::CaseSensitive); } diff --git a/tests/config/config.d/display_name.xml b/tests/config/config.d/display_name.xml index ba4d49c4838..d386a8efaf2 100644 --- a/tests/config/config.d/display_name.xml +++ b/tests/config/config.d/display_name.xml @@ -1,3 +1,3 @@ functional-tests - \ No newline at end of file + diff --git a/tests/queries/0_stateless/02313_displayname_test.reference b/tests/queries/0_stateless/02313_displayname.reference similarity index 50% rename from tests/queries/0_stateless/02313_displayname_test.reference rename to tests/queries/0_stateless/02313_displayname.reference index 0c0e11ecf8e..604a4944b33 100644 --- a/tests/queries/0_stateless/02313_displayname_test.reference +++ b/tests/queries/0_stateless/02313_displayname.reference @@ -1,2 +1 @@ functional-tests -functional-tests diff --git a/tests/queries/0_stateless/02313_displayname_test.sql b/tests/queries/0_stateless/02313_displayname.sql similarity index 50% rename from tests/queries/0_stateless/02313_displayname_test.sql rename to tests/queries/0_stateless/02313_displayname.sql index f6db937f1b8..6b6411b798b 100644 --- a/tests/queries/0_stateless/02313_displayname_test.sql +++ b/tests/queries/0_stateless/02313_displayname.sql @@ -1,2 +1 @@ -select displayname(); select displayName(); From ebb51a2023c5873ef9e3192cf0cd7a6997082229 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 7 Nov 2022 10:53:36 +0000 Subject: [PATCH 205/526] Use config.d/display_name.xml in tests --- tests/config/install.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/install.sh b/tests/config/install.sh index 9a172f0be15..876d05507fe 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -51,6 +51,7 @@ ln -sf $SRC_PATH/config.d/system_unfreeze.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_zero_copy_replication.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/nlp.xml $DEST_SERVER_PATH/config.d/ ln -sf $SRC_PATH/config.d/enable_keeper_map.xml $DEST_SERVER_PATH/config.d/ +ln -sf $SRC_PATH/config.d/display_name.xml $DEST_SERVER_PATH/config.d/ # Not supported with fasttest. if [ "${DEST_SERVER_PATH}" = "/etc/clickhouse-server" ] From 544d98a0ea182afa51256b8736aaaa02b064c356 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 7 Nov 2022 13:35:24 +0000 Subject: [PATCH 206/526] Fix for flaky test 00502_custom_partitioning_replicated_zookeeper_long --- ...0502_custom_partitioning_replicated_zookeeper_long.sql | 6 +++--- ...alter_enum_partition_key_replicated_zookeeper_long.sql | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 2883215ff94..0ee8ba07006 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -15,7 +15,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA not_partitioned_replica1_00502; +SYSTEM SYNC REPLICA not_partitioned_replica2_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica2_00502' AND active ORDER BY name; @@ -42,7 +42,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_week_replica1; +SYSTEM SYNC REPLICA partitioned_by_week_replica2; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica2' AND active ORDER BY name; @@ -68,7 +68,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; SELECT 'Parts after OPTIMIZE:'; diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 6d569ca6352..2b3e17a8776 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -2,8 +2,8 @@ SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test2; +DROP TABLE IF EXISTS test SYNC; +DROP TABLE IF EXISTS test2 SYNC; CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; @@ -50,5 +50,5 @@ ALTER TABLE test RENAME COLUMN y TO z; -- { serverError 524 } ALTER TABLE test DROP COLUMN x; -- { serverError 47 } ALTER TABLE test DROP COLUMN y; -- { serverError 47 } -DROP TABLE test; -DROP TABLE test2; +DROP TABLE test SYNC; +DROP TABLE test2 SYNC; From 70ce95236385a67a265f6e2124c595607ef8a891 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Mon, 7 Nov 2022 13:49:21 +0000 Subject: [PATCH 207/526] Automatic style fix --- tests/ci/jepsen_check.py | 34 +++++++++++++++++++++------------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 6f9f77e2cdd..25c77a1d0b5 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -57,8 +57,7 @@ def _parse_jepsen_output(path): current_type = "FAIL" if ( - line.startswith("store/clickhouse") - or line.startswith("clickhouse") + line.startswith("store/clickhouse") or line.startswith("clickhouse") ) and current_type: test_results.append((line.strip(), current_type)) @@ -153,12 +152,15 @@ def get_run_command( if __name__ == "__main__": logging.basicConfig(level=logging.INFO) parser = argparse.ArgumentParser( - prog = 'Jepsen Check', - description = 'Check that uses Jepsen. Both Keeper and Server can be tested.') - parser.add_argument('program', help='What should be tested. Valid values "keeper", "server"') + prog="Jepsen Check", + description="Check that uses Jepsen. Both Keeper and Server can be tested.", + ) + parser.add_argument( + "program", help='What should be tested. Valid values "keeper", "server"' + ) args = parser.parse_args() - if args.program != 'server' and args.program != 'keeper': + if args.program != "server" and args.program != "keeper": logging.warning(f"Invalid argument '{args.program}'") sys.exit(0) @@ -179,7 +181,7 @@ if __name__ == "__main__": gh = Github(get_best_robot_token(), per_page=100) - check_name = KEEPER_CHECK_NAME if args.program == 'keeper' else SERVER_CHECK_NAME + check_name = KEEPER_CHECK_NAME if args.program == "keeper" else SERVER_CHECK_NAME rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status(): @@ -193,11 +195,17 @@ if __name__ == "__main__": if not os.path.exists(result_path): os.makedirs(result_path) - instances = prepare_autoscaling_group_and_get_hostnames(KEEPER_DESIRED_INSTANCE_COUNT if args.program == 'keeper' else SERVER_DESIRED_INSTANCE_COUNT) - nodes_path = save_nodes_to_file(instances[:KEEPER_DESIRED_INSTANCE_COUNT], TEMP_PATH) + instances = prepare_autoscaling_group_and_get_hostnames( + KEEPER_DESIRED_INSTANCE_COUNT + if args.program == "keeper" + else SERVER_DESIRED_INSTANCE_COUNT + ) + nodes_path = save_nodes_to_file( + instances[:KEEPER_DESIRED_INSTANCE_COUNT], TEMP_PATH + ) # always use latest - docker_image = KEEPER_IMAGE_NAME if args.program == 'keeper' else SERVER_IMAGE_NAME + docker_image = KEEPER_IMAGE_NAME if args.program == "keeper" else SERVER_IMAGE_NAME build_name = get_build_name_for_check(check_name) @@ -223,9 +231,9 @@ if __name__ == "__main__": logging.warning("Cannot fetch build in 30 minutes, exiting") sys.exit(0) - extra_args = '' - if args.program == 'server': - extra_args = f'-e KEEPER_NODE={instances[-1]}' + extra_args = "" + if args.program == "server": + extra_args = f"-e KEEPER_NODE={instances[-1]}" with SSHKey(key_value=get_parameter_from_ssm("jepsen_ssh_key") + "\n"): ssh_auth_sock = os.environ["SSH_AUTH_SOCK"] From 900894e251ca0644884a5e814cbdc8933af69c11 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 7 Nov 2022 14:03:11 +0000 Subject: [PATCH 208/526] Fix style --- tests/ci/jepsen_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/jepsen_check.py b/tests/ci/jepsen_check.py index 25c77a1d0b5..4116d15bba6 100644 --- a/tests/ci/jepsen_check.py +++ b/tests/ci/jepsen_check.py @@ -161,7 +161,7 @@ if __name__ == "__main__": args = parser.parse_args() if args.program != "server" and args.program != "keeper": - logging.warning(f"Invalid argument '{args.program}'") + logging.warning("Invalid argument '%s'", args.program) sys.exit(0) stopwatch = Stopwatch() From a8098db25de62eb1d5ffae6a1c4f3ad988f357f3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 4 Nov 2022 09:09:48 +0000 Subject: [PATCH 209/526] Minor cleanups Semantics are unchanged. Some special case handling was changed to early-out, because of that the indentation changed but the logic is the same as before. --- src/Functions/FunctionsStringSearch.h | 27 +- src/Functions/MatchImpl.h | 353 ++++++++++++-------------- src/Functions/notLike.cpp | 3 +- 3 files changed, 184 insertions(+), 199 deletions(-) diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index 68425ee496e..d189a0bacf4 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -91,23 +91,30 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { if (arguments.size() < 2 || 3 < arguments.size()) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 2 or 3.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception( + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3.", + getName(), arguments.size()); if (!isStringOrFixedString(arguments[0])) throw Exception( - "Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[0]->getName(), getName()); if (!isString(arguments[1])) throw Exception( - "Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[1]->getName(), getName()); if (arguments.size() >= 3) { if (!isUnsignedInteger(arguments[2])) throw Exception( - "Illegal type " + arguments[2]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of argument of function {}", + arguments[2]->getName(), getName()); } return std::make_shared>(); @@ -196,9 +203,11 @@ public: vec_res); else throw Exception( - "Illegal columns " + arguments[0].column->getName() + " and " - + arguments[1].column->getName() + " of arguments of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN); + ErrorCodes::ILLEGAL_COLUMN, + "Illegal columns {} and {} of arguments of function {}", + arguments[0].column->getName(), + arguments[1].column->getName(), + getName()); return col_res; } diff --git a/src/Functions/MatchImpl.h b/src/Functions/MatchImpl.h index 6862a097d0e..edb0df3ae34 100644 --- a/src/Functions/MatchImpl.h +++ b/src/Functions/MatchImpl.h @@ -25,7 +25,7 @@ namespace impl /// Is the [I]LIKE expression reduced to finding a substring in a string? inline bool likePatternIsSubstring(std::string_view pattern, String & res) { - if (pattern.size() < 2 || pattern.front() != '%' || pattern.back() != '%') + if (pattern.size() < 2 || !pattern.starts_with('%') || !pattern.ends_with('%')) return false; res.clear(); @@ -101,9 +101,7 @@ struct MatchImpl static constexpr bool case_insensitive = (case_ == MatchTraits::Case::Insensitive); static constexpr bool negate = (result_ == MatchTraits::Result::Negate); - using Searcher = std::conditional_t; + using Searcher = std::conditional_t; static void vectorConstant( const ColumnString::Chars & haystack_data, @@ -115,13 +113,12 @@ struct MatchImpl const size_t haystack_size = haystack_offsets.size(); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) return; - /// A simple case where the [I]LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -158,105 +155,101 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t prev_offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, + 0, + haystack_offsets[i] - prev_offset - 1, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + prev_offset = haystack_offsets[i]; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack_data.data(); + const UInt8 * const end = haystack_data.begin() + haystack_data.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + /// The current index in the array of strings. + size_t i = 0; - if (required_substring.empty()) + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + /// Determine which index it refers to. + while (begin + haystack_offsets[i] <= pos) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); + res[i] = negate; + ++i; } - else + + /// We check that the entry does not pass through the boundaries of strings. + if (pos + required_substring.size() < begin + haystack_offsets[i]) { - size_t prev_offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else { + const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); + size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = str_size; + const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack_data[prev_offset]), haystack_offsets[i] - prev_offset - 1}, - 0, - haystack_offsets[i] - prev_offset - 1, + {str_data, str_size}, + start_pos, + end_pos, re2_st::RE2::UNANCHORED, nullptr, 0); res[i] = negate ^ match; - - prev_offset = haystack_offsets[i]; } } + else + res[i] = negate; + + pos = begin + haystack_offsets[i]; + ++i; } - else - { - /// NOTE This almost matches with the case of impl::likePatternIsSubstring. - const UInt8 * const begin = haystack_data.data(); - const UInt8 * const end = haystack_data.begin() + haystack_data.size(); - const UInt8 * pos = begin; - - /// The current index in the array of strings. - size_t i = 0; - - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Determine which index it refers to. - while (begin + haystack_offsets[i] <= pos) - { - res[i] = negate; - ++i; - } - - /// We check that the entry does not pass through the boundaries of strings. - if (pos + required_substring.size() < begin + haystack_offsets[i]) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(&haystack_data[haystack_offsets[i - 1]]); - size_t str_size = haystack_offsets[i] - haystack_offsets[i - 1] - 1; - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = str_size; - - const bool match = regexp.getRE2()->Match( - {str_data, str_size}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = begin + haystack_offsets[i]; - ++i; - } - - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); - } + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -274,7 +267,7 @@ struct MatchImpl if (haystack.empty()) return; - /// A simple case where the LIKE expression reduces to finding a substring in a string + /// Special case that the [I]LIKE expression reduces to finding a substring in a string String strstr_pattern; if (is_like && impl::likePatternIsSubstring(needle, strstr_pattern)) { @@ -316,109 +309,105 @@ struct MatchImpl /// Tail, in which there can be no substring. if (i < res.size()) memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + + return; + } + + const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + + String required_substring; + bool is_trivial; + bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + + regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + + if (required_substring.empty()) + { + if (!regexp.getRE2()) /// An empty regexp. Always matches. + memset(res.data(), !negate, haystack_size * sizeof(res[0])); + else + { + size_t offset = 0; + for (size_t i = 0; i < haystack_size; ++i) + { + const bool match = regexp.getRE2()->Match( + {reinterpret_cast(&haystack[offset]), N}, + 0, + N, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + + offset += N; + } + } } else { - const auto & regexp = Regexps::Regexp(Regexps::createRegexp(needle)); + /// NOTE This almost matches with the case of likePatternIsSubstring. - String required_substring; - bool is_trivial; - bool required_substring_is_prefix; /// for `anchored` execution of the regexp. + const UInt8 * const begin = haystack.data(); + const UInt8 * const end = haystack.data() + haystack.size(); + const UInt8 * pos = begin; - regexp.getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix); + size_t i = 0; + const UInt8 * next_pos = begin; - if (required_substring.empty()) + /// If required substring is larger than string size - it cannot be found. + if (required_substring.size() <= N) { - if (!regexp.getRE2()) /// An empty regexp. Always matches. + Searcher searcher(required_substring.data(), required_substring.size(), end - pos); + + /// We will search for the next occurrence in all rows at once. + while (pos < end && end != (pos = searcher.search(pos, end - pos))) { - if (haystack_size) - memset(res.data(), !negate, haystack_size * sizeof(res[0])); - } - else - { - size_t offset = 0; - for (size_t i = 0; i < haystack_size; ++i) + /// Let's determine which index it refers to. + while (next_pos + N <= pos) { - const bool match = regexp.getRE2()->Match( - {reinterpret_cast(&haystack[offset]), N}, - 0, - N, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - - offset += N; - } - } - } - else - { - /// NOTE This almost matches with the case of likePatternIsSubstring. - - const UInt8 * const begin = haystack.data(); - const UInt8 * const end = haystack.data() + haystack.size(); - const UInt8 * pos = begin; - - size_t i = 0; - const UInt8 * next_pos = begin; - - /// If required substring is larger than string size - it cannot be found. - if (required_substring.size() <= N) - { - Searcher searcher(required_substring.data(), required_substring.size(), end - pos); - - /// We will search for the next occurrence in all rows at once. - while (pos < end && end != (pos = searcher.search(pos, end - pos))) - { - /// Let's determine which index it refers to. - while (next_pos + N <= pos) - { - res[i] = negate; - next_pos += N; - ++i; - } + res[i] = negate; next_pos += N; - - if (pos + required_substring.size() <= next_pos) - { - /// And if it does not, if necessary, we check the regexp. - - if (is_trivial) - res[i] = !negate; - else - { - const char * str_data = reinterpret_cast(next_pos - N); - - /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, - * so that it can match when `required_substring` occurs into the string several times, - * and at the first occurrence, the regexp is not a match. - */ - const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; - const size_t end_pos = N; - - const bool match = regexp.getRE2()->Match( - {str_data, N}, - start_pos, - end_pos, - re2_st::RE2::UNANCHORED, - nullptr, - 0); - res[i] = negate ^ match; - } - } - else - res[i] = negate; - - pos = next_pos; ++i; } - } + next_pos += N; - /// Tail, in which there can be no substring. - if (i < res.size()) - memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); + if (pos + required_substring.size() <= next_pos) + { + /// And if it does not, if necessary, we check the regexp. + if (is_trivial) + res[i] = !negate; + else + { + const char * str_data = reinterpret_cast(next_pos - N); + + /** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp, + * so that it can match when `required_substring` occurs into the string several times, + * and at the first occurrence, the regexp is not a match. + */ + const size_t start_pos = (required_substring_is_prefix) ? (reinterpret_cast(pos) - str_data) : 0; + const size_t end_pos = N; + + const bool match = regexp.getRE2()->Match( + {str_data, N}, + start_pos, + end_pos, + re2_st::RE2::UNANCHORED, + nullptr, + 0); + res[i] = negate ^ match; + } + } + else + res[i] = negate; + + pos = next_pos; + ++i; + } } + + /// Tail, in which there can be no substring. + if (i < res.size()) + memset(&res[i], negate, (res.size() - i) * sizeof(res[0])); } } @@ -434,7 +423,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack_offsets.empty()) @@ -481,9 +469,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -502,15 +488,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; @@ -546,7 +528,6 @@ struct MatchImpl assert(haystack_size == needle_offset.size()); assert(haystack_size == res.size()); - assert(start_pos_ == nullptr); if (haystack.empty()) @@ -593,9 +574,7 @@ struct MatchImpl if (required_substr.empty()) { if (!regexp->getRE2()) /// An empty regexp. Always matches. - { res[i] = !negate; - } else { const bool match = regexp->getRE2()->Match( @@ -614,15 +593,11 @@ struct MatchImpl const auto * match = searcher.search(cur_haystack_data, cur_haystack_length); if (match == cur_haystack_data + cur_haystack_length) - { res[i] = negate; // no match - } else { if (is_trivial) - { res[i] = !negate; // no wildcards in pattern - } else { const size_t start_pos = (required_substring_is_prefix) ? (match - cur_haystack_data) : 0; diff --git a/src/Functions/notLike.cpp b/src/Functions/notLike.cpp index a546b511a0b..200890d77d8 100644 --- a/src/Functions/notLike.cpp +++ b/src/Functions/notLike.cpp @@ -12,7 +12,8 @@ struct NameNotLike static constexpr auto name = "notLike"; }; -using FunctionNotLike = FunctionsStringSearch>; +using NotLikeImpl = MatchImpl; +using FunctionNotLike = FunctionsStringSearch; } From 8fa5cae01a37c625a6675cdd9b6ef34f200d65c8 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Mon, 7 Nov 2022 14:11:02 +0000 Subject: [PATCH 210/526] Fix style check --- src/Analyzer/QueryTreePassManager.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/QueryTreePassManager.cpp b/src/Analyzer/QueryTreePassManager.cpp index d59e7457ed2..9dc9cda44a3 100644 --- a/src/Analyzer/QueryTreePassManager.cpp +++ b/src/Analyzer/QueryTreePassManager.cpp @@ -28,6 +28,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; } namespace From 3d0f0e3896b3da9e99bba4bdf6d0761f41bfc138 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 7 Nov 2022 13:44:05 +0000 Subject: [PATCH 211/526] Apply another fix --- src/Parsers/ExpressionListParsers.cpp | 12 ++++++++++-- .../02476_fix_lambda_parsing.reference | 8 ++++++-- .../0_stateless/02476_fix_lambda_parsing.sh | 15 +++++++++++---- 3 files changed, 27 insertions(+), 8 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 0f103690e37..44886ec4577 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -694,7 +694,7 @@ public: return false; /// 2. If there is already tuple do nothing - if (elements.size() == 1 && tryGetFunctionName(elements.back()) == "tuple") + if (tryGetFunctionName(elements.back()) == "tuple") { pushOperand(elements.back()); elements.pop_back(); @@ -1050,12 +1050,20 @@ public: if (!mergeElement()) return false; - // Special case for (('a', 'b')) -> tuple(('a', 'b')) if (!is_tuple && elements.size() == 1) + { + // Special case for (('a', 'b')) = tuple(('a', 'b')) if (auto * literal = elements[0]->as()) if (literal->value.getType() == Field::Types::Tuple) is_tuple = true; + // Special case for f(x, (y) -> z) = f(x, tuple(y) -> z) + auto test_pos = pos; + auto test_expected = expected; + if (parseOperator(test_pos, "->", test_expected)) + is_tuple = true; + } + finished = true; } diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.reference b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference index fce203907eb..18cb46ce23c 100644 --- a/tests/queries/0_stateless/02476_fix_lambda_parsing.reference +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.reference @@ -1,4 +1,8 @@ -SELECT f((x, tuple(y)) -> z) -SELECT f((x, tuple(y)) -> z) +SELECT f(x, y -> z) +SELECT f(x, y -> z) SELECT f((x, y) -> z) SELECT f((x, y) -> z) +SELECT f((x, y) -> z) +SELECT f(x, (x, y) -> z) +SELECT f(x, (x, y) -> z) +CREATE FUNCTION func AS x -> plus(x, (x -> ('2' + 2)) -> plus(1), 1) diff --git a/tests/queries/0_stateless/02476_fix_lambda_parsing.sh b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh index 2f134010991..641ef59a170 100755 --- a/tests/queries/0_stateless/02476_fix_lambda_parsing.sh +++ b/tests/queries/0_stateless/02476_fix_lambda_parsing.sh @@ -8,7 +8,14 @@ set -e format="$CLICKHOUSE_FORMAT" -echo "SELECT f((x, tuple(y)) -> z)" | $format -echo "select f(x, tuple(y) -> z)" | $format -echo "select f(x, y -> z)" | $format -echo "select f((x, y) -> z)" | $format +echo "SELECT f(x, tuple(y) -> z)" | $format +echo "SELECT f(x, (y) -> z)" | $format + +echo "SELECT f(x, y -> z)" | $format +echo "SELECT f((x, y) -> z)" | $format +echo "SELECT f(tuple(x, y) -> z)" | $format + +echo "SELECT f(x, (x, y) -> z)" | $format +echo "SELECT f(x, tuple(x, y) -> z)" | $format + +echo "CREATE FUNCTION func AS x -> plus(x, (x -> ('2' + 2)) -> plus(1), 1)" | $format | $format From 891e1675093aea0eefd62dc8f5bb62709b566aa3 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Nov 2022 15:17:16 +0100 Subject: [PATCH 212/526] Update StorageHudi.cpp --- src/Storages/StorageHudi.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index d31d04beef5..986bb6b955c 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; + extern const int LOGICAL_ERROR; } StorageHudi::StorageHudi( From 34f90ff6efb6a773cbca1d360d5c7469df92eb8e Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 7 Nov 2022 09:26:50 -0500 Subject: [PATCH 213/526] update pages that refer to dictionaries --- .../table-engines/integrations/index.md | 2 +- .../table-engines/integrations/mysql.md | 2 +- .../engines/table-engines/integrations/odbc.md | 2 +- .../table-engines/integrations/postgresql.md | 2 +- .../example-datasets/cell-towers.md | 2 +- docs/en/interfaces/formats.md | 2 +- docs/en/operations/named-collections.md | 6 +++--- .../settings.md | 4 ++-- .../external-dicts-dict-layout.md | 2 +- .../external-dicts-dict-polygon.md | 2 +- .../functions/ext-dict-functions.md | 18 +++++++++--------- .../functions/ym-dict-functions.md | 2 +- .../en/sql-reference/statements/select/join.md | 2 +- docs/en/sql-reference/statements/show.md | 2 +- docs/en/sql-reference/table-functions/mysql.md | 2 +- docs/en/sql-reference/table-functions/odbc.md | 2 +- .../table-functions/postgresql.md | 2 +- 17 files changed, 28 insertions(+), 28 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index 7e67bcb6249..09e89209ea9 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -6,7 +6,7 @@ sidebar_label: Integrations # Table Engines for Integrations -ClickHouse provides various means for integrating with external systems, including table engines. Like with all other table engines, the configuration is done using `CREATE TABLE` or `ALTER TABLE` queries. Then from a user perspective, the configured integration looks like a normal table, but queries to it are proxied to the external system. This transparent querying is one of the key advantages of this approach over alternative integration methods, like external dictionaries or table functions, which require to use custom query methods on each use. +ClickHouse provides various means for integrating with external systems, including table engines. Like with all other table engines, the configuration is done using `CREATE TABLE` or `ALTER TABLE` queries. Then from a user perspective, the configured integration looks like a normal table, but queries to it are proxied to the external system. This transparent querying is one of the key advantages of this approach over alternative integration methods, like dictionaries or table functions, which require to use custom query methods on each use. List of supported integrations: diff --git a/docs/en/engines/table-engines/integrations/mysql.md b/docs/en/engines/table-engines/integrations/mysql.md index 7c9c4cfea53..9f637c50989 100644 --- a/docs/en/engines/table-engines/integrations/mysql.md +++ b/docs/en/engines/table-engines/integrations/mysql.md @@ -180,6 +180,6 @@ Default value: `300`. ## See Also {#see-also} - [The mysql table function](../../../sql-reference/table-functions/mysql.md) -- [Using MySQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) +- [Using MySQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/mysql/) diff --git a/docs/en/engines/table-engines/integrations/odbc.md b/docs/en/engines/table-engines/integrations/odbc.md index 043d5170654..e21a64bc5b2 100644 --- a/docs/en/engines/table-engines/integrations/odbc.md +++ b/docs/en/engines/table-engines/integrations/odbc.md @@ -126,7 +126,7 @@ SELECT * FROM odbc_t ## See Also {#see-also} -- [ODBC external dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) +- [ODBC dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) - [ODBC table function](../../../sql-reference/table-functions/odbc.md) [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/odbc/) diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index 4bb8033de9c..c07512cf0ce 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -174,6 +174,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) **See Also** - [The `postgresql` table function](../../../sql-reference/table-functions/postgresql.md) -- [Using PostgreSQL as a source of external dictionary](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [Using PostgreSQL as a dictionary source](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) [Original article](https://clickhouse.com/docs/en/engines/table-engines/integrations/postgresql/) diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index 3d993c3e224..26bed30e3b8 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -129,7 +129,7 @@ SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 So, the top countries are: the USA, Germany, and Russia. -You may want to create an [External Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) in ClickHouse to decode these values. +You may want to create a [Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) in ClickHouse to decode these values. ## Use case: Incorporate geo data {#use-case} diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 58e986cc2f3..56708def497 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -5,7 +5,7 @@ sidebar_label: Input and Output Formats title: Formats for Input and Output Data --- -ClickHouse can accept and return data in various formats. A format supported for input can be used to parse the data provided to `INSERT`s, to perform `SELECT`s from a file-backed table such as File, URL or HDFS, or to read an external dictionary. A format supported for output can be used to arrange the +ClickHouse can accept and return data in various formats. A format supported for input can be used to parse the data provided to `INSERT`s, to perform `SELECT`s from a file-backed table such as File, URL or HDFS, or to read a dictionary. A format supported for output can be used to arrange the results of a `SELECT`, and to perform `INSERT`s into a file-backed table. The supported formats are: diff --git a/docs/en/operations/named-collections.md b/docs/en/operations/named-collections.md index f605045a0ad..cbb8d0a4c02 100644 --- a/docs/en/operations/named-collections.md +++ b/docs/en/operations/named-collections.md @@ -130,7 +130,7 @@ SHOW TABLES FROM mydatabase; └────────┘ ``` -### Example of using named collections with an external dictionary with source MySQL +### Example of using named collections with a dictionary with source MySQL ```sql CREATE DICTIONARY dict (A Int64, B String) @@ -213,7 +213,7 @@ SHOW TABLES FROM mydatabase └──────┘ ``` -### Example of using named collections with an external dictionary with source POSTGRESQL +### Example of using named collections with a dictionary with source POSTGRESQL ```sql CREATE DICTIONARY dict (a Int64, b String) @@ -270,7 +270,7 @@ SELECT * FROM remote(remote1, database = default, table = test); └───┴───┘ ``` -### Example of using named collections with an external dictionary with source ClickHouse +### Example of using named collections with a dictionary with source ClickHouse ```sql CREATE DICTIONARY dict(a Int64, b String) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index dcda7536935..5faf3819d7e 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -268,14 +268,14 @@ The path to the table in ZooKeeper. ## dictionaries_config {#server_configuration_parameters-dictionaries_config} -The path to the config file for external dictionaries. +The path to the config file for dictionaries. Path: - Specify the absolute path or the path relative to the server config file. - The path can contain wildcards \* and ?. -See also “[External dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md)”. +See also “[Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md)”. **Example** diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 19884e7676d..49a8620b609 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -22,7 +22,7 @@ ClickHouse generates an exception for errors with dictionaries. Examples of erro - The dictionary being accessed could not be loaded. - Error querying a `cached` dictionary. -You can view the list of external dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. +You can view the list of dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. :::tip If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 2124f816883..42c4a7faa73 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -83,7 +83,7 @@ To respond to the query, there is a corresponding cell, and the index for the po - `POLYGON`. Synonym to `POLYGON_INDEX_CELL`. -Dictionary queries are carried out using standard [functions](../../../sql-reference/functions/ext-dict-functions.md) for working with external dictionaries. +Dictionary queries are carried out using standard [functions](../../../sql-reference/functions/ext-dict-functions.md) for working with dictionaries. An important difference is that here the keys will be the points for which you want to find the polygon containing them. **Example** diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 728e26d6958..1c33638da09 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -1,20 +1,20 @@ --- slug: /en/sql-reference/functions/ext-dict-functions sidebar_position: 58 -sidebar_label: External Dictionaries +sidebar_label: Dictionaries --- +# Functions for Working with Dictionaries + :::note For dictionaries created with [DDL queries](../../sql-reference/statements/create/dictionary.md), the `dict_name` parameter must be fully specified, like `.`. Otherwise, the current database is used. ::: -# Functions for Working with External Dictionaries - -For information on connecting and configuring external dictionaries, see [External dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +For information on connecting and configuring dictionaries, see [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). ## dictGet, dictGetOrDefault, dictGetOrNull -Retrieves values from an external dictionary. +Retrieves values from a dictionary. ``` sql dictGet('dict_name', attr_names, id_expr) @@ -52,7 +52,7 @@ Create a text file `ext-dict-test.csv` containing the following: The first column is `id`, the second column is `c1`. -Configure the external dictionary: +Configure the dictionary: ``` xml @@ -112,7 +112,7 @@ Create a text file `ext-dict-mult.csv` containing the following: The first column is `id`, the second is `c1`, the third is `c2`. -Configure the external dictionary: +Configure the dictionary: ``` xml @@ -185,7 +185,7 @@ INSERT INTO range_key_dictionary_source_table VALUES(2, toDate('2019-05-20'), to INSERT INTO range_key_dictionary_source_table VALUES(3, toDate('2019-05-20'), toDate('2019-05-20'), 'Third', 'Third'); ``` -Create the external dictionary: +Create the dictionary: ```sql CREATE DICTIONARY range_key_dictionary @@ -226,7 +226,7 @@ Result: **See Also** -- [External Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) +- [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) ## dictHas diff --git a/docs/en/sql-reference/functions/ym-dict-functions.md b/docs/en/sql-reference/functions/ym-dict-functions.md index 04df3db571e..f92ad5db2ad 100644 --- a/docs/en/sql-reference/functions/ym-dict-functions.md +++ b/docs/en/sql-reference/functions/ym-dict-functions.md @@ -131,7 +131,7 @@ Type: `UInt32`. ### regionToPopulation(id\[, geobase\]) Gets the population for a region. -The population can be recorded in files with the geobase. See the section “External dictionaries”. +The population can be recorded in files with the geobase. See the section “Dictionaries”. If the population is not recorded for the region, it returns 0. In the geobase, the population might be recorded for child regions, but not for parent regions. diff --git a/docs/en/sql-reference/statements/select/join.md b/docs/en/sql-reference/statements/select/join.md index 1890ff081d8..62d3e9fd69a 100644 --- a/docs/en/sql-reference/statements/select/join.md +++ b/docs/en/sql-reference/statements/select/join.md @@ -282,7 +282,7 @@ Each time a query is run with the same `JOIN`, the subquery is run again because In some cases, it is more efficient to use [IN](../../../sql-reference/operators/in.md) instead of `JOIN`. -If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is an “external dictionaries” feature that you should use instead of `JOIN`. For more information, see the [External dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. +If you need a `JOIN` for joining with dimension tables (these are relatively small tables that contain dimension properties, such as names for advertising campaigns), a `JOIN` might not be very convenient due to the fact that the right table is re-accessed for every query. For such cases, there is a “dictionaries” feature that you should use instead of `JOIN`. For more information, see the [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. ### Memory Limitations diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index 87248bb115b..0efad3d460f 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -198,7 +198,7 @@ Result: ## SHOW DICTIONARIES -Displays a list of [external dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +Displays a list of [Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). ``` sql SHOW DICTIONARIES [FROM ] [LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] diff --git a/docs/en/sql-reference/table-functions/mysql.md b/docs/en/sql-reference/table-functions/mysql.md index f867cda45bd..de1567c052e 100644 --- a/docs/en/sql-reference/table-functions/mysql.md +++ b/docs/en/sql-reference/table-functions/mysql.md @@ -110,5 +110,5 @@ SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); **See Also** - [The ‘MySQL’ table engine](../../engines/table-engines/integrations/mysql.md) -- [Using MySQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) +- [Using MySQL as a dictionary source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) diff --git a/docs/en/sql-reference/table-functions/odbc.md b/docs/en/sql-reference/table-functions/odbc.md index f8c46fe44d8..7e13424bc8a 100644 --- a/docs/en/sql-reference/table-functions/odbc.md +++ b/docs/en/sql-reference/table-functions/odbc.md @@ -101,5 +101,5 @@ SELECT * FROM odbc('DSN=mysqlconn', 'test', 'test') ## See Also -- [ODBC external dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) +- [ODBC dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-odbc) - [ODBC table engine](../../engines/table-engines/integrations/odbc.md). diff --git a/docs/en/sql-reference/table-functions/postgresql.md b/docs/en/sql-reference/table-functions/postgresql.md index 367edbe9a00..e98869de739 100644 --- a/docs/en/sql-reference/table-functions/postgresql.md +++ b/docs/en/sql-reference/table-functions/postgresql.md @@ -130,6 +130,6 @@ CREATE TABLE pg_table_schema_with_dots (a UInt32) **See Also** - [The PostgreSQL table engine](../../engines/table-engines/integrations/postgresql.md) -- [Using PostgreSQL as a source of external dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) +- [Using PostgreSQL as a dictionary source](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-postgresql) [Original article](https://clickhouse.com/docs/en/sql-reference/table-functions/postgresql/) From d4cdc38b71dc6e443045070bbec7166beb9cc369 Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Mon, 7 Nov 2022 22:49:58 +0800 Subject: [PATCH 214/526] better --- .../System/StorageSystemDetachedParts.cpp | 30 +++++++------------ 1 file changed, 10 insertions(+), 20 deletions(-) diff --git a/src/Storages/System/StorageSystemDetachedParts.cpp b/src/Storages/System/StorageSystemDetachedParts.cpp index 11c4f7843ac..75bf3fdf72a 100644 --- a/src/Storages/System/StorageSystemDetachedParts.cpp +++ b/src/Storages/System/StorageSystemDetachedParts.cpp @@ -36,28 +36,11 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr /// Files or directories of detached part may not exist. Only count the size of existing files. if (disk->isFile(from)) { - try - { - total_size += disk->getFileSize(from); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + total_size += disk->getFileSize(from); } else { - DirectoryIteratorPtr it; - try - { - it = disk->iterateDirectory(from); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - - for (; it->isValid(); it->next()) + for (auto it = disk->iterateDirectory(from); it->isValid(); it->next()) calculateTotalSizeOnDiskImpl(disk, fs::path(from) / it->name(), total_size); } } @@ -65,7 +48,14 @@ static void calculateTotalSizeOnDiskImpl(const DiskPtr & disk, const String & fr static UInt64 calculateTotalSizeOnDisk(const DiskPtr & disk, const String & from) { UInt64 total_size = 0; - calculateTotalSizeOnDiskImpl(disk, from, total_size); + try + { + calculateTotalSizeOnDiskImpl(disk, from, total_size); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } return total_size; } From 58a12300ef46c7d2dd4712d332a701f2b7b03603 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Mon, 7 Nov 2022 15:33:53 +0100 Subject: [PATCH 215/526] Update src/Functions/FunctionsStringSearch.h Co-authored-by: Vladimir C --- src/Functions/FunctionsStringSearch.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/FunctionsStringSearch.h b/src/Functions/FunctionsStringSearch.h index d189a0bacf4..d8da525e63a 100644 --- a/src/Functions/FunctionsStringSearch.h +++ b/src/Functions/FunctionsStringSearch.h @@ -93,7 +93,7 @@ public: if (arguments.size() < 2 || 3 < arguments.size()) throw Exception( ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, - "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3.", + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", getName(), arguments.size()); if (!isStringOrFixedString(arguments[0])) From a04b9fa9470ced4e50c78c8e204f67b8b7a2e980 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 7 Nov 2022 17:25:04 +0100 Subject: [PATCH 216/526] Add some guard rails around aggregation memory management --- src/AggregateFunctions/AggregateFunctionAggThrow.cpp | 2 ++ src/AggregateFunctions/AggregateFunctionDistinct.h | 5 +++++ src/AggregateFunctions/AggregateFunctionForEach.h | 2 +- src/AggregateFunctions/AggregateFunctionMap.h | 5 +++++ src/AggregateFunctions/IAggregateFunction.h | 11 ++++++++++- src/Processors/Transforms/WindowTransform.cpp | 2 ++ 6 files changed, 25 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index e74f93cd9b0..ae1bbfddf75 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -76,6 +76,8 @@ public: data(place).~Data(); } + bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } + void add(AggregateDataPtr __restrict, const IColumn **, size_t, Arena *) const override { } diff --git a/src/AggregateFunctions/AggregateFunctionDistinct.h b/src/AggregateFunctions/AggregateFunctionDistinct.h index 9884e92f425..2d7362ba4cc 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinct.h +++ b/src/AggregateFunctions/AggregateFunctionDistinct.h @@ -239,6 +239,11 @@ public: nested_func->destroy(getNestedPlace(place)); } + bool hasTrivialDestructor() const override + { + return std::is_trivially_destructible_v && nested_func->hasTrivialDestructor(); + } + void destroyUpToState(AggregateDataPtr __restrict place) const noexcept override { this->data(place).~Data(); diff --git a/src/AggregateFunctions/AggregateFunctionForEach.h b/src/AggregateFunctions/AggregateFunctionForEach.h index 62794ac0f53..c91c4dd7c86 100644 --- a/src/AggregateFunctions/AggregateFunctionForEach.h +++ b/src/AggregateFunctions/AggregateFunctionForEach.h @@ -174,7 +174,7 @@ public: bool hasTrivialDestructor() const override { - return nested_func->hasTrivialDestructor(); + return std::is_trivially_destructible_v && nested_func->hasTrivialDestructor(); } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override diff --git a/src/AggregateFunctions/AggregateFunctionMap.h b/src/AggregateFunctions/AggregateFunctionMap.h index d349fc05944..f60cc71e78e 100644 --- a/src/AggregateFunctions/AggregateFunctionMap.h +++ b/src/AggregateFunctions/AggregateFunctionMap.h @@ -228,6 +228,11 @@ public: destroyImpl(place); } + bool hasTrivialDestructor() const override + { + return std::is_trivially_destructible_v && nested_func->hasTrivialDestructor(); + } + void destroyUpToState(AggregateDataPtr __restrict place) const noexcept override { destroyImpl(place); diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index 7d2fe6ae8e3..b3fd055b28d 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -685,7 +685,16 @@ public: static constexpr bool DateTime64Supported = true; IAggregateFunctionDataHelper(const DataTypes & argument_types_, const Array & parameters_) - : IAggregateFunctionHelper(argument_types_, parameters_) {} + : IAggregateFunctionHelper(argument_types_, parameters_) + { + /// To prevent derived classes changing the destroy() without updating hasTrivialDestructor() to match it + /// Enforce that either both of them are changed or none are + constexpr bool declares_destroy_and_hasTrivialDestructor = + std::is_same_v == + std::is_same_v; + static_assert(declares_destroy_and_hasTrivialDestructor, + "destroy() and hasTrivialDestructor() methods of an aggregate function must be either both overridden or not"); + } void create(AggregateDataPtr __restrict place) const override /// NOLINT { diff --git a/src/Processors/Transforms/WindowTransform.cpp b/src/Processors/Transforms/WindowTransform.cpp index 83b0b202d74..4d3eb1f0bbd 100644 --- a/src/Processors/Transforms/WindowTransform.cpp +++ b/src/Processors/Transforms/WindowTransform.cpp @@ -1579,6 +1579,8 @@ struct StatefulWindowFunction : public WindowFunction state->~State(); } + bool hasTrivialDestructor() const override { return std::is_trivially_destructible_v; } + State & getState(const WindowFunctionWorkspace & workspace) { return *static_cast(static_cast(workspace.aggregate_function_state.data())); From 2daec0b45e781366178a857319990898dd8ef14e Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Nov 2022 18:05:40 +0000 Subject: [PATCH 217/526] S3 request per second rate throttling + refactoring --- src/Backups/BackupIO_S3.cpp | 9 ++-- src/Backups/BackupIO_S3.h | 3 +- src/Common/ProfileEvents.cpp | 2 +- src/Common/Throttler.cpp | 2 - src/Common/Throttler.h | 2 + src/Coordination/KeeperSnapshotManagerS3.cpp | 6 ++- src/Core/Settings.h | 4 ++ .../ObjectStorages/S3/S3ObjectStorage.cpp | 15 ++++--- src/Disks/ObjectStorages/S3/diskSettings.cpp | 34 +++++++++++---- src/Disks/ObjectStorages/S3/diskSettings.h | 2 +- src/Disks/ObjectStorages/S3/parseConfig.h | 27 ------------ .../ObjectStorages/S3/registerDiskS3.cpp | 13 ++---- src/IO/ReadBufferFromS3.cpp | 10 ++--- src/IO/ReadBufferFromS3.h | 11 ++--- src/IO/S3/PocoHTTPClient.cpp | 26 +++++++++++- src/IO/S3/PocoHTTPClient.h | 17 +++++++- src/IO/S3/tests/gtest_aws_s3_client.cpp | 8 +++- src/IO/S3Common.cpp | 31 ++++++++++++-- src/IO/S3Common.h | 5 ++- src/Storages/StorageS3.cpp | 42 ++++++++++--------- src/Storages/StorageS3.h | 7 ++-- src/Storages/StorageS3Settings.cpp | 21 ++++++++++ src/Storages/StorageS3Settings.h | 7 +++- 23 files changed, 198 insertions(+), 106 deletions(-) delete mode 100644 src/Disks/ObjectStorages/S3/parseConfig.h diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 12038a8a30c..3f723c98a2a 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -46,7 +46,7 @@ namespace context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); + /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}); client_configuration.endpointOverride = s3_uri.endpoint; client_configuration.maxConnections = static_cast(context->getSettingsRef().s3_max_connections); @@ -86,8 +86,8 @@ BackupReaderS3::BackupReaderS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) - , max_single_read_retries(context_->getSettingsRef().s3_max_single_read_retries) , read_settings(context_->getReadSettings()) + , rw_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).rw_settings) { } @@ -115,7 +115,7 @@ UInt64 BackupReaderS3::getFileSize(const String & file_name) std::unique_ptr BackupReaderS3::readFile(const String & file_name) { return std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, max_single_read_retries, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, rw_settings, read_settings); } @@ -123,7 +123,6 @@ BackupWriterS3::BackupWriterS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) - , max_single_read_retries(context_->getSettingsRef().s3_max_single_read_retries) , read_settings(context_->getReadSettings()) , rw_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).rw_settings) { @@ -318,7 +317,7 @@ bool BackupWriterS3::fileContentsEqual(const String & file_name, const String & try { auto in = std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, max_single_read_retries, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, rw_settings, read_settings); String actual_file_contents(expected_file_contents.size(), ' '); return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) && (actual_file_contents == expected_file_contents) && in->eof(); diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 471ddcc06e6..17a365da396 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -39,8 +39,8 @@ public: private: S3::URI s3_uri; std::shared_ptr client; - UInt64 max_single_read_retries; ReadSettings read_settings; + S3Settings::ReadWriteSettings rw_settings; }; @@ -82,7 +82,6 @@ private: S3::URI s3_uri; std::shared_ptr client; - UInt64 max_single_read_retries; ReadSettings read_settings; S3Settings::ReadWriteSettings rw_settings; }; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 2f801e496fa..a3ceecf8041 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -61,7 +61,7 @@ M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \ M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ - M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform the 'max_network_bandwidth' setting.") \ + M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_network_bandwidth' and other throttling settings.") \ \ M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \ \ diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index b38777efc03..79625d4894c 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -20,8 +20,6 @@ namespace ErrorCodes /// Just 10^9. static constexpr auto NS = 1000000000UL; -static const size_t default_burst_seconds = 1; - Throttler::Throttler(size_t max_speed_, const std::shared_ptr & parent_) : max_speed(max_speed_) , max_burst(max_speed_ * default_burst_seconds) diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 9b6eff13506..708e9b939fa 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -17,6 +17,8 @@ namespace DB class Throttler { public: + static const size_t default_burst_seconds = 1; + Throttler(size_t max_speed_, size_t max_burst_, const std::shared_ptr & parent_ = nullptr) : max_speed(max_speed_), max_burst(max_burst_), limit_exceeded_exception_message(""), tokens(max_burst), parent(parent_) {} diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 2e19d496407..8777ef341ff 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -93,7 +93,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auth_settings.region, RemoteHostFilter(), s3_max_redirects, enable_s3_requests_logging, - /* for_disk_s3 = */ false); + /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}); client_configuration.endpointOverride = new_uri.endpoint; @@ -194,13 +194,15 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa lock_writer.finalize(); // We read back the written UUID, if it's the same we can upload the file + S3Settings::ReadWriteSettings rw_settings; + rw_settings.max_single_read_retries = 1; ReadBufferFromS3 lock_reader { s3_client->client, s3_client->uri.bucket, lock_file, "", - 1, + rw_settings, {} }; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5c6ca1a1d57..e36ff1985fb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -90,6 +90,10 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ + M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate. Zero means unlimited.", 0) \ + M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ + M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate. Zero means unlimited.", 0) \ + M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 0c421ee03d7..21b7ff4f0f1 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -175,7 +175,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT bucket, path, version_id, - settings_ptr->s3_settings.max_single_read_retries, + settings_ptr->s3_settings, disk_read_settings, /* use_external_buffer */true, /* offset */0, @@ -212,7 +212,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT bucket, object.absolute_path, version_id, - settings_ptr->s3_settings.max_single_read_retries, + settings_ptr->s3_settings, patchSettings(read_settings)); } @@ -627,17 +627,20 @@ void S3ObjectStorage::startup() void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { - s3_settings.set(getSettings(config, config_prefix, context)); - client.set(getClient(config, config_prefix, context)); + auto new_s3_settings = getSettings(config, config_prefix, context); + auto new_client = getClient(config, config_prefix, context, *new_s3_settings); + s3_settings.set(std::move(new_s3_settings)); + client.set(std::move(new_client)); applyRemoteThrottlingSettings(context); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { + auto new_s3_settings = getSettings(config, config_prefix, context); + auto new_client = getClient(config, config_prefix, context, *new_s3_settings); return std::make_unique( - getClient(config, config_prefix, context), - getSettings(config, config_prefix, context), + std::move(new_client), std::move(new_s3_settings), version_id, s3_capabilities, new_namespace, S3::URI(Poco::URI(config.getString(config_prefix + ".endpoint"))).endpoint); } diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index e61987163d2..707675c01fd 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -32,14 +33,23 @@ namespace ErrorCodes std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { + const Settings & settings = context->getSettingsRef(); S3Settings::ReadWriteSettings rw_settings; - rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries); - rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size); - rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", context->getSettingsRef().s3_upload_part_size_multiply_factor); - rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold); - rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size); - rw_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", context->getSettingsRef().s3_check_objects_after_upload); - rw_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", context->getSettingsRef().s3_max_unexpected_write_error_retries); + rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries); + rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size); + rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor); + rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold); + rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size); + rw_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload); + rw_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries); + + // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. + if (UInt64 max_get_rps = config.getUInt64(config_prefix + ".s3_max_get_rps", settings.s3_max_get_rps)) + rw_settings.get_request_throttler = std::make_shared( + max_get_rps, config.getUInt64(config_prefix + ".s3_max_get_burst", settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); + if (UInt64 max_put_rps = config.getUInt64(config_prefix + ".s3_max_put_rps", settings.s3_max_put_rps)) + rw_settings.put_request_throttler = std::make_shared( + max_put_rps, config.getUInt64(config_prefix + ".s3_max_put_burst", settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); return std::make_unique( rw_settings, @@ -112,14 +122,20 @@ std::shared_ptr getProxyConfiguration(const String & pre } -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) +std::unique_ptr getClient( + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + ContextPtr context, + const S3ObjectStorageSettings & settings) { S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ true); + /* for_disk_s3 = */ true, + settings.s3_settings.get_request_throttler, + settings.s3_settings.put_request_throttler); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 05ba8819f83..04eb7aced8e 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -22,7 +22,7 @@ struct S3ObjectStorageSettings; std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); +std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); } diff --git a/src/Disks/ObjectStorages/S3/parseConfig.h b/src/Disks/ObjectStorages/S3/parseConfig.h deleted file mode 100644 index 1defc673c2e..00000000000 --- a/src/Disks/ObjectStorages/S3/parseConfig.h +++ /dev/null @@ -1,27 +0,0 @@ -#pragma once - -#include "config.h" - -#if USE_AWS_S3 - -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - - -std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); - -std::shared_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); - - -} diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index e09aef22122..e73accbb956 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -130,21 +130,16 @@ void registerDiskS3(DiskFactory & factory) chassert(type == "s3" || type == "s3_plain"); MetadataStoragePtr metadata_storage; + auto settings = getSettings(config, config_prefix, context); + auto client = getClient(config, config_prefix, context, *settings); if (type == "s3_plain") { - s3_storage = std::make_shared( - getClient(config, config_prefix, context), - getSettings(config, config_prefix, context), - uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); + s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); metadata_storage = std::make_shared(s3_storage, uri.key); } else { - s3_storage = std::make_shared( - getClient(config, config_prefix, context), - getSettings(config, config_prefix, context), - uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); - + s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); metadata_storage = std::make_shared(metadata_disk, uri.key); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index c49941b025d..fa748469ca0 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -45,7 +45,7 @@ ReadBufferFromS3::ReadBufferFromS3( const String & bucket_, const String & key_, const String & version_id_, - UInt64 max_single_read_retries_, + const S3Settings::ReadWriteSettings & s3_settings_, const ReadSettings & settings_, bool use_external_buffer_, size_t offset_, @@ -56,7 +56,7 @@ ReadBufferFromS3::ReadBufferFromS3( , bucket(bucket_) , key(key_) , version_id(version_id_) - , max_single_read_retries(max_single_read_retries_) + , s3_settings(s3_settings_) , offset(offset_) , read_until_position(read_until_position_) , read_settings(settings_) @@ -105,7 +105,7 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; (attempt < max_single_read_retries) && !next_result; ++attempt) + for (size_t attempt = 0; attempt < s3_settings.max_single_read_retries && !next_result; ++attempt) { Stopwatch watch; try @@ -166,7 +166,7 @@ bool ReadBufferFromS3::nextImpl() attempt, e.message()); - if (attempt + 1 == max_single_read_retries) + if (attempt + 1 == s3_settings.max_single_read_retries) throw; /// Pause before next attempt. @@ -349,7 +349,7 @@ SeekableReadBufferPtr ReadBufferS3Factory::getReader() bucket, key, version_id, - s3_max_single_read_retries, + s3_settings, read_settings, false /*use_external_buffer*/, next_range->first, diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index cc836bba495..3e1a26f7713 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include "config.h" #if USE_AWS_S3 @@ -33,7 +34,7 @@ private: String bucket; String key; String version_id; - UInt64 max_single_read_retries; + const S3Settings::ReadWriteSettings s3_settings; /// These variables are atomic because they can be used for `logging only` /// (where it is not important to get consistent result) @@ -52,7 +53,7 @@ public: const String & bucket_, const String & key_, const String & version_id_, - UInt64 max_single_read_retries_, + const S3Settings::ReadWriteSettings & s3_settings_, const ReadSettings & settings_, bool use_external_buffer = false, size_t offset_ = 0, @@ -100,7 +101,7 @@ public: const String & version_id_, size_t range_step_, size_t object_size_, - UInt64 s3_max_single_read_retries_, + const S3Settings::ReadWriteSettings & s3_settings_, const ReadSettings & read_settings_) : client_ptr(client_ptr_) , bucket(bucket_) @@ -110,7 +111,7 @@ public: , range_generator(object_size_, range_step_) , range_step(range_step_) , object_size(object_size_) - , s3_max_single_read_retries(s3_max_single_read_retries_) + , s3_settings(s3_settings_) { assert(range_step > 0); assert(range_step < object_size); @@ -135,7 +136,7 @@ private: size_t range_step; size_t object_size; - UInt64 s3_max_single_read_retries; + const S3Settings::ReadWriteSettings s3_settings; }; } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 7d053bebe4a..33dd3250c9f 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -11,6 +11,7 @@ #include #include +#include #include #include #include @@ -76,12 +77,16 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, bool enable_s3_requests_logging_, - bool for_disk_s3_) + bool for_disk_s3_, + const ThrottlerPtr & get_request_throttler_, + const ThrottlerPtr & put_request_throttler_) : force_region(force_region_) , remote_host_filter(remote_host_filter_) , s3_max_redirects(s3_max_redirects_) , enable_s3_requests_logging(enable_s3_requests_logging_) , for_disk_s3(for_disk_s3_) + , get_request_throttler(get_request_throttler_) + , put_request_throttler(put_request_throttler_) { } @@ -128,6 +133,8 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , s3_max_redirects(client_configuration.s3_max_redirects) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) + , get_request_throttler(client_configuration.get_request_throttler) + , put_request_throttler(client_configuration.put_request_throttler) , extra_headers(client_configuration.extra_headers) { } @@ -245,6 +252,23 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Make request to: {}", uri); + switch (request.GetMethod()) + { + case Aws::Http::HttpMethod::HTTP_GET: + case Aws::Http::HttpMethod::HTTP_HEAD: + if (get_request_throttler) + get_request_throttler->add(1); + break; + case Aws::Http::HttpMethod::HTTP_PUT: + case Aws::Http::HttpMethod::HTTP_POST: + case Aws::Http::HttpMethod::HTTP_PATCH: + if (put_request_throttler) + put_request_throttler->add(1); + break; + case Aws::Http::HttpMethod::HTTP_DELETE: + break; // Not throttled + } + addMetric(request, S3MetricType::Count); CurrentMetrics::Increment metric_increment{CurrentMetrics::S3Requests}; diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index 5649638285d..ed6e1793c2c 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -8,6 +8,7 @@ #if USE_AWS_S3 #include +#include #include #include #include @@ -48,6 +49,8 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration unsigned int s3_max_redirects; bool enable_s3_requests_logging; bool for_disk_s3; + ThrottlerPtr get_request_throttler; + ThrottlerPtr put_request_throttler; HeaderCollection extra_headers; void updateSchemeAndRegion(); @@ -60,7 +63,9 @@ private: const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, bool enable_s3_requests_logging_, - bool for_disk_s3_ + bool for_disk_s3_, + const ThrottlerPtr & get_request_throttler_, + const ThrottlerPtr & put_request_throttler_ ); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. @@ -154,6 +159,16 @@ private: unsigned int s3_max_redirects; bool enable_s3_requests_logging; bool for_disk_s3; + + /// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler + /// (i.e. throttles GetObject, HeadObject) + ThrottlerPtr get_request_throttler; + + /// Limits put request per second rate for PUT, COPY, POST, LIST requests + /// (i.e. throttles PutObject, CopyObject, ListObjects, CreateMultipartUpload, UploadPartCopy, UploadPart, CompleteMultipartUpload) + /// NOTE: DELETE and CANCEL requests are not throttled by either put or get throttler + ThrottlerPtr put_request_throttler; + const HeaderCollection extra_headers; }; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 9b2a65d84fc..96a7c8b3dfb 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -88,7 +88,9 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) remote_host_filter, s3_max_redirects, enable_s3_requests_logging, - /* for_disk_s3 = */ false + /* for_disk_s3 = */ false, + /* get_request_throttler = */ {}, + /* put_request_throttler = */ {} ); client_configuration.endpointOverride = uri.endpoint; @@ -113,12 +115,14 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) ASSERT_TRUE(client); DB::ReadSettings read_settings; + DB::S3Settings::ReadWriteSettings rw_settings; + rw_settings.max_single_read_retries = max_single_read_retries; DB::ReadBufferFromS3 read_buffer( client, uri.bucket, uri.key, version_id, - max_single_read_retries, + rw_settings, read_settings ); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 859f5ce796b..5af09275df4 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -573,7 +573,14 @@ public: /// AWS API tries credentials providers one by one. Some of providers (like ProfileConfigFileAWSCredentialsProvider) can be /// quite verbose even if nobody configured them. So we use our provider first and only after it use default providers. { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + configuration.region, + configuration.remote_host_filter, + configuration.s3_max_redirects, + configuration.enable_s3_requests_logging, + configuration.for_disk_s3, + configuration.get_request_throttler, + configuration.put_request_throttler); AddProvider(std::make_shared(aws_client_configuration)); } @@ -610,7 +617,14 @@ public: } else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true") { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( + configuration.region, + configuration.remote_host_filter, + configuration.s3_max_redirects, + configuration.enable_s3_requests_logging, + configuration.for_disk_s3, + configuration.get_request_throttler, + configuration.put_request_throttler); /// See MakeDefaultHttpResourceClientConfiguration(). /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside @@ -731,9 +745,18 @@ namespace S3 const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, bool enable_s3_requests_logging, - bool for_disk_s3) + bool for_disk_s3, + const ThrottlerPtr & get_request_throttler, + const ThrottlerPtr & put_request_throttler) { - return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging, for_disk_s3); + return PocoHTTPClientConfiguration( + force_region, + remote_host_filter, + s3_max_redirects, + enable_s3_requests_logging, + for_disk_s3, + get_request_throttler, + put_request_throttler); } URI::URI(const Poco::URI & uri_) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 93e5eb78c7f..01a6b8d5d82 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -17,6 +17,7 @@ #include #include +#include namespace Aws::S3 { @@ -88,7 +89,9 @@ public: const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, bool enable_s3_requests_logging, - bool for_disk_s3); + bool for_disk_s3, + const ThrottlerPtr & get_request_throttler, + const ThrottlerPtr & put_request_throttler); private: ClientFactory(); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9bbccf5f582..9309e3f7384 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -100,7 +100,8 @@ public: const Block & virtual_header_, ContextPtr context_, std::unordered_map * object_infos_, - Strings * read_keys_) + Strings * read_keys_, + const S3Settings::ReadWriteSettings & rw_settings_) : WithContext(context_) , client(client_) , globbed_uri(globbed_uri_) @@ -108,6 +109,7 @@ public: , virtual_header(virtual_header_) , object_infos(object_infos_) , read_keys(read_keys_) + , rw_settings(rw_settings_) { if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -258,6 +260,7 @@ private: bool is_finished{false}; std::unordered_map * object_infos; Strings * read_keys; + S3Settings::ReadWriteSettings rw_settings; }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( @@ -267,8 +270,9 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( const Block & virtual_header, ContextPtr context, std::unordered_map * object_infos_, - Strings * read_keys_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_)) + Strings * read_keys_, + const S3Settings::ReadWriteSettings & rw_settings_) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_, rw_settings_)) { } @@ -381,7 +385,7 @@ StorageS3Source::StorageS3Source( std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - UInt64 max_single_read_retries_, + const S3Settings::ReadWriteSettings & rw_settings_, String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -397,7 +401,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) - , max_single_read_retries(max_single_read_retries_) + , rw_settings(rw_settings_) , compression_hint(std::move(compression_hint_)) , client(client_) , sample_block(sample_block_) @@ -463,7 +467,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k if (!use_parallel_download || object_too_small) { LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); - return std::make_unique(client, bucket, key, version_id, max_single_read_retries, getContext()->getReadSettings()); + return std::make_unique(client, bucket, key, version_id, rw_settings, getContext()->getReadSettings()); } assert(object_size > 0); @@ -475,7 +479,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k } auto factory = std::make_unique( - client, bucket, key, version_id, download_buffer_size, object_size, max_single_read_retries, getContext()->getReadSettings()); + client, bucket, key, version_id, download_buffer_size, object_size, rw_settings, getContext()->getReadSettings()); LOG_TRACE( log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size); @@ -815,7 +819,7 @@ std::shared_ptr StorageS3::createFileIterator( { /// Iterate through disclosed globs and make a source for each file auto glob_iterator = std::make_shared( - *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys); + *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys, s3_configuration.rw_settings); return std::make_shared([glob_iterator]() { return glob_iterator->next(); }); } else @@ -905,7 +909,7 @@ Pipe StorageS3::read( format_settings, columns_description, max_block_size, - s3_configuration.rw_settings.max_single_read_retries, + s3_configuration.rw_settings, compression_method, s3_configuration.client, s3_configuration.uri.bucket, @@ -1022,9 +1026,7 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - const auto & config_rw_settings = settings.rw_settings; - - if (upd.rw_settings != config_rw_settings) + if (upd.rw_settings != settings.rw_settings) upd.rw_settings = settings.rw_settings; upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); @@ -1045,7 +1047,9 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration ctx->getRemoteHostFilter(), static_cast(ctx->getGlobalContext()->getSettingsRef().s3_max_redirects), ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false); + /* for_disk_s3 = */ false, + upd.rw_settings.get_request_throttler, + upd.rw_settings.put_request_throttler); client_configuration.endpointOverride = upd.uri.endpoint; client_configuration.maxConnections = static_cast(upd.rw_settings.max_connections); @@ -1082,15 +1086,15 @@ void StorageS3::processNamedCollectionResult(StorageS3Configuration & configurat else if (arg_name == "max_single_read_retries") configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); else if (arg_name == "min_upload_part_size") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); + configuration.rw_settings.min_upload_part_size = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); else if (arg_name == "upload_part_size_multiply_factor") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); + configuration.rw_settings.upload_part_size_multiply_factor = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); else if (arg_name == "upload_part_size_multiply_parts_count_threshold") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); + configuration.rw_settings.upload_part_size_multiply_parts_count_threshold = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); else if (arg_name == "max_single_part_upload_size") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); + configuration.rw_settings.max_single_part_upload_size = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); else if (arg_name == "max_connections") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_connections"); + configuration.rw_settings.max_connections = checkAndGetLiteralArgument(arg_value, "max_connections"); else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].", @@ -1220,7 +1224,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod( std::make_unique( - s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.rw_settings.max_single_read_retries, ctx->getReadSettings()), + s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.rw_settings, ctx->getReadSettings()), chooseCompressionMethod(key, compression_method), zstd_window_log_max); }; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 23947a32092..76391cb2695 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,7 +43,8 @@ public: const Block & virtual_header, ContextPtr context, std::unordered_map * object_infos = nullptr, - Strings * read_keys_ = nullptr); + Strings * read_keys_ = nullptr, + const S3Settings::ReadWriteSettings & rw_settings_ = {}); String next(); @@ -79,7 +80,7 @@ public: std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - UInt64 max_single_read_retries_, + const S3Settings::ReadWriteSettings & rw_settings_, String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -102,7 +103,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - UInt64 max_single_read_retries; + S3Settings::ReadWriteSettings rw_settings; String compression_hint; std::shared_ptr client; Block sample_block; diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 65e9bb1ab8c..9e670c65e17 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -56,6 +57,14 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U rw_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); rw_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); + // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. + if (UInt64 max_get_rps = get_uint_for_key(key, "max_get_rps", true, settings.s3_max_get_rps)) + rw_settings.get_request_throttler = std::make_shared( + max_get_rps, get_uint_for_key(key, "max_get_burst", true, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); + if (UInt64 max_put_rps = get_uint_for_key(key, "max_put_rps", true, settings.s3_max_put_rps)) + rw_settings.put_request_throttler = std::make_shared( + max_put_rps, get_uint_for_key(key, "max_put_burst", true, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); + s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(rw_settings)}); } } @@ -87,6 +96,12 @@ S3Settings::ReadWriteSettings::ReadWriteSettings(const Settings & settings) max_connections = settings.s3_max_connections; check_objects_after_upload = settings.s3_check_objects_after_upload; max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; + if (settings.s3_max_get_rps) + get_request_throttler = std::make_shared( + settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); + if (settings.s3_max_put_rps) + put_request_throttler = std::make_shared( + settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); } void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & settings) @@ -106,6 +121,12 @@ void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & s if (!max_unexpected_write_error_retries) max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; check_objects_after_upload = settings.s3_check_objects_after_upload; + if (!get_request_throttler && settings.s3_max_get_rps) + get_request_throttler = std::make_shared( + settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); + if (!put_request_throttler && settings.s3_max_put_rps) + put_request_throttler = std::make_shared( + settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); } } diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 2da4a1d7590..40ba11a19ba 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include @@ -33,6 +34,8 @@ struct S3Settings size_t max_connections = 0; bool check_objects_after_upload = false; size_t max_unexpected_write_error_retries = 0; + ThrottlerPtr get_request_throttler; + ThrottlerPtr put_request_throttler; ReadWriteSettings() = default; explicit ReadWriteSettings(const Settings & settings); @@ -46,7 +49,9 @@ struct S3Settings && max_single_part_upload_size == other.max_single_part_upload_size && max_connections == other.max_connections && check_objects_after_upload == other.check_objects_after_upload - && max_unexpected_write_error_retries == other.max_unexpected_write_error_retries; + && max_unexpected_write_error_retries == other.max_unexpected_write_error_retries + && get_request_throttler == other.get_request_throttler + && put_request_throttler == other.put_request_throttler; } void updateFromSettingsIfEmpty(const Settings & settings); From da114e7ab7ebff76449ce0f3274d4c805a1d6a56 Mon Sep 17 00:00:00 2001 From: serxa Date: Mon, 7 Nov 2022 18:29:00 +0000 Subject: [PATCH 218/526] fix bandwidth throttlers initialization order --- src/Interpreters/ProcessList.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessList.cpp b/src/Interpreters/ProcessList.cpp index 551d20f835a..84f5570349b 100644 --- a/src/Interpreters/ProcessList.cpp +++ b/src/Interpreters/ProcessList.cpp @@ -263,6 +263,11 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as user_process_list.user_memory_tracker.setSoftLimit(settings.memory_overcommit_ratio_denominator_for_user); user_process_list.user_memory_tracker.setDescription("(for user)"); + if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) + { + total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); + } + if (!user_process_list.user_throttler) { if (settings.max_network_bandwidth_for_user) @@ -270,11 +275,6 @@ ProcessList::EntryPtr ProcessList::insert(const String & query_, const IAST * as else if (settings.max_network_bandwidth_for_all_users) user_process_list.user_throttler = total_network_throttler; } - - if (!total_network_throttler && settings.max_network_bandwidth_for_all_users) - { - total_network_throttler = std::make_shared(settings.max_network_bandwidth_for_all_users); - } } return res; From 2c7a7fa406a4601fa67360fc8148fd7d8d0bee76 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 7 Nov 2022 18:38:41 +0000 Subject: [PATCH 219/526] Disable fault injection and retries function tests settings --- tests/config/users.d/insert_keeper_retries.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml index 462c9df5248..f7b652bf2ef 100644 --- a/tests/config/users.d/insert_keeper_retries.xml +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -1,8 +1,8 @@ - 20 - 0.01 + 0 + 0.0 From 9210e586d24ea572d76ec58729d89e3bb5c0c4dc Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 7 Nov 2022 20:27:18 +0100 Subject: [PATCH 220/526] fix --- src/Common/ZooKeeper/ZooKeeper.h | 39 ++++++++++++------- .../MergeTree/EphemeralLockInZooKeeper.cpp | 1 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 11 +++--- .../MergeTree/ReplicatedMergeTreeSink.h | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../0_stateless/01158_zookeeper_log_long.sql | 2 +- 6 files changed, 36 insertions(+), 21 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 968d10ad9a5..c9529b78f8a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -76,7 +76,7 @@ using GetPriorityForLoadBalancing = DB::GetPriorityForLoadBalancing; template concept ZooKeeperResponse = std::derived_from; -template +template struct MultiReadResponses { template @@ -96,7 +96,17 @@ struct MultiReadResponses if constexpr (std::same_as) return dynamic_cast(*resp[index]); else + { + if constexpr (try_multi) + { + /// We should not ignore errors except ZNONODE + /// for consistency with exists, tryGet and tryGetChildren + const auto & error = resp[index].error; + if (error != Coordination::Error::ZOK && error != Coordination::Error::ZNONODE) + throw KeeperException(error); + } return resp[index]; + } }, responses); } @@ -144,6 +154,7 @@ class ZooKeeper public: using Ptr = std::shared_ptr; + using ErrorsList = std::initializer_list; ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); @@ -217,7 +228,7 @@ public: bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr); bool existsWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); - using MultiExistsResponse = MultiReadResponses; + using MultiExistsResponse = MultiReadResponses; template MultiExistsResponse exists(TIter start, TIter end) { @@ -233,7 +244,8 @@ public: std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const EventPtr & watch = nullptr); std::string getWatch(const std::string & path, Coordination::Stat * stat, Coordination::WatchCallback watch_callback); - using MultiGetResponse = MultiReadResponses; + using MultiGetResponse = MultiReadResponses; + using MultiTryGetResponse = MultiReadResponses; template MultiGetResponse get(TIter start, TIter end) @@ -264,13 +276,13 @@ public: Coordination::Error * code = nullptr); template - MultiGetResponse tryGet(TIter start, TIter end) + MultiTryGetResponse tryGet(TIter start, TIter end) { return multiRead( start, end, zkutil::makeGetRequest, [&](const auto & path) { return asyncTryGet(path); }); } - MultiGetResponse tryGet(const std::vector & paths) + MultiTryGetResponse tryGet(const std::vector & paths) { return tryGet(paths.begin(), paths.end()); } @@ -297,7 +309,8 @@ public: Coordination::WatchCallback watch_callback, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); - using MultiGetChildrenResponse = MultiReadResponses; + using MultiGetChildrenResponse = MultiReadResponses; + using MultiTryGetChildrenResponse = MultiReadResponses; template MultiGetChildrenResponse @@ -333,7 +346,7 @@ public: Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL); template - MultiGetChildrenResponse + MultiTryGetChildrenResponse tryGetChildren(TIter start, TIter end, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) { return multiRead( @@ -343,7 +356,7 @@ public: [&](const auto & path) { return asyncTryGetChildren(path, list_request_type); }); } - MultiGetChildrenResponse + MultiTryGetChildrenResponse tryGetChildren(const std::vector & paths, Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) { return tryGetChildren(paths.begin(), paths.end(), list_request_type); @@ -511,7 +524,7 @@ private: using AsyncFunction = std::function(const std::string &)>; template - MultiReadResponses multiRead(TIter start, TIter end, RequestFactory request_factory, AsyncFunction async_fun) + MultiReadResponses multiRead(TIter start, TIter end, RequestFactory request_factory, AsyncFunction async_fun) { if (getApiVersion() >= DB::KeeperApiVersion::WITH_MULTI_READ) { @@ -523,12 +536,12 @@ private: { Coordination::Responses responses; tryMulti(requests, responses); - return MultiReadResponses{std::move(responses)}; + return MultiReadResponses{std::move(responses)}; } else { auto responses = multi(requests); - return MultiReadResponses{std::move(responses)}; + return MultiReadResponses{std::move(responses)}; } } @@ -536,14 +549,14 @@ private: std::vector> future_responses; if (responses_size == 0) - return MultiReadResponses(std::move(future_responses)); + return MultiReadResponses(std::move(future_responses)); future_responses.reserve(responses_size); for (auto it = start; it != end; ++it) future_responses.push_back(async_fun(*it)); - return MultiReadResponses{std::move(future_responses)}; + return MultiReadResponses{std::move(future_responses)}; } std::unique_ptr impl; diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 802d648655f..6ddb35d109e 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -141,6 +141,7 @@ void EphemeralLocksInAllPartitions::unlock() return; std::vector futures; + futures.reserve(locks.size()); for (const auto & lock : locks) { futures.push_back(zookeeper->asyncRemove(lock.path)); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index b3f5a421763..53654285699 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -469,7 +469,7 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_value, replicas_num); + waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); } else { @@ -636,7 +636,7 @@ void ReplicatedMergeTreeSink::commitPart( storage.updateQuorum(part->name, false); } - waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_value, replicas_num); + waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); } } @@ -658,7 +658,7 @@ void ReplicatedMergeTreeSink::waitForQuorum( zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, - const std::string & is_active_node_value, + Int32 is_active_node_version, size_t replicas_num) const { /// We are waiting for quorum to be satisfied. @@ -691,9 +691,10 @@ void ReplicatedMergeTreeSink::waitForQuorum( /// And what if it is possible that the current replica at this time has ceased to be active /// and the quorum is marked as failed and deleted? + Coordination::Stat stat; String value; - if (!zookeeper->tryGet(storage.replica_path + "/is_active", value, nullptr) - || value != is_active_node_value) + if (!zookeeper->tryGet(storage.replica_path + "/is_active", value, &stat) + || stat.version != is_active_node_version) throw Exception("Replica become inactive while waiting for quorum", ErrorCodes::NO_ACTIVE_REPLICAS); } catch (...) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index da87ddc0d63..b094b373f97 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -85,7 +85,7 @@ private: /// Also checks that replica still alive. void waitForQuorum( zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, - const std::string & quorum_path, const std::string & is_active_node_value, size_t replicas_num) const; + const std::string & quorum_path, int is_active_node_version, size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a70e6f30989..c547ad29f3d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6484,7 +6484,7 @@ void StorageReplicatedMergeTree::getClearBlocksInPartitionOps( if (startsWith(block_id, partition_prefix)) paths_to_get.push_back(fs::path(zookeeper_path) / "blocks" / block_id); - auto results = zookeeper.get(paths_to_get); + auto results = zookeeper.tryGet(paths_to_get); for (size_t i = 0; i < paths_to_get.size(); ++i) { diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 501ab805a5a..5f6ca762ae7 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -27,7 +27,7 @@ select 'blocks'; select type, has_watch, op_num, path, is_ephemeral, is_sequential, version, requests_size, request_idx, error, watch_type, watch_state, path_created, stat_version, stat_cversion, stat_dataLength, stat_numChildren from system.zookeeper_log -where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks%' and op_num not in (1, 12, 500)) +where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks/%' and op_num not in (1, 12, 500)) order by xid, type, request_idx; drop table rmt; From 6e4daa67d541d3ba3fd8f41cb5dab295829c3e3d Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Nov 2022 20:46:53 +0100 Subject: [PATCH 221/526] better tests --- ...346_alter_enum_partition_key_replicated_zookeeper_long.sql | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 2b3e17a8776..1aa96016007 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -1,6 +1,7 @@ -- Tags: long, replica SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries +SET replication_alter_partitions_sync=2; DROP TABLE IF EXISTS test SYNC; DROP TABLE IF EXISTS test2 SYNC; @@ -10,7 +11,6 @@ CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = Replica INSERT INTO test VALUES ('hello', 'test'); SELECT * FROM test; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -19,7 +19,6 @@ ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 3); INSERT INTO test VALUES ('goodbye', 'test'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -33,7 +32,6 @@ ALTER TABLE test MODIFY COLUMN x Int8; INSERT INTO test VALUES (111, 'abc'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; From 257e766e0e9aa0bd021dcdca81ead4317aa87246 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Nov 2022 20:39:33 +0100 Subject: [PATCH 222/526] Trying to fix two flaky tests --- .../MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp | 4 ++++ tests/integration/test_restore_replica/test.py | 9 ++++----- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 626295d7255..192f0d23f96 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -112,6 +112,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() && now - last_refresh_time < REFRESH_STATE_MINIMUM_INTERVAL_SECONDS) return; + LOG_DEBUG(storage.log, "Updating strategy picker state"); + auto zookeeper = storage.getZooKeeper(); auto all_replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas"); @@ -154,6 +156,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() last_refresh_time = now; current_replica_index = current_replica_index_tmp; active_replicas = active_replicas_tmp; + + LOG_DEBUG(storage.log, "Strategy picker state updated, current replica: {}, active replicas: [{}]", current_replica_index, fmt::join(active_replicas, ", ")); } diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py index 0b11cdf7512..31c503f6184 100644 --- a/tests/integration/test_restore_replica/test.py +++ b/tests/integration/test_restore_replica/test.py @@ -7,6 +7,9 @@ from helpers.test_tools import assert_eq_with_retry def fill_nodes(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS test SYNC") + for node in nodes: node.query( """ @@ -29,11 +32,7 @@ nodes = [node_1, node_2, node_3] def fill_table(): - node_1.query("TRUNCATE TABLE test") - - for node in nodes: - node.query("SYSTEM SYNC REPLICA test") - + fill_nodes(nodes) check_data(0, 0) # it will create multiple parts in each partition and probably cause merges From de6014e4a56b61f3928bc641437012420de84c69 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 7 Nov 2022 22:54:08 +0100 Subject: [PATCH 223/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 5c9ab1af24e..78176890841 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db 2>> server.log & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db 2>&1 > server.log & server_pid=$! kill -0 $server_pid From d695240d5351d75b6b424fb3223385713b6b68f1 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Mon, 7 Nov 2022 23:07:00 +0100 Subject: [PATCH 224/526] Update StorageHudi.cpp --- src/Storages/StorageHudi.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 986bb6b955c..121856c4a57 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -45,7 +45,7 @@ StorageHudi::StorageHudi( StorageS3::updateS3Configuration(context_, base_configuration); auto keys = getKeysFromS3(); - auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(std::move(keys), configuration_.format); + auto new_uri = base_configuration.uri.uri.toString() + generateQueryFromKeys(keys, configuration_.format); LOG_DEBUG(log, "New uri: {}", new_uri); LOG_DEBUG(log, "Table path: {}", table_path); @@ -158,7 +158,7 @@ String StorageHudi::generateQueryFromKeys(const std::vector & keys, const String partition_path = key_path.parent_path(); /// Every filename contains metadata split by "_", timestamp is after last "_". - const auto delim = key.find_last_of("_") + 1; + const auto delim = key.find_last_of('_') + 1; if (delim == std::string::npos) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected format of metadata files"); const auto timestamp = parse(key.substr(delim + 1)); From 48cc1d8492c5838958ec14a7da6a2490ca696986 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky <43110995+evillique@users.noreply.github.com> Date: Tue, 8 Nov 2022 00:15:19 +0100 Subject: [PATCH 225/526] Update DateLUTImpl.h --- src/Common/DateLUTImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/DateLUTImpl.h b/src/Common/DateLUTImpl.h index 3afbb6735dc..2f8aa487621 100644 --- a/src/Common/DateLUTImpl.h +++ b/src/Common/DateLUTImpl.h @@ -1331,7 +1331,7 @@ public: } template - inline auto addQuarters(DateOrTime d, Int64 delta) const + inline auto NO_SANITIZE_UNDEFINED addQuarters(DateOrTime d, Int64 delta) const { return addMonths(d, delta * 3); } From ab0174f3e864e069639c1df975466cf2ce0f6fca Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 08:51:01 +0000 Subject: [PATCH 226/526] Add test for issue #42520 --- .../02477_number_to_date.reference | 25 +++++++++++++++++++ .../0_stateless/02477_number_to_date.sql | 17 +++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 tests/queries/0_stateless/02477_number_to_date.reference create mode 100644 tests/queries/0_stateless/02477_number_to_date.sql diff --git a/tests/queries/0_stateless/02477_number_to_date.reference b/tests/queries/0_stateless/02477_number_to_date.reference new file mode 100644 index 00000000000..9161909966c --- /dev/null +++ b/tests/queries/0_stateless/02477_number_to_date.reference @@ -0,0 +1,25 @@ +-- { echoOn } +select toDate(1666249120::Float); +2022-10-20 +select toDate(1666249120::Double); +2022-10-20 +select toDate(1666249120::UInt32); +2022-10-20 +select toDate32(1666249120::Float); +2022-10-20 +select toDate32(1666249120::Double); +2022-10-20 +select toDate32(1666249120::UInt32); +2022-10-20 +select toDateTime(1666249120::Float); +2022-10-20 06:58:08 +select toDateTime(1666249120::Double); +2022-10-20 06:58:40 +select toDateTime(1666249120::UInt32); +2022-10-20 06:58:40 +select toDateTime64(1666249120::Float, 3); +2022-10-20 06:58:49.984 +select toDateTime64(1666249120::Double, 3); +2022-10-20 06:58:40.000 +select toDateTime64(1666249120::UInt32, 3); +2022-10-20 06:58:40.000 diff --git a/tests/queries/0_stateless/02477_number_to_date.sql b/tests/queries/0_stateless/02477_number_to_date.sql new file mode 100644 index 00000000000..b365bf5a014 --- /dev/null +++ b/tests/queries/0_stateless/02477_number_to_date.sql @@ -0,0 +1,17 @@ +-- { echoOn } +select toDate(1666249120::Float); +select toDate(1666249120::Double); +select toDate(1666249120::UInt32); + +select toDate32(1666249120::Float); +select toDate32(1666249120::Double); +select toDate32(1666249120::UInt32); + +select toDateTime(1666249120::Float); +select toDateTime(1666249120::Double); +select toDateTime(1666249120::UInt32); + +select toDateTime64(1666249120::Float, 3); +select toDateTime64(1666249120::Double, 3); +select toDateTime64(1666249120::UInt32, 3); +-- { echoOff } From 9ec313b05c2dbf8d699491013c8454b4470668cc Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger Date: Tue, 8 Nov 2022 18:32:53 +0800 Subject: [PATCH 227/526] remove generic driver lld Signed-off-by: Lloyd-Pottiger --- cmake/tools.cmake | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/cmake/tools.cmake b/cmake/tools.cmake index 23f34bb24cd..e8fecd9f30b 100644 --- a/cmake/tools.cmake +++ b/cmake/tools.cmake @@ -57,13 +57,14 @@ if (NOT LINKER_NAME) if (COMPILER_GCC) find_program (LLD_PATH NAMES "ld.lld") find_program (GOLD_PATH NAMES "ld.gold") - # llvm lld is a generic driver. - # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead - elseif (COMPILER_CLANG AND OS_LINUX) - find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld.lld" "lld") - find_program (GOLD_PATH NAMES "ld.gold" "gold") - elseif (COMPILER_CLANG AND OS_DARWIN) - find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "lld-${COMPILER_VERSION_MAJOR}" "ld64.lld" "lld") + elseif (COMPILER_CLANG) + # llvm lld is a generic driver. + # Invoke ld.lld (Unix), ld64.lld (macOS), lld-link (Windows), wasm-ld (WebAssembly) instead + if (OS_LINUX) + find_program (LLD_PATH NAMES "ld.lld-${COMPILER_VERSION_MAJOR}" "ld.lld") + elseif (OS_DARWIN) + find_program (LLD_PATH NAMES "ld64.lld-${COMPILER_VERSION_MAJOR}" "ld64.lld") + endif () find_program (GOLD_PATH NAMES "ld.gold" "gold") endif () endif() From ff083e1b55190c1db38d8f39c040242dd4e3d4fb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 18:17:54 +0100 Subject: [PATCH 228/526] Forbit launching release.py from branches but `master` --- tests/ci/release.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/ci/release.py b/tests/ci/release.py index c0c248aa513..fd4bda3eae4 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -118,6 +118,8 @@ class Release: except subprocess.CalledProcessError: logging.fatal("Repo contains uncommitted changes") raise + if self._git.branch != "master": + raise Exception("the script must be launched only from master") self.set_release_branch() From 7549b223173d387aa65dc41c307aaf4515394577 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 18:20:39 +0100 Subject: [PATCH 229/526] Add a script to mark commit as ready for release --- tests/ci/mark_release_ready.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 tests/ci/mark_release_ready.py diff --git a/tests/ci/mark_release_ready.py b/tests/ci/mark_release_ready.py new file mode 100644 index 00000000000..be1771e62bd --- /dev/null +++ b/tests/ci/mark_release_ready.py @@ -0,0 +1,25 @@ +#!/usr/bin/env python3 + +from commit_status_helper import get_commit +from env_helper import GITHUB_JOB_URL +from get_robot_token import get_best_robot_token +from github_helper import GitHub +from pr_info import PRInfo + +RELEASE_READY_STATUS = "Ready for release" + + +def main(): + pr_info = PRInfo() + gh = GitHub(get_best_robot_token(), per_page=100) + commit = get_commit(gh, pr_info.sha) + commit.create_status( + context=RELEASE_READY_STATUS, + description="the release can be created from the commit", + state="success", + target_url=GITHUB_JOB_URL(), + ) + + +if __name__ == "__main__": + main() From aebb30c7d74a01e8aae746c932b76a67adf5a4d2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 21:28:05 +0100 Subject: [PATCH 230/526] Add CI jobs to mark commit as release ready --- .github/workflows/master.yml | 18 ++++++++++++++++++ .github/workflows/release_branches.yml | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 3d43a960534..08177eace76 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -1056,6 +1056,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -3069,6 +3086,7 @@ jobs: needs: - DockerHubPush - BuilderReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index abe85d3e72d..8148905cec7 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -615,6 +615,23 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" + MarkReleaseReady: + needs: + - BuilderBinDarwin + - BuilderBinDarwinAarch64 + - BuilderDebRelease + - BuilderDebAarch64 + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Mark Commit Release Ready + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 mark_release_ready.py ############################################################################################## ########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## @@ -1888,6 +1905,7 @@ jobs: - DockerServerImages - BuilderReport - BuilderSpecialReport + - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 - FunctionalStatelessTestDebug2 From 7069919706749db9662eeb8b24cf898b883e12eb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 8 Nov 2022 11:16:01 +0100 Subject: [PATCH 231/526] Add BuilderSpecialReport to dependencies of FinishCheck --- .github/workflows/master.yml | 1 + .github/workflows/pull_request.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 08177eace76..da84500559a 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -3086,6 +3086,7 @@ jobs: needs: - DockerHubPush - BuilderReport + - BuilderSpecialReport - MarkReleaseReady - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 09ca64977f0..d3580f4c4d7 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -3579,6 +3579,7 @@ jobs: - DockerServerImages - CheckLabels - BuilderReport + - BuilderSpecialReport - FastTest - FunctionalStatelessTestDebug0 - FunctionalStatelessTestDebug1 From f8bb9ce787f56d6a97e070fac7b090f26537fc00 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 10:49:45 +0000 Subject: [PATCH 232/526] Try to fix time zone difference --- .../02477_number_to_date.reference | 25 ------------------- .../0_stateless/02477_number_to_date.sql | 17 ------------- 2 files changed, 42 deletions(-) delete mode 100644 tests/queries/0_stateless/02477_number_to_date.reference delete mode 100644 tests/queries/0_stateless/02477_number_to_date.sql diff --git a/tests/queries/0_stateless/02477_number_to_date.reference b/tests/queries/0_stateless/02477_number_to_date.reference deleted file mode 100644 index 9161909966c..00000000000 --- a/tests/queries/0_stateless/02477_number_to_date.reference +++ /dev/null @@ -1,25 +0,0 @@ --- { echoOn } -select toDate(1666249120::Float); -2022-10-20 -select toDate(1666249120::Double); -2022-10-20 -select toDate(1666249120::UInt32); -2022-10-20 -select toDate32(1666249120::Float); -2022-10-20 -select toDate32(1666249120::Double); -2022-10-20 -select toDate32(1666249120::UInt32); -2022-10-20 -select toDateTime(1666249120::Float); -2022-10-20 06:58:08 -select toDateTime(1666249120::Double); -2022-10-20 06:58:40 -select toDateTime(1666249120::UInt32); -2022-10-20 06:58:40 -select toDateTime64(1666249120::Float, 3); -2022-10-20 06:58:49.984 -select toDateTime64(1666249120::Double, 3); -2022-10-20 06:58:40.000 -select toDateTime64(1666249120::UInt32, 3); -2022-10-20 06:58:40.000 diff --git a/tests/queries/0_stateless/02477_number_to_date.sql b/tests/queries/0_stateless/02477_number_to_date.sql deleted file mode 100644 index b365bf5a014..00000000000 --- a/tests/queries/0_stateless/02477_number_to_date.sql +++ /dev/null @@ -1,17 +0,0 @@ --- { echoOn } -select toDate(1666249120::Float); -select toDate(1666249120::Double); -select toDate(1666249120::UInt32); - -select toDate32(1666249120::Float); -select toDate32(1666249120::Double); -select toDate32(1666249120::UInt32); - -select toDateTime(1666249120::Float); -select toDateTime(1666249120::Double); -select toDateTime(1666249120::UInt32); - -select toDateTime64(1666249120::Float, 3); -select toDateTime64(1666249120::Double, 3); -select toDateTime64(1666249120::UInt32, 3); --- { echoOff } From d7cd038e759b48293482caa59c70c423c1bb0451 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 11:59:12 +0000 Subject: [PATCH 233/526] Integrate tests into 02462_int_to_date + rename to 02462_num_to_datetype --- .../0_stateless/02462_int_to_date.reference | 4 -- .../queries/0_stateless/02462_int_to_date.sql | 4 -- .../02462_number_to_datetype.reference | 49 +++++++++++++++++++ .../0_stateless/02462_number_to_datetype.sql | 29 +++++++++++ 4 files changed, 78 insertions(+), 8 deletions(-) delete mode 100644 tests/queries/0_stateless/02462_int_to_date.reference delete mode 100644 tests/queries/0_stateless/02462_int_to_date.sql create mode 100644 tests/queries/0_stateless/02462_number_to_datetype.reference create mode 100644 tests/queries/0_stateless/02462_number_to_datetype.sql diff --git a/tests/queries/0_stateless/02462_int_to_date.reference b/tests/queries/0_stateless/02462_int_to_date.reference deleted file mode 100644 index f31441cf3b8..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.reference +++ /dev/null @@ -1,4 +0,0 @@ -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 -20221011 2022-10-11 1665519765 Int32 -20221011 2022-10-11 1665519765 UInt32 diff --git a/tests/queries/0_stateless/02462_int_to_date.sql b/tests/queries/0_stateless/02462_int_to_date.sql deleted file mode 100644 index cd470ca12f6..00000000000 --- a/tests/queries/0_stateless/02462_int_to_date.sql +++ /dev/null @@ -1,4 +0,0 @@ -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp; -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); -select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); diff --git a/tests/queries/0_stateless/02462_number_to_datetype.reference b/tests/queries/0_stateless/02462_number_to_datetype.reference new file mode 100644 index 00000000000..baa03dafdc9 --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.reference @@ -0,0 +1,49 @@ +-- { echoOn } +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt64 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Int32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 UInt32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519700 Float32 +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 1665519765 Float64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt64 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Int32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 UInt32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:24 1665519700 Float32 +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45 1665519765 Float64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt64 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Int32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 UInt32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:21:54.304 1665519700 Float32 +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +20221011 2022-10-11 22:22:45.000 1665519765 Float64 diff --git a/tests/queries/0_stateless/02462_number_to_datetype.sql b/tests/queries/0_stateless/02462_number_to_datetype.sql new file mode 100644 index 00000000000..d877157314c --- /dev/null +++ b/tests/queries/0_stateless/02462_number_to_datetype.sql @@ -0,0 +1,29 @@ +-- { echoOn } +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- { echoOff } From c3e63f77844d6656ecc5c7389019f9da817642fe Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 8 Nov 2022 12:04:12 +0000 Subject: [PATCH 234/526] Add section headings --- .../queries/0_stateless/02462_number_to_datetype.reference | 5 +++++ tests/queries/0_stateless/02462_number_to_datetype.sql | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/02462_number_to_datetype.reference b/tests/queries/0_stateless/02462_number_to_datetype.reference index baa03dafdc9..22cee1308ff 100644 --- a/tests/queries/0_stateless/02462_number_to_datetype.reference +++ b/tests/queries/0_stateless/02462_number_to_datetype.reference @@ -1,4 +1,6 @@ -- { echoOn } + +-- toDate select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Int64 select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -11,6 +13,7 @@ select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTim 20221011 2022-10-11 1665519700 Float32 select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Float64 +-- toDate32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Int64 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -23,6 +26,7 @@ select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recor 20221011 2022-10-11 1665519700 Float32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 1665519765 Float64 +-- toDateTime select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45 1665519765 Int64 select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -35,6 +39,7 @@ select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(r 20221011 2022-10-11 22:22:24 1665519700 Float32 select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45 1665519765 Float64 +-- toDateTime64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); 20221011 2022-10-11 22:22:45.000 1665519765 Int64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); diff --git a/tests/queries/0_stateless/02462_number_to_datetype.sql b/tests/queries/0_stateless/02462_number_to_datetype.sql index d877157314c..5d361f2b0ba 100644 --- a/tests/queries/0_stateless/02462_number_to_datetype.sql +++ b/tests/queries/0_stateless/02462_number_to_datetype.sql @@ -1,4 +1,6 @@ -- { echoOn } + +-- toDate select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -6,6 +8,7 @@ select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTim select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate(recordTimestamp, 'Europe/Amsterdam')), toDate(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDate32 select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -13,6 +16,7 @@ select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recor select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDate32(recordTimestamp, 'Europe/Amsterdam')), toDate32(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDateTime select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); @@ -20,10 +24,12 @@ select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(r select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime(recordTimestamp, 'Europe/Amsterdam')), toDateTime(recordTimestamp, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); +-- toDateTime64 select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toUInt32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat32(1665519765) as recordTimestamp, toTypeName(recordTimestamp); select toYYYYMMDD(toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam')), toDateTime64(recordTimestamp, 3, 'Europe/Amsterdam'), toFloat64(1665519765) as recordTimestamp, toTypeName(recordTimestamp); + -- { echoOff } From 3b3e27f22ac940031a56648e9a4c677b97e6762e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Nov 2022 14:25:23 +0100 Subject: [PATCH 235/526] Analyzer improve ARRAY JOIN with JOIN --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 126 +++++++++++++----- src/Analyzer/Utils.cpp | 37 ----- src/Analyzer/Utils.h | 5 - src/Planner/CollectTableExpressionData.cpp | 12 +- src/Planner/PlannerJoinTree.cpp | 19 +-- src/Planner/TableExpressionData.h | 27 +++- ...75_analyzer_array_join_with_join.reference | 28 ++++ .../02475_analyzer_array_join_with_join.sql | 71 ++++++++++ ...nalyzer_join_with_unused_columns.reference | 9 ++ ...2476_analyzer_join_with_unused_columns.sql | 19 +++ 10 files changed, 262 insertions(+), 91 deletions(-) create mode 100644 tests/queries/0_stateless/02475_analyzer_array_join_with_join.reference create mode 100644 tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql create mode 100644 tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.reference create mode 100644 tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9db2d66d99d..1b680c1fcdb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2330,30 +2330,43 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi const auto & from_array_join_node = table_expression_node->as(); auto resolved_identifier = tryResolveIdentifierFromJoinTreeNode(identifier_lookup, from_array_join_node.getTableExpression(), scope); - /** Special case when qualified or unqualified identifier point to array join expression without alias. - * - * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; - * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; - * - * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. - */ - if (!scope.table_expressions_in_resolve_process.contains(table_expression_node.get()) && resolved_identifier) + if (scope.table_expressions_in_resolve_process.contains(table_expression_node.get())) + return resolved_identifier; + + for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) { - for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) + /** Allow JOIN with USING with ARRAY JOIN. + * + * SELECT * FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id + * SELECT * FROM test_table_1 AS t1 ARRAY JOIN t1.id AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id + */ + if (identifier_lookup.identifier.isShort() && + array_join_expression->getAlias() == identifier_lookup.identifier.getFullName()) { - auto & array_join_column_expression = array_join_expression->as(); - if (array_join_column_expression.hasAlias()) - continue; + resolved_identifier = array_join_expression; + break; + } - auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow(); - if (array_join_column_inner_expression.get() == resolved_identifier.get() || - array_join_column_inner_expression->isEqual(*resolved_identifier)) - { - auto array_join_column = array_join_column_expression.getColumn(); - auto result = std::make_shared(array_join_column, table_expression_node); + if (!resolved_identifier) + continue; - return result; - } + /** Special case when qualified or unqualified identifier point to array join expression without alias. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; + * + * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. + */ + auto & array_join_column_expression = array_join_expression->as(); + if (array_join_column_expression.hasAlias()) + continue; + + auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow(); + if (array_join_column_inner_expression.get() == resolved_identifier.get() || + array_join_column_inner_expression->isEqual(*resolved_identifier)) + { + resolved_identifier = array_join_expression; + break; } } @@ -2868,6 +2881,9 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( auto table_expressions_stack = buildTableExpressionsStack(nearest_query_scope_query_node->getJoinTree()); std::vector table_expressions_column_nodes_with_names_stack; + std::unordered_set left_table_expression_column_names_to_skip; + std::unordered_set right_table_expression_column_names_to_skip; + for (auto & table_expression : table_expressions_stack) { QueryTreeNodesWithNames matched_expression_nodes_with_column_names; @@ -2893,8 +2909,14 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( auto left_table_expression_columns = std::move(table_expressions_column_nodes_with_names_stack.back()); table_expressions_column_nodes_with_names_stack.pop_back(); - std::unordered_set column_names_to_skip; + left_table_expression_column_names_to_skip.clear(); + right_table_expression_column_names_to_skip.clear(); + /** If there is JOIN with USING we need to match only single USING column and do not use left table expression + * and right table expression column with same name. + * + * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id); + */ if (!table_expression_in_resolve_process && join_node->isUsingJoinExpression()) { auto & join_using_list = join_node->getJoinExpression()->as(); @@ -2902,22 +2924,64 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( for (auto & join_using_node : join_using_list.getNodes()) { auto & column_node = join_using_node->as(); - const auto & column_name = column_node.getColumnName(); + const auto & using_column_name = column_node.getColumnName(); - if (!matcher_node_typed.isMatchingColumn(column_name)) + if (!matcher_node_typed.isMatchingColumn(using_column_name)) continue; - column_names_to_skip.insert(column_name); + const auto & join_using_column_nodes_list = column_node.getExpressionOrThrow()->as(); + const auto & join_using_column_nodes = join_using_column_nodes_list.getNodes(); - QueryTreeNodePtr column_source = getColumnSourceForJoinNodeWithUsing(table_expression); - auto matched_column_node = std::make_shared(column_node.getColumn(), column_source); - matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), column_name); + QueryTreeNodePtr matched_column_node; + + if (isRight(join_node->getKind())) + matched_column_node = join_using_column_nodes.at(1); + else + matched_column_node = join_using_column_nodes.at(0); + + /** It is possible that in USING there is JOIN with array joined column. + * SELECT * FROM (SELECT [0] AS value) AS t1 ARRAY JOIN value AS id INNER JOIN test_table USING (id); + * In such example match `value` column from t1, and all columns from test_table except `id`. + * + * SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN id INNER JOIN test_table USING (id); + * In such example, match `id` column from ARRAY JOIN, and all columns from test_table except `id`. + * + * SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); + * In such example match `id` column from ARRAY JOIN, and all columns from test_table except `id`. + * + * SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + * In such example match `id` column from t1, and all columns from test_table except `id`. + */ + auto & matched_column_node_typed = matched_column_node->as(); + auto matched_column_source = matched_column_node->as().getColumnSource(); + + if (matched_column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) + { + auto * array_join_inner_expression_column = matched_column_node_typed.getExpressionOrThrow()->as(); + + if (array_join_inner_expression_column && array_join_inner_expression_column->getColumnName() == using_column_name) + { + left_table_expression_column_names_to_skip.insert(using_column_name); + right_table_expression_column_names_to_skip.insert(using_column_name); + matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), using_column_name); + } + else + { + right_table_expression_column_names_to_skip.insert(using_column_name); + } + } + else + { + left_table_expression_column_names_to_skip.insert(using_column_name); + right_table_expression_column_names_to_skip.insert(using_column_name); + matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), using_column_name); + } } } for (auto && left_table_column : left_table_expression_columns) { - if (column_names_to_skip.contains(left_table_column.second)) + if (left_table_expression_column_names_to_skip.contains(left_table_column.second)) continue; matched_expression_nodes_with_column_names.push_back(std::move(left_table_column)); @@ -2925,7 +2989,7 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( for (auto && right_table_column : right_table_expression_columns) { - if (column_names_to_skip.contains(right_table_column.second)) + if (right_table_expression_column_names_to_skip.contains(right_table_column.second)) continue; matched_expression_nodes_with_column_names.push_back(std::move(right_table_column)); @@ -5056,8 +5120,10 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, if (!common_type) throw Exception(ErrorCodes::NO_COMMON_TYPE, - "JOIN {} cannot infer common type in USING for identifier '{}'. In scope {}", + "JOIN {} cannot infer common type for {} and {} in USING for identifier '{}'. In scope {}", join_node.formatASTForErrorMessage(), + result_left_table_expression->getResultType()->getName(), + result_right_table_expression->getResultType()->getName(), identifier_full_name, scope.scope_node->formatASTForErrorMessage()); diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index b504a5b5787..4e277fa8624 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -289,41 +289,4 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod return result; } -QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node) -{ - QueryTreeNodePtr column_source_node = join_node; - - while (true) - { - auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::TABLE || - column_source_node_type == QueryTreeNodeType::TABLE_FUNCTION || - column_source_node_type == QueryTreeNodeType::QUERY || - column_source_node_type == QueryTreeNodeType::UNION) - { - break; - } - else if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN) - { - auto & array_join_node = column_source_node->as(); - column_source_node = array_join_node.getTableExpression(); - continue; - } - else if (column_source_node_type == QueryTreeNodeType::JOIN) - { - auto & join_node_typed = column_source_node->as(); - column_source_node = isRight(join_node_typed.getKind()) ? join_node_typed.getRightTableExpression() : join_node_typed.getLeftTableExpression(); - continue; - } - else - { - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected node type for table expression. Expected table, table function, query, union, join or array join. Actual {}", - column_source_node->getNodeTypeName()); - } - } - - return column_source_node; -} - } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index 325a7d2fcc8..dbb2e7d5b59 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -31,9 +31,4 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); */ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); -/** Get column source for JOIN node with USING. - * Example: SELECT id FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 USING (id); - */ -QueryTreeNodePtr getColumnSourceForJoinNodeWithUsing(const QueryTreeNodePtr & join_node); - } diff --git a/src/Planner/CollectTableExpressionData.cpp b/src/Planner/CollectTableExpressionData.cpp index 30ccc541507..81ce3d325f7 100644 --- a/src/Planner/CollectTableExpressionData.cpp +++ b/src/Planner/CollectTableExpressionData.cpp @@ -39,17 +39,16 @@ public: auto column_source_node = column_node->getColumnSource(); auto column_source_node_type = column_source_node->getNodeType(); - if (column_source_node_type == QueryTreeNodeType::ARRAY_JOIN || - column_source_node_type == QueryTreeNodeType::LAMBDA) + if (column_source_node_type == QueryTreeNodeType::LAMBDA) return; /// JOIN using expression - if (column_node->hasExpression() && column_source_node->getNodeType() == QueryTreeNodeType::JOIN) + if (column_node->hasExpression() && column_source_node_type == QueryTreeNodeType::JOIN) return; auto & table_expression_data = planner_context.getOrCreateTableExpressionData(column_source_node); - if (column_node->hasExpression()) + if (column_node->hasExpression() && column_source_node_type != QueryTreeNodeType::ARRAY_JOIN) { /// Replace ALIAS column with expression table_expression_data.addAliasColumnName(column_node->getColumnName()); @@ -61,9 +60,10 @@ public: if (column_source_node_type != QueryTreeNodeType::TABLE && column_source_node_type != QueryTreeNodeType::TABLE_FUNCTION && column_source_node_type != QueryTreeNodeType::QUERY && - column_source_node_type != QueryTreeNodeType::UNION) + column_source_node_type != QueryTreeNodeType::UNION && + column_source_node_type != QueryTreeNodeType::ARRAY_JOIN) throw Exception(ErrorCodes::LOGICAL_ERROR, - "Expected table, table function, query or union column source. Actual {}", + "Expected table, table function, array join, query or union column source. Actual {}", column_source_node->formatASTForErrorMessage()); bool column_already_exists = table_expression_data.hasColumn(column_node->getColumnName()); diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4cb446a65a0..205c6c5e740 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -185,18 +185,19 @@ QueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expression, } auto rename_actions_dag = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + ActionsDAG::NodeRawConstPtrs updated_actions_dag_outputs; for (auto & output_node : rename_actions_dag->getOutputs()) { const auto * column_identifier = table_expression_data.getColumnIdentifierOrNull(output_node->result_name); - if (!column_identifier) continue; - const auto * node_to_rename = output_node; - output_node = &rename_actions_dag->addAlias(*node_to_rename, *column_identifier); + updated_actions_dag_outputs.push_back(&rename_actions_dag->addAlias(*output_node, *column_identifier)); } + rename_actions_dag->getOutputs() = std::move(updated_actions_dag_outputs); + auto rename_step = std::make_unique(query_plan.getCurrentDataStream(), rename_actions_dag); rename_step->setStepDescription("Change column names to column identifiers"); query_plan.addStep(std::move(rename_step)); @@ -639,17 +640,17 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, ActionsDAGPtr array_join_action_dag = std::make_shared(plan_output_columns); PlannerActionsVisitor actions_visitor(planner_context); - NameSet array_join_columns; + NameSet array_join_column_names; for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) { - auto & array_join_expression_column = array_join_expression->as(); - const auto & array_join_column_name = array_join_expression_column.getColumnName(); - array_join_columns.insert(array_join_column_name); + const auto & array_join_column_identifier = planner_context->getColumnNodeIdentifierOrThrow(array_join_expression); + array_join_column_names.insert(array_join_column_identifier); + auto & array_join_expression_column = array_join_expression->as(); auto expression_dag_index_nodes = actions_visitor.visit(array_join_action_dag, array_join_expression_column.getExpressionOrThrow()); for (auto & expression_dag_index_node : expression_dag_index_nodes) { - const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_name); + const auto * array_join_column_node = &array_join_action_dag->addAlias(*expression_dag_index_node, array_join_column_identifier); array_join_action_dag->getOutputs().push_back(array_join_column_node); } } @@ -659,7 +660,7 @@ QueryPlan buildQueryPlanForArrayJoinNode(QueryTreeNodePtr table_expression, array_join_actions->setStepDescription("ARRAY JOIN actions"); plan.addStep(std::move(array_join_actions)); - auto array_join_action = std::make_shared(array_join_columns, array_join_node.isLeft(), planner_context->getQueryContext()); + auto array_join_action = std::make_shared(array_join_column_names, array_join_node.isLeft(), planner_context->getQueryContext()); auto array_join_step = std::make_unique(plan.getCurrentDataStream(), std::move(array_join_action)); array_join_step->setStepDescription("ARRAY JOIN"); plan.addStep(std::move(array_join_step)); diff --git a/src/Planner/TableExpressionData.h b/src/Planner/TableExpressionData.h index 0918c35a8ef..57eaa28e072 100644 --- a/src/Planner/TableExpressionData.h +++ b/src/Planner/TableExpressionData.h @@ -16,6 +16,25 @@ using ColumnIdentifier = std::string; /** Table expression data is created for each table expression that take part in query. * Table expression data has information about columns that participate in query, their name to identifier mapping, * and additional table expression properties. + * + * Table expression can be table, table function, query, union, array join node. + * + * Examples: + * SELECT * FROM (SELECT 1); + * (SELECT 1) - table expression. + * + * SELECT * FROM test_table; + * test_table - table expression. + * + * SELECT * FROM view(SELECT 1); + * view(SELECT 1) - table expression. + * + * SELECT * FROM (SELECT 1) JOIN (SELECT 2); + * (SELECT 1) - table expression. + * (SELECT 2) - table expression. + * + * SELECT array, a FROM (SELECT [1] AS array) ARRAY JOIN array AS a; + * ARRAY JOIN array AS a - table expression. */ class TableExpressionData { @@ -164,19 +183,19 @@ public: } private: - /// Valid for table, table function, query, union table expression nodes + /// Valid for table, table function, query, union, array join table expression nodes NamesAndTypesList columns; - /// Valid for table, table function, query, union table expression nodes + /// Valid for table, table function, query, union, array join table expression nodes NameSet columns_names; /// Valid only for table table expression node NameSet alias_columns_names; - /// Valid for table, table function, query, union table expression nodes + /// Valid for table, table function, query, union table, array join expression nodes ColumnNameToColumnIdentifier column_name_to_column_identifier; - /// Valid for table, table function, query, union table expression nodes + /// Valid for table, table function, query, union table, array join expression nodes ColumnIdentifierToColumnName column_identifier_to_column_name; /// Is storage remote diff --git a/tests/queries/0_stateless/02475_analyzer_array_join_with_join.reference b/tests/queries/0_stateless/02475_analyzer_array_join_with_join.reference new file mode 100644 index 00000000000..d679934d4e6 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_array_join_with_join.reference @@ -0,0 +1,28 @@ +0 +0 +-- +0 +-- +-- +-- +1 +-- +0 Value_0 +-- +1 Value_1 +-- +0 Value_0 +-- +1 Value_1 +-- +[0] Value_0 +-- +[1] Value_1 +-- +[0] Value_0 +-- +[0] Value_1 +-- +[5] Value_0 +-- +[5] Value_1 diff --git a/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql b/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql new file mode 100644 index 00000000000..80ae3dfc072 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql @@ -0,0 +1,71 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (0, 'Value_0'); +INSERT INTO test_table VALUES (1, 'Value_1'); + +SELECT * FROM (SELECT [dummy, dummy] AS dummy FROM system.one) AS subquery ARRAY JOIN dummy INNER JOIN system.one USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.reference b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.reference new file mode 100644 index 00000000000..c6240227a2e --- /dev/null +++ b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.reference @@ -0,0 +1,9 @@ +1 3 +-- +2 4 +-- +1 3 +-- +2 4 +-- +1 2 3 4 diff --git a/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql new file mode 100644 index 00000000000..ca937e01238 --- /dev/null +++ b/tests/queries/0_stateless/02476_analyzer_join_with_unused_columns.sql @@ -0,0 +1,19 @@ +SET allow_experimental_analyzer = 1; + +SELECT subquery_1.id, subquery_2.id FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; + +SELECT '--'; + +SELECT subquery_1.value, subquery_2.value FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; + +SELECT '--'; + +SELECT COLUMNS('id') FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; + +SELECT '--'; + +SELECT COLUMNS('value') FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id, 2 AS value) AS subquery_1, (SELECT 3 AS id, 4 AS value) AS subquery_2; From 583c65d37741abb209e5cd6c4850a66994e8d555 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 1 Nov 2022 15:14:07 +0100 Subject: [PATCH 236/526] Analyzer subquery in JOIN TREE with aggregation --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 12 +++++++++--- .../02475_analyzer_join_tree_subquery.reference | 3 +++ .../02475_analyzer_join_tree_subquery.sql | 7 +++++++ 3 files changed, 19 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference create mode 100644 tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 9db2d66d99d..4f24051df79 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -5527,9 +5527,15 @@ void QueryAnalyzer::resolveQuery(const QueryTreeNodePtr & query_node, Identifier * 3. Check that there are no columns that are not specified in GROUP BY keys. * 4. Validate GROUP BY modifiers. */ - assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); - assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + auto join_tree_node_type = query_node_typed.getJoinTree()->getNodeType(); + bool join_tree_is_subquery = join_tree_node_type == QueryTreeNodeType::QUERY || join_tree_node_type == QueryTreeNodeType::UNION; + + if (!join_tree_is_subquery) + { + assertNoAggregateFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoGroupingFunction(query_node_typed.getJoinTree(), "in JOIN TREE"); + assertNoWindowFunctionNodes(query_node_typed.getJoinTree(), "in JOIN TREE"); + } if (query_node_typed.hasWhere()) { diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference new file mode 100644 index 00000000000..623ba8fde99 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.reference @@ -0,0 +1,3 @@ +45 +-- +45 diff --git a/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql new file mode 100644 index 00000000000..eda90529166 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_join_tree_subquery.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer = 1; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT * FROM subquery; + +SELECT '--'; + +WITH subquery AS (SELECT sum(number) FROM numbers(10)) SELECT (SELECT * FROM subquery); From 0aa4fd3c2db9edf7bc69ff8d8ed30e9720d23ad1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Nov 2022 12:53:57 +0100 Subject: [PATCH 237/526] Fixed tests --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 674e02b16ec..4091d37ec11 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -781,6 +781,11 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( ReadFromMergeTree::IndexStats & index_stats) { const Settings & settings = context->getSettingsRef(); + + /// TODO: Analyzer syntax analyzer result + if (settings.allow_experimental_analyzer) + return; + std::optional partition_pruner; std::optional minmax_idx_condition; DataTypes minmax_columns_types; From 6d5d9ff421cbe7497c6c94f032eb2f9807fe2799 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 8 Nov 2022 13:48:23 +0000 Subject: [PATCH 238/526] rename ReadWriteSettings -> RequestSettings --- src/Backups/BackupIO_S3.cpp | 14 ++--- src/Backups/BackupIO_S3.h | 4 +- src/Coordination/KeeperSnapshotManagerS3.cpp | 12 ++--- .../ObjectStorages/S3/S3ObjectStorage.cpp | 12 ++--- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 6 +-- src/Disks/ObjectStorages/S3/diskSettings.cpp | 26 ++++----- src/IO/ReadBufferFromS3.cpp | 10 ++-- src/IO/ReadBufferFromS3.h | 10 ++-- src/IO/S3/tests/gtest_aws_s3_client.cpp | 6 +-- src/IO/WriteBufferFromS3.cpp | 18 +++---- src/IO/WriteBufferFromS3.h | 4 +- .../ExternalDataSourceConfiguration.h | 2 +- src/Storages/StorageS3.cpp | 54 +++++++++---------- src/Storages/StorageS3.h | 12 ++--- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageS3Settings.cpp | 26 ++++----- src/Storages/StorageS3Settings.h | 12 ++--- 17 files changed, 115 insertions(+), 115 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 3f723c98a2a..f794500980b 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -87,7 +87,7 @@ BackupReaderS3::BackupReaderS3( : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , read_settings(context_->getReadSettings()) - , rw_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).rw_settings) + , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) { } @@ -115,7 +115,7 @@ UInt64 BackupReaderS3::getFileSize(const String & file_name) std::unique_ptr BackupReaderS3::readFile(const String & file_name) { return std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, rw_settings, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); } @@ -124,9 +124,9 @@ BackupWriterS3::BackupWriterS3( : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) , read_settings(context_->getReadSettings()) - , rw_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).rw_settings) + , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) { - rw_settings.updateFromSettingsIfEmpty(context_->getSettingsRef()); + request_settings.updateFromSettingsIfEmpty(context_->getSettingsRef()); } DataSourceDescription BackupWriterS3::getDataSourceDescription() const @@ -212,7 +212,7 @@ void BackupWriterS3::copyObjectMultipartImpl( std::vector part_tags; - size_t upload_part_size = rw_settings.min_upload_part_size; + size_t upload_part_size = request_settings.min_upload_part_size; for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size) { Aws::S3::Model::UploadPartCopyRequest part_request; @@ -317,7 +317,7 @@ bool BackupWriterS3::fileContentsEqual(const String & file_name, const String & try { auto in = std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, rw_settings, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); String actual_file_contents(expected_file_contents.size(), ' '); return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) && (actual_file_contents == expected_file_contents) && in->eof(); @@ -335,7 +335,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, - rw_settings, + request_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index 17a365da396..99d7558ac9e 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -40,7 +40,7 @@ private: S3::URI s3_uri; std::shared_ptr client; ReadSettings read_settings; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; }; @@ -83,7 +83,7 @@ private: S3::URI s3_uri; std::shared_ptr client; ReadSettings read_settings; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; }; } diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index 8777ef341ff..cc837f5f496 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -135,8 +135,8 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa if (s3_client == nullptr) return; - S3Settings::ReadWriteSettings read_write_settings; - read_write_settings.upload_part_size_multiply_parts_count_threshold = 10000; + S3Settings::RequestSettings request_settings_1; + request_settings_1.upload_part_size_multiply_parts_count_threshold = 10000; const auto create_writer = [&](const auto & key) { @@ -145,7 +145,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa s3_client->client, s3_client->uri.bucket, key, - read_write_settings + request_settings_1 }; }; @@ -194,15 +194,15 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa lock_writer.finalize(); // We read back the written UUID, if it's the same we can upload the file - S3Settings::ReadWriteSettings rw_settings; - rw_settings.max_single_read_retries = 1; + S3Settings::RequestSettings request_settings_2; + request_settings_2.max_single_read_retries = 1; ReadBufferFromS3 lock_reader { s3_client->client, s3_client->uri.bucket, lock_file, "", - rw_settings, + request_settings_2, {} }; diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 21b7ff4f0f1..bbb5c4cddc5 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -175,7 +175,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT bucket, path, version_id, - settings_ptr->s3_settings, + settings_ptr->request_settings, disk_read_settings, /* use_external_buffer */true, /* offset */0, @@ -212,7 +212,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT bucket, object.absolute_path, version_id, - settings_ptr->s3_settings, + settings_ptr->request_settings, patchSettings(read_settings)); } @@ -238,7 +238,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN client.get(), bucket, object.absolute_path, - settings_ptr->s3_settings, + settings_ptr->request_settings, attributes, buf_size, std::move(scheduler), @@ -473,7 +473,7 @@ void S3ObjectStorage::copyObjectImpl( throwIfError(outcome); auto settings_ptr = s3_settings.get(); - if (settings_ptr->s3_settings.check_objects_after_upload) + if (settings_ptr->request_settings.check_objects_after_upload) { auto object_head = requestObjectHeadData(dst_bucket, dst_key); if (!object_head.IsSuccess()) @@ -517,7 +517,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( std::vector part_tags; - size_t upload_part_size = settings_ptr->s3_settings.min_upload_part_size; + size_t upload_part_size = settings_ptr->request_settings.min_upload_part_size; for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size) { ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); @@ -570,7 +570,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( throwIfError(outcome); } - if (settings_ptr->s3_settings.check_objects_after_upload) + if (settings_ptr->request_settings.check_objects_after_upload) { auto object_head = requestObjectHeadData(dst_bucket, dst_key); if (!object_head.IsSuccess()) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 6b1e8289b15..a737d3bc114 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -23,17 +23,17 @@ struct S3ObjectStorageSettings S3ObjectStorageSettings() = default; S3ObjectStorageSettings( - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, int32_t objects_chunk_size_to_delete_) - : s3_settings(s3_settings_) + : request_settings(request_settings_) , min_bytes_for_seek(min_bytes_for_seek_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {} - S3Settings::ReadWriteSettings s3_settings; + S3Settings::RequestSettings request_settings; uint64_t min_bytes_for_seek; int32_t list_object_keys_size; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index 707675c01fd..ca2e9d04926 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -34,25 +34,25 @@ namespace ErrorCodes std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { const Settings & settings = context->getSettingsRef(); - S3Settings::ReadWriteSettings rw_settings; - rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries); - rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size); - rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor); - rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold); - rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size); - rw_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload); - rw_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries); + S3Settings::RequestSettings request_settings; + request_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries); + request_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size); + request_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor); + request_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold); + request_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size); + request_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload); + request_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries); // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. if (UInt64 max_get_rps = config.getUInt64(config_prefix + ".s3_max_get_rps", settings.s3_max_get_rps)) - rw_settings.get_request_throttler = std::make_shared( + request_settings.get_request_throttler = std::make_shared( max_get_rps, config.getUInt64(config_prefix + ".s3_max_get_burst", settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); if (UInt64 max_put_rps = config.getUInt64(config_prefix + ".s3_max_put_rps", settings.s3_max_put_rps)) - rw_settings.put_request_throttler = std::make_shared( + request_settings.put_request_throttler = std::make_shared( max_put_rps, config.getUInt64(config_prefix + ".s3_max_put_burst", settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); return std::make_unique( - rw_settings, + request_settings, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); @@ -134,8 +134,8 @@ std::unique_ptr getClient( static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ true, - settings.s3_settings.get_request_throttler, - settings.s3_settings.put_request_throttler); + settings.request_settings.get_request_throttler, + settings.request_settings.put_request_throttler); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index fa748469ca0..c14fbecf223 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -45,7 +45,7 @@ ReadBufferFromS3::ReadBufferFromS3( const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, const ReadSettings & settings_, bool use_external_buffer_, size_t offset_, @@ -56,7 +56,7 @@ ReadBufferFromS3::ReadBufferFromS3( , bucket(bucket_) , key(key_) , version_id(version_id_) - , s3_settings(s3_settings_) + , request_settings(request_settings_) , offset(offset_) , read_until_position(read_until_position_) , read_settings(settings_) @@ -105,7 +105,7 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; attempt < s3_settings.max_single_read_retries && !next_result; ++attempt) + for (size_t attempt = 0; attempt < request_settings.max_single_read_retries && !next_result; ++attempt) { Stopwatch watch; try @@ -166,7 +166,7 @@ bool ReadBufferFromS3::nextImpl() attempt, e.message()); - if (attempt + 1 == s3_settings.max_single_read_retries) + if (attempt + 1 == request_settings.max_single_read_retries) throw; /// Pause before next attempt. @@ -349,7 +349,7 @@ SeekableReadBufferPtr ReadBufferS3Factory::getReader() bucket, key, version_id, - s3_settings, + request_settings, read_settings, false /*use_external_buffer*/, next_range->first, diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 3e1a26f7713..17b13bf7d62 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -34,7 +34,7 @@ private: String bucket; String key; String version_id; - const S3Settings::ReadWriteSettings s3_settings; + const S3Settings::RequestSettings request_settings; /// These variables are atomic because they can be used for `logging only` /// (where it is not important to get consistent result) @@ -53,7 +53,7 @@ public: const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, const ReadSettings & settings_, bool use_external_buffer = false, size_t offset_ = 0, @@ -101,7 +101,7 @@ public: const String & version_id_, size_t range_step_, size_t object_size_, - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, const ReadSettings & read_settings_) : client_ptr(client_ptr_) , bucket(bucket_) @@ -111,7 +111,7 @@ public: , range_generator(object_size_, range_step_) , range_step(range_step_) , object_size(object_size_) - , s3_settings(s3_settings_) + , request_settings(request_settings_) { assert(range_step > 0); assert(range_step < object_size); @@ -136,7 +136,7 @@ private: size_t range_step; size_t object_size; - const S3Settings::ReadWriteSettings s3_settings; + const S3Settings::RequestSettings request_settings; }; } diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 96a7c8b3dfb..2d298ca5df2 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -115,14 +115,14 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) ASSERT_TRUE(client); DB::ReadSettings read_settings; - DB::S3Settings::ReadWriteSettings rw_settings; - rw_settings.max_single_read_retries = max_single_read_retries; + DB::S3Settings::RequestSettings request_settings; + request_settings.max_single_read_retries = max_single_read_retries; DB::ReadBufferFromS3 read_buffer( client, uri.bucket, uri.key, version_id, - rw_settings, + request_settings, read_settings ); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index f823015bd7d..d3a5b185248 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -71,7 +71,7 @@ WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_, size_t buffer_size_, ThreadPoolCallbackRunner schedule_, @@ -79,10 +79,10 @@ WriteBufferFromS3::WriteBufferFromS3( : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) , key(key_) - , s3_settings(s3_settings_) + , request_settings(request_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , upload_part_size(s3_settings_.min_upload_part_size) + , upload_part_size(request_settings_.min_upload_part_size) , schedule(std::move(schedule_)) , write_settings(write_settings_) { @@ -107,7 +107,7 @@ void WriteBufferFromS3::nextImpl() write_settings.remote_throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. - if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) + if (multipart_upload_id.empty() && last_part_size > request_settings.max_single_part_upload_size) createMultipartUpload(); if (!multipart_upload_id.empty() && last_part_size > upload_part_size) @@ -122,8 +122,8 @@ void WriteBufferFromS3::nextImpl() void WriteBufferFromS3::allocateBuffer() { - if (total_parts_uploaded != 0 && total_parts_uploaded % s3_settings.upload_part_size_multiply_parts_count_threshold == 0) - upload_part_size *= s3_settings.upload_part_size_multiply_factor; + if (total_parts_uploaded != 0 && total_parts_uploaded % request_settings.upload_part_size_multiply_parts_count_threshold == 0) + upload_part_size *= request_settings.upload_part_size_multiply_factor; temporary_buffer = Aws::MakeShared("temporary buffer"); temporary_buffer->exceptions(std::ios::badbit); @@ -177,7 +177,7 @@ void WriteBufferFromS3::finalizeImpl() if (!multipart_upload_id.empty()) completeMultipartUpload(); - if (s3_settings.check_objects_after_upload) + if (request_settings.check_objects_after_upload) { LOG_TRACE(log, "Checking object {} exists after upload", key); @@ -367,7 +367,7 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(s3_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); @@ -473,7 +473,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) { - size_t max_retry = std::max(s3_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index 28f831856d7..b4b5a6d37a3 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -50,7 +50,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::ReadWriteSettings & s3_settings_, + const S3Settings::RequestSettings & request_settings_, std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, ThreadPoolCallbackRunner schedule_ = {}, @@ -88,7 +88,7 @@ private: const String bucket; const String key; - const S3Settings::ReadWriteSettings s3_settings; + const S3Settings::RequestSettings request_settings; const std::shared_ptr client_ptr; const std::optional> object_metadata; diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index 5736336983a..a8c27e3d1d4 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -118,7 +118,7 @@ struct URLBasedDataSourceConfiguration struct StorageS3Configuration : URLBasedDataSourceConfiguration { S3::AuthSettings auth_settings; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 9309e3f7384..cc13838a4c7 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -101,7 +101,7 @@ public: ContextPtr context_, std::unordered_map * object_infos_, Strings * read_keys_, - const S3Settings::ReadWriteSettings & rw_settings_) + const S3Settings::RequestSettings & request_settings_) : WithContext(context_) , client(client_) , globbed_uri(globbed_uri_) @@ -109,7 +109,7 @@ public: , virtual_header(virtual_header_) , object_infos(object_infos_) , read_keys(read_keys_) - , rw_settings(rw_settings_) + , request_settings(request_settings_) { if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -260,7 +260,7 @@ private: bool is_finished{false}; std::unordered_map * object_infos; Strings * read_keys; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( @@ -271,8 +271,8 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( ContextPtr context, std::unordered_map * object_infos_, Strings * read_keys_, - const S3Settings::ReadWriteSettings & rw_settings_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_, rw_settings_)) + const S3Settings::RequestSettings & request_settings_) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_, request_settings_)) { } @@ -385,7 +385,7 @@ StorageS3Source::StorageS3Source( std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - const S3Settings::ReadWriteSettings & rw_settings_, + const S3Settings::RequestSettings & request_settings_, String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -401,7 +401,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) - , rw_settings(rw_settings_) + , request_settings(request_settings_) , compression_hint(std::move(compression_hint_)) , client(client_) , sample_block(sample_block_) @@ -467,7 +467,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k if (!use_parallel_download || object_too_small) { LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); - return std::make_unique(client, bucket, key, version_id, rw_settings, getContext()->getReadSettings()); + return std::make_unique(client, bucket, key, version_id, request_settings, getContext()->getReadSettings()); } assert(object_size > 0); @@ -479,7 +479,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k } auto factory = std::make_unique( - client, bucket, key, version_id, download_buffer_size, object_size, rw_settings, getContext()->getReadSettings()); + client, bucket, key, version_id, download_buffer_size, object_size, request_settings, getContext()->getReadSettings()); LOG_TRACE( log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size); @@ -589,7 +589,7 @@ public: s3_configuration_.client, bucket, key, - s3_configuration_.rw_settings, + s3_configuration_.request_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), @@ -753,7 +753,7 @@ StorageS3::StorageS3( bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) - , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} , keys({s3_configuration.uri.key}) , format_name(configuration_.format) , compression_method(configuration_.compression_method) @@ -819,7 +819,7 @@ std::shared_ptr StorageS3::createFileIterator( { /// Iterate through disclosed globs and make a source for each file auto glob_iterator = std::make_shared( - *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys, s3_configuration.rw_settings); + *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys, s3_configuration.request_settings); return std::make_shared([glob_iterator]() { return glob_iterator->next(); }); } else @@ -909,7 +909,7 @@ Pipe StorageS3::read( format_settings, columns_description, max_block_size, - s3_configuration.rw_settings, + s3_configuration.request_settings, compression_method, s3_configuration.client, s3_configuration.uri.bucket, @@ -1026,10 +1026,10 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - if (upd.rw_settings != settings.rw_settings) - upd.rw_settings = settings.rw_settings; + if (upd.request_settings != settings.request_settings) + upd.request_settings = settings.request_settings; - upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + upd.request_settings.updateFromSettingsIfEmpty(ctx->getSettings()); if (upd.client) { @@ -1048,11 +1048,11 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration static_cast(ctx->getGlobalContext()->getSettingsRef().s3_max_redirects), ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, /* for_disk_s3 = */ false, - upd.rw_settings.get_request_throttler, - upd.rw_settings.put_request_throttler); + upd.request_settings.get_request_throttler, + upd.request_settings.put_request_throttler); client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = static_cast(upd.rw_settings.max_connections); + client_configuration.maxConnections = static_cast(upd.request_settings.max_connections); auto credentials = Aws::Auth::AWSCredentials(upd.auth_settings.access_key_id, upd.auth_settings.secret_access_key); auto headers = upd.auth_settings.headers; @@ -1084,17 +1084,17 @@ void StorageS3::processNamedCollectionResult(StorageS3Configuration & configurat else if (arg_name == "use_environment_credentials") configuration.auth_settings.use_environment_credentials = checkAndGetLiteralArgument(arg_value, "use_environment_credentials"); else if (arg_name == "max_single_read_retries") - configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); + configuration.request_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); else if (arg_name == "min_upload_part_size") - configuration.rw_settings.min_upload_part_size = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); + configuration.request_settings.min_upload_part_size = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); else if (arg_name == "upload_part_size_multiply_factor") - configuration.rw_settings.upload_part_size_multiply_factor = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); + configuration.request_settings.upload_part_size_multiply_factor = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); else if (arg_name == "upload_part_size_multiply_parts_count_threshold") - configuration.rw_settings.upload_part_size_multiply_parts_count_threshold = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); + configuration.request_settings.upload_part_size_multiply_parts_count_threshold = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); else if (arg_name == "max_single_part_upload_size") - configuration.rw_settings.max_single_part_upload_size = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); + configuration.request_settings.max_single_part_upload_size = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); else if (arg_name == "max_connections") - configuration.rw_settings.max_connections = checkAndGetLiteralArgument(arg_value, "max_connections"); + configuration.request_settings.max_connections = checkAndGetLiteralArgument(arg_value, "max_connections"); else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].", @@ -1162,7 +1162,7 @@ ColumnsDescription StorageS3::getTableStructureFromData( S3Configuration s3_configuration{ configuration.url, configuration.auth_settings, - S3Settings::ReadWriteSettings(ctx->getSettingsRef()), + S3Settings::RequestSettings(ctx->getSettingsRef()), configuration.headers}; updateS3Configuration(ctx, s3_configuration); @@ -1224,7 +1224,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod( std::make_unique( - s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.rw_settings, ctx->getReadSettings()), + s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.request_settings, ctx->getReadSettings()), chooseCompressionMethod(key, compression_method), zstd_window_log_max); }; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 76391cb2695..8c15ede2b0b 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -44,7 +44,7 @@ public: ContextPtr context, std::unordered_map * object_infos = nullptr, Strings * read_keys_ = nullptr, - const S3Settings::ReadWriteSettings & rw_settings_ = {}); + const S3Settings::RequestSettings & request_settings_ = {}); String next(); @@ -80,7 +80,7 @@ public: std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - const S3Settings::ReadWriteSettings & rw_settings_, + const S3Settings::RequestSettings & request_settings_, String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -103,7 +103,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; String compression_hint; std::shared_ptr client; Block sample_block; @@ -187,7 +187,7 @@ public: std::shared_ptr client; S3::AuthSettings auth_settings; - S3Settings::ReadWriteSettings rw_settings; + S3Settings::RequestSettings request_settings; /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. @@ -199,11 +199,11 @@ public: S3Configuration( const String & url_, const S3::AuthSettings & auth_settings_, - const S3Settings::ReadWriteSettings & rw_settings_, + const S3Settings::RequestSettings & request_settings_, const HeaderCollection & headers_from_ast_) : uri(S3::URI(url_)) , auth_settings(auth_settings_) - , rw_settings(rw_settings_) + , request_settings(request_settings_) , static_configuration(!auth_settings_.access_key_id.empty()) , headers_from_ast(headers_from_ast_) {} }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 3b8c8b1cb92..920116527ee 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -46,7 +46,7 @@ StorageS3Cluster::StorageS3Cluster( const ConstraintsDescription & constraints_, ContextPtr context_) : IStorage(table_id_) - , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} , filename(configuration_.url) , cluster_name(configuration_.cluster_name) , format_name(configuration_.format) diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 9e670c65e17..e24b892dbac 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -48,24 +48,24 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U auto auth_settings = S3::AuthSettings::loadFromConfig(config_elem + "." + key, config); - S3Settings::ReadWriteSettings rw_settings; - rw_settings.max_single_read_retries = get_uint_for_key(key, "max_single_read_retries", true, settings.s3_max_single_read_retries); - rw_settings.min_upload_part_size = get_uint_for_key(key, "min_upload_part_size", true, settings.s3_min_upload_part_size); - rw_settings.upload_part_size_multiply_factor = get_uint_for_key(key, "upload_part_size_multiply_factor", true, settings.s3_upload_part_size_multiply_factor); - rw_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); - rw_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); - rw_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); - rw_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); + S3Settings::RequestSettings request_settings; + request_settings.max_single_read_retries = get_uint_for_key(key, "max_single_read_retries", true, settings.s3_max_single_read_retries); + request_settings.min_upload_part_size = get_uint_for_key(key, "min_upload_part_size", true, settings.s3_min_upload_part_size); + request_settings.upload_part_size_multiply_factor = get_uint_for_key(key, "upload_part_size_multiply_factor", true, settings.s3_upload_part_size_multiply_factor); + request_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); + request_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); + request_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); + request_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. if (UInt64 max_get_rps = get_uint_for_key(key, "max_get_rps", true, settings.s3_max_get_rps)) - rw_settings.get_request_throttler = std::make_shared( + request_settings.get_request_throttler = std::make_shared( max_get_rps, get_uint_for_key(key, "max_get_burst", true, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); if (UInt64 max_put_rps = get_uint_for_key(key, "max_put_rps", true, settings.s3_max_put_rps)) - rw_settings.put_request_throttler = std::make_shared( + request_settings.put_request_throttler = std::make_shared( max_put_rps, get_uint_for_key(key, "max_put_burst", true, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); - s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(rw_settings)}); + s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); } } } @@ -86,7 +86,7 @@ S3Settings StorageS3Settings::getSettings(const String & endpoint) const return {}; } -S3Settings::ReadWriteSettings::ReadWriteSettings(const Settings & settings) +S3Settings::RequestSettings::RequestSettings(const Settings & settings) { max_single_read_retries = settings.s3_max_single_read_retries; min_upload_part_size = settings.s3_min_upload_part_size; @@ -104,7 +104,7 @@ S3Settings::ReadWriteSettings::ReadWriteSettings(const Settings & settings) settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); } -void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & settings) +void S3Settings::RequestSettings::updateFromSettingsIfEmpty(const Settings & settings) { if (!max_single_read_retries) max_single_read_retries = settings.s3_max_single_read_retries; diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 40ba11a19ba..509a690ef8c 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -24,7 +24,7 @@ struct Settings; struct S3Settings { - struct ReadWriteSettings + struct RequestSettings { size_t max_single_read_retries = 0; size_t min_upload_part_size = 0; @@ -37,10 +37,10 @@ struct S3Settings ThrottlerPtr get_request_throttler; ThrottlerPtr put_request_throttler; - ReadWriteSettings() = default; - explicit ReadWriteSettings(const Settings & settings); + RequestSettings() = default; + explicit RequestSettings(const Settings & settings); - inline bool operator==(const ReadWriteSettings & other) const + inline bool operator==(const RequestSettings & other) const { return max_single_read_retries == other.max_single_read_retries && min_upload_part_size == other.min_upload_part_size @@ -58,11 +58,11 @@ struct S3Settings }; S3::AuthSettings auth_settings; - ReadWriteSettings rw_settings; + RequestSettings request_settings; inline bool operator==(const S3Settings & other) const { - return auth_settings == other.auth_settings && rw_settings == other.rw_settings; + return auth_settings == other.auth_settings && request_settings == other.request_settings; } }; From eb19df0094bc6bda65d5b39c222fb8c501d02f75 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Nov 2022 15:21:25 +0100 Subject: [PATCH 239/526] fix --- src/Storages/MergeTree/EphemeralLockInZooKeeper.h | 1 + src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 1 - src/Storages/MergeTree/ReplicatedMergeTreeSink.h | 1 - src/Storages/StorageReplicatedMergeTree.cpp | 4 ++-- 4 files changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index aaa1fc2af24..5945fa10d91 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -35,6 +35,7 @@ public: /// Fake "secondary node" names for blocks with and without "deduplication_path" static constexpr const char * LEGACY_LOCK_INSERT = "abandonable_lock-insert"; static constexpr const char * LEGACY_LOCK_OTHER = "abandonable_lock-other"; + static constexpr const char * LEGACY_LOCK_PREFIX = "/temp/abandonable_lock-"; EphemeralLockInZooKeeper(EphemeralLockInZooKeeper && rhs) noexcept { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 53654285699..5482052e184 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -146,7 +146,6 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z if (is_active.error == Coordination::Error::ZNONODE || host.error == Coordination::Error::ZNONODE) throw Exception("Replica is not active right now", ErrorCodes::READONLY); - quorum_info.is_active_node_value = is_active.data; quorum_info.is_active_node_version = is_active.stat.version; quorum_info.host_node_version = host.stat.version; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index b094b373f97..e3ec5d14a52 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -63,7 +63,6 @@ private: struct QuorumInfo { String status_path; - String is_active_node_value; int is_active_node_version = -1; int host_node_version = -1; }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index c547ad29f3d..3b85581a157 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -6426,7 +6426,7 @@ void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( paths_to_get.push_back(partition_path / block); } - auto results = zookeeper.get(paths_to_get); + auto results = zookeeper.tryGet(paths_to_get); for (size_t i = 0; i < paths_to_get.size(); ++i) { auto & result = results[i]; @@ -6454,7 +6454,7 @@ void StorageReplicatedMergeTree::clearLockedBlockNumbersInPartition( "probably it is created by a replica that running newer version of ClickHouse. " "Cannot remove it, will wait for this lock to disappear. Upgrade remaining hosts in the cluster to address this warning."; - if (result.data.starts_with(zookeeper_path + "/temp/abandonable_lock-")) + if (result.data.starts_with(zookeeper_path + EphemeralLockInZooKeeper::LEGACY_LOCK_PREFIX)) LOG_WARNING(log, old_version_warning, paths_to_get[i], result.data); else LOG_WARNING(log, new_version_warning, paths_to_get[i], result.data); From 2de26daa5630e38076f96ba600cb1594edab32d7 Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 8 Nov 2022 14:31:29 +0000 Subject: [PATCH 240/526] fix build --- src/Backups/BackupIO_S3.cpp | 8 ++++---- src/Storages/StorageDelta.cpp | 6 ++++-- src/Storages/StorageHudi.cpp | 2 +- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index e953b441c1b..5a4e804a778 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -247,10 +247,10 @@ void BackupWriterS3::copyObjectMultipartImpl( position = next_position; - if (part_number % rw_settings.upload_part_size_multiply_parts_count_threshold == 0) + if (part_number % request_settings.upload_part_size_multiply_parts_count_threshold == 0) { - upload_part_size *= rw_settings.upload_part_size_multiply_factor; - upload_part_size = std::min(upload_part_size, rw_settings.max_upload_part_size); + upload_part_size *= request_settings.upload_part_size_multiply_factor; + upload_part_size = std::min(upload_part_size, request_settings.max_upload_part_size); } } @@ -293,7 +293,7 @@ void BackupWriterS3::copyFileNative(DiskPtr from_disk, const String & file_name_ auto file_path = fs::path(s3_uri.key) / file_name_to; auto head = requestObjectHeadData(source_bucket, objects[0].absolute_path).GetResult(); - if (static_cast(head.GetContentLength()) < rw_settings.max_single_operation_copy_size) + if (static_cast(head.GetContentLength()) < request_settings.max_single_operation_copy_size) { copyObjectImpl( source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head); diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index e8287a2fd61..c077b24c610 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -151,12 +151,14 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { /// TODO: add parallel downloads + S3Settings::RequestSettings request_settings; + request_settings.max_single_read_retries = 10; return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, - /* max single read retries */10, + request_settings, context->getReadSettings()); } @@ -187,7 +189,7 @@ StorageDelta::StorageDelta( ContextPtr context_, std::optional format_settings_) : IStorage(table_id_) - , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) { diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 121856c4a57..4b20e4cbd22 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -37,7 +37,7 @@ StorageHudi::StorageHudi( ContextPtr context_, std::optional format_settings_) : IStorage(table_id_) - , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) { From 6f059be748d21d2201f2413fe2927ba1c6049a01 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 8 Nov 2022 15:33:23 +0100 Subject: [PATCH 241/526] Fix bug with temporary projection parts deletion --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 368af55aa15..42f15abd113 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1629,6 +1629,12 @@ void IMergeTreeDataPart::remove() auto can_remove_callback = [this] () { + if (isProjectionPart() && is_temp) + { + LOG_TRACE(storage.log, "Temporary projection part {} can be removed", name); + return CanRemoveDescription{.can_remove_anything = true, .files_not_to_remove = {} }; + } + auto [can_remove, files_not_to_remove] = canRemovePart(); if (!can_remove) LOG_TRACE(storage.log, "Blobs of part {} cannot be removed", name); @@ -1642,8 +1648,8 @@ void IMergeTreeDataPart::remove() if (!isStoredOnDisk()) return; - if (isProjectionPart()) - LOG_WARNING(storage.log, "Projection part {} should be removed by its parent {}.", name, parent_part->name); + if (isProjectionPart() && !is_temp) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection part {} should be removed by its parent {}.", name, parent_part->name); metadata_manager->deleteAll(false); metadata_manager->assertAllDeleted(false); From a4cd562914b3f9c7f16ac488b04dc933dca3515c Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 10:12:29 -0500 Subject: [PATCH 242/526] fix next button --- docs/en/operations/system-tables/index.md | 4 ++-- docs/en/operations/utilities/index.md | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md index e08a727a62a..5fc302cad34 100644 --- a/docs/en/operations/system-tables/index.md +++ b/docs/en/operations/system-tables/index.md @@ -1,7 +1,8 @@ --- slug: /en/operations/system-tables/ sidebar_position: 52 -sidebar_label: System Tables +sidebar_label: Overview +pagination_next: 'en/operations/system-tables/asynchronous_metric_log' --- # System Tables @@ -72,4 +73,3 @@ If procfs is supported and enabled on the system, ClickHouse server collects the - `OSReadBytes` - `OSWriteBytes` -[Original article](https://clickhouse.com/docs/en/operations/system-tables/) diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index df4af30768c..9de68923ea4 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -1,10 +1,11 @@ --- slug: /en/operations/utilities/ sidebar_position: 56 -sidebar_label: Utilities +sidebar_label: Overview +pagination_next: 'en/operations/utilities/clickhouse-copier' --- -# ClickHouse Utility +# ClickHouse Utilities - [clickhouse-local](../../operations/utilities/clickhouse-local.md) — Allows running SQL queries on data without starting the ClickHouse server, similar to how `awk` does this. - [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. From de01a0dd7345ef900466ae7a375b769b64177a26 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 8 Nov 2022 16:21:04 +0100 Subject: [PATCH 243/526] Add test --- ...jection_materialize_and_zero_copy.reference | 0 ...77_projection_materialize_and_zero_copy.sql | 18 ++++++++++++++++++ 2 files changed, 18 insertions(+) create mode 100644 tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.reference create mode 100644 tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.reference b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql new file mode 100644 index 00000000000..922b613888f --- /dev/null +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS t; + +create table t (c1 Int64, c2 String, c3 DateTime, c4 Int8, c5 String, c6 String, c7 String, c8 String, c9 String, c10 String, c11 String, c12 String, c13 Int8, c14 Int64, c15 String, c16 String, c17 String, c18 Int64, c19 Int64, c20 Int64) engine ReplicatedMergeTree('/clickhouse/test/{database}/test_02477', '1') order by c18 +SETTINGS allow_remote_fs_zero_copy_replication=1; + +insert into t (c1, c18) select number, -number from numbers(2000000); + +alter table t add projection p_norm (select * order by c1); + +optimize table t final; + +alter table t materialize projection p_norm settings mutations_sync = 1; + +SYSTEM FLUSH LOGS; + +SELECT * FROM system.text_log WHERE event_time >= now() - 30 and message like '%BAD_DATA_PART_NAME%'; + +DROP TABLE IF EXISTS t; From a31a79020f314666466169dfd18b08971d8d50a9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 8 Nov 2022 16:27:26 +0100 Subject: [PATCH 244/526] Add comment --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 42f15abd113..f38105ce1f6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1629,6 +1629,8 @@ void IMergeTreeDataPart::remove() auto can_remove_callback = [this] () { + /// Temporary projections are "subparts" which are generated during projections materialization + /// We can always remove them without any additional checks. if (isProjectionPart() && is_temp) { LOG_TRACE(storage.log, "Temporary projection part {} can be removed", name); @@ -1648,6 +1650,8 @@ void IMergeTreeDataPart::remove() if (!isStoredOnDisk()) return; + /// Projections should be never removed by themselves, they will be removed + /// with by parent part. if (isProjectionPart() && !is_temp) throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection part {} should be removed by its parent {}.", name, parent_part->name); From e659fb5d85d13c8d6e1b5e0719f3a27c4bfee8c1 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Nov 2022 19:11:44 +0300 Subject: [PATCH 245/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 78176890841..f004e7464a5 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db 2>&1 > server.log & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db > server.log 2>&1 & server_pid=$! kill -0 $server_pid From 050f3ca7cf514e358e2d852a56f81ffcb4162908 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 8 Nov 2022 17:25:03 +0100 Subject: [PATCH 246/526] Fix msan warning --- base/glibc-compatibility/musl/getauxval.c | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/base/glibc-compatibility/musl/getauxval.c b/base/glibc-compatibility/musl/getauxval.c index eba12604b4d..44a9f979f99 100644 --- a/base/glibc-compatibility/musl/getauxval.c +++ b/base/glibc-compatibility/musl/getauxval.c @@ -10,6 +10,12 @@ #include "syscall.h" +#if defined(__has_feature) +#if __has_feature(memory_sanitizer) +#include +#endif +#endif + #define ARRAY_SIZE(a) sizeof((a))/sizeof((a[0])) /// Suppress TSan since it is possible for this code to be called from multiple threads, From e0f1fd93f3e43c05b8fb8d2f760f84fb2a22bad9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Nov 2022 18:25:13 +0100 Subject: [PATCH 247/526] Updated tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 147 +++++++++++++----- .../02475_analyzer_array_join_with_join.sql | 71 --------- ...77_analyzer_array_join_with_join.reference | 128 +++++++++++++++ .../02477_analyzer_array_join_with_join.sql | 143 +++++++++++++++++ 4 files changed, 375 insertions(+), 114 deletions(-) delete mode 100644 tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql create mode 100644 tests/queries/0_stateless/02477_analyzer_array_join_with_join.reference create mode 100644 tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 1b680c1fcdb..9af1d30e956 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2136,8 +2136,8 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id if (qualified_identifier_with_removed_part.empty()) break; - if (scope.context->getSettingsRef().prefer_column_name_to_alias - && scope.alias_name_to_expression_node.contains(qualified_identifier_with_removed_part[0])) + IdentifierLookup bind_to_aliases_identifier_lookup = {qualified_identifier_with_removed_part, IdentifierLookupContext::EXPRESSION}; + if (tryBindIdentifierToAliases(bind_to_aliases_identifier_lookup, scope)) break; bool can_remove_qualificator = true; @@ -2333,39 +2333,45 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromArrayJoin(const Identifi if (scope.table_expressions_in_resolve_process.contains(table_expression_node.get())) return resolved_identifier; - for (const auto & array_join_expression : from_array_join_node.getJoinExpressions().getNodes()) + const auto & array_join_column_expressions = from_array_join_node.getJoinExpressions(); + const auto & array_join_column_expressions_nodes = array_join_column_expressions.getNodes(); + + /** Allow JOIN with USING with ARRAY JOIN. + * + * SELECT * FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id + * SELECT * FROM test_table_1 AS t1 ARRAY JOIN t1.id AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id + */ + for (const auto & array_join_column_expression : array_join_column_expressions_nodes) { - /** Allow JOIN with USING with ARRAY JOIN. - * - * SELECT * FROM test_table_1 AS t1 ARRAY JOIN [1,2,3] AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id - * SELECT * FROM test_table_1 AS t1 ARRAY JOIN t1.id AS id INNER JOIN test_table_2 AS t2 ON t1.id = t2.id - */ + auto & array_join_column_expression_typed = array_join_column_expression->as(); + if (identifier_lookup.identifier.isShort() && - array_join_expression->getAlias() == identifier_lookup.identifier.getFullName()) - { - resolved_identifier = array_join_expression; - break; - } + array_join_column_expression_typed.getAlias() == identifier_lookup.identifier.getFullName()) + return array_join_column_expression; + } - if (!resolved_identifier) + if (!resolved_identifier) + return nullptr; + + /** Special case when qualified or unqualified identifier point to array join expression without alias. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; + * + * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. + */ + for (const auto & array_join_column_expression : array_join_column_expressions_nodes) + { + auto & array_join_column_expression_typed = array_join_column_expression->as(); + + if (array_join_column_expression_typed.hasAlias()) continue; - /** Special case when qualified or unqualified identifier point to array join expression without alias. - * - * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; - * SELECT id, value, value_array, test_table.value_array, default.test_table.value_array FROM test_table ARRAY JOIN value_array; - * - * value_array, test_table.value_array, default.test_table.value_array must be resolved into array join expression. - */ - auto & array_join_column_expression = array_join_expression->as(); - if (array_join_column_expression.hasAlias()) - continue; - - auto & array_join_column_inner_expression = array_join_column_expression.getExpressionOrThrow(); + auto & array_join_column_inner_expression = array_join_column_expression_typed.getExpressionOrThrow(); if (array_join_column_inner_expression.get() == resolved_identifier.get() || array_join_column_inner_expression->isEqual(*resolved_identifier)) { - resolved_identifier = array_join_expression; + resolved_identifier = array_join_column_expression; break; } } @@ -2889,7 +2895,48 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( QueryTreeNodesWithNames matched_expression_nodes_with_column_names; if (auto * array_join_node = table_expression->as()) + { + size_t table_expressions_column_nodes_with_names_stack_size = table_expressions_column_nodes_with_names_stack.size(); + if (table_expressions_column_nodes_with_names_stack_size < 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Expected at least 1 table expressions on stack before ARRAY JOIN processing. Actual {}", + table_expressions_column_nodes_with_names_stack_size); + + auto & table_expression_column_nodes_with_names = table_expressions_column_nodes_with_names_stack.back(); + + const auto & array_join_column_list = array_join_node->getJoinExpressions(); + const auto & array_join_column_nodes = array_join_column_list.getNodes(); + + /** Special case with ARRAY JOIN column without alias. + * + * CREATE TABLE test_table (id UInt64, value String, value_array Array(UInt8)) ENGINE=TinyLog; + * SELECT * FROM test_table ARRAY JOIN value_array; + * + * In matched columns `value_array` must be resolved into array join column. + */ + for (const auto & array_join_column_node : array_join_column_nodes) + { + if (array_join_column_node->hasAlias()) + continue; + + auto array_join_column_inner_expression = array_join_column_node->as().getExpressionOrThrow(); + if (array_join_column_inner_expression->getNodeType() != QueryTreeNodeType::COLUMN) + continue; + + for (auto & table_expressions_column_node_with_name : table_expression_column_nodes_with_names) + { + auto & table_expression_column_node = table_expressions_column_node_with_name.first; + + if (table_expression_column_node.get() == array_join_column_inner_expression.get() || + table_expression_column_node->isEqual(*array_join_column_inner_expression)) + { + table_expression_column_node = array_join_column_node; + } + } + } + continue; + } bool table_expression_in_resolve_process = scope.table_expressions_in_resolve_process.contains(table_expression.get()); @@ -2947,28 +2994,19 @@ QueryAnalyzer::QueryTreeNodesWithNames QueryAnalyzer::resolveUnqualifiedMatcher( * In such example, match `id` column from ARRAY JOIN, and all columns from test_table except `id`. * * SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); - * In such example match `id` column from ARRAY JOIN, and all columns from test_table except `id`. + * In such example match `id` column from t1, and all columns from test_table except `id`. * * SELECT * FROM (SELECT [0] AS id) AS t1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); * In such example match `id` column from t1, and all columns from test_table except `id`. */ - auto & matched_column_node_typed = matched_column_node->as(); auto matched_column_source = matched_column_node->as().getColumnSource(); - if (matched_column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN) + if (matched_column_source->getNodeType() == QueryTreeNodeType::ARRAY_JOIN && matched_column_node->hasAlias()) { - auto * array_join_inner_expression_column = matched_column_node_typed.getExpressionOrThrow()->as(); - - if (array_join_inner_expression_column && array_join_inner_expression_column->getColumnName() == using_column_name) - { + if (isRight(join_node->getKind())) left_table_expression_column_names_to_skip.insert(using_column_name); - right_table_expression_column_names_to_skip.insert(using_column_name); - matched_expression_nodes_with_column_names.emplace_back(std::move(matched_column_node), using_column_name); - } else - { right_table_expression_column_names_to_skip.insert(using_column_name); - } } else { @@ -5028,6 +5066,8 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, resolveQueryJoinTreeNode(array_join_node.getTableExpression(), scope, expressions_visitor); validateJoinTableExpressionWithoutAlias(join_tree_node, array_join_node.getTableExpression(), scope); + std::unordered_set array_join_column_names; + /// Wrap array join expressions into column nodes, where array join expression is inner expression. for (auto & array_join_expression : array_join_node.getJoinExpressions().getNodes()) @@ -5056,16 +5096,37 @@ void QueryAnalyzer::resolveQueryJoinTreeNode(QueryTreeNodePtr & join_tree_node, result_type = assert_cast(*result_type).getNestedType(); - auto array_join_expression_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); - ++array_join_expressions_counter; + String array_join_column_name; - auto array_join_column = std::make_shared(NameAndTypePair{array_join_expression_name, result_type}, array_join_expression, join_tree_node); + if (!array_join_expression_alias.empty()) + { + array_join_column_name = array_join_expression_alias; + } + else if (auto * array_join_expression_inner_column = array_join_expression->as()) + { + array_join_column_name = array_join_expression_inner_column->getColumnName(); + } + else + { + array_join_column_name = "__array_join_expression_" + std::to_string(array_join_expressions_counter); + ++array_join_expressions_counter; + } + + if (array_join_column_names.contains(array_join_column_name)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "ARRAY JOIN {} multiple columns with name {}. In scope {}", + array_join_node.formatASTForErrorMessage(), + array_join_column_name, + scope.scope_node->formatASTForErrorMessage()); + array_join_column_names.emplace(array_join_column_name); + + auto array_join_column = std::make_shared(NameAndTypePair{array_join_column_name, result_type}, array_join_expression, join_tree_node); array_join_expression = std::move(array_join_column); array_join_expression->setAlias(array_join_expression_alias); auto it = scope.alias_name_to_expression_node.find(array_join_expression_alias); if (it != scope.alias_name_to_expression_node.end()) - it->second = std::make_shared(NameAndTypePair{array_join_expression_name, result_type}, join_tree_node); + it->second = array_join_expression; } break; diff --git a/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql b/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql deleted file mode 100644 index 80ae3dfc072..00000000000 --- a/tests/queries/0_stateless/02475_analyzer_array_join_with_join.sql +++ /dev/null @@ -1,71 +0,0 @@ -SET allow_experimental_analyzer = 1; - -DROP TABLE IF EXISTS test_table; -CREATE TABLE test_table -( - id UInt64, - value String -) ENGINE=MergeTree ORDER BY id; - -INSERT INTO test_table VALUES (0, 'Value_0'); -INSERT INTO test_table VALUES (1, 'Value_1'); - -SELECT * FROM (SELECT [dummy, dummy] AS dummy FROM system.one) AS subquery ARRAY JOIN dummy INNER JOIN system.one USING (dummy); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy); - -SELECT '--'; - -SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN system.one USING (dummy); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy); - -SELECT '--'; - -SELECT * FROM (SELECT [1] AS dummy) AS subquery_1 ARRAY JOIN dummy INNER JOIN (SELECT 1 AS dummy) AS subquery_2 USING (dummy); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [1] AS value) AS subquery_1 ARRAY JOIN value AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); - -SELECT '--'; - -SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); - -DROP TABLE test_table; diff --git a/tests/queries/0_stateless/02477_analyzer_array_join_with_join.reference b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.reference new file mode 100644 index 00000000000..f7084c4617c --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.reference @@ -0,0 +1,128 @@ +-- { echoOn } + +SELECT * FROM test_table ARRAY JOIN value_array; +0 Value_0 1 +0 Value_0 2 +0 Value_0 3 +SELECT '--'; +-- +SELECT *, value_array_element FROM test_table ARRAY JOIN value_array AS value_array_element; +0 Value_0 [1,2,3] 1 +0 Value_0 [1,2,3] 2 +0 Value_0 [1,2,3] 3 +SELECT '--'; +-- +SELECT *, value_array FROM test_table ARRAY JOIN value_array AS value_array; +0 Value_0 [1,2,3] 1 +0 Value_0 [1,2,3] 2 +0 Value_0 [1,2,3] 3 +SELECT '--'; +-- +SELECT *, value_array FROM test_table ARRAY JOIN [4,5,6] AS value_array; +0 Value_0 [1,2,3] 4 +0 Value_0 [1,2,3] 5 +0 Value_0 [1,2,3] 6 +SELECT '--'; +-- +SELECT *, value_array, value_element FROM test_table ARRAY JOIN value_array, [4,5,6] AS value_element; +0 Value_0 1 1 4 +0 Value_0 2 2 5 +0 Value_0 3 3 6 +SELECT '--'; +-- +SELECT * FROM (SELECT [dummy, dummy] AS dummy FROM system.one) AS subquery ARRAY JOIN dummy INNER JOIN system.one USING (dummy); +0 +0 +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 0 AS id) AS subquery_2 USING (id); +0 +SELECT '--'; +-- +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 0 AS id) AS subquery_2 USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); +1 +SELECT '--'; +-- +SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1,2,3] AS id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); +[5] +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table USING (id); +0 Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] 0 +SELECT '--'; +-- +SELECT *, id FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT *, id FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] 0 +SELECT '--'; +-- +SELECT *, id FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); +[0] Value_0 [1,2,3] 0 +SELECT '--'; +-- +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); +[5] Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); +[5] Value_0 [1,2,3] 0 +SELECT '--'; +-- +SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); +SELECT '--'; +-- +SELECT * FROM (SELECT [5] AS id_array) AS subquery ARRAY JOIN id_array, [0] AS id INNER JOIN test_table USING (id); +5 Value_0 [1,2,3] +SELECT '--'; +-- +SELECT * FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id); +[[0]] Value_0 [1,2,3] +SELECT '--'; +-- +SELECT *, id FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id); +[[0]] Value_0 [1,2,3] 0 diff --git a/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql new file mode 100644 index 00000000000..7e4ecb98ae8 --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_array_join_with_join.sql @@ -0,0 +1,143 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String, + value_array Array(UInt64) +) ENGINE=MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (0, 'Value_0', [1,2,3]); + +-- { echoOn } + +SELECT * FROM test_table ARRAY JOIN value_array; + +SELECT '--'; + +SELECT *, value_array_element FROM test_table ARRAY JOIN value_array AS value_array_element; + +SELECT '--'; + +SELECT *, value_array FROM test_table ARRAY JOIN value_array AS value_array; + +SELECT '--'; + +SELECT *, value_array FROM test_table ARRAY JOIN [4,5,6] AS value_array; + +SELECT '--'; + +SELECT *, value_array, value_element FROM test_table ARRAY JOIN value_array, [4,5,6] AS value_element; + +SELECT '--'; + +SELECT * FROM (SELECT [dummy, dummy] AS dummy FROM system.one) AS subquery ARRAY JOIN dummy INNER JOIN system.one USING (dummy); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 0 AS id) AS subquery_2 USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 0 AS id) AS subquery_2 USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery_1 ARRAY JOIN id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id) AS subquery_1 ARRAY JOIN [1,2,3] AS id INNER JOIN (SELECT 1 AS id) AS subquery_2 USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [1] AS id) AS subquery ARRAY JOIN id AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [0] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [1] AS value) AS subquery ARRAY JOIN value AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [0] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [5] AS id) AS subquery ARRAY JOIN [1] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [5] AS id_array) AS subquery ARRAY JOIN id_array, [0] AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT * FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id); + +SELECT '--'; + +SELECT *, id FROM (SELECT [[0]] AS id) AS subquery ARRAY JOIN id AS id_nested_array ARRAY JOIN id_nested_array AS id INNER JOIN test_table USING (id); + +-- { echoOff } + +DROP TABLE test_table; From a3c1049f9d286f7efad1eb77774e67ddba181717 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 8 Nov 2022 19:42:13 +0100 Subject: [PATCH 248/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index f004e7464a5..9b9f0240760 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db > server.log 2>&1 & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db > >(tail --lines=+0 > server.log) 2>&1 & server_pid=$! kill -0 $server_pid From 236f3329bc4bd7f0241a47224a7adb93321319ac Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 8 Nov 2022 22:54:58 +0300 Subject: [PATCH 249/526] Update run-fuzzer.sh --- docker/test/fuzzer/run-fuzzer.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index 9b9f0240760..e57b671342c 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -178,7 +178,7 @@ function fuzz # interferes with gdb export CLICKHOUSE_WATCHDOG_ENABLE=0 # NOTE: we use process substitution here to preserve keep $! as a pid of clickhouse-server - clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db > >(tail --lines=+0 > server.log) 2>&1 & + clickhouse-server --config-file db/config.xml --pid-file /var/run/clickhouse-server/clickhouse-server.pid -- --path db 2>&1 | pigz > server.log.gz & server_pid=$! kill -0 $server_pid @@ -297,7 +297,7 @@ quit # The server has died. task_exit_code=210 echo "failure" > status.txt - if ! grep --text -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: AddressSanitizer:.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*\|.*_LIBCPP_ASSERT.*" server.log > description.txt + if ! zgrep --text -ao "Received signal.*\|Logical error.*\|Assertion.*failed\|Failed assertion.*\|.*runtime error: .*\|.*is located.*\|SUMMARY: AddressSanitizer:.*\|SUMMARY: MemorySanitizer:.*\|SUMMARY: ThreadSanitizer:.*\|.*_LIBCPP_ASSERT.*" server.log.gz > description.txt then echo "Lost connection to server. See the logs." > description.txt fi @@ -392,7 +392,7 @@ th { cursor: pointer; }

AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}

From 5a3d4cd72ed005a729e4c0ebc9430d341efc0878 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 8 Nov 2022 19:57:29 +0000 Subject: [PATCH 250/526] Fix typos. Add comments. --- .../Optimizations/optimizeReadInOrder.cpp | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index a1184607979..d426c12dbdc 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -87,7 +87,7 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) using FixedColumns = std::unordered_set; /// Right now we find only simple cases like 'and(..., and(..., and(column = value, ...), ...' -void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fiexd_columns) +void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) { std::stack stack; stack.push(&filter_expression); @@ -107,7 +107,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr else if (name == "equals") { const ActionsDAG::Node * maybe_fixed_column = nullptr; - bool is_singe = true; + bool is_single = true; for (const auto & child : node->children) { if (!child->column) @@ -115,14 +115,14 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr if (!maybe_fixed_column) maybe_fixed_column = child; else - is_singe = false; + is_single = false; } } - if (maybe_fixed_column && is_singe) + if (maybe_fixed_column && is_single) { //std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; - fiexd_columns.insert(maybe_fixed_column); + fixed_columns.insert(maybe_fixed_column); const ActionsDAG::Node * maybe_injective = maybe_fixed_column; while (maybe_injective->type == ActionsDAG::ActionType::FUNCTION @@ -130,7 +130,7 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr && maybe_injective->function_base->isInjective({})) { maybe_injective = maybe_injective->children.front(); - fiexd_columns.insert(maybe_injective); + fixed_columns.insert(maybe_injective); } } } @@ -364,15 +364,20 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG auto it = matches.find(child); if (it == matches.end()) { + /// If match map does not contain a child, it was not visited. stack.push(Frame{child, {}}); break; } + /// A node from found match may be nullptr. + /// It means that node is visited, but no match was found. frame.mapped_children.push_back(it->second.node); } if (frame.mapped_children.size() < frame.node->children.size()) continue; + /// Create an empty match for current node. + /// natch.node will be set if match is found. auto & match = matches[frame.node]; if (frame.node->type == ActionsDAG::ActionType::INPUT) @@ -567,6 +572,9 @@ InputOrderInfoPtr buildInputOrderInfo( if (!dag) { + /// This is possible if there were no Expression or Filter steps in Plan. + /// Examlpe: SELECT * FROM tab ORDER BY a, b + if (sort_column_node->type != ActionsDAG::ActionType::INPUT) break; From 4c89fccd703d750153c7ad1ed493d0db6b00ba92 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 8 Nov 2022 21:09:02 +0000 Subject: [PATCH 251/526] fix flaky test + 01346_alter_enum_partition_key_replicated_zookeeper_long --- .../01346_alter_enum_partition_key_replicated_zookeeper_long.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 1aa96016007..f20156fd9e3 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -11,6 +11,7 @@ CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = Replica INSERT INTO test VALUES ('hello', 'test'); SELECT * FROM test; +SYSTEM SYNC REPLICA test2; SELECT * FROM test2; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; From 090b153b1e297e24f7cd5c44a973492eab6a8784 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 17:37:20 -0500 Subject: [PATCH 252/526] a dir named index with index.md broke nav --- .../sql-reference/statements/alter/{index.md => overview.md} | 0 .../statements/alter/{index/index.md => skipping-index.md} | 3 ++- 2 files changed, 2 insertions(+), 1 deletion(-) rename docs/en/sql-reference/statements/alter/{index.md => overview.md} (100%) rename docs/en/sql-reference/statements/alter/{index/index.md => skipping-index.md} (95%) diff --git a/docs/en/sql-reference/statements/alter/index.md b/docs/en/sql-reference/statements/alter/overview.md similarity index 100% rename from docs/en/sql-reference/statements/alter/index.md rename to docs/en/sql-reference/statements/alter/overview.md diff --git a/docs/en/sql-reference/statements/alter/index/index.md b/docs/en/sql-reference/statements/alter/skipping-index.md similarity index 95% rename from docs/en/sql-reference/statements/alter/index/index.md rename to docs/en/sql-reference/statements/alter/skipping-index.md index 03d4bd47e71..e23cf20cbed 100644 --- a/docs/en/sql-reference/statements/alter/index/index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -1,5 +1,6 @@ --- -slug: /en/sql-reference/statements/alter/index +slug: /en/sql-reference/statements/alter/skipping-index + toc_hidden_folder: true sidebar_position: 42 sidebar_label: INDEX From d2253663356af1149960662942a2b5c87108804f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 18:04:22 -0500 Subject: [PATCH 253/526] update links --- .../database-engines/materialized-mysql.md | 78 +++++++++---------- .../statements/alter/skipping-index.md | 4 +- .../en/sql-reference/statements/alter/view.md | 2 +- docs/en/sql-reference/statements/grant.md | 2 +- docs/en/sql-reference/statements/kill.md | 2 +- 5 files changed, 44 insertions(+), 44 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index c8aa65bdd91..5ddcf6bfb41 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -77,15 +77,15 @@ While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = ## Virtual Columns {#virtual-columns} -When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](../../engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. +When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](/docs/en/sql-reference/engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. ### \_version -`_version` — Transaction counter. Type [UInt64](../../sql-reference/data-types/int-uint.md). +`_version` — Transaction counter. Type [UInt64](/docs/en/sql-reference/data-types/int-uint.md). ### \_sign -`_sign` — Deletion mark. Type [Int8](../../sql-reference/data-types/int-uint.md). Possible values: +`_sign` — Deletion mark. Type [Int8](/docs/en/sql-reference/data-types/int-uint.md). Possible values: - `1` — Row is not deleted, - `-1` — Row is deleted. @@ -93,29 +93,29 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( | MySQL | ClickHouse | |-------------------------|--------------------------------------------------------------| -| TINY | [Int8](../../sql-reference/data-types/int-uint.md) | -| SHORT | [Int16](../../sql-reference/data-types/int-uint.md) | -| INT24 | [Int32](../../sql-reference/data-types/int-uint.md) | -| LONG | [UInt32](../../sql-reference/data-types/int-uint.md) | -| LONGLONG | [UInt64](../../sql-reference/data-types/int-uint.md) | -| FLOAT | [Float32](../../sql-reference/data-types/float.md) | -| DOUBLE | [Float64](../../sql-reference/data-types/float.md) | -| DECIMAL, NEWDECIMAL | [Decimal](../../sql-reference/data-types/decimal.md) | -| DATE, NEWDATE | [Date](../../sql-reference/data-types/date.md) | -| DATETIME, TIMESTAMP | [DateTime](../../sql-reference/data-types/datetime.md) | -| DATETIME2, TIMESTAMP2 | [DateTime64](../../sql-reference/data-types/datetime64.md) | -| YEAR | [UInt16](../../sql-reference/data-types/int-uint.md) | -| TIME | [Int64](../../sql-reference/data-types/int-uint.md) | -| ENUM | [Enum](../../sql-reference/data-types/enum.md) | -| STRING | [String](../../sql-reference/data-types/string.md) | -| VARCHAR, VAR_STRING | [String](../../sql-reference/data-types/string.md) | -| BLOB | [String](../../sql-reference/data-types/string.md) | -| GEOMETRY | [String](../../sql-reference/data-types/string.md) | -| BINARY | [FixedString](../../sql-reference/data-types/fixedstring.md) | -| BIT | [UInt64](../../sql-reference/data-types/int-uint.md) | -| SET | [UInt64](../../sql-reference/data-types/int-uint.md) | +| TINY | [Int8](/docs/en/sql-reference/data-types/int-uint.md) | +| SHORT | [Int16](/docs/en/sql-reference/data-types/int-uint.md) | +| INT24 | [Int32](/docs/en/sql-reference/data-types/int-uint.md) | +| LONG | [UInt32](/docs/en/sql-reference/data-types/int-uint.md) | +| LONGLONG | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | +| FLOAT | [Float32](/docs/en/sql-reference/data-types/float.md) | +| DOUBLE | [Float64](/docs/en/sql-reference/data-types/float.md) | +| DECIMAL, NEWDECIMAL | [Decimal](/docs/en/sql-reference/data-types/decimal.md) | +| DATE, NEWDATE | [Date](/docs/en/sql-reference/data-types/date.md) | +| DATETIME, TIMESTAMP | [DateTime](/docs/en/sql-reference/data-types/datetime.md) | +| DATETIME2, TIMESTAMP2 | [DateTime64](/docs/en/sql-reference/data-types/datetime64.md) | +| YEAR | [UInt16](/docs/en/sql-reference/data-types/int-uint.md) | +| TIME | [Int64](/docs/en/sql-reference/data-types/int-uint.md) | +| ENUM | [Enum](/docs/en/sql-reference/data-types/enum.md) | +| STRING | [String](/docs/en/sql-reference/data-types/string.md) | +| VARCHAR, VAR_STRING | [String](/docs/en/sql-reference/data-types/string.md) | +| BLOB | [String](/docs/en/sql-reference/data-types/string.md) | +| GEOMETRY | [String](/docs/en/sql-reference/data-types/string.md) | +| BINARY | [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) | +| BIT | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | +| SET | [UInt64](/docs/en/sql-reference/data-types/int-uint.md) | -[Nullable](../../sql-reference/data-types/nullable.md) is supported. +[Nullable](/docs/en/sql-reference/data-types/nullable.md) is supported. The data of TIME type in MySQL is converted to microseconds in ClickHouse. @@ -133,7 +133,7 @@ Apart of the data types limitations there are few restrictions comparing to `MyS ### DDL Queries {#ddl-queries} -MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](../../sql-reference/statements/alter/index.md), [CREATE](../../sql-reference/statements/create/index.md), [DROP](../../sql-reference/statements/drop), [RENAME](../../sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. +MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](/docs/en/sql-reference/statements/alter/overview.md), [CREATE](/docs/en/sql-reference/statements/create/index.md), [DROP](/docs/en/sql-reference/statements/drop.md), [RENAME](/docs/en/sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. ### Data Replication {#data-replication} @@ -151,7 +151,7 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ `SELECT` query from `MaterializedMySQL` tables has some specifics: - If `_version` is not specified in the `SELECT` query, the - [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier is used, so only rows with + [FINAL](/docs/en/sql-reference/statements/select/from.md/#select-from-final) modifier is used, so only rows with `MAX(_version)` are returned for each primary key value. - If `_sign` is not specified in the `SELECT` query, `WHERE _sign=1` is used by default. So the deleted rows are not @@ -164,7 +164,7 @@ MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ MySQL `PRIMARY KEY` and `INDEX` clauses are converted into `ORDER BY` tuples in ClickHouse tables. ClickHouse has only one physical order, which is determined by `ORDER BY` clause. To create a new physical order, use -[materialized views](../../sql-reference/statements/create/view.md#materialized). +[materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized). **Notes** @@ -173,7 +173,7 @@ ClickHouse has only one physical order, which is determined by `ORDER BY` clause MySQL binlog. - Replication can be easily broken. - Manual operations on database and tables are forbidden. -- `MaterializedMySQL` is affected by the [optimize_on_insert](../../operations/settings/settings.md#optimize-on-insert) +- `MaterializedMySQL` is affected by the [optimize_on_insert](/docs/en/operations/settings/settings.md/#optimize-on-insert) setting. Data is merged in the corresponding table in the `MaterializedMySQL` database when a table in the MySQL server changes. @@ -187,19 +187,19 @@ These are the schema conversion manipulations you can do with table overrides fo * Modify column type. Must be compatible with the original type, or replication will fail. For example, you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String). - * Modify [column TTL](../table-engines/mergetree-family/mergetree/#mergetree-column-ttl). - * Modify [column compression codec](../../sql-reference/statements/create/table/#codecs). - * Add [ALIAS columns](../../sql-reference/statements/create/table/#alias). - * Add [skipping indexes](../table-engines/mergetree-family/mergetree/#table_engine-mergetree-data_skipping-indexes) - * Add [projections](../table-engines/mergetree-family/mergetree/#projections). Note that projection optimizations are + * Modify [column TTL](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). + * Modify [column compression codec](/docs/en/sql-reference/statements/create/table.md/#codecs). + * Add [ALIAS columns](/docs/en/sql-reference/statements/create/table.md/#alias). + * Add [skipping indexes](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-data_skipping-indexes) + * Add [projections](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#projections). Note that projection optimizations are disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here. `INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/) may be more useful in this case. - * Modify [PARTITION BY](../table-engines/mergetree-family/custom-partitioning-key/) - * Modify [ORDER BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Modify [PRIMARY KEY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [SAMPLE BY](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) - * Add [table TTL](../table-engines/mergetree-family/mergetree/#mergetree-query-clauses) + * Modify [PARTITION BY](/docs/en/sql-reference/table-engines/mergetree-family/custom-partitioning-key/) + * Modify [ORDER BY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Modify [PRIMARY KEY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [SAMPLE BY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [table TTL](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index e23cf20cbed..1a61e1b73ec 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -14,12 +14,12 @@ The following operations are available: - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](../../../../sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing indices metadata via ZooKeeper. :::note -Index manipulation is supported only for tables with [`*MergeTree`](../../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../../engines/table-engines/mergetree-family/replication.md) variants). +Index manipulation is supported only for tables with [`*MergeTree`](/docs/en/sql-reference/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/sql-reference/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index e382cdace30..fd6045a026e 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -8,7 +8,7 @@ sidebar_label: VIEW You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement. Use it when the materialized view was created without the `TO [db.]name` clause. The `allow_experimental_alter_materialized_view_structure` setting must be enabled. -If a materialized view uses the `TO [db.]name` construction, you must [DETACH](../detach.md) the view, run [ALTER TABLE](index.md) query for the target table, and then [ATTACH](../attach.md) the previously detached (`DETACH`) view. +If a materialized view uses the `TO [db.]name` construction, you must [DETACH](../detach.md) the view, run [ALTER TABLE](overview.md) query for the target table, and then [ATTACH](../attach.md) the previously detached (`DETACH`) view. **Example** diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index 546a8b0958d..bd35efb3deb 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -261,7 +261,7 @@ The granted privilege allows `john` to insert data to the `x` and/or `y` columns ### ALTER -Allows executing [ALTER](../../sql-reference/statements/alter/index.md) queries according to the following hierarchy of privileges: +Allows executing [ALTER](../../sql-reference/statements/alter/overview.md) queries according to the following hierarchy of privileges: - `ALTER`. Level: `COLUMN`. - `ALTER TABLE`. Level: `GROUP` diff --git a/docs/en/sql-reference/statements/kill.md b/docs/en/sql-reference/statements/kill.md index 294724dfa50..733125c78f3 100644 --- a/docs/en/sql-reference/statements/kill.md +++ b/docs/en/sql-reference/statements/kill.md @@ -51,7 +51,7 @@ KILL MUTATION [ON CLUSTER cluster] [FORMAT format] ``` -Tries to cancel and remove [mutations](../../sql-reference/statements/alter/index.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query. +Tries to cancel and remove [mutations](../../sql-reference/statements/alter/overview.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query. A test query (`TEST`) only checks the user’s rights and displays a list of mutations to stop. From ecb4c2b9fbc08f1d641ccafa976b1f331ac8462e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 9 Nov 2022 00:49:11 +0100 Subject: [PATCH 254/526] Only look for logs with Error level --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 922b613888f..027eb699ad6 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -13,6 +13,6 @@ alter table t materialize projection p_norm settings mutations_sync = 1; SYSTEM FLUSH LOGS; -SELECT * FROM system.text_log WHERE event_time >= now() - 30 and message like '%BAD_DATA_PART_NAME%'; +SELECT * FROM system.text_log WHERE event_time >= now() - 30 and level == 'Error' and message like '%BAD_DATA_PART_NAME%'; DROP TABLE IF EXISTS t; From 761274d85747074e09dbe82665bb7e62e4eeacd8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Nov 2022 01:14:03 +0100 Subject: [PATCH 255/526] Fix wrong logic --- .../MergeTree/ReplicatedMergeTreeSink.cpp | 16 +++++++++++----- src/Storages/MergeTree/ReplicatedMergeTreeSink.h | 3 ++- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index a6fd0a1e3e4..362d56f8a9d 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -297,7 +297,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjecti try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num); + commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; @@ -331,7 +331,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", replicas_num); + commitPart(zookeeper, part, "", replicas_num, true); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -345,7 +345,8 @@ void ReplicatedMergeTreeSink::commitPart( const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num) + size_t replicas_num, + bool writing_existing_part) { /// It is possible that we alter a part with different types of source columns. /// In this case, if column was not altered, the result type will be different with what we have in metadata. @@ -377,8 +378,13 @@ void ReplicatedMergeTreeSink::commitPart( throw Exception( ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); - retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); - return; + /// When we attach existing parts it's okay to be in read-only mode + /// For example during RESTORE REPLICA. + if (!writing_existing_part) + { + retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); + return; + } } if (retries_ctl.isRetry()) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 51aa4350c68..268c9db1b6e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -82,7 +82,8 @@ private: const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num); + size_t replicas_num, + bool writing_existing_part); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. From 45c9debeaff24c41968c5a4a7f173e9dbc706dc6 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 19:17:58 -0500 Subject: [PATCH 256/526] update links --- .../database-engines/materialized-mysql.md | 18 +- .../engines/table-engines/log-family/index.md | 10 +- .../mergetree-family/mergetree.md | 114 +++---- .../mergetree-family/replication.md | 26 +- docs/en/engines/table-engines/special/join.md | 28 +- docs/en/operations/settings/settings.md | 308 +++++++++--------- docs/en/operations/storing-data.md | 18 +- docs/en/operations/system-tables/mutations.md | 34 +- docs/en/operations/system-tables/parts.md | 22 +- .../operations/system-tables/parts_columns.md | 2 +- .../sql-reference/statements/alter/column.md | 22 +- .../sql-reference/statements/alter/delete.md | 12 +- .../statements/alter/overview.md | 44 +-- .../statements/alter/partition.md | 22 +- .../statements/alter/projection.md | 10 +- .../statements/alter/skipping-index.md | 4 +- .../sql-reference/statements/alter/update.md | 10 +- docs/en/sql-reference/statements/index.md | 44 +-- 18 files changed, 374 insertions(+), 374 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 5ddcf6bfb41..0411286cd23 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -77,7 +77,7 @@ While turning on `gtid_mode` you should also specify `enforce_gtid_consistency = ## Virtual Columns {#virtual-columns} -When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](/docs/en/sql-reference/engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. +When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree](/docs/en/engines/table-engines/mergetree-family/replacingmergetree.md) tables are used with virtual `_sign` and `_version` columns. ### \_version @@ -187,19 +187,19 @@ These are the schema conversion manipulations you can do with table overrides fo * Modify column type. Must be compatible with the original type, or replication will fail. For example, you can modify a UInt32 column to UInt64, but you can not modify a String column to Array(String). - * Modify [column TTL](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). + * Modify [column TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). * Modify [column compression codec](/docs/en/sql-reference/statements/create/table.md/#codecs). * Add [ALIAS columns](/docs/en/sql-reference/statements/create/table.md/#alias). - * Add [skipping indexes](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-data_skipping-indexes) - * Add [projections](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#projections). Note that projection optimizations are + * Add [skipping indexes](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-data_skipping-indexes) + * Add [projections](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#projections). Note that projection optimizations are disabled when using `SELECT ... FINAL` (which MaterializedMySQL does by default), so their utility is limited here. `INDEX ... TYPE hypothesis` as [described in the v21.12 blog post]](https://clickhouse.com/blog/en/2021/clickhouse-v21.12-released/) may be more useful in this case. - * Modify [PARTITION BY](/docs/en/sql-reference/table-engines/mergetree-family/custom-partitioning-key/) - * Modify [ORDER BY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) - * Modify [PRIMARY KEY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) - * Add [SAMPLE BY](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) - * Add [table TTL](/docs/en/sql-reference/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Modify [PARTITION BY](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key/) + * Modify [ORDER BY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Modify [PRIMARY KEY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [SAMPLE BY](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) + * Add [table TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) ```sql CREATE DATABASE db_name ENGINE = MaterializedMySQL(...) diff --git a/docs/en/engines/table-engines/log-family/index.md b/docs/en/engines/table-engines/log-family/index.md index 98bc4dbad04..269ad0c98f1 100644 --- a/docs/en/engines/table-engines/log-family/index.md +++ b/docs/en/engines/table-engines/log-family/index.md @@ -10,11 +10,11 @@ These engines were developed for scenarios when you need to quickly write many s Engines of the family: -- [StripeLog](../../../engines/table-engines/log-family/stripelog.md) -- [Log](../../../engines/table-engines/log-family/log.md) -- [TinyLog](../../../engines/table-engines/log-family/tinylog.md) +- [StripeLog](/docs/en/engines/table-engines/log-family/stripelog.md) +- [Log](/docs/en/engines/table-engines/log-family/log.md) +- [TinyLog](/docs/en/engines/table-engines/log-family/tinylog.md) -`Log` family table engines can store data to [HDFS](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-hdfs) or [S3](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-s3) distributed file systems. +`Log` family table engines can store data to [HDFS](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-hdfs) or [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3) distributed file systems. ## Common Properties {#common-properties} @@ -28,7 +28,7 @@ Engines: During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. -- Do not support [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations). +- Do not support [mutations](/docs/en/sql-reference/statements/alter/overview.md/#alter-mutations). - Do not support indexes. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 486baac2310..807a19605c4 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -16,20 +16,20 @@ Main features: This allows you to create a small sparse index that helps find data faster. -- Partitions can be used if the [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. +- Partitions can be used if the [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) is specified. ClickHouse supports certain operations with partitions that are more efficient than general operations on the same data with the same result. ClickHouse also automatically cuts off the partition data where the partitioning key is specified in the query. - Data replication support. - The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](../../../engines/table-engines/mergetree-family/replication.md). + The family of `ReplicatedMergeTree` tables provides data replication. For more information, see [Data replication](/docs/en/engines/table-engines/mergetree-family/replication.md). - Data sampling support. If necessary, you can set the data sampling method in the table. :::info -The [Merge](../../../engines/table-engines/special/merge.md#merge) engine does not belong to the `*MergeTree` family. +The [Merge](/docs/en/engines/table-engines/special/merge.md/#merge) engine does not belong to the `*MergeTree` family. ::: ## Creating a Table {#table_engine-mergetree-creating-a-table} @@ -57,7 +57,7 @@ ORDER BY expr [SETTINGS name=value, ...] ``` -For a description of parameters, see the [CREATE query description](../../../sql-reference/statements/create/table.md). +For a description of parameters, see the [CREATE query description](/docs/en/sql-reference/statements/create/table.md). ### Query Clauses {#mergetree-query-clauses} @@ -77,9 +77,9 @@ Use the `ORDER BY tuple()` syntax, if you do not need sorting. See [Selecting th #### PARTITION BY -`PARTITION BY` — The [partitioning key](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression). +`PARTITION BY` — The [partitioning key](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md). Optional. In most cases you don't need partition key, and in most other cases you don't need partition key more granular than by months. Partitioning does not speed up queries (in contrast to the ORDER BY expression). You should never use too granular partitioning. Don't partition your data by client identifiers or names (instead make client identifier or name the first column in the ORDER BY expression). -For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](../../../sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. +For partitioning by month, use the `toYYYYMM(date_column)` expression, where `date_column` is a column with a date of the type [Date](/docs/en/sql-reference/data-types/date.md). The partition names here have the `"YYYYMM"` format. #### PRIMARY KEY @@ -127,7 +127,7 @@ Additional parameters that control the behavior of the `MergeTree` (optional): #### use_minimalistic_part_header_in_zookeeper -`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](../../../operations/server-configuration-parameters/settings.md#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. +`use_minimalistic_part_header_in_zookeeper` — Storage method of the data parts headers in ZooKeeper. If `use_minimalistic_part_header_in_zookeeper=1`, then ZooKeeper stores less data. For more information, see the [setting description](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings-use_minimalistic_part_header_in_zookeeper) in “Server configuration parameters”. #### min_merge_bytes_to_use_direct_io @@ -166,15 +166,15 @@ Additional parameters that control the behavior of the `MergeTree` (optional): #### max_compress_block_size -`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](../../../operations/settings/settings.md#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. +`max_compress_block_size` — Maximum size of blocks of uncompressed data before compressing for writing to a table. You can also specify this setting in the global settings (see [max_compress_block_size](/docs/en/operations/settings/settings.md/#max-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. #### min_compress_block_size -`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](../../../operations/settings/settings.md#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. +`min_compress_block_size` — Minimum size of blocks of uncompressed data required for compression when writing the next mark. You can also specify this setting in the global settings (see [min_compress_block_size](/docs/en/operations/settings/settings.md/#min-compress-block-size) setting). The value specified when table is created overrides the global value for this setting. #### max_partitions_to_read -`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](../../../operations/settings/merge-tree-settings.md#max-partitions-to-read) in the global setting. +`max_partitions_to_read` — Limits the maximum number of partitions that can be accessed in one query. You can also specify setting [max_partitions_to_read](/docs/en/operations/settings/merge-tree-settings.md/#max-partitions-to-read) in the global setting. **Example of Sections Setting** @@ -184,7 +184,7 @@ ENGINE MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDa In the example, we set partitioning by month. -We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If you define a [SAMPLE](../../../sql-reference/statements/select/sample.md#select-sample-clause) clause when selecting the data, ClickHouse will return an evenly pseudorandom data sample for a subset of users. +We also set an expression for sampling as a hash by the user ID. This allows you to pseudorandomize the data in the table for each `CounterID` and `EventDate`. If you define a [SAMPLE](/docs/en/sql-reference/statements/select/sample.md/#select-sample-clause) clause when selecting the data, ClickHouse will return an evenly pseudorandom data sample for a subset of users. The `index_granularity` setting can be omitted because 8192 is the default value. @@ -207,9 +207,9 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **MergeTree() Parameters** -- `date-column` — The name of a column of the [Date](../../../sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. +- `date-column` — The name of a column of the [Date](/docs/en/sql-reference/data-types/date.md) type. ClickHouse automatically creates partitions by month based on this column. The partition names are in the `"YYYYMM"` format. - `sampling_expression` — An expression for sampling. -- `(primary, key)` — Primary key. Type: [Tuple()](../../../sql-reference/data-types/tuple.md) +- `(primary, key)` — Primary key. Type: [Tuple()](/docs/en/sql-reference/data-types/tuple.md) - `index_granularity` — The granularity of an index. The number of data rows between the “marks” of an index. The value 8192 is appropriate for most tasks. **Example** @@ -262,7 +262,7 @@ Sparse indexes allow you to work with a very large number of table rows, because ClickHouse does not require a unique primary key. You can insert multiple rows with the same primary key. -You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](../../../operations/settings/settings.md#allow-nullable-key) setting. The [NULLS_LAST](../../../sql-reference/statements/select/order-by.md#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. +You can use `Nullable`-typed expressions in the `PRIMARY KEY` and `ORDER BY` clauses but it is strongly discouraged. To allow this feature, turn on the [allow_nullable_key](/docs/en/operations/settings/settings.md/#allow-nullable-key) setting. The [NULLS_LAST](/docs/en/sql-reference/statements/select/order-by.md/#sorting-of-special-values) principle applies for `NULL` values in the `ORDER BY` clause. ### Selecting the Primary Key {#selecting-the-primary-key} @@ -279,26 +279,26 @@ The number of columns in the primary key is not explicitly limited. Depending on ClickHouse sorts data by primary key, so the higher the consistency, the better the compression. -- Provide additional logic when merging data parts in the [CollapsingMergeTree](../../../engines/table-engines/mergetree-family/collapsingmergetree.md#table_engine-collapsingmergetree) and [SummingMergeTree](../../../engines/table-engines/mergetree-family/summingmergetree.md) engines. +- Provide additional logic when merging data parts in the [CollapsingMergeTree](/docs/en/engines/table-engines/mergetree-family/collapsingmergetree.md/#table_engine-collapsingmergetree) and [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) engines. In this case it makes sense to specify the *sorting key* that is different from the primary key. A long primary key will negatively affect the insert performance and memory consumption, but extra columns in the primary key do not affect ClickHouse performance during `SELECT` queries. -You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](../../../operations/settings/settings.md#settings-max-insert-threads). +You can create a table without a primary key using the `ORDER BY tuple()` syntax. In this case, ClickHouse stores data in the order of inserting. If you want to save data order when inserting data by `INSERT ... SELECT` queries, set [max_insert_threads = 1](/docs/en/operations/settings/settings.md/#settings-max-insert-threads). -To select data in the initial order, use [single-threaded](../../../operations/settings/settings.md#settings-max_threads) `SELECT` queries. +To select data in the initial order, use [single-threaded](/docs/en/operations/settings/settings.md/#settings-max_threads) `SELECT` queries. ### Choosing a Primary Key that Differs from the Sorting Key {#choosing-a-primary-key-that-differs-from-the-sorting-key} It is possible to specify a primary key (an expression with values that are written in the index file for each mark) that is different from the sorting key (an expression for sorting the rows in data parts). In this case the primary key expression tuple must be a prefix of the sorting key expression tuple. -This feature is helpful when using the [SummingMergeTree](../../../engines/table-engines/mergetree-family/summingmergetree.md) and -[AggregatingMergeTree](../../../engines/table-engines/mergetree-family/aggregatingmergetree.md) table engines. In a common case when using these engines, the table has two types of columns: *dimensions* and *measures*. Typical queries aggregate values of measure columns with arbitrary `GROUP BY` and filtering by dimensions. Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns and this list must be frequently updated with newly added dimensions. +This feature is helpful when using the [SummingMergeTree](/docs/en/engines/table-engines/mergetree-family/summingmergetree.md) and +[AggregatingMergeTree](/docs/en/engines/table-engines/mergetree-family/aggregatingmergetree.md) table engines. In a common case when using these engines, the table has two types of columns: *dimensions* and *measures*. Typical queries aggregate values of measure columns with arbitrary `GROUP BY` and filtering by dimensions. Because SummingMergeTree and AggregatingMergeTree aggregate rows with the same value of the sorting key, it is natural to add all dimensions to it. As a result, the key expression consists of a long list of columns and this list must be frequently updated with newly added dimensions. In this case it makes sense to leave only a few columns in the primary key that will provide efficient range scans and add the remaining dimension columns to the sorting key tuple. -[ALTER](../../../sql-reference/statements/alter/index.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. +[ALTER](/docs/en/sql-reference/statements/alter/overview.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. ### Use of Indexes and Partitions in Queries {#use-of-indexes-and-partitions-in-queries} @@ -342,7 +342,7 @@ In the example below, the index can’t be used. SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](../../../operations/settings/settings.md#settings-force_index_by_date) and [force_primary_key](../../../operations/settings/settings.md#force-primary-key). +To check whether ClickHouse can use the index when running a query, use the settings [force_index_by_date](/docs/en/operations/settings/settings.md/#settings-force_index_by_date) and [force_primary_key](/docs/en/operations/settings/settings.md/#force-primary-key). The key for partitioning by month allows reading only those data blocks which contain dates from the proper range. In this case, the data block may contain data for many dates (up to an entire month). Within a block, data is sorted by primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. @@ -400,7 +400,7 @@ Stores unique values of the specified expression (no more than `max_rows` rows, #### `ngrambf_v1(n, size_of_bloom_filter_in_bytes, number_of_hash_functions, random_seed)` -Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](../../../sql-reference/data-types/string.md), [FixedString](../../../sql-reference/data-types/fixedstring.md) and [Map](../../../sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. +Stores a [Bloom filter](https://en.wikipedia.org/wiki/Bloom_filter) that contains all ngrams from a block of data. Works only with datatypes: [String](/docs/en/sql-reference/data-types/string.md), [FixedString](/docs/en/sql-reference/data-types/fixedstring.md) and [Map](/docs/en/sql-reference/data-types/map.md). Can be used for optimization of `EQUALS`, `LIKE` and `IN` expressions. - `n` — ngram size, - `size_of_bloom_filter_in_bytes` — Bloom filter size in bytes (you can use large values here, for example, 256 or 512, because it can be compressed well). @@ -417,11 +417,11 @@ The optional `false_positive` parameter is the probability of receiving a false Supported data types: `Int*`, `UInt*`, `Float*`, `Enum`, `Date`, `DateTime`, `String`, `FixedString`, `Array`, `LowCardinality`, `Nullable`, `UUID`, `Map`. -For `Map` data type client can specify if index should be created for keys or values using [mapKeys](../../../sql-reference/functions/tuple-map-functions.md#mapkeys) or [mapValues](../../../sql-reference/functions/tuple-map-functions.md#mapvalues) function. +For `Map` data type client can specify if index should be created for keys or values using [mapKeys](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapkeys) or [mapValues](/docs/en/sql-reference/functions/tuple-map-functions.md/#mapvalues) function. There are also special-purpose and experimental indexes to support approximate nearest neighbor (ANN) queries. See [here](annindexes.md) for details. -The following functions can use the filter: [equals](../../../sql-reference/functions/comparison-functions.md), [notEquals](../../../sql-reference/functions/comparison-functions.md), [in](../../../sql-reference/functions/in-functions), [notIn](../../../sql-reference/functions/in-functions), [has](../../../sql-reference/functions/array-functions#hasarr-elem), [hasAny](../../../sql-reference/functions/array-functions#hasany), [hasAll](../../../sql-reference/functions/array-functions#hasall). +The following functions can use the filter: [equals](/docs/en/sql-reference/functions/comparison-functions.md), [notEquals](/docs/en/sql-reference/functions/comparison-functions.md), [in](/docs/en/sql-reference/functions/in-functions), [notIn](/docs/en/sql-reference/functions/in-functions), [has](/docs/en/sql-reference/functions/array-functions#hasarr-elem), [hasAny](/docs/en/sql-reference/functions/array-functions#hasany), [hasAll](/docs/en/sql-reference/functions/array-functions#hasall). Example of index creation for `Map` data type @@ -445,21 +445,21 @@ The `set` index can be used with all functions. Function subsets for other index | Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter | |------------------------------------------------------------------------------------------------------------|-------------|--------|-------------|-------------|---------------| -| [equals (=, ==)](../../../sql-reference/functions/comparison-functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notEquals(!=, <>)](../../../sql-reference/functions/comparison-functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [like](../../../sql-reference/functions/string-search-functions.md#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [notLike](../../../sql-reference/functions/string-search-functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [startsWith](../../../sql-reference/functions/string-functions.md#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | -| [endsWith](../../../sql-reference/functions/string-functions.md#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | -| [multiSearchAny](../../../sql-reference/functions/string-search-functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | -| [in](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [notIn](../../../sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | -| [less (<)](../../../sql-reference/functions/comparison-functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greater (>)](../../../sql-reference/functions/comparison-functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [lessOrEquals (<=)](../../../sql-reference/functions/comparison-functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [greaterOrEquals (>=)](../../../sql-reference/functions/comparison-functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [empty](../../../sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | -| [notEmpty](../../../sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [equals (=, ==)](/docs/en/sql-reference/functions/comparison-functions.md/#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notEquals(!=, <>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [like](/docs/en/sql-reference/functions/string-search-functions.md/#function-like) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [notLike](/docs/en/sql-reference/functions/string-search-functions.md/#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [startsWith](/docs/en/sql-reference/functions/string-functions.md/#startswith) | ✔ | ✔ | ✔ | ✔ | ✗ | +| [endsWith](/docs/en/sql-reference/functions/string-functions.md/#endswith) | ✗ | ✗ | ✔ | ✔ | ✗ | +| [multiSearchAny](/docs/en/sql-reference/functions/string-search-functions.md/#function-multisearchany) | ✗ | ✗ | ✔ | ✗ | ✗ | +| [in](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [notIn](/docs/en/sql-reference/functions/in-functions#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ | +| [less (<)](/docs/en/sql-reference/functions/comparison-functions.md/#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greater (>)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [lessOrEquals (<=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [greaterOrEquals (>=)](/docs/en/sql-reference/functions/comparison-functions.md/#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [empty](/docs/en/sql-reference/functions/array-functions#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ | +| [notEmpty](/docs/en/sql-reference/functions/array-functions#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ | | hasToken | ✗ | ✗ | ✗ | ✔ | ✗ | Functions with a constant argument that is less than ngram size can’t be used by `ngrambf_v1` for query optimization. @@ -485,16 +485,16 @@ For example: ## Approximate Nearest Neighbor Search Indexes [experimental] {#table_engines-ANNIndex} -In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](../../../engines/table-engines/mergetree-family/annindexes.md). +In addition to skip indices, there are also [Approximate Nearest Neighbor Search Indexes](/docs/en/engines/table-engines/mergetree-family/annindexes.md). ## Projections {#projections} -Projections are like [materialized views](../../../sql-reference/statements/create/view.md#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. +Projections are like [materialized views](/docs/en/sql-reference/statements/create/view.md/#materialized) but defined in part-level. It provides consistency guarantees along with automatic usage in queries. :::note -When you are implementing projections you should also consider the [force_optimize_projection](../../../operations/settings/settings.md#force-optimize-projection) setting. +When you are implementing projections you should also consider the [force_optimize_projection](/docs/en/operations/settings/settings.md/#force-optimize-projection) setting. ::: -Projections are not supported in the `SELECT` statements with the [FINAL](../../../sql-reference/statements/select/from.md#select-from-final) modifier. +Projections are not supported in the `SELECT` statements with the [FINAL](/docs/en/sql-reference/statements/select/from.md/#select-from-final) modifier. ### Projection Query {#projection-query} A projection query is what defines a projection. It implicitly selects data from the parent table. @@ -504,7 +504,7 @@ A projection query is what defines a projection. It implicitly selects data from SELECT [GROUP BY] [ORDER BY] ``` -Projections can be modified or dropped with the [ALTER](../../../sql-reference/statements/alter/projection.md) statement. +Projections can be modified or dropped with the [ALTER](/docs/en/sql-reference/statements/alter/projection.md) statement. ### Projection Storage {#projection-storage} Projections are stored inside the part directory. It's similar to an index but contains a subdirectory that stores an anonymous `MergeTree` table's part. The table is induced by the definition query of the projection. If there is a `GROUP BY` clause, the underlying storage engine becomes [AggregatingMergeTree](aggregatingmergetree.md), and all aggregate functions are converted to `AggregateFunction`. If there is an `ORDER BY` clause, the `MergeTree` table uses it as its primary key expression. During the merge process the projection part is merged via its storage's merge routine. The checksum of the parent table's part is combined with the projection's part. Other maintenance jobs are similar to skip indices. @@ -526,7 +526,7 @@ Determines the lifetime of values. The `TTL` clause can be set for the whole table and for each individual column. Table-level `TTL` can also specify the logic of automatic moving data between disks and volumes, or recompressing parts where all the data has been expired. -Expressions must evaluate to [Date](../../../sql-reference/data-types/date.md) or [DateTime](../../../sql-reference/data-types/datetime.md) data type. +Expressions must evaluate to [Date](/docs/en/sql-reference/data-types/date.md) or [DateTime](/docs/en/sql-reference/data-types/datetime.md) data type. **Syntax** @@ -537,7 +537,7 @@ TTL time_column TTL time_column + interval ``` -To define `interval`, use [time interval](../../../sql-reference/operators/index.md#operators-datetime) operators, for example: +To define `interval`, use [time interval](/docs/en/sql-reference/operators/index.md/#operators-datetime) operators, for example: ``` sql TTL date_time + INTERVAL 1 MONTH @@ -684,11 +684,11 @@ Data with an expired `TTL` is removed when ClickHouse merges data parts. When ClickHouse detects that data is expired, it performs an off-schedule merge. To control the frequency of such merges, you can set `merge_with_ttl_timeout`. If the value is too low, it will perform many off-schedule merges that may consume a lot of resources. -If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](../../../sql-reference/statements/optimize.md) query before `SELECT`. +If you perform the `SELECT` query between merges, you may get expired data. To avoid it, use the [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) query before `SELECT`. **See Also** -- [ttl_only_drop_parts](../../../operations/settings/settings.md#ttl_only_drop_parts) setting +- [ttl_only_drop_parts](/docs/en/operations/settings/settings.md/#ttl_only_drop_parts) setting ## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} @@ -697,16 +697,16 @@ If you perform the `SELECT` query between merges, you may get expired data. To a `MergeTree` family table engines can store data on multiple block devices. For example, it can be useful when the data of a certain table are implicitly split into “hot” and “cold”. The most recent data is regularly requested but requires only a small amount of space. On the contrary, the fat-tailed historical data is requested rarely. If several disks are available, the “hot” data may be located on fast disks (for example, NVMe SSDs or in memory), while the “cold” data - on relatively slow ones (for example, HDD). -Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../../sql-reference/statements/alter/partition.md#alter_move-partition) queries. +Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition) queries. ### Terms {#terms} - Disk — Block device mounted to the filesystem. -- Default disk — Disk that stores the path specified in the [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) server setting. +- Default disk — Disk that stores the path specified in the [path](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-path) server setting. - Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). - Storage policy — Set of volumes and the rules for moving data between them. -The names given to the described entities can be found in the system tables, [system.storage_policies](../../../operations/system-tables/storage_policies.md#system_tables-storage_policies) and [system.disks](../../../operations/system-tables/disks.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. +The names given to the described entities can be found in the system tables, [system.storage_policies](/docs/en/operations/system-tables/storage_policies.md/#system_tables-storage_policies) and [system.disks](/docs/en/operations/system-tables/disks.md/#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. ### Configuration {#table_engine-mergetree-multiple-volumes_configure} @@ -853,16 +853,16 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' The `default` storage policy implies using only one volume, which consists of only one disk given in ``. You could change storage policy after table creation with [ALTER TABLE ... MODIFY SETTING] query, new policy should include all old disks and volumes with same names. -The number of threads performing background moves of data parts can be changed by [background_move_pool_size](../../../operations/settings/settings.md#background_move_pool_size) setting. +The number of threads performing background moves of data parts can be changed by [background_move_pool_size](/docs/en/operations/settings/settings.md/#background_move_pool_size) setting. ### Details {#details} In the case of `MergeTree` tables, data is getting to disk in different ways: - As a result of an insert (`INSERT` query). -- During background merges and [mutations](../../../sql-reference/statements/alter/index.md#alter-mutations). +- During background merges and [mutations](/docs/en/sql-reference/statements/alter/overview.md/#alter-mutations). - When downloading from another replica. -- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](../../../sql-reference/statements/alter/partition.md#alter_freeze-partition). +- As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy: @@ -872,16 +872,16 @@ In all these cases except for mutations and partition freezing, a part is stored Under the hood, mutations and partition freezing make use of [hard links](https://en.wikipedia.org/wiki/Hard_link). Hard links between different disks are not supported, therefore in such cases the resulting parts are stored on the same disks as the initial ones. In the background, parts are moved between volumes on the basis of the amount of free space (`move_factor` parameter) according to the order the volumes are declared in the configuration file. -Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](../../../operations/system-tables/part_log.md#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](../../../operations/system-tables/parts.md#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. +Data is never transferred from the last one and into the first one. One may use system tables [system.part_log](/docs/en/operations/system-tables/part_log.md/#system_tables-part-log) (field `type = MOVE_PART`) and [system.parts](/docs/en/operations/system-tables/parts.md/#system_tables-parts) (fields `path` and `disk`) to monitor background moves. Also, the detailed information can be found in server logs. -User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](../../../sql-reference/statements/alter/partition.md#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. +User can force moving a part or a partition from one volume to another using the query [ALTER TABLE … MOVE PART\|PARTITION … TO VOLUME\|DISK …](/docs/en/sql-reference/statements/alter/partition.md/#alter_move-partition), all the restrictions for background operations are taken into account. The query initiates a move on its own and does not wait for background operations to be completed. User will get an error message if not enough free space is available or if any of the required conditions are not met. Moving data does not interfere with data replication. Therefore, different storage policies can be specified for the same table on different replicas. After the completion of background merges and mutations, old parts are removed only after a certain amount of time (`old_parts_lifetime`). During this time, they are not moved to other volumes or disks. Therefore, until the parts are finally removed, they are still taken into account for evaluation of the occupied disk space. -User can assign new big parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](../../../operations/settings/merge-tree-settings.md#min-bytes-to-rebalance-partition-over-jbod) setting. +User can assign new big parts to different disks of a [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures) volume in a balanced way using the [min_bytes_to_rebalance_partition_over_jbod](/docs/en/operations/settings/merge-tree-settings.md/#min-bytes-to-rebalance-partition-over-jbod) setting. ## Using S3 for Data Storage {#table_engine-mergetree-s3} diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 06faceab8ec..0e208629c2e 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -20,7 +20,7 @@ Replication works at the level of an individual table, not the entire server. A Replication does not depend on sharding. Each shard has its own independent replication. -Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](../../../sql-reference/statements/alter/index.md#query_language_queries_alter)). +Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/overview.md/#query_language_queries_alter)). `CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: @@ -28,9 +28,9 @@ Compressed data for `INSERT` and `ALTER` queries is replicated (for more informa - The `DROP TABLE` query deletes the replica located on the server where the query is run. - The `RENAME` query renames the table on one of the replicas. In other words, replicated tables can have different names on different replicas. -ClickHouse uses [ClickHouse Keeper](../../../guides/sre/keeper/clickhouse-keeper.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. +ClickHouse uses [ClickHouse Keeper](/docs/en/guides/sre/keeper/clickhouse-keeper.md) for storing replicas meta information. It is possible to use ZooKeeper version 3.4.5 or newer, but ClickHouse Keeper is recommended. -To use replication, set parameters in the [zookeeper](../../../operations/server-configuration-parameters/settings.md#server-settings_zookeeper) server configuration section. +To use replication, set parameters in the [zookeeper](/docs/en/operations/server-configuration-parameters/settings.md/#server-settings_zookeeper) server configuration section. :::warning Don’t neglect the security setting. ClickHouse supports the `digest` [ACL scheme](https://zookeeper.apache.org/doc/current/zookeeperProgrammers.html#sc_ZooKeeperAccessControl) of the ZooKeeper security subsystem. @@ -95,21 +95,21 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn’t set in the config file, you can’t create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../../../operations/settings/settings.md#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../../../operations/settings/settings.md#settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper is not used in `SELECT` queries because replication does not affect the performance of `SELECT` and queries run just as fast as they do for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](/docs/en/operations/settings/settings.md/#settings-fallback_to_stale_replicas_for_distributed_queries). For each `INSERT` query, approximately ten entries are added to ZooKeeper through several transactions. (To be more precise, this is for each inserted block of data; an INSERT query contains one block or one block per `max_insert_block_size = 1048576` rows.) This leads to slightly longer latencies for `INSERT` compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one `INSERT` per second, it does not create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred `INSERTs` per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. For very large clusters, you can use different ZooKeeper clusters for different shards. However, from our experience this has not proven necessary based on production clusters with approximately 300 servers. -Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) setting. +Replication is asynchronous and multi-master. `INSERT` queries (as well as `ALTER`) can be sent to any available server. Data is inserted on the server where the query is run, and then it is copied to the other servers. Because it is asynchronous, recently inserted data appears on the other replicas with some latency. If part of the replicas are not available, the data is written when they become available. If a replica is available, the latency is the amount of time it takes to transfer the block of compressed data over the network. The number of threads performing background tasks for replicated tables can be set by [background_schedule_pool_size](/docs/en/operations/settings/settings.md/#background_schedule_pool_size) setting. -`ReplicatedMergeTree` engine uses a separate thread pool for replicated fetches. Size of the pool is limited by the [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size) setting which can be tuned with a server restart. +`ReplicatedMergeTree` engine uses a separate thread pool for replicated fetches. Size of the pool is limited by the [background_fetches_pool_size](/docs/en/operations/settings/settings.md/#background_fetches_pool_size) setting which can be tuned with a server restart. By default, an INSERT query waits for confirmation of writing the data from only one replica. If the data was successfully written to only one replica and the server with this replica ceases to exist, the stored data will be lost. To enable getting confirmation of data writes from multiple replicas, use the `insert_quorum` option. Each block of data is written atomically. The INSERT query is divided into blocks up to `max_insert_block_size = 1048576` rows. In other words, if the `INSERT` query has less than 1048576 rows, it is made atomically. -Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application does not know if the data was written to the DB, so the `INSERT` query can simply be repeated. It does not matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-merge_tree) server settings. +Data blocks are deduplicated. For multiple writes of the same data block (data blocks of the same size containing the same rows in the same order), the block is only written once. The reason for this is in case of network failures when the client application does not know if the data was written to the DB, so the `INSERT` query can simply be repeated. It does not matter which replica INSERTs were sent to with identical data. `INSERTs` are idempotent. Deduplication parameters are controlled by [merge_tree](/docs/en/operations/server-configuration-parameters/settings.md/#server_configuration_parameters-merge_tree) server settings. During replication, only the source data to insert is transferred over the network. Further data transformation (merging) is coordinated and performed on all the replicas in the same way. This minimizes network usage, which means that replication works well when replicas reside in different datacenters. (Note that duplicating data in different datacenters is the main goal of replication.) @@ -165,7 +165,7 @@ CREATE TABLE table_name -As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the [macros](../../../operations/server-configuration-parameters/settings.md#macros) section of the configuration file. +As the example shows, these parameters can contain substitutions in curly brackets. The substituted values are taken from the [macros](/docs/en/operations/server-configuration-parameters/settings.md/#macros) section of the configuration file. Example: @@ -295,10 +295,10 @@ If the data in ClickHouse Keeper was lost or damaged, you can save data by movin **See Also** -- [background_schedule_pool_size](../../../operations/settings/settings.md#background_schedule_pool_size) -- [background_fetches_pool_size](../../../operations/settings/settings.md#background_fetches_pool_size) -- [execute_merges_on_single_replica_time_threshold](../../../operations/settings/settings.md#execute-merges-on-single-replica-time-threshold) -- [max_replicated_fetches_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth) -- [max_replicated_sends_network_bandwidth](../../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth) +- [background_schedule_pool_size](/docs/en/operations/settings/settings.md/#background_schedule_pool_size) +- [background_fetches_pool_size](/docs/en/operations/settings/settings.md/#background_fetches_pool_size) +- [execute_merges_on_single_replica_time_threshold](/docs/en/operations/settings/settings.md/#execute-merges-on-single-replica-time-threshold) +- [max_replicated_fetches_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth) +- [max_replicated_sends_network_bandwidth](/docs/en/operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth) [Original article](https://clickhouse.com/docs/en/operations/table_engines/replication/) diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 161896e5550..36b2e59bcf8 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -6,10 +6,10 @@ sidebar_label: Join # Join Table Engine -Optional prepared data structure for usage in [JOIN](../../../sql-reference/statements/select/join.md#select-join) operations. +Optional prepared data structure for usage in [JOIN](/docs/en/sql-reference/statements/select/join.md/#select-join) operations. :::note -This is not an article about the [JOIN clause](../../../sql-reference/statements/select/join.md#select-join) itself. +This is not an article about the [JOIN clause](/docs/en/sql-reference/statements/select/join.md/#select-join) itself. ::: ## Creating a Table {#creating-a-table} @@ -22,17 +22,17 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] ) ENGINE = Join(join_strictness, join_type, k1[, k2, ...]) ``` -See the detailed description of the [CREATE TABLE](../../../sql-reference/statements/create/table.md#create-table-query) query. +See the detailed description of the [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md/#create-table-query) query. ## Engine Parameters ### join_strictness -`join_strictness` – [JOIN strictness](../../../sql-reference/statements/select/join.md#select-join-types). +`join_strictness` – [JOIN strictness](/docs/en/sql-reference/statements/select/join.md/#select-join-types). ### join_type -`join_type` – [JOIN type](../../../sql-reference/statements/select/join.md#select-join-types). +`join_type` – [JOIN type](/docs/en/sql-reference/statements/select/join.md/#select-join-types). ### Key columns @@ -55,11 +55,11 @@ You can use `INSERT` queries to add data to the `Join`-engine tables. If the tab Main use-cases for `Join`-engine tables are following: - Place the table to the right side in a `JOIN` clause. -- Call the [joinGet](../../../sql-reference/functions/other-functions.md#joinget) function, which lets you extract data from the table the same way as from a dictionary. +- Call the [joinGet](/docs/en/sql-reference/functions/other-functions.md/#joinget) function, which lets you extract data from the table the same way as from a dictionary. ### Deleting Data {#deleting-data} -`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](../../../sql-reference/statements/alter/index.md#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. +`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. ### Limitations and Settings {#join-limitations-and-settings} @@ -67,30 +67,30 @@ When creating a table, the following settings are applied: #### join_use_nulls -[join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) +[join_use_nulls](/docs/en/operations/settings/settings.md/#join_use_nulls) #### max_rows_in_join -[max_rows_in_join](../../../operations/settings/query-complexity.md#settings-max_rows_in_join) +[max_rows_in_join](/docs/en/operations/settings/query-complexity.md/#settings-max_rows_in_join) #### max_bytes_in_join -[max_bytes_in_join](../../../operations/settings/query-complexity.md#settings-max_bytes_in_join) +[max_bytes_in_join](/docs/en/operations/settings/query-complexity.md/#settings-max_bytes_in_join) #### join_overflow_mode -[join_overflow_mode](../../../operations/settings/query-complexity.md#settings-join_overflow_mode) +[join_overflow_mode](/docs/en/operations/settings/query-complexity.md/#settings-join_overflow_mode) #### join_any_take_last_row -[join_any_take_last_row](../../../operations/settings/settings.md#settings-join_any_take_last_row) +[join_any_take_last_row](/docs/en/operations/settings/settings.md/#settings-join_any_take_last_row) #### join_use_nulls -[persistent](../../../operations/settings/settings.md#persistent) +[persistent](/docs/en/operations/settings/settings.md/#persistent) The `Join`-engine tables can’t be used in `GLOBAL JOIN` operations. -The `Join`-engine allows to specify [join_use_nulls](../../../operations/settings/settings.md#join_use_nulls) setting in the `CREATE TABLE` statement. [SELECT](../../../sql-reference/statements/select/index.md) query should have the same `join_use_nulls` value. +The `Join`-engine allows to specify [join_use_nulls](/docs/en/operations/settings/settings.md/#join_use_nulls) setting in the `CREATE TABLE` statement. [SELECT](/docs/en/sql-reference/statements/select/index.md) query should have the same `join_use_nulls` value. ## Usage Examples {#example} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7f7c14817ba..38077a2b891 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -70,7 +70,7 @@ Another use case of `prefer_global_in_and_join` is accessing tables created by **See also:** -- [Distributed subqueries](../../sql-reference/operators/in.md#select-distributed-subqueries) for more information on how to use `GLOBAL IN`/`GLOBAL JOIN` +- [Distributed subqueries](../../sql-reference/operators/in.md/#select-distributed-subqueries) for more information on how to use `GLOBAL IN`/`GLOBAL JOIN` ## enable_optimize_predicate_expression {#enable-optimize-predicate-expression} @@ -170,7 +170,7 @@ It makes sense to disable it if the server has millions of tiny tables that are ## function_range_max_elements_in_block {#settings-function_range_max_elements_in_block} -Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). +Sets the safety threshold for data volume generated by function [range](../../sql-reference/functions/array-functions.md/#range). Defines the maximum number of values generated by function per block of data (sum of array sizes for every row in a block). Possible values: @@ -273,10 +273,10 @@ Default value: 0. ## insert_null_as_default {#insert_null_as_default} -Enables or disables the insertion of [default values](../../sql-reference/statements/create/table.md#create-default-values) instead of [NULL](../../sql-reference/syntax.md#null-literal) into columns with not [nullable](../../sql-reference/data-types/nullable.md#data_type-nullable) data type. +Enables or disables the insertion of [default values](../../sql-reference/statements/create/table.md/#create-default-values) instead of [NULL](../../sql-reference/syntax.md/#null-literal) into columns with not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable) data type. If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. -This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md#insert_query_insert-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. +This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md/#insert_query_insert-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. Possible values: @@ -287,7 +287,7 @@ Default value: `1`. ## join_default_strictness {#settings-join_default_strictness} -Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md#select-join). +Sets default strictness for [JOIN clauses](../../sql-reference/statements/select/join.md/#select-join). Possible values: @@ -322,7 +322,7 @@ When using `partial_merge` algorithm, ClickHouse sorts the data and dumps it to - `direct` - can be applied when the right storage supports key-value requests. -The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. +The `direct` algorithm performs a lookup in the right table using rows from the left table as keys. It's supported only by special storage such as [Dictionary](../../engines/table-engines/special/dictionary.md/#dictionary) or [EmbeddedRocksDB](../../engines/table-engines/integrations/embedded-rocksdb.md) and only the `LEFT` and `INNER` JOINs. - `auto` — try `hash` join and switch on the fly to another algorithm if the memory limit is violated. @@ -348,7 +348,7 @@ Default value: 0. See also: -- [JOIN clause](../../sql-reference/statements/select/join.md#select-join) +- [JOIN clause](../../sql-reference/statements/select/join.md/#select-join) - [Join table engine](../../engines/table-engines/special/join.md) - [join_default_strictness](#settings-join_default_strictness) @@ -359,7 +359,7 @@ Sets the type of [JOIN](../../sql-reference/statements/select/join.md) behaviour Possible values: - 0 — The empty cells are filled with the default value of the corresponding field type. -- 1 — `JOIN` behaves the same way as in standard SQL. The type of the corresponding field is converted to [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable), and empty cells are filled with [NULL](../../sql-reference/syntax.md). +- 1 — `JOIN` behaves the same way as in standard SQL. The type of the corresponding field is converted to [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable), and empty cells are filled with [NULL](../../sql-reference/syntax.md). Default value: 0. @@ -431,7 +431,7 @@ Default value: 0. See also: -- [JOIN strictness](../../sql-reference/statements/select/join.md#join-settings) +- [JOIN strictness](../../sql-reference/statements/select/join.md/#join-settings) ## temporary_files_codec {#temporary_files_codec} @@ -532,7 +532,7 @@ Default value: 8. If ClickHouse should read more than `merge_tree_max_rows_to_use_cache` rows in one query, it does not use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible values: @@ -544,7 +544,7 @@ Default value: 128 ✕ 8192. If ClickHouse should read more than `merge_tree_max_bytes_to_use_cache` bytes in one query, it does not use the cache of uncompressed blocks. -The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. +The cache of uncompressed blocks stores data extracted for queries. ClickHouse uses this cache to speed up responses to repeated small queries. This setting protects the cache from trashing by queries that read a large amount of data. The [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) server setting defines the size of the cache of uncompressed blocks. Possible values: @@ -594,7 +594,7 @@ Default value: `1`. Setting up query logging. -Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query-log) server configuration parameter. +Queries sent to ClickHouse with this setup are logged according to the rules in the [query_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query-log) server configuration parameter. Example: @@ -639,7 +639,7 @@ log_queries_min_type='EXCEPTION_WHILE_PROCESSING' Setting up query threads logging. -Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_thread_log) server configuration parameter. +Query threads log into [system.query_thread_log](../../operations/system-tables/query_thread_log.md) table. This setting have effect only when [log_queries](#settings-log-queries) is true. Queries’ threads run by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_thread_log) server configuration parameter. Possible values: @@ -658,7 +658,7 @@ log_query_threads=1 Setting up query views logging. -When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-query_views_log) server configuration parameter. +When a query run by ClickHouse with this setup on has associated views (materialized or live views), they are logged in the [query_views_log](../../operations/server-configuration-parameters/settings.md/#server_configuration_parameters-query_views_log) server configuration parameter. Example: @@ -884,7 +884,7 @@ Default value: `5`. ## max_replicated_fetches_network_bandwidth_for_server {#max_replicated_fetches_network_bandwidth_for_server} -Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_fetches_network_bandwidth) setting. +Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) fetches for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_fetches_network_bandwidth](../../operations/settings/merge-tree-settings.md/#max_replicated_fetches_network_bandwidth) setting. The setting isn't followed perfectly accurately. @@ -905,7 +905,7 @@ Could be used for throttling speed when replicating the data to add or replace n ## max_replicated_sends_network_bandwidth_for_server {#max_replicated_sends_network_bandwidth_for_server} -Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md#max_replicated_sends_network_bandwidth) setting. +Limits the maximum speed of data exchange over the network in bytes per second for [replicated](../../engines/table-engines/mergetree-family/replication.md) sends for the server. Only has meaning at server startup. You can also limit the speed for a particular table with [max_replicated_sends_network_bandwidth](../../operations/settings/merge-tree-settings.md/#max_replicated_sends_network_bandwidth) setting. The setting isn't followed perfectly accurately. @@ -955,7 +955,7 @@ For more information, see the section “Extreme values”. ## kafka_max_wait_ms {#kafka-max-wait-ms} -The wait time in milliseconds for reading messages from [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) before retry. +The wait time in milliseconds for reading messages from [Kafka](../../engines/table-engines/integrations/kafka.md/#kafka) before retry. Possible values: @@ -977,7 +977,7 @@ Default value: false. ## use_uncompressed_cache {#setting-use_uncompressed_cache} Whether to use a cache of uncompressed blocks. Accepts 0 or 1. By default, 0 (disabled). -Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. +Using the uncompressed cache (only for tables in the MergeTree family) can significantly reduce latency and increase throughput when working with a large number of short queries. Enable this setting for users who send frequent short requests. Also pay attention to the [uncompressed_cache_size](../../operations/server-configuration-parameters/settings.md/#server-settings-uncompressed_cache_size) configuration parameter (only set in the config file) – the size of uncompressed cache blocks. By default, it is 8 GiB. The uncompressed cache is filled in as needed and the least-used data is automatically deleted. For queries that read at least a somewhat large volume of data (one million rows or more), the uncompressed cache is disabled automatically to save space for truly small queries. This means that you can keep the ‘use_uncompressed_cache’ setting always set to 1. @@ -1124,7 +1124,7 @@ This setting is useful for replicated tables with a sampling key. A query may be - The cluster latency distribution has a long tail, so that querying more servers increases the query overall latency. :::warning -This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md#max_parallel_replica-subqueries) for more details. +This setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain requirements. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. ::: ## compile_expressions {#compile-expressions} @@ -1261,7 +1261,7 @@ Possible values: Default value: 1. By default, blocks inserted into replicated tables by the `INSERT` statement are deduplicated (see [Data Replication](../../engines/table-engines/mergetree-family/replication.md)). -For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). +For the replicated tables by default the only 100 of the most recent blocks for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). ## deduplicate_blocks_in_dependent_materialized_views {#settings-deduplicate-blocks-in-dependent-materialized-views} @@ -1296,7 +1296,7 @@ Default value: empty string (disabled) `insert_deduplication_token` is used for deduplication _only_ when not empty. -For the replicated tables by default the only 100 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). +For the replicated tables by default the only 100 of the most recent inserts for each partition are deduplicated (see [replicated_deduplication_window](merge-tree-settings.md/#replicated-deduplication-window), [replicated_deduplication_window_seconds](merge-tree-settings.md/#replicated-deduplication-window-seconds)). For not replicated tables see [non_replicated_deduplication_window](merge-tree-settings.md/#non-replicated-deduplication-window). Example: @@ -1373,15 +1373,15 @@ Default value: 0. ## count_distinct_implementation {#settings-count_distinct_implementation} -Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) construction. +Specifies which of the `uniq*` functions should be used to perform the [COUNT(DISTINCT …)](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) construction. Possible values: -- [uniq](../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) -- [uniqCombined](../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined) -- [uniqCombined64](../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) -- [uniqHLL12](../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12) -- [uniqExact](../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact) +- [uniq](../../sql-reference/aggregate-functions/reference/uniq.md/#agg_function-uniq) +- [uniqCombined](../../sql-reference/aggregate-functions/reference/uniqcombined.md/#agg_function-uniqcombined) +- [uniqCombined64](../../sql-reference/aggregate-functions/reference/uniqcombined64.md/#agg_function-uniqcombined64) +- [uniqHLL12](../../sql-reference/aggregate-functions/reference/uniqhll12.md/#agg_function-uniqhll12) +- [uniqExact](../../sql-reference/aggregate-functions/reference/uniqexact.md/#agg_function-uniqexact) Default value: `uniqExact`. @@ -1616,14 +1616,14 @@ Enables or disables optimization by transforming some functions to reading subco These functions can be transformed: -- [length](../../sql-reference/functions/array-functions.md#array_functions-length) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [empty](../../sql-reference/functions/array-functions.md#function-empty) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [notEmpty](../../sql-reference/functions/array-functions.md#function-notempty) to read the [size0](../../sql-reference/data-types/array.md#array-size) subcolumn. -- [isNull](../../sql-reference/operators/index.md#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [isNotNull](../../sql-reference/operators/index.md#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [count](../../sql-reference/aggregate-functions/reference/count.md) to read the [null](../../sql-reference/data-types/nullable.md#finding-null) subcolumn. -- [mapKeys](../../sql-reference/functions/tuple-map-functions.md#mapkeys) to read the [keys](../../sql-reference/data-types/map.md#map-subcolumns) subcolumn. -- [mapValues](../../sql-reference/functions/tuple-map-functions.md#mapvalues) to read the [values](../../sql-reference/data-types/map.md#map-subcolumns) subcolumn. +- [length](../../sql-reference/functions/array-functions.md/#array_functions-length) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [empty](../../sql-reference/functions/array-functions.md/#function-empty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [notEmpty](../../sql-reference/functions/array-functions.md/#function-notempty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. +- [isNull](../../sql-reference/operators/index.md/#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [isNotNull](../../sql-reference/operators/index.md/#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [count](../../sql-reference/aggregate-functions/reference/count.md) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) to read the [keys](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. +- [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) to read the [values](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. Possible values: @@ -1782,7 +1782,7 @@ Default value: 1000000000 nanoseconds (once a second). See also: -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} @@ -1805,7 +1805,7 @@ Default value: 1000000000 nanoseconds. See also: -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## allow_introspection_functions {#settings-allow_introspection_functions} @@ -1821,11 +1821,11 @@ Default value: 0. **See Also** - [Sampling Query Profiler](../../operations/optimizing-performance/sampling-query-profiler.md) -- System table [trace_log](../../operations/system-tables/trace_log.md#system_tables-trace_log) +- System table [trace_log](../../operations/system-tables/trace_log.md/#system_tables-trace_log) ## input_format_parallel_parsing {#input-format-parallel-parsing} -Enables or disables order-preserving parallel parsing of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats. +Enables or disables order-preserving parallel parsing of data formats. Supported only for [TSV](../../interfaces/formats.md/#tabseparated), [TKSV](../../interfaces/formats.md/#tskv), [CSV](../../interfaces/formats.md/#csv) and [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) formats. Possible values: @@ -1836,7 +1836,7 @@ Default value: `1`. ## output_format_parallel_formatting {#output-format-parallel-formatting} -Enables or disables parallel formatting of data formats. Supported only for [TSV](../../interfaces/formats.md#tabseparated), [TKSV](../../interfaces/formats.md#tskv), [CSV](../../interfaces/formats.md#csv) and [JSONEachRow](../../interfaces/formats.md#jsoneachrow) formats. +Enables or disables parallel formatting of data formats. Supported only for [TSV](../../interfaces/formats.md/#tabseparated), [TKSV](../../interfaces/formats.md/#tskv), [CSV](../../interfaces/formats.md/#csv) and [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) formats. Possible values: @@ -1878,7 +1878,7 @@ Default value: 0. ## insert_distributed_sync {#insert_distributed_sync} -Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table. +Enables or disables synchronous data insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table. By default, when inserting data into a `Distributed` table, the ClickHouse server sends data to cluster nodes in asynchronous mode. When `insert_distributed_sync=1`, the data is processed synchronously, and the `INSERT` operation succeeds only after all the data is saved on all shards (at least one replica for each shard if `internal_replication` is true). @@ -1891,12 +1891,12 @@ Default value: `0`. **See Also** -- [Distributed Table Engine](../../engines/table-engines/special/distributed.md#distributed) -- [Managing Distributed Tables](../../sql-reference/statements/system.md#query-language-system-distributed) +- [Distributed Table Engine](../../engines/table-engines/special/distributed.md/#distributed) +- [Managing Distributed Tables](../../sql-reference/statements/system.md/#query-language-system-distributed) ## insert_shard_id {#insert_shard_id} -If not `0`, specifies the shard of [Distributed](../../engines/table-engines/special/distributed.md#distributed) table into which the data will be inserted synchronously. +If not `0`, specifies the shard of [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table into which the data will be inserted synchronously. If `insert_shard_id` value is incorrect, the server will throw an exception. @@ -1909,7 +1909,7 @@ SELECT uniq(shard_num) FROM system.clusters WHERE cluster = 'requested_cluster'; Possible values: - 0 — Disabled. -- Any number from `1` to `shards_num` of corresponding [Distributed](../../engines/table-engines/special/distributed.md#distributed) table. +- Any number from `1` to `shards_num` of corresponding [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table. Default value: `0`. @@ -1969,7 +1969,7 @@ Default value: 16. ## background_move_pool_size {#background_move_pool_size} -Sets the number of threads performing background moves of data parts for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. +Sets the number of threads performing background moves of data parts for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes)-engine tables. This setting is applied at the ClickHouse server start and can’t be changed in a user session. Possible values: @@ -1979,7 +1979,7 @@ Default value: 8. ## background_schedule_pool_size {#background_schedule_pool_size} -Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md#server-settings-dns-cache-update-period). This setting is applied at ClickHouse server start and can’t be changed in a user session. +Sets the number of threads performing background tasks for [replicated](../../engines/table-engines/mergetree-family/replication.md) tables, [Kafka](../../engines/table-engines/integrations/kafka.md) streaming, [DNS cache updates](../../operations/server-configuration-parameters/settings.md/#server-settings-dns-cache-update-period). This setting is applied at ClickHouse server start and can’t be changed in a user session. Possible values: @@ -2036,12 +2036,12 @@ Default value: 16. **See Also** -- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine. -- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine. +- [Kafka](../../engines/table-engines/integrations/kafka.md/#kafka) engine. +- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md/#rabbitmq-engine) engine. ## validate_polygons {#validate_polygons} -Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. +Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md/#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. Possible values: @@ -2052,7 +2052,7 @@ Default value: 1. ## transform_null_in {#transform_null_in} -Enables equality of [NULL](../../sql-reference/syntax.md#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. +Enables equality of [NULL](../../sql-reference/syntax.md/#null-literal) values for [IN](../../sql-reference/operators/in.md) operator. By default, `NULL` values can’t be compared because `NULL` means undefined value. Thus, comparison `expr = NULL` must always return `false`. With this setting `NULL = NULL` returns `true` for `IN` operator. @@ -2106,7 +2106,7 @@ Result: **See Also** -- [NULL Processing in IN Operators](../../sql-reference/operators/in.md#in-null-processing) +- [NULL Processing in IN Operators](../../sql-reference/operators/in.md/#in-null-processing) ## low_cardinality_max_dictionary_size {#low_cardinality_max_dictionary_size} @@ -2133,7 +2133,7 @@ Default value: 0. ## low_cardinality_allow_in_native_format {#low_cardinality_allow_in_native_format} -Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md#native) format. +Allows or restricts using the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) data type with the [Native](../../interfaces/formats.md/#native) format. If usage of `LowCardinality` is restricted, ClickHouse server converts `LowCardinality`-columns to ordinary ones for `SELECT` queries, and convert ordinary columns to `LowCardinality`-columns for `INSERT` queries. @@ -2197,7 +2197,7 @@ Default value: 268435456. ## optimize_read_in_order {#optimize_read_in_order} -Enables [ORDER BY](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries for reading data from [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. +Enables [ORDER BY](../../sql-reference/statements/select/order-by.md/#optimize_read_in_order) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries for reading data from [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables. Possible values: @@ -2208,7 +2208,7 @@ Default value: `1`. **See Also** -- [ORDER BY Clause](../../sql-reference/statements/select/order-by.md#optimize_read_in_order) +- [ORDER BY Clause](../../sql-reference/statements/select/order-by.md/#optimize_read_in_order) ## optimize_aggregation_in_order {#optimize_aggregation_in_order} @@ -2223,11 +2223,11 @@ Default value: `0`. **See Also** -- [GROUP BY optimization](../../sql-reference/statements/select/group-by.md#aggregation-in-order) +- [GROUP BY optimization](../../sql-reference/statements/select/group-by.md/#aggregation-in-order) ## mutations_sync {#mutations_sync} -Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md#mutations)) synchronously. +Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/overview.md/#mutations)) synchronously. Possible values: @@ -2239,8 +2239,8 @@ Default value: `0`. **See Also** -- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [Mutations](../../sql-reference/statements/alter/index.md#mutations) +- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) +- [Mutations](../../sql-reference/statements/alter/overview.md/#mutations) ## ttl_only_drop_parts {#ttl_only_drop_parts} @@ -2261,8 +2261,8 @@ Default value: `0`. **See Also** -- [CREATE TABLE query clauses and settings](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-query-clauses) (`merge_with_ttl_timeout` setting) -- [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md#mergetree-table-ttl) +- [CREATE TABLE query clauses and settings](../../engines/table-engines/mergetree-family/mergetree.md/#mergetree-query-clauses) (`merge_with_ttl_timeout` setting) +- [Table TTL](../../engines/table-engines/mergetree-family/mergetree.md/#mergetree-table-ttl) ## lock_acquire_timeout {#lock_acquire_timeout} @@ -2279,7 +2279,7 @@ Default value: `120` seconds. ## cast_keep_nullable {#cast_keep_nullable} -Enables or disables keeping of the `Nullable` data type in [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) operations. +Enables or disables keeping of the `Nullable` data type in [CAST](../../sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) operations. When the setting is enabled and the argument of `CAST` function is `Nullable`, the result is also transformed to `Nullable` type. When the setting is disabled, the result always has the destination type exactly. @@ -2324,7 +2324,7 @@ Result: **See Also** -- [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast) function +- [CAST](../../sql-reference/functions/type-conversion-functions.md/#type_conversion_function-cast) function ## system_events_show_zero_values {#system_events_show_zero_values} @@ -2369,7 +2369,7 @@ Result ## persistent {#persistent} -Disables persistency for the [Set](../../engines/table-engines/special/set.md#set) and [Join](../../engines/table-engines/special/join.md#join) table engines. +Disables persistency for the [Set](../../engines/table-engines/special/set.md/#set) and [Join](../../engines/table-engines/special/join.md/#join) table engines. Reduces the I/O overhead. Suitable for scenarios that pursue performance and do not require persistence. @@ -2382,7 +2382,7 @@ Default value: `1`. ## allow_nullable_key {#allow-nullable-key} -Allows using of the [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engines-mergetree) tables. +Allows using of the [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable)-typed values in a sorting and a primary key for [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md/#table_engines-mergetree) tables. Possible values: @@ -2401,7 +2401,7 @@ Do not enable this feature in version `<= 21.8`. It's not properly implemented a ## aggregate_functions_null_for_empty {#aggregate_functions_null_for_empty} -Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. +Enables or disables rewriting all aggregate functions in a query, adding [-OrNull](../../sql-reference/aggregate-functions/combinators.md/#agg-functions-combinator-ornull) suffix to them. Enable it for SQL standard compatibility. It is implemented via query rewrite (similar to [count_distinct_implementation](#settings-count_distinct_implementation) setting) to get consistent results for distributed queries. Possible values: @@ -2448,7 +2448,7 @@ See examples in [UNION](../../sql-reference/statements/select/union.md). ## data_type_default_nullable {#data_type_default_nullable} -Allows data types without explicit modifiers [NULL or NOT NULL](../../sql-reference/statements/create/table.md#null-modifiers) in column definition will be [Nullable](../../sql-reference/data-types/nullable.md#data_type-nullable). +Allows data types without explicit modifiers [NULL or NOT NULL](../../sql-reference/statements/create/table.md/#null-modifiers) in column definition will be [Nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable). Possible values: @@ -2478,7 +2478,7 @@ It can be useful when merges are CPU bounded not IO bounded (performing heavy da ## max_final_threads {#max-final-threads} -Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. +Sets the maximum number of parallel threads for the `SELECT` query data read phase with the [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. Possible values: @@ -2551,7 +2551,7 @@ Result: └─────────────┘ ``` -Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) behaviour. +Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md/#materialized) and [MaterializedMySQL](../../engines/database-engines/materialized-mysql.md) behaviour. ## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists} @@ -2608,7 +2608,7 @@ Default value: `0`. ## allow_experimental_live_view {#allow-experimental-live-view} -Allows creation of experimental [live views](../../sql-reference/statements/create/view.md#live-view). +Allows creation of experimental [live views](../../sql-reference/statements/create/view.md/#live-view). Possible values: @@ -2619,19 +2619,19 @@ Default value: `0`. ## live_view_heartbeat_interval {#live-view-heartbeat-interval} -Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md#live-view) is alive . +Sets the heartbeat interval in seconds to indicate [live view](../../sql-reference/statements/create/view.md/#live-view) is alive . Default value: `15`. ## max_live_view_insert_blocks_before_refresh {#max-live-view-insert-blocks-before-refresh} -Sets the maximum number of inserted blocks after which mergeable blocks are dropped and query for [live view](../../sql-reference/statements/create/view.md#live-view) is re-executed. +Sets the maximum number of inserted blocks after which mergeable blocks are dropped and query for [live view](../../sql-reference/statements/create/view.md/#live-view) is re-executed. Default value: `64`. ## periodic_live_view_refresh {#periodic-live-view-refresh} -Sets the interval in seconds after which periodically refreshed [live view](../../sql-reference/statements/create/view.md#live-view) is forced to refresh. +Sets the interval in seconds after which periodically refreshed [live view](../../sql-reference/statements/create/view.md/#live-view) is forced to refresh. Default value: `60`. @@ -2670,7 +2670,7 @@ Default value: 180. ## check_query_single_value_result {#check_query_single_value_result} -Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md#checking-mergetree-tables) query result for `MergeTree` family engines . +Defines the level of detail for the [CHECK TABLE](../../sql-reference/statements/check-table.md/#checking-mergetree-tables) query result for `MergeTree` family engines . Possible values: @@ -2681,7 +2681,7 @@ Default value: `0`. ## prefer_column_name_to_alias {#prefer-column-name-to-alias} -Enables or disables using the original column names instead of aliases in query expressions and clauses. It especially matters when alias is the same as the column name, see [Expression Aliases](../../sql-reference/syntax.md#notes-on-usage). Enable this setting to make aliases syntax rules in ClickHouse more compatible with most other database engines. +Enables or disables using the original column names instead of aliases in query expressions and clauses. It especially matters when alias is the same as the column name, see [Expression Aliases](../../sql-reference/syntax.md/#notes-on-usage). Enable this setting to make aliases syntax rules in ClickHouse more compatible with most other database engines. Possible values: @@ -2725,7 +2725,7 @@ Result: ## limit {#limit} -Sets the maximum number of rows to get from the query result. It adjusts the value set by the [LIMIT](../../sql-reference/statements/select/limit.md#limit-clause) clause, so that the limit, specified in the query, cannot exceed the limit, set by this setting. +Sets the maximum number of rows to get from the query result. It adjusts the value set by the [LIMIT](../../sql-reference/statements/select/limit.md/#limit-clause) clause, so that the limit, specified in the query, cannot exceed the limit, set by this setting. Possible values: @@ -2736,7 +2736,7 @@ Default value: `0`. ## offset {#offset} -Sets the number of rows to skip before starting to return rows from the query. It adjusts the offset set by the [OFFSET](../../sql-reference/statements/select/offset.md#offset-fetch) clause, so that these two values are summarized. +Sets the number of rows to skip before starting to return rows from the query. It adjusts the offset set by the [OFFSET](../../sql-reference/statements/select/offset.md/#offset-fetch) clause, so that these two values are summarized. Possible values: @@ -2773,7 +2773,7 @@ Result: ## optimize_syntax_fuse_functions {#optimize_syntax_fuse_functions} -Enables to fuse aggregate functions with identical argument. It rewrites query contains at least two aggregate functions from [sum](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md#agg_function-count) or [avg](../../sql-reference/aggregate-functions/reference/avg.md#agg_function-avg) with identical argument to [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md#agg_function-sumCount). +Enables to fuse aggregate functions with identical argument. It rewrites query contains at least two aggregate functions from [sum](../../sql-reference/aggregate-functions/reference/sum.md/#agg_function-sum), [count](../../sql-reference/aggregate-functions/reference/count.md/#agg_function-count) or [avg](../../sql-reference/aggregate-functions/reference/avg.md/#agg_function-avg) with identical argument to [sumCount](../../sql-reference/aggregate-functions/reference/sumcount.md/#agg_function-sumCount). Possible values: @@ -2932,7 +2932,7 @@ If the setting is set to `0`, the table function does not make Nullable columns ## allow_experimental_projection_optimization {#allow-experimental-projection-optimization} -Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md#projections) optimization when processing `SELECT` queries. +Enables or disables [projection](../../engines/table-engines/mergetree-family/mergetree.md/#projections) optimization when processing `SELECT` queries. Possible values: @@ -2943,7 +2943,7 @@ Default value: `1`. ## force_optimize_projection {#force-optimize-projection} -Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting). +Enables or disables the obligatory use of [projections](../../engines/table-engines/mergetree-family/mergetree.md/#projections) in `SELECT` queries, when projection optimization is enabled (see [allow_experimental_projection_optimization](#allow-experimental-projection-optimization) setting). Possible values: @@ -2954,7 +2954,7 @@ Default value: `0`. ## replication_alter_partitions_sync {#replication-alter-partitions-sync} -Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. +Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/overview.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. Possible values: @@ -2966,7 +2966,7 @@ Default value: `1`. ## replication_wait_for_inactive_replica_timeout {#replication-wait-for-inactive-replica-timeout} -Specifies how long (in seconds) to wait for inactive replicas to execute [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. +Specifies how long (in seconds) to wait for inactive replicas to execute [ALTER](../../sql-reference/statements/alter/overview.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. Possible values: @@ -2978,7 +2978,7 @@ Default value: `120` seconds. ## regexp_max_matches_per_row {#regexp-max-matches-per-row} -Sets the maximum number of matches for a single regular expression per row. Use it to protect against memory overload when using greedy regular expression in the [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md#extractallgroups-horizontal) function. +Sets the maximum number of matches for a single regular expression per row. Use it to protect against memory overload when using greedy regular expression in the [extractAllGroupsHorizontal](../../sql-reference/functions/string-search-functions.md/#extractallgroups-horizontal) function. Possible values: @@ -3010,7 +3010,7 @@ Default value: `1`. ## short_circuit_function_evaluation {#short-circuit-function-evaluation} -Allows calculating the [if](../../sql-reference/functions/conditional-functions.md#if), [multiIf](../../sql-reference/functions/conditional-functions.md#multiif), [and](../../sql-reference/functions/logical-functions.md#logical-and-function), and [or](../../sql-reference/functions/logical-functions.md#logical-or-function) functions according to a [short scheme](https://en.wikipedia.org/wiki/Short-circuit_evaluation). This helps optimize the execution of complex expressions in these functions and prevent possible exceptions (such as division by zero when it is not expected). +Allows calculating the [if](../../sql-reference/functions/conditional-functions.md/#if), [multiIf](../../sql-reference/functions/conditional-functions.md/#multiif), [and](../../sql-reference/functions/logical-functions.md/#logical-and-function), and [or](../../sql-reference/functions/logical-functions.md/#logical-or-function) functions according to a [short scheme](https://en.wikipedia.org/wiki/Short-circuit_evaluation). This helps optimize the execution of complex expressions in these functions and prevent possible exceptions (such as division by zero when it is not expected). Possible values: @@ -3022,7 +3022,7 @@ Default value: `enable`. ## max_hyperscan_regexp_length {#max-hyperscan-regexp-length} -Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Defines the maximum length for each regular expression in the [hyperscan multi-match functions](../../sql-reference/functions/string-search-functions.md/#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: @@ -3065,7 +3065,7 @@ Exception: Regexp length too large. ## max_hyperscan_regexp_total_length {#max-hyperscan-regexp-total-length} -Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md#multimatchanyhaystack-pattern1-pattern2-patternn). +Sets the maximum length total of all regular expressions in each [hyperscan multi-match function](../../sql-reference/functions/string-search-functions.md/#multimatchanyhaystack-pattern1-pattern2-patternn). Possible values: @@ -3142,8 +3142,8 @@ Result: ## enable_extended_results_for_datetime_functions {#enable-extended-results-for-datetime-functions} Enables or disables returning results of type: -- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md#tolastdayofmonth). -- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md#timeslot). +- `Date32` with extended range (compared to type `Date`) for functions [toStartOfYear](../../sql-reference/functions/date-time-functions.md/#tostartofyear), [toStartOfISOYear](../../sql-reference/functions/date-time-functions.md/#tostartofisoyear), [toStartOfQuarter](../../sql-reference/functions/date-time-functions.md/#tostartofquarter), [toStartOfMonth](../../sql-reference/functions/date-time-functions.md/#tostartofmonth), [toStartOfWeek](../../sql-reference/functions/date-time-functions.md/#tostartofweek), [toMonday](../../sql-reference/functions/date-time-functions.md/#tomonday) and [toLastDayOfMonth](../../sql-reference/functions/date-time-functions.md/#tolastdayofmonth). +- `DateTime64` with extended range (compared to type `DateTime`) for functions [toStartOfDay](../../sql-reference/functions/date-time-functions.md/#tostartofday), [toStartOfHour](../../sql-reference/functions/date-time-functions.md/#tostartofhour), [toStartOfMinute](../../sql-reference/functions/date-time-functions.md/#tostartofminute), [toStartOfFiveMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffiveminutes), [toStartOfTenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoftenminutes), [toStartOfFifteenMinutes](../../sql-reference/functions/date-time-functions.md/#tostartoffifteenminutes) and [timeSlot](../../sql-reference/functions/date-time-functions.md/#timeslot). Possible values: @@ -3167,7 +3167,7 @@ Default value: `1`. ## optimize_move_to_prewhere_if_final {#optimize_move_to_prewhere_if_final} -Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md#select-from-final) modifier. +Enables or disables automatic [PREWHERE](../../sql-reference/statements/select/prewhere.md) optimization in [SELECT](../../sql-reference/statements/select/index.md) queries with [FINAL](../../sql-reference/statements/select/from.md/#select-from-final) modifier. Works only for [*MergeTree](../../engines/table-engines/mergetree-family/index.md) tables. @@ -3184,7 +3184,7 @@ Default value: `0`. ## describe_include_subcolumns {#describe_include_subcolumns} -Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md#finding-null) or an [Array](../../sql-reference/data-types/array.md#array-size) data type. +Enables describing subcolumns for a [DESCRIBE](../../sql-reference/statements/describe-table.md) query. For example, members of a [Tuple](../../sql-reference/data-types/tuple.md) or subcolumns of a [Map](../../sql-reference/data-types/map.md/#map-subcolumns), [Nullable](../../sql-reference/data-types/nullable.md/#finding-null) or an [Array](../../sql-reference/data-types/array.md/#array-size) data type. Possible values: @@ -3283,7 +3283,7 @@ Default value: `0`. ## alter_partition_verbose_result {#alter-partition-verbose-result} Enables or disables the display of information about the parts to which the manipulation operations with partitions and parts have been successfully applied. -Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alter/partition.md#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition). +Applicable to [ATTACH PARTITION|PART](../../sql-reference/statements/alter/partition.md/#alter_attach-partition) and to [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md/#alter_freeze-partition). Possible values: @@ -3418,11 +3418,11 @@ When writing data, ClickHouse throws an exception if input data contain columns Supported formats: -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) -- [TSKV](../../interfaces/formats.md#tskv) +- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) +- [TSKV](../../interfaces/formats.md/#tskv) - All formats with suffixes WithNames/WithNamesAndTypes -- [JSONColumns](../../interfaces/formats.md#jsoncolumns) -- [MySQLDump](../../interfaces/formats.md#mysqldump) +- [JSONColumns](../../interfaces/formats.md/#jsoncolumns) +- [MySQLDump](../../interfaces/formats.md/#mysqldump) Possible values: @@ -3439,18 +3439,18 @@ To improve insert performance, we recommend disabling this check if you are sure Supported formats: -- [CSVWithNames](../../interfaces/formats.md#csvwithnames) -- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) -- [TabSeparatedWithNames](../../interfaces/formats.md#tabseparatedwithnames) -- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) -- [JSONCompactEachRowWithNames](../../interfaces/formats.md#jsoncompacteachrowwithnames) -- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) -- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md#jsoncompactstringseachrowwithnames) -- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) -- [RowBinaryWithNames](../../interfaces/formats.md#rowbinarywithnames) -- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes) -- [CustomSeparatedWithNames](../../interfaces/formats.md#customseparatedwithnames) -- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) +- [CSVWithNames](../../interfaces/formats.md/#csvwithnames) +- [CSVWithNamesAndTypes](../../interfaces/formats.md/#csvwithnamesandtypes) +- [TabSeparatedWithNames](../../interfaces/formats.md/#tabseparatedwithnames) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md/#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNames](../../interfaces/formats.md/#jsoncompacteachrowwithnames) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNames](../../interfaces/formats.md/#jsoncompactstringseachrowwithnames) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNames](../../interfaces/formats.md/#rowbinarywithnames) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md/#rowbinarywithnamesandtypes) +- [CustomSeparatedWithNames](../../interfaces/formats.md/#customseparatedwithnames) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md/#customseparatedwithnamesandtypes) Possible values: @@ -3465,12 +3465,12 @@ Controls whether format parser should check if data types from the input data ma Supported formats: -- [CSVWithNamesAndTypes](../../interfaces/formats.md#csvwithnamesandtypes) -- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md#tabseparatedwithnamesandtypes) -- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompacteachrowwithnamesandtypes) -- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md#jsoncompactstringseachrowwithnamesandtypes) -- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) -- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md#customseparatedwithnamesandtypes) +- [CSVWithNamesAndTypes](../../interfaces/formats.md/#csvwithnamesandtypes) +- [TabSeparatedWithNamesAndTypes](../../interfaces/formats.md/#tabseparatedwithnamesandtypes) +- [JSONCompactEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompacteachrowwithnamesandtypes) +- [JSONCompactStringsEachRowWithNamesAndTypes](../../interfaces/formats.md/#jsoncompactstringseachrowwithnamesandtypes) +- [RowBinaryWithNamesAndTypes](../../interfaces/formats.md/#rowbinarywithnamesandtypes-rowbinarywithnamesandtypes) +- [CustomSeparatedWithNamesAndTypes](../../interfaces/formats.md/#customseparatedwithnamesandtypes) Possible values: @@ -3481,7 +3481,7 @@ Default value: 1. ## input_format_defaults_for_omitted_fields {#input_format_defaults_for_omitted_fields} -When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md#jsoneachrow), [CSV](../../interfaces/formats.md#csv), [TabSeparated](../../interfaces/formats.md#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. +When performing `INSERT` queries, replace omitted input column values with default values of the respective columns. This option only applies to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow), [CSV](../../interfaces/formats.md/#csv), [TabSeparated](../../interfaces/formats.md/#tabseparated) formats and formats with `WithNames`/`WithNamesAndTypes` suffixes. :::note When this option is enabled, extended table metadata are sent from server to client. It consumes additional computing resources on the server and can reduce performance. @@ -3496,7 +3496,7 @@ Default value: 1. ## input_format_null_as_default {#input_format_null_as_default} -Enables or disables the initialization of [NULL](../../sql-reference/syntax.md#null-literal) fields with [default values](../../sql-reference/statements/create/table.md#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md#data_type-nullable). +Enables or disables the initialization of [NULL](../../sql-reference/syntax.md/#null-literal) fields with [default values](../../sql-reference/statements/create/table.md/#create-default-values), if data type of these fields is not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable). If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. This setting is applicable to [INSERT ... VALUES](../../sql-reference/statements/insert-into.md) queries for text input formats. @@ -3663,7 +3663,7 @@ Enabled by default ## insert_distributed_one_random_shard {#insert_distributed_one_random_shard} -Enables or disables random shard insertion into a [Distributed](../../engines/table-engines/special/distributed.md#distributed) table when there is no distributed key. +Enables or disables random shard insertion into a [Distributed](../../engines/table-engines/special/distributed.md/#distributed) table when there is no distributed key. By default, when inserting data into a `Distributed` table with more than one shard, the ClickHouse server will reject any insertion request if there is no distributed key. When `insert_distributed_one_random_shard = 1`, insertions are allowed and data is forwarded randomly among all shards. @@ -3682,7 +3682,7 @@ Enables or disables the insertion of JSON data with nested objects. Supported formats: -- [JSONEachRow](../../interfaces/formats.md#jsoneachrow) +- [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) Possible values: @@ -3693,7 +3693,7 @@ Default value: 0. See also: -- [Usage of Nested Structures](../../interfaces/formats.md#jsoneachrow-nested) with the `JSONEachRow` format. +- [Usage of Nested Structures](../../interfaces/formats.md/#jsoneachrow-nested) with the `JSONEachRow` format. ### input_format_json_read_bools_as_numbers {#input_format_json_read_bools_as_numbers} @@ -3716,7 +3716,7 @@ Enabled by default. ### output_format_json_quote_64bit_integers {#output_format_json_quote_64bit_integers} -Controls quoting of 64-bit or bigger [integers](../../sql-reference/data-types/int-uint.md) (like `UInt64` or `Int128`) when they are output in a [JSON](../../interfaces/formats.md#json) format. +Controls quoting of 64-bit or bigger [integers](../../sql-reference/data-types/int-uint.md) (like `UInt64` or `Int128`) when they are output in a [JSON](../../interfaces/formats.md/#json) format. Such integers are enclosed in quotes by default. This behavior is compatible with most JavaScript implementations. Possible values: @@ -3734,7 +3734,7 @@ Disabled by default. ### output_format_json_quote_denormals {#output_format_json_quote_denormals} -Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md#json) output format. +Enables `+nan`, `-nan`, `+inf`, `-inf` outputs in [JSON](../../interfaces/formats.md/#json) output format. Possible values: @@ -3851,7 +3851,7 @@ Disabled by default. ### output_format_json_array_of_rows {#output_format_json_array_of_rows} -Enables the ability to output all rows as a JSON array in the [JSONEachRow](../../interfaces/formats.md#jsoneachrow) format. +Enables the ability to output all rows as a JSON array in the [JSONEachRow](../../interfaces/formats.md/#jsoneachrow) format. Possible values: @@ -3904,7 +3904,7 @@ Disabled by default. ### format_json_object_each_row_column_for_object_name {#format_json_object_each_row_column_for_object_name} -The name of column that will be used for storing/writing object names in [JSONObjectEachRow](../../interfaces/formats.md#jsonobjecteachrow) format. +The name of column that will be used for storing/writing object names in [JSONObjectEachRow](../../interfaces/formats.md/#jsonobjecteachrow) format. Column type should be String. If value is empty, default names `row_{i}`will be used for object names. Default value: ''. @@ -4005,7 +4005,7 @@ Disabled by default. ### format_tsv_null_representation {#format_tsv_null_representation} -Defines the representation of `NULL` for [TSV](../../interfaces/formats.md#tabseparated) output and input formats. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [TSV](../../interfaces/formats.md/#tabseparated) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -4159,7 +4159,7 @@ Default value: `0`. ### format_csv_null_representation {#format_csv_null_representation} -Defines the representation of `NULL` for [CSV](../../interfaces/formats.md#csv) output and input formats. User can set any string as a value, for example, `My NULL`. +Defines the representation of `NULL` for [CSV](../../interfaces/formats.md/#csv) output and input formats. User can set any string as a value, for example, `My NULL`. Default value: `\N`. @@ -4198,7 +4198,7 @@ My NULL ### input_format_values_interpret_expressions {#input_format_values_interpret_expressions} -Enables or disables the full SQL parser if the fast stream parser can’t parse the data. This setting is used only for the [Values](../../interfaces/formats.md#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section. +Enables or disables the full SQL parser if the fast stream parser can’t parse the data. This setting is used only for the [Values](../../interfaces/formats.md/#data-format-values) format at the data insertion. For more information about syntax parsing, see the [Syntax](../../sql-reference/syntax.md) section. Possible values: @@ -4248,7 +4248,7 @@ Ok. ### input_format_values_deduce_templates_of_expressions {#input_format_values_deduce_templates_of_expressions} -Enables or disables template deduction for SQL expressions in [Values](../../interfaces/formats.md#data-format-values) format. It allows parsing and interpreting expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse tries to deduce the template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. +Enables or disables template deduction for SQL expressions in [Values](../../interfaces/formats.md/#data-format-values) format. It allows parsing and interpreting expressions in `Values` much faster if expressions in consecutive rows have the same structure. ClickHouse tries to deduce the template of an expression, parse the following rows using this template and evaluate the expression on a batch of successfully parsed rows. Possible values: @@ -4293,7 +4293,7 @@ Default value: 1. ### input_format_arrow_import_nested {#input_format_arrow_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md#data_types-matching-arrow) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Arrow](../../interfaces/formats.md/#data_types-matching-arrow) input format. Possible values: @@ -4322,7 +4322,7 @@ Disabled by default. ### output_format_arrow_low_cardinality_as_dictionary {#output_format_arrow_low_cardinality_as_dictionary} -Allows to convert the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) type to the `DICTIONARY` type of the [Arrow](../../interfaces/formats.md#data-format-arrow) format for `SELECT` queries. +Allows to convert the [LowCardinality](../../sql-reference/data-types/lowcardinality.md) type to the `DICTIONARY` type of the [Arrow](../../interfaces/formats.md/#data-format-arrow) format for `SELECT` queries. Possible values: @@ -4341,7 +4341,7 @@ Disabled by default. ### input_format_orc_import_nested {#input_format_orc_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md#data-format-orc) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [ORC](../../interfaces/formats.md/#data-format-orc) input format. Possible values: @@ -4384,7 +4384,7 @@ Disabled by default. ## input_format_parquet_import_nested {#input_format_parquet_import_nested} -Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md#data-format-parquet) input format. +Enables or disables the ability to insert the data into [Nested](../../sql-reference/data-types/nested-data-structures/nested.md) columns as an array of structs in [Parquet](../../interfaces/formats.md/#data-format-parquet) input format. Possible values: @@ -4481,7 +4481,7 @@ Disabled by default. ### input_format_avro_allow_missing_fields {#input_format_avro_allow_missing_fields} -Enables using fields that are not specified in [Avro](../../interfaces/formats.md#data-format-avro) or [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format schema. When a field is not found in the schema, ClickHouse uses the default value instead of throwing an exception. +Enables using fields that are not specified in [Avro](../../interfaces/formats.md/#data-format-avro) or [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format schema. When a field is not found in the schema, ClickHouse uses the default value instead of throwing an exception. Possible values: @@ -4492,7 +4492,7 @@ Default value: 0. ### format_avro_schema_registry_url {#format_avro_schema_registry_url} -Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent) format. +Sets [Confluent Schema Registry](https://docs.confluent.io/current/schema-registry/index.html) URL to use with [AvroConfluent](../../interfaces/formats.md/#data-format-avro-confluent) format. Default value: `Empty`. @@ -4549,7 +4549,7 @@ Default value: `250`. ### output_format_pretty_max_value_width {#output_format_pretty_max_value_width} -Limits the width of value displayed in [Pretty](../../interfaces/formats.md#pretty) formats. If the value width exceeds the limit, the value is cut. +Limits the width of value displayed in [Pretty](../../interfaces/formats.md/#pretty) formats. If the value width exceeds the limit, the value is cut. Possible values: @@ -4625,7 +4625,7 @@ SELECT * FROM a; ### output_format_pretty_row_numbers {#output_format_pretty_row_numbers} -Adds row numbers to output in the [Pretty](../../interfaces/formats.md#pretty) format. +Adds row numbers to output in the [Pretty](../../interfaces/formats.md/#pretty) format. Possible values: @@ -4670,52 +4670,52 @@ Delimiter between rows (for Template format). ### format_custom_escaping_rule {#format_custom_escaping_rule} -Sets the field escaping rule for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the field escaping rule for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Possible values: -- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md#tabseparated). -- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md#data-format-values). -- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md#csv). -- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md#jsoneachrow). -- `'XML'` — Similarly to [XML](../../interfaces/formats.md#xml). -- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md#tabseparatedraw). +- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated). +- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values). +- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv). +- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow). +- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml). +- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw). Default value: `'Escaped'`. ### format_custom_field_delimiter {#format_custom_field_delimiter} -Sets the character that is interpreted as a delimiter between the fields for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter between the fields for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `'\t'`. ### format_custom_row_before_delimiter {#format_custom_row_before_delimiter} -Sets the character that is interpreted as a delimiter before the field of the first column for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter before the field of the first column for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_row_after_delimiter {#format_custom_row_after_delimiter} -Sets the character that is interpreted as a delimiter after the field of the last column for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter after the field of the last column for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `'\n'`. ### format_custom_row_between_delimiter {#format_custom_row_between_delimiter} -Sets the character that is interpreted as a delimiter between the rows for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a delimiter between the rows for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_result_before_delimiter {#format_custom_result_before_delimiter} -Sets the character that is interpreted as a prefix before the result set for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a prefix before the result set for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. ### format_custom_result_after_delimiter {#format_custom_result_after_delimiter} -Sets the character that is interpreted as a suffix after the result set for [CustomSeparated](../../interfaces/formats.md#format-customseparated) data format. +Sets the character that is interpreted as a suffix after the result set for [CustomSeparated](../../interfaces/formats.md/#format-customseparated) data format. Default value: `''`. @@ -4727,12 +4727,12 @@ Field escaping rule. Possible values: -- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md#tabseparated). -- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md#data-format-values). -- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md#csv). -- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md#jsoneachrow). -- `'XML'` — Similarly to [XML](../../interfaces/formats.md#xml). -- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md#tabseparatedraw). +- `'Escaped'` — Similarly to [TSV](../../interfaces/formats.md/#tabseparated). +- `'Quoted'` — Similarly to [Values](../../interfaces/formats.md/#data-format-values). +- `'CSV'` — Similarly to [CSV](../../interfaces/formats.md/#csv). +- `'JSON'` — Similarly to [JSONEachRow](../../interfaces/formats.md/#jsoneachrow). +- `'XML'` — Similarly to [XML](../../interfaces/formats.md/#xml). +- `'Raw'` — Extracts subpatterns as a whole, no escaping rules, similarly to [TSVRaw](../../interfaces/formats.md/#tabseparatedraw). Default value: `Raw`. @@ -4746,7 +4746,7 @@ Disabled by default. ### format_capn_proto_enum_comparising_mode {#format_capn_proto_enum_comparising_mode} -Determines how to map ClickHouse `Enum` data type and [CapnProto](../../interfaces/formats.md#capnproto) `Enum` data type from schema. +Determines how to map ClickHouse `Enum` data type and [CapnProto](../../interfaces/formats.md/#capnproto) `Enum` data type from schema. Possible values: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index 43623577e66..c0acd544fa9 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -7,13 +7,13 @@ title: "External Disks for Storing Data" Data, processed in ClickHouse, is usually stored in the local file system — on the same machine with the ClickHouse server. That requires large-capacity disks, which can be expensive enough. To avoid that you can store the data remotely — on [Amazon S3](https://aws.amazon.com/s3/) disks or in the Hadoop Distributed File System ([HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.html)). -To work with data stored on `Amazon S3` disks use [S3](../engines/table-engines/integrations/s3.md) table engine, and to work with data in the Hadoop Distributed File System — [HDFS](../engines/table-engines/integrations/hdfs.md) table engine. +To work with data stored on `Amazon S3` disks use [S3](/docs/en/engines/table-engines/integrations/s3.md) table engine, and to work with data in the Hadoop Distributed File System — [HDFS](/docs/en/engines/table-engines/integrations/hdfs.md) table engine. To load data from a web server with static files use a disk with type [web](#storing-data-on-webserver). ## Configuring HDFS {#configuring-hdfs} -[MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`. +[MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) family table engines can store data to HDFS using a disk with type `HDFS`. Configuration markup: @@ -53,7 +53,7 @@ Optional parameters: ## Using Virtual File System for Data Encryption {#encrypted-virtual-file-system} -You can encrypt the data stored on [S3](../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. +You can encrypt the data stored on [S3](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-s3), or [HDFS](#configuring-hdfs) external disks, or on a local disk. To turn on the encryption mode, in the configuration file you must define a disk with the type `encrypted` and choose a disk on which the data will be saved. An `encrypted` disk ciphers all written files on the fly, and when you read files from an `encrypted` disk it deciphers them automatically. So you can work with an `encrypted` disk like with a normal one. Example of disk configuration: @@ -80,14 +80,14 @@ Required parameters: - `type` — `encrypted`. Otherwise the encrypted disk is not created. - `disk` — Type of disk for data storage. -- `key` — The key for encryption and decryption. Type: [Uint64](../sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encrypt in hexadecimal form. +- `key` — The key for encryption and decryption. Type: [Uint64](/docs/en/sql-reference/data-types/int-uint.md). You can use `key_hex` parameter to encrypt in hexadecimal form. You can specify multiple keys using the `id` attribute (see example above). Optional parameters: - `path` — Path to the location on the disk where the data will be saved. If not specified, the data will be saved in the root directory. - `current_key_id` — The key used for encryption. All the specified keys can be used for decryption, and you can always switch to another key while maintaining access to previously encrypted data. -- `algorithm` — [Algorithm](../sql-reference/statements/create/table.md#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes. +- `algorithm` — [Algorithm](/docs/en/sql-reference/statements/create/table.md/#create-query-encryption-codecs) for encryption. Possible values: `AES_128_CTR`, `AES_192_CTR` or `AES_256_CTR`. Default value: `AES_128_CTR`. The key length depends on the algorithm: `AES_128_CTR` — 16 bytes, `AES_192_CTR` — 24 bytes, `AES_256_CTR` — 32 bytes. Example of disk configuration: @@ -265,9 +265,9 @@ Cache profile events: There is a tool `clickhouse-static-files-uploader`, which prepares a data directory for a given table (`SELECT data_paths FROM system.tables WHERE name = 'table_name'`). For each table you need, you get a directory of files. These files can be uploaded to, for example, a web server with static files. After this preparation, you can load this table into any ClickHouse server via `DiskWeb`. -This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](../sql-reference/statements/create/table.md), [ALTER TABLE](../sql-reference/statements/alter/index.md), [RENAME TABLE](../sql-reference/statements/rename.md#misc_operations-rename_table), [DETACH TABLE](../sql-reference/statements/detach.md) and [TRUNCATE TABLE](../sql-reference/statements/truncate.md). +This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md), [ALTER TABLE](/docs/en/sql-reference/statements/alter/overview.md), [RENAME TABLE](/docs/en/sql-reference/statements/rename.md/#misc_operations-rename_table), [DETACH TABLE](/docs/en/sql-reference/statements/detach.md) and [TRUNCATE TABLE](/docs/en/sql-reference/statements/truncate.md). -Web server storage is supported only for the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) and [Log](../engines/table-engines/log-family/log.md) engine families. To access the data stored on a `web` disk, use the [storage_policy](../engines/table-engines/mergetree-family/mergetree.md#terms) setting when executing the query. For example, `ATTACH TABLE table_web UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'`. +Web server storage is supported only for the [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) engine families. To access the data stored on a `web` disk, use the [storage_policy](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#terms) setting when executing the query. For example, `ATTACH TABLE table_web UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'`. A ready test case. You need to add this configuration to config: @@ -451,7 +451,7 @@ Optional parameters: - `remote_fs_read_backoff_threashold` — The maximum wait time when trying to read data for remote disk. Default value: `10000` seconds. - `remote_fs_read_backoff_max_tries` — The maximum number of attempts to read with backoff. Default value: `5`. -If a query fails with an exception `DB:Exception Unreachable URL`, then you can try to adjust the settings: [http_connection_timeout](../operations/settings/settings.md#http_connection_timeout), [http_receive_timeout](../operations/settings/settings.md#http_receive_timeout), [keep_alive_timeout](../operations/server-configuration-parameters/settings.md#keep-alive-timeout). +If a query fails with an exception `DB:Exception Unreachable URL`, then you can try to adjust the settings: [http_connection_timeout](/docs/en/operations/settings/settings.md/#http_connection_timeout), [http_receive_timeout](/docs/en/operations/settings/settings.md/#http_receive_timeout), [keep_alive_timeout](/docs/en/operations/server-configuration-parameters/settings.md/#keep-alive-timeout). To get files for upload run: `clickhouse static-files-disk-uploader --metadata-path --output-dir ` (`--metadata-path` can be found in query `SELECT data_paths FROM system.tables WHERE name = 'table_name'`). @@ -460,7 +460,7 @@ When loading files by `endpoint`, they must be loaded into `/store/` p If URL is not reachable on disk load when the server is starting up tables, then all errors are caught. If in this case there were errors, tables can be reloaded (become visible) via `DETACH TABLE table_name` -> `ATTACH TABLE table_name`. If metadata was successfully loaded at server startup, then tables are available straight away. -Use [http_max_single_read_retries](../operations/settings/settings.md#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read. +Use [http_max_single_read_retries](/docs/en/operations/settings/settings.md/#http-max-single-read-retries) setting to limit the maximum number of retries during a single HTTP read. ## Zero-copy Replication (not ready for production) {#zero-copy} diff --git a/docs/en/operations/system-tables/mutations.md b/docs/en/operations/system-tables/mutations.md index 45447f3644e..98237562f37 100644 --- a/docs/en/operations/system-tables/mutations.md +++ b/docs/en/operations/system-tables/mutations.md @@ -3,31 +3,31 @@ slug: /en/operations/system-tables/mutations --- # mutations -The table contains information about [mutations](../../sql-reference/statements/alter/index.md#mutations) of [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. +The table contains information about [mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. Columns: -- `database` ([String](../../sql-reference/data-types/string.md)) — The name of the database to which the mutation was applied. +- `database` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the database to which the mutation was applied. -- `table` ([String](../../sql-reference/data-types/string.md)) — The name of the table to which the mutation was applied. +- `table` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the table to which the mutation was applied. -- `mutation_id` ([String](../../sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `/mutations/` directory in ClickHouse Keeper. For non-replicated tables the IDs correspond to file names in the data directory of the table. +- `mutation_id` ([String](/docs/en/sql-reference/data-types/string.md)) — The ID of the mutation. For replicated tables these IDs correspond to znode names in the `/mutations/` directory in ClickHouse Keeper. For non-replicated tables the IDs correspond to file names in the data directory of the table. -- `command` ([String](../../sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`). +- `command` ([String](/docs/en/sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`). -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. +- `create_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. -- `block_numbers.partition_id` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — For mutations of replicated tables, the array contains the partitions' IDs (one record for each partition). For mutations of non-replicated tables the array is empty. +- `block_numbers.partition_id` ([Array](/docs/en/sql-reference/data-types/array.md)([String](/docs/en/sql-reference/data-types/string.md))) — For mutations of replicated tables, the array contains the partitions' IDs (one record for each partition). For mutations of non-replicated tables the array is empty. -- `block_numbers.number` ([Array](../../sql-reference/data-types/array.md)([Int64](../../sql-reference/data-types/int-uint.md))) — For mutations of replicated tables, the array contains one record for each partition, with the block number that was acquired by the mutation. Only parts that contain blocks with numbers less than this number will be mutated in the partition. +- `block_numbers.number` ([Array](/docs/en/sql-reference/data-types/array.md)([Int64](/docs/en/sql-reference/data-types/int-uint.md))) — For mutations of replicated tables, the array contains one record for each partition, with the block number that was acquired by the mutation. Only parts that contain blocks with numbers less than this number will be mutated in the partition. In non-replicated tables, block numbers in all partitions form a single sequence. This means that for mutations of non-replicated tables, the column will contain one record with a single block number acquired by the mutation. -- `parts_to_do_names` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — An array of names of data parts that need to be mutated for the mutation to complete. +- `parts_to_do_names` ([Array](/docs/en/sql-reference/data-types/array.md)([String](/docs/en/sql-reference/data-types/string.md))) — An array of names of data parts that need to be mutated for the mutation to complete. -- `parts_to_do` ([Int64](../../sql-reference/data-types/int-uint.md)) — The number of data parts that need to be mutated for the mutation to complete. +- `parts_to_do` ([Int64](/docs/en/sql-reference/data-types/int-uint.md)) — The number of data parts that need to be mutated for the mutation to complete. -- `is_done` ([UInt8](../../sql-reference/data-types/int-uint.md)) — The flag whether the mutation is done or not. Possible values: +- `is_done` ([UInt8](/docs/en/sql-reference/data-types/int-uint.md)) — The flag whether the mutation is done or not. Possible values: - `1` if the mutation is completed, - `0` if the mutation is still in process. @@ -37,16 +37,16 @@ Even if `parts_to_do = 0` it is possible that a mutation of a replicated table i If there were problems with mutating some data parts, the following columns contain additional information: -- `latest_failed_part` ([String](../../sql-reference/data-types/string.md)) — The name of the most recent part that could not be mutated. +- `latest_failed_part` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the most recent part that could not be mutated. -- `latest_fail_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. +- `latest_fail_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. -- `latest_fail_reason` ([String](../../sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure. +- `latest_fail_reason` ([String](/docs/en/sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure. **See Also** -- [Mutations](../../sql-reference/statements/alter/index.md#mutations) -- [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine -- [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family +- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) +- [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) table engine +- [ReplicatedMergeTree](/docs/en/engines/table-engines/mergetree-family/replication.md) family [Original article](https://clickhouse.com/docs/en/operations/system-tables/mutations) diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index f1d60896a2e..9b6a3495e1e 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/overview.md/#query_language_queries_alter) query. Formats: @@ -75,7 +75,7 @@ Columns: - `primary_key_bytes_in_memory_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md)) – The amount of memory (in bytes) reserved for primary key values. -- `is_frozen` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md#alter_freeze-partition) +- `is_frozen` ([UInt8](../../sql-reference/data-types/int-uint.md)) – Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. For more details, see [FREEZE PARTITION](../../sql-reference/statements/alter/partition.md/#alter_freeze-partition) - `database` ([String](../../sql-reference/data-types/string.md)) – Name of the database. @@ -87,25 +87,25 @@ Columns: - `disk_name` ([String](../../sql-reference/data-types/string.md)) – Name of a disk that stores the data part. -- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of compressed files. +- `hash_of_all_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of compressed files. -- `hash_of_uncompressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.). +- `hash_of_uncompressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.). -- `uncompressed_hash_of_compressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md#hash_functions-siphash128) of data in the compressed files as if they were uncompressed. +- `uncompressed_hash_of_compressed_files` ([String](../../sql-reference/data-types/string.md)) – [sipHash128](../../sql-reference/functions/hash-functions.md/#hash_functions-siphash128) of data in the compressed files as if they were uncompressed. -- `delete_ttl_info_min` ([DateTime](../../sql-reference/data-types/datetime.md)) — The minimum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `delete_ttl_info_min` ([DateTime](../../sql-reference/data-types/datetime.md)) — The minimum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `delete_ttl_info_max` ([DateTime](../../sql-reference/data-types/datetime.md)) — The maximum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `delete_ttl_info_max` ([DateTime](../../sql-reference/data-types/datetime.md)) — The maximum value of the date and time key for [TTL DELETE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). :::warning The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: -- `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). -- `move_ttl_info.max` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the maximum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl). +- `move_ttl_info.max` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the maximum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). - `bytes` ([UInt64](../../sql-reference/data-types/int-uint.md)) – Alias for `bytes_on_disk`. @@ -166,6 +166,6 @@ move_ttl_info.max: [] **See Also** - [MergeTree family](../../engines/table-engines/mergetree-family/mergetree.md) -- [TTL for Columns and Tables](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-ttl) +- [TTL for Columns and Tables](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl) [Original article](https://clickhouse.com/docs/en/operations/system-tables/parts) diff --git a/docs/en/operations/system-tables/parts_columns.md b/docs/en/operations/system-tables/parts_columns.md index 68757ddfbff..c909d075f17 100644 --- a/docs/en/operations/system-tables/parts_columns.md +++ b/docs/en/operations/system-tables/parts_columns.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/overview.md/#query_language_queries_alter) query. Formats: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index cc278465437..a19b946b3cd 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -35,11 +35,11 @@ These actions are described in detail below. ADD COLUMN [IF NOT EXISTS] name [type] [default_expr] [codec] [AFTER name_after | FIRST] ``` -Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md#codecs) and `default_expr` (see the section [Default expressions](../../../sql-reference/statements/create/table.md#create-default-values)). +Adds a new column to the table with the specified `name`, `type`, [`codec`](../create/table.md/#codecs) and `default_expr` (see the section [Default expressions](/docs/en/sql-reference/statements/create/table.md/#create-default-values)). If the `IF NOT EXISTS` clause is included, the query won’t return an error if the column already exists. If you specify `AFTER name_after` (the name of another column), the column is added after the specified one in the list of table columns. If you want to add a column to the beginning of the table use the `FIRST` clause. Otherwise, the column is added to the end of the table. For a chain of actions, `name_after` can be the name of a column that is added in one of the previous actions. -Adding a column just changes the table structure, without performing any actions with data. The data does not appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](../../../engines/table-engines/mergetree-family/mergetree.md)). +Adding a column just changes the table structure, without performing any actions with data. The data does not appear on the disk after `ALTER`. If the data is missing for a column when reading from the table, it is filled in with default values (by performing the default expression if there is one, or using zeros or empty strings). The column appears on the disk after merging data parts (see [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md)). This approach allows us to complete the `ALTER` query instantly, without increasing the volume of old data. @@ -76,7 +76,7 @@ Deletes the column with the name `name`. If the `IF EXISTS` clause is specified, Deletes data from the file system. Since this deletes entire files, the query is completed almost instantly. :::warning -You can’t delete a column if it is referenced by [materialized view](../../../sql-reference/statements/create/view.md#materialized). Otherwise, it returns an error. +You can’t delete a column if it is referenced by [materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized). Otherwise, it returns an error. ::: Example: @@ -107,7 +107,7 @@ ALTER TABLE visits RENAME COLUMN webBrowser TO browser CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` -Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md#how-to-set-partition-expression). +Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md/#how-to-set-partition-expression). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. @@ -127,7 +127,7 @@ Adds a comment to the column. If the `IF EXISTS` clause is specified, the query Each column can have one comment. If a comment already exists for the column, a new comment overwrites the previous comment. -Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](../../../sql-reference/statements/describe-table.md) query. +Comments are stored in the `comment_expression` column returned by the [DESCRIBE TABLE](/docs/en/sql-reference/statements/describe-table.md) query. Example: @@ -152,15 +152,15 @@ This query changes the `name` column properties: - TTL -For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md#codecs). +For examples of columns compression CODECS modifying, see [Column Compression Codecs](../create/table.md/#codecs). -For examples of columns TTL modifying, see [Column TTL](../../../engines/table-engines/mergetree-family/mergetree.md#mergetree-column-ttl). +For examples of columns TTL modifying, see [Column TTL](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#mergetree-column-ttl). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. The query also can change the order of the columns using `FIRST | AFTER` clause, see [ADD COLUMN](#alter_add-column) description. -When changing the type, values are converted as if the [toType](../../../sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly. +When changing the type, values are converted as if the [toType](/docs/en/sql-reference/functions/type-conversion-functions.md) functions were applied to them. If only the default expression is changed, the query does not do anything complex, and is completed almost instantly. Example: @@ -246,7 +246,7 @@ SELECT groupArray(x), groupArray(s) FROM tmp; **See Also** -- [MATERIALIZED](../../statements/create/table.md#materialized). +- [MATERIALIZED](/docs/en/sql-reference/statements/create/table.md/#materialized). ## Limitations @@ -254,8 +254,8 @@ The `ALTER` query lets you create and delete separate elements (columns) in nest There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`). -If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](../../../sql-reference/statements/insert-into.md#insert_query_insert-select) query, then switch the tables using the [RENAME](../../../sql-reference/statements/rename.md#rename-table) query and delete the old table. You can use the [clickhouse-copier](../../../operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. +If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](/docs/en/sql-reference/statements/insert-into.md/#insert_query_insert-select) query, then switch the tables using the [RENAME](/docs/en/sql-reference/statements/rename.md/#rename-table) query and delete the old table. You can use the [clickhouse-copier](/docs/en/operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running. -For tables that do not store data themselves (such as [Merge](../../../sql-reference/statements/alter/index.md) and [Distributed](../../../sql-reference/statements/alter/index.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. +For tables that do not store data themselves (such as [Merge](/docs/en/sql-reference/statements/alter/overview.md) and [Distributed](/docs/en/sql-reference/statements/alter/overview.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index ba5d01d9b4d..05b33253bfe 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -10,21 +10,21 @@ sidebar_label: DELETE ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr ``` -Deletes data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). :::note -The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. `ALTER TABLE` is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](../delete.md), which performs a lightweight delete and can be considerably faster. +The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. `ALTER TABLE` is considered a heavyweight operation that requires the underlying data to be merged before it is deleted. For MergeTree tables, consider using the [`DELETE FROM` query](/docs/en/sql-reference/statements/delete.md), which performs a lightweight delete and can be considerably faster. ::: The `filter_expr` must be of type `UInt8`. The query deletes rows in the table for which this expression takes a non-zero value. One query can contain several commands separated by commas. -The synchronicity of the query processing is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. By default, it is asynchronous. +The synchronicity of the query processing is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. By default, it is asynchronous. **See also** -- [Mutations](../../../sql-reference/statements/alter/index.md#mutations) -- [Synchronicity of ALTER Queries](../../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting +- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) +- [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/alter/overview.md b/docs/en/sql-reference/statements/alter/overview.md index 4027429cf0d..1c4d62f3190 100644 --- a/docs/en/sql-reference/statements/alter/overview.md +++ b/docs/en/sql-reference/statements/alter/overview.md @@ -8,43 +8,43 @@ sidebar_label: ALTER Most `ALTER TABLE` queries modify table settings or data: -- [COLUMN](../../../sql-reference/statements/alter/column.md) -- [PARTITION](../../../sql-reference/statements/alter/partition.md) -- [DELETE](../../../sql-reference/statements/alter/delete.md) -- [UPDATE](../../../sql-reference/statements/alter/update.md) -- [ORDER BY](../../../sql-reference/statements/alter/order-by.md) -- [INDEX](../../../sql-reference/statements/alter/index/index.md) -- [CONSTRAINT](../../../sql-reference/statements/alter/constraint.md) -- [TTL](../../../sql-reference/statements/alter/ttl.md) +- [COLUMN](/docs/en/sql-reference/statements/alter/column.md) +- [PARTITION](/docs/en/sql-reference/statements/alter/partition.md) +- [DELETE](/docs/en/sql-reference/statements/alter/delete.md) +- [UPDATE](/docs/en/sql-reference/statements/alter/update.md) +- [ORDER BY](/docs/en/sql-reference/statements/alter/order-by.md) +- [INDEX](/docs/en/sql-reference/statements/alter/skipping-index.md) +- [CONSTRAINT](/docs/en/sql-reference/statements/alter/constraint.md) +- [TTL](/docs/en/sql-reference/statements/alter/ttl.md) :::note -Most `ALTER TABLE` queries are supported only for [\*MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](../../../engines/table-engines/special/merge.md) and [Distributed](../../../engines/table-engines/special/distributed.md). +Most `ALTER TABLE` queries are supported only for [\*MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables, as well as [Merge](/docs/en/engines/table-engines/special/merge.md) and [Distributed](/docs/en/engines/table-engines/special/distributed.md). ::: These `ALTER` statements manipulate views: -- [ALTER TABLE ... MODIFY QUERY](../../../sql-reference/statements/alter/view.md) — Modifies a [Materialized view](../create/view.md#materialized) structure. -- [ALTER LIVE VIEW](../../../sql-reference/statements/alter/view.md#alter-live-view) — Refreshes a [Live view](../create/view.md#live-view). +- [ALTER TABLE ... MODIFY QUERY](/docs/en/sql-reference/statements/alter/view.md) — Modifies a [Materialized view](/docs/en/sql-reference/statements/create/view.md/#materialized) structure. +- [ALTER LIVE VIEW](/docs/en/sql-reference/statements/alter/view.md/#alter-live-view) — Refreshes a [Live view](/docs/en/sql-reference/statements/create/view.md/#live-view). These `ALTER` statements modify entities related to role-based access control: -- [USER](../../../sql-reference/statements/alter/user.md) -- [ROLE](../../../sql-reference/statements/alter/role.md) -- [QUOTA](../../../sql-reference/statements/alter/quota.md) -- [ROW POLICY](../../../sql-reference/statements/alter/row-policy.md) -- [SETTINGS PROFILE](../../../sql-reference/statements/alter/settings-profile.md) +- [USER](/docs/en/sql-reference/statements/alter/user.md) +- [ROLE](/docs/en/sql-reference/statements/alter/role.md) +- [QUOTA](/docs/en/sql-reference/statements/alter/quota.md) +- [ROW POLICY](/docs/en/sql-reference/statements/alter/row-policy.md) +- [SETTINGS PROFILE](/docs/en/sql-reference/statements/alter/settings-profile.md) -[ALTER TABLE ... MODIFY COMMENT](../../../sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not. +[ALTER TABLE ... MODIFY COMMENT](/docs/en/sql-reference/statements/alter/comment.md) statement adds, modifies, or removes comments to the table, regardless if it was set before or not. ## Mutations -`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](../../../sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](../../../sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](../../../engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. +`ALTER` queries that are intended to manipulate table data are implemented with a mechanism called “mutations”, most notably [ALTER TABLE … DELETE](/docs/en/sql-reference/statements/alter/delete.md) and [ALTER TABLE … UPDATE](/docs/en/sql-reference/statements/alter/update.md). They are asynchronous background processes similar to merges in [MergeTree](/docs/en/engines/table-engines/mergetree-family/index.md) tables that to produce new “mutated” versions of parts. For `*MergeTree` tables mutations execute by **rewriting whole data parts**. There is no atomicity - parts are substituted for mutated parts as soon as they are ready and a `SELECT` query that started executing during a mutation will see data from parts that have already been mutated along with data from parts that have not been mutated yet. Mutations are totally ordered by their creation order and are applied to each part in that order. Mutations are also partially ordered with `INSERT INTO` queries: data that was inserted into the table before the mutation was submitted will be mutated and data that was inserted after that will not be mutated. Note that mutations do not block inserts in any way. -A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for non-replicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](../../../operations/system-tables/mutations.md#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](../../../sql-reference/statements/kill.md#kill-mutation) query. +A mutation query returns immediately after the mutation entry is added (in case of replicated tables to ZooKeeper, for non-replicated tables - to the filesystem). The mutation itself executes asynchronously using the system profile settings. To track the progress of mutations you can use the [`system.mutations`](/docs/en/operations/system-tables/mutations.md/#system_tables-mutations) table. A mutation that was successfully submitted will continue to execute even if ClickHouse servers are restarted. There is no way to roll back the mutation once it is submitted, but if the mutation is stuck for some reason it can be cancelled with the [`KILL MUTATION`](/docs/en/sql-reference/statements/kill.md/#kill-mutation) query. Entries for finished mutations are not deleted right away (the number of preserved entries is determined by the `finished_mutations_to_keep` storage engine parameter). Older mutation entries are deleted. @@ -52,12 +52,12 @@ Entries for finished mutations are not deleted right away (the number of preserv For non-replicated tables, all `ALTER` queries are performed synchronously. For replicated tables, the query just adds instructions for the appropriate actions to `ZooKeeper`, and the actions themselves are performed as soon as possible. However, the query can wait for these actions to be completed on all the replicas. -For all `ALTER` queries, you can use the [replication_alter_partitions_sync](../../../operations/settings/settings.md#replication-alter-partitions-sync) setting to set up waiting. +For all `ALTER` queries, you can use the [replication_alter_partitions_sync](/docs/en/operations/settings/settings.md/#replication-alter-partitions-sync) setting to set up waiting. -You can specify how long (in seconds) to wait for inactive replicas to execute all `ALTER` queries with the [replication_wait_for_inactive_replica_timeout](../../../operations/settings/settings.md#replication-wait-for-inactive-replica-timeout) setting. +You can specify how long (in seconds) to wait for inactive replicas to execute all `ALTER` queries with the [replication_wait_for_inactive_replica_timeout](/docs/en/operations/settings/settings.md/#replication-wait-for-inactive-replica-timeout) setting. :::note For all `ALTER` queries, if `replication_alter_partitions_sync = 2` and some replicas are not active for more than the time, specified in the `replication_wait_for_inactive_replica_timeout` setting, then an exception `UNFINISHED` is thrown. ::: -For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. +For `ALTER TABLE ... UPDATE|DELETE` queries the synchronicity is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 2d89c1d5d18..20eb76a8c8e 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -5,7 +5,7 @@ sidebar_label: PARTITION title: "Manipulating Partitions and Parts" --- -The following operations with [partitions](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md) are available: +The following operations with [partitions](/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md) are available: - [DETACH PARTITION\|PART](#detach-partitionpart) — Moves a partition or part to the `detached` directory and forget it. - [DROP PARTITION\|PART](#drop-partitionpart) — Deletes a partition or part. @@ -43,7 +43,7 @@ Read about setting the partition expression in a section [How to set the partiti After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it. -This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](../../../operations/system-tables/replicas.md#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replicas (as multiple leaders are allowed). +This query is replicated – it moves the data to the `detached` directory on all replicas. Note that you can execute this query only on a leader replica. To find out if a replica is a leader, perform the `SELECT` query to the [system.replicas](/docs/en/operations/system-tables/replicas.md/#system_tables-replicas) table. Alternatively, it is easier to make a `DETACH` query on all replicas - all the replicas throw an exception, except the leader replicas (as multiple leaders are allowed). ## DROP PARTITION\|PART @@ -175,7 +175,7 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a - if the `WITH NAME` parameter is specified, then the value of the `'backup_name'` parameter is used instead of the incremental number. :::note -If you use [a set of disks for data storage in a table](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. +If you use [a set of disks for data storage in a table](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. ::: The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs `chmod` for all files, forbidding writing into them. @@ -249,7 +249,7 @@ Although the query is called `ALTER TABLE`, it does not change the table structu ## MOVE PARTITION\|PART -Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes). +Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-multiple-volumes). ``` sql ALTER TABLE table_name [ON CLUSTER cluster] MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' @@ -270,7 +270,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' ## UPDATE IN PARTITION -Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). Syntax: @@ -286,11 +286,11 @@ ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; ### See Also -- [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) +- [UPDATE](/docs/en/sql-reference/statements/alter/update.md/#alter-table-update-statements) ## DELETE IN PARTITION -Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). Syntax: @@ -306,7 +306,7 @@ ALTER TABLE mt DELETE IN PARTITION 2 WHERE p = 2; ### See Also -- [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) +- [DELETE](/docs/en/sql-reference/statements/alter/delete.md/#alter-mutations) ## How to Set Partition Expression @@ -315,16 +315,16 @@ You can specify the partition expression in `ALTER ... PARTITION` queries in dif - As a value from the `partition` column of the `system.parts` table. For example, `ALTER TABLE visits DETACH PARTITION 201901`. - As a tuple of expressions or constants that matches (in types) the table partitioning keys tuple. In the case of a single element partitioning key, the expression should be wrapped in the `tuple (...)` function. For example, `ALTER TABLE visits DETACH PARTITION tuple(toYYYYMM(toDate('2019-01-25')))`. - Using the partition ID. Partition ID is a string identifier of the partition (human-readable, if possible) that is used as the names of partitions in the file system and in ZooKeeper. The partition ID must be specified in the `PARTITION ID` clause, in a single quotes. For example, `ALTER TABLE visits DETACH PARTITION ID '201901'`. -- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](../../../operations/system-tables/detached_parts.md#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. +- In the [ALTER ATTACH PART](#alter_attach-partition) and [DROP DETACHED PART](#alter_drop-detached) query, to specify the name of a part, use string literal with a value from the `name` column of the [system.detached_parts](/docs/en/operations/system-tables/detached_parts.md/#system_tables-detached_parts) table. For example, `ALTER TABLE visits ATTACH PART '201901_1_1_0'`. Usage of quotes when specifying the partition depends on the type of partition expression. For example, for the `String` type, you have to specify its name in quotes (`'`). For the `Date` and `Int*` types no quotes are needed. -All the rules above are also true for the [OPTIMIZE](../../../sql-reference/statements/optimize.md) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example: +All the rules above are also true for the [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) query. If you need to specify the only partition when optimizing a non-partitioned table, set the expression `PARTITION tuple()`. For example: ``` sql OPTIMIZE TABLE table_not_partitioned PARTITION tuple() FINAL; ``` -`IN PARTITION` specifies the partition to which the [UPDATE](../../../sql-reference/statements/alter/update.md#alter-table-update-statements) or [DELETE](../../../sql-reference/statements/alter/delete.md#alter-mutations) expressions are applied as a result of the `ALTER TABLE` query. New parts are created only from the specified partition. In this way, `IN PARTITION` helps to reduce the load when the table is divided into many partitions, and you only need to update the data point-by-point. +`IN PARTITION` specifies the partition to which the [UPDATE](/docs/en/sql-reference/statements/alter/update.md/#alter-table-update-statements) or [DELETE](/docs/en/sql-reference/statements/alter/delete.md/#alter-mutations) expressions are applied as a result of the `ALTER TABLE` query. New parts are created only from the specified partition. In this way, `IN PARTITION` helps to reduce the load when the table is divided into many partitions, and you only need to update the data point-by-point. The examples of `ALTER ... PARTITION` queries are demonstrated in the tests [`00502_custom_partitioning_local`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/ClickHouse/ClickHouse/blob/master/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index ff8ecf3a77f..952afc7f764 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -5,7 +5,7 @@ sidebar_label: PROJECTION title: "Manipulating Projections" --- -The following operations with [projections](../../../engines/table-engines/mergetree-family/mergetree.md#projections) are available: +The following operations with [projections](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#projections) are available: ## ADD PROJECTION @@ -13,15 +13,15 @@ The following operations with [projections](../../../engines/table-engines/merge ## DROP PROJECTION -`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). ## MATERIALIZE PROJECTION -`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). ## CLEAR PROJECTION -`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. @@ -29,5 +29,5 @@ The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only Also, they are replicated, syncing projections metadata via ClickHouse Keeper or ZooKeeper. :::note -Projection manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). +Projection manipulation is supported only for tables with [`*MergeTree`](/docs/en/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index 1a61e1b73ec..47b3999160e 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -14,12 +14,12 @@ The following operations are available: - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. Also, they are replicated, syncing indices metadata via ZooKeeper. :::note -Index manipulation is supported only for tables with [`*MergeTree`](/docs/en/sql-reference/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/sql-reference/engines/table-engines/mergetree-family/replication.md) variants). +Index manipulation is supported only for tables with [`*MergeTree`](/docs/en/engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](/docs/en/engines/table-engines/mergetree-family/replication.md) variants). ::: diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index e4fb872ae24..475a1527842 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -10,7 +10,7 @@ sidebar_label: UPDATE ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` -Manipulates data matching the specified filtering expression. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). :::note The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. @@ -20,11 +20,11 @@ The `filter_expr` must be of type `UInt8`. This query updates values of specifie One query can contain several commands separated by commas. -The synchronicity of the query processing is defined by the [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting. By default, it is asynchronous. +The synchronicity of the query processing is defined by the [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting. By default, it is asynchronous. **See also** -- [Mutations](../../../sql-reference/statements/alter/index.md#mutations) -- [Synchronicity of ALTER Queries](../../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) -- [mutations_sync](../../../operations/settings/settings.md#mutations_sync) setting +- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) +- [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index bfb90f4a89f..4583b4ef1e9 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -8,25 +8,25 @@ sidebar_label: Statements Statements represent various kinds of action you can perform using SQL queries. Each kind of statement has it’s own syntax and usage details that are described separately: -- [SELECT](../../sql-reference/statements/select/index.md) -- [INSERT INTO](../../sql-reference/statements/insert-into.md) -- [CREATE](../../sql-reference/statements/create/index.md) -- [ALTER](../../sql-reference/statements/alter/index.md) -- [SYSTEM](../../sql-reference/statements/system.md) -- [SHOW](../../sql-reference/statements/show.md) -- [GRANT](../../sql-reference/statements/grant.md) -- [REVOKE](../../sql-reference/statements/revoke.md) -- [ATTACH](../../sql-reference/statements/attach.md) -- [CHECK TABLE](../../sql-reference/statements/check-table.md) -- [DESCRIBE TABLE](../../sql-reference/statements/describe-table.md) -- [DETACH](../../sql-reference/statements/detach.md) -- [DROP](../../sql-reference/statements/drop.md) -- [EXISTS](../../sql-reference/statements/exists.md) -- [KILL](../../sql-reference/statements/kill.md) -- [OPTIMIZE](../../sql-reference/statements/optimize.md) -- [RENAME](../../sql-reference/statements/rename.md) -- [SET](../../sql-reference/statements/set.md) -- [SET ROLE](../../sql-reference/statements/set-role.md) -- [TRUNCATE](../../sql-reference/statements/truncate.md) -- [USE](../../sql-reference/statements/use.md) -- [EXPLAIN](../../sql-reference/statements/explain.md) +- [SELECT](/docs/en/sql-reference/statements/select/index.md) +- [INSERT INTO](/docs/en/sql-reference/statements/insert-into.md) +- [CREATE](/docs/en/sql-reference/statements/create/index.md) +- [ALTER](/docs/en/sql-reference/statements/alter/overview.md) +- [SYSTEM](/docs/en/sql-reference/statements/system.md) +- [SHOW](/docs/en/sql-reference/statements/show.md) +- [GRANT](/docs/en/sql-reference/statements/grant.md) +- [REVOKE](/docs/en/sql-reference/statements/revoke.md) +- [ATTACH](/docs/en/sql-reference/statements/attach.md) +- [CHECK TABLE](/docs/en/sql-reference/statements/check-table.md) +- [DESCRIBE TABLE](/docs/en/sql-reference/statements/describe-table.md) +- [DETACH](/docs/en/sql-reference/statements/detach.md) +- [DROP](/docs/en/sql-reference/statements/drop.md) +- [EXISTS](/docs/en/sql-reference/statements/exists.md) +- [KILL](/docs/en/sql-reference/statements/kill.md) +- [OPTIMIZE](/docs/en/sql-reference/statements/optimize.md) +- [RENAME](/docs/en/sql-reference/statements/rename.md) +- [SET](/docs/en/sql-reference/statements/set.md) +- [SET ROLE](/docs/en/sql-reference/statements/set-role.md) +- [TRUNCATE](/docs/en/sql-reference/statements/truncate.md) +- [USE](/docs/en/sql-reference/statements/use.md) +- [EXPLAIN](/docs/en/sql-reference/statements/explain.md) From c822c8161ff1a5c57ad4077c2324bd841b9a04fc Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 8 Nov 2022 20:21:26 -0500 Subject: [PATCH 257/526] fix nav --- docs/en/engines/database-engines/materialized-mysql.md | 2 +- docs/en/engines/table-engines/log-family/index.md | 2 +- .../table-engines/mergetree-family/mergetree.md | 4 ++-- .../table-engines/mergetree-family/replication.md | 2 +- docs/en/engines/table-engines/special/join.md | 2 +- docs/en/operations/settings/settings.md | 10 +++++----- docs/en/operations/storing-data.md | 2 +- docs/en/operations/system-tables/mutations.md | 4 ++-- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/system-tables/parts_columns.md | 2 +- docs/en/sql-reference/statements/alter/column.md | 2 +- docs/en/sql-reference/statements/alter/delete.md | 6 +++--- .../statements/alter/{overview.md => index.md} | 0 docs/en/sql-reference/statements/alter/partition.md | 4 ++-- docs/en/sql-reference/statements/alter/projection.md | 6 +++--- .../sql-reference/statements/alter/skipping-index.md | 2 +- docs/en/sql-reference/statements/alter/update.md | 6 +++--- docs/en/sql-reference/statements/alter/view.md | 2 +- docs/en/sql-reference/statements/grant.md | 2 +- docs/en/sql-reference/statements/index.md | 2 +- docs/en/sql-reference/statements/kill.md | 2 +- 21 files changed, 33 insertions(+), 33 deletions(-) rename docs/en/sql-reference/statements/alter/{overview.md => index.md} (100%) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 0411286cd23..7dd43858416 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -133,7 +133,7 @@ Apart of the data types limitations there are few restrictions comparing to `MyS ### DDL Queries {#ddl-queries} -MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](/docs/en/sql-reference/statements/alter/overview.md), [CREATE](/docs/en/sql-reference/statements/create/index.md), [DROP](/docs/en/sql-reference/statements/drop.md), [RENAME](/docs/en/sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. +MySQL DDL queries are converted into the corresponding ClickHouse DDL queries ([ALTER](/docs/en/sql-reference/statements/alter/index.md), [CREATE](/docs/en/sql-reference/statements/create/index.md), [DROP](/docs/en/sql-reference/statements/drop.md), [RENAME](/docs/en/sql-reference/statements/rename.md)). If ClickHouse cannot parse some DDL query, the query is ignored. ### Data Replication {#data-replication} diff --git a/docs/en/engines/table-engines/log-family/index.md b/docs/en/engines/table-engines/log-family/index.md index 269ad0c98f1..486c41c2496 100644 --- a/docs/en/engines/table-engines/log-family/index.md +++ b/docs/en/engines/table-engines/log-family/index.md @@ -28,7 +28,7 @@ Engines: During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. -- Do not support [mutations](/docs/en/sql-reference/statements/alter/overview.md/#alter-mutations). +- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). - Do not support indexes. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 807a19605c4..7dfb5a9fed7 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -298,7 +298,7 @@ This feature is helpful when using the [SummingMergeTree](/docs/en/engines/table In this case it makes sense to leave only a few columns in the primary key that will provide efficient range scans and add the remaining dimension columns to the sorting key tuple. -[ALTER](/docs/en/sql-reference/statements/alter/overview.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. +[ALTER](/docs/en/sql-reference/statements/alter/index.md) of the sorting key is a lightweight operation because when a new column is simultaneously added to the table and to the sorting key, existing data parts do not need to be changed. Since the old sorting key is a prefix of the new sorting key and there is no data in the newly added column, the data is sorted by both the old and new sorting keys at the moment of table modification. ### Use of Indexes and Partitions in Queries {#use-of-indexes-and-partitions-in-queries} @@ -860,7 +860,7 @@ The number of threads performing background moves of data parts can be changed b In the case of `MergeTree` tables, data is getting to disk in different ways: - As a result of an insert (`INSERT` query). -- During background merges and [mutations](/docs/en/sql-reference/statements/alter/overview.md/#alter-mutations). +- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). - When downloading from another replica. - As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 0e208629c2e..67b595d0fa0 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -20,7 +20,7 @@ Replication works at the level of an individual table, not the entire server. A Replication does not depend on sharding. Each shard has its own independent replication. -Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/overview.md/#query_language_queries_alter)). +Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/index.md/#query_language_queries_alter)). `CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 36b2e59bcf8..0e51a8b7696 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -59,7 +59,7 @@ Main use-cases for `Join`-engine tables are following: ### Deleting Data {#deleting-data} -`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. +`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. ### Limitations and Settings {#join-limitations-and-settings} diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 38077a2b891..d2f0f46f637 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2227,7 +2227,7 @@ Default value: `0`. ## mutations_sync {#mutations_sync} -Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/overview.md/#mutations)) synchronously. +Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md/#mutations)) synchronously. Possible values: @@ -2239,8 +2239,8 @@ Default value: `0`. **See Also** -- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) -- [Mutations](../../sql-reference/statements/alter/overview.md/#mutations) +- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [Mutations](../../sql-reference/statements/alter/index.md/#mutations) ## ttl_only_drop_parts {#ttl_only_drop_parts} @@ -2954,7 +2954,7 @@ Default value: `0`. ## replication_alter_partitions_sync {#replication-alter-partitions-sync} -Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/overview.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. +Allows to set up waiting for actions to be executed on replicas by [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. Possible values: @@ -2966,7 +2966,7 @@ Default value: `1`. ## replication_wait_for_inactive_replica_timeout {#replication-wait-for-inactive-replica-timeout} -Specifies how long (in seconds) to wait for inactive replicas to execute [ALTER](../../sql-reference/statements/alter/overview.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. +Specifies how long (in seconds) to wait for inactive replicas to execute [ALTER](../../sql-reference/statements/alter/index.md), [OPTIMIZE](../../sql-reference/statements/optimize.md) or [TRUNCATE](../../sql-reference/statements/truncate.md) queries. Possible values: diff --git a/docs/en/operations/storing-data.md b/docs/en/operations/storing-data.md index c0acd544fa9..203fe4e42d2 100644 --- a/docs/en/operations/storing-data.md +++ b/docs/en/operations/storing-data.md @@ -265,7 +265,7 @@ Cache profile events: There is a tool `clickhouse-static-files-uploader`, which prepares a data directory for a given table (`SELECT data_paths FROM system.tables WHERE name = 'table_name'`). For each table you need, you get a directory of files. These files can be uploaded to, for example, a web server with static files. After this preparation, you can load this table into any ClickHouse server via `DiskWeb`. -This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md), [ALTER TABLE](/docs/en/sql-reference/statements/alter/overview.md), [RENAME TABLE](/docs/en/sql-reference/statements/rename.md/#misc_operations-rename_table), [DETACH TABLE](/docs/en/sql-reference/statements/detach.md) and [TRUNCATE TABLE](/docs/en/sql-reference/statements/truncate.md). +This is a read-only disk. Its data is only read and never modified. A new table is loaded to this disk via `ATTACH TABLE` query (see example below). Local disk is not actually used, each `SELECT` query will result in a `http` request to fetch required data. All modification of the table data will result in an exception, i.e. the following types of queries are not allowed: [CREATE TABLE](/docs/en/sql-reference/statements/create/table.md), [ALTER TABLE](/docs/en/sql-reference/statements/alter/index.md), [RENAME TABLE](/docs/en/sql-reference/statements/rename.md/#misc_operations-rename_table), [DETACH TABLE](/docs/en/sql-reference/statements/detach.md) and [TRUNCATE TABLE](/docs/en/sql-reference/statements/truncate.md). Web server storage is supported only for the [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) and [Log](/docs/en/engines/table-engines/log-family/log.md) engine families. To access the data stored on a `web` disk, use the [storage_policy](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#terms) setting when executing the query. For example, `ATTACH TABLE table_web UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'`. diff --git a/docs/en/operations/system-tables/mutations.md b/docs/en/operations/system-tables/mutations.md index 98237562f37..782d7c42ad2 100644 --- a/docs/en/operations/system-tables/mutations.md +++ b/docs/en/operations/system-tables/mutations.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/mutations --- # mutations -The table contains information about [mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. +The table contains information about [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. Columns: @@ -45,7 +45,7 @@ If there were problems with mutating some data parts, the following columns cont **See Also** -- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) - [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) table engine - [ReplicatedMergeTree](/docs/en/engines/table-engines/mergetree-family/replication.md) family diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index 9b6a3495e1e..cbabd9b27b1 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/overview.md/#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. Formats: diff --git a/docs/en/operations/system-tables/parts_columns.md b/docs/en/operations/system-tables/parts_columns.md index c909d075f17..d934e01f245 100644 --- a/docs/en/operations/system-tables/parts_columns.md +++ b/docs/en/operations/system-tables/parts_columns.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/overview.md/#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. Formats: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index a19b946b3cd..6bca0dbff42 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -258,4 +258,4 @@ If the `ALTER` query is not sufficient to make the table changes you need, you c The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running. -For tables that do not store data themselves (such as [Merge](/docs/en/sql-reference/statements/alter/overview.md) and [Distributed](/docs/en/sql-reference/statements/alter/overview.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. +For tables that do not store data themselves (such as [Merge](/docs/en/sql-reference/statements/alter/index.md) and [Distributed](/docs/en/sql-reference/statements/alter/index.md)), `ALTER` just changes the table structure, and does not change the structure of subordinate tables. For example, when running ALTER for a `Distributed` table, you will also need to run `ALTER` for the tables on all remote servers. diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 05b33253bfe..4dcab030d13 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -10,7 +10,7 @@ sidebar_label: DELETE ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr ``` -Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). :::note @@ -25,6 +25,6 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do **See also** -- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) -- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/alter/overview.md b/docs/en/sql-reference/statements/alter/index.md similarity index 100% rename from docs/en/sql-reference/statements/alter/overview.md rename to docs/en/sql-reference/statements/alter/index.md diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 20eb76a8c8e..146c15e776e 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -270,7 +270,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' ## UPDATE IN PARTITION -Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). Syntax: @@ -290,7 +290,7 @@ ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; ## DELETE IN PARTITION -Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). Syntax: diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 952afc7f764..7913c7cb7e4 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -13,15 +13,15 @@ The following operations with [projections](/docs/en/engines/table-engines/merge ## DROP PROJECTION -`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). ## MATERIALIZE PROJECTION -`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). ## CLEAR PROJECTION -`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index 47b3999160e..2dadffc4527 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -14,7 +14,7 @@ The following operations are available: - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index 475a1527842..f40b72f7ab3 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -10,7 +10,7 @@ sidebar_label: UPDATE ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` -Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/overview.md/#mutations). +Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). :::note The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. @@ -24,7 +24,7 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do **See also** -- [Mutations](/docs/en/sql-reference/statements/alter/overview.md/#mutations) -- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/overview.md/#synchronicity-of-alter-queries) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/alter/view.md b/docs/en/sql-reference/statements/alter/view.md index fd6045a026e..e382cdace30 100644 --- a/docs/en/sql-reference/statements/alter/view.md +++ b/docs/en/sql-reference/statements/alter/view.md @@ -8,7 +8,7 @@ sidebar_label: VIEW You can modify `SELECT` query that was specified when a [materialized view](../create/view.md#materialized) was created with the `ALTER TABLE … MODIFY QUERY` statement. Use it when the materialized view was created without the `TO [db.]name` clause. The `allow_experimental_alter_materialized_view_structure` setting must be enabled. -If a materialized view uses the `TO [db.]name` construction, you must [DETACH](../detach.md) the view, run [ALTER TABLE](overview.md) query for the target table, and then [ATTACH](../attach.md) the previously detached (`DETACH`) view. +If a materialized view uses the `TO [db.]name` construction, you must [DETACH](../detach.md) the view, run [ALTER TABLE](index.md) query for the target table, and then [ATTACH](../attach.md) the previously detached (`DETACH`) view. **Example** diff --git a/docs/en/sql-reference/statements/grant.md b/docs/en/sql-reference/statements/grant.md index bd35efb3deb..546a8b0958d 100644 --- a/docs/en/sql-reference/statements/grant.md +++ b/docs/en/sql-reference/statements/grant.md @@ -261,7 +261,7 @@ The granted privilege allows `john` to insert data to the `x` and/or `y` columns ### ALTER -Allows executing [ALTER](../../sql-reference/statements/alter/overview.md) queries according to the following hierarchy of privileges: +Allows executing [ALTER](../../sql-reference/statements/alter/index.md) queries according to the following hierarchy of privileges: - `ALTER`. Level: `COLUMN`. - `ALTER TABLE`. Level: `GROUP` diff --git a/docs/en/sql-reference/statements/index.md b/docs/en/sql-reference/statements/index.md index 4583b4ef1e9..b286d8c932d 100644 --- a/docs/en/sql-reference/statements/index.md +++ b/docs/en/sql-reference/statements/index.md @@ -11,7 +11,7 @@ Statements represent various kinds of action you can perform using SQL queries. - [SELECT](/docs/en/sql-reference/statements/select/index.md) - [INSERT INTO](/docs/en/sql-reference/statements/insert-into.md) - [CREATE](/docs/en/sql-reference/statements/create/index.md) -- [ALTER](/docs/en/sql-reference/statements/alter/overview.md) +- [ALTER](/docs/en/sql-reference/statements/alter/index.md) - [SYSTEM](/docs/en/sql-reference/statements/system.md) - [SHOW](/docs/en/sql-reference/statements/show.md) - [GRANT](/docs/en/sql-reference/statements/grant.md) diff --git a/docs/en/sql-reference/statements/kill.md b/docs/en/sql-reference/statements/kill.md index 733125c78f3..294724dfa50 100644 --- a/docs/en/sql-reference/statements/kill.md +++ b/docs/en/sql-reference/statements/kill.md @@ -51,7 +51,7 @@ KILL MUTATION [ON CLUSTER cluster] [FORMAT format] ``` -Tries to cancel and remove [mutations](../../sql-reference/statements/alter/overview.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query. +Tries to cancel and remove [mutations](../../sql-reference/statements/alter/index.md#alter-mutations) that are currently executing. Mutations to cancel are selected from the [`system.mutations`](../../operations/system-tables/mutations.md#system_tables-mutations) table using the filter specified by the `WHERE` clause of the `KILL` query. A test query (`TEST`) only checks the user’s rights and displays a list of mutations to stop. From 901e1e2fefae8a609cd26a5f24f59b4a20a9fa09 Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 9 Nov 2022 12:01:51 +0800 Subject: [PATCH 258/526] better logging for mark range filtering on projection parts --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4091d37ec11..9abea8c226d 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1466,6 +1466,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( } size_t used_key_size = key_condition.getMaxKeyColumn() + 1; + String part_name = part->isProjectionPart() ? fmt::format("{}.{}", part->name, part->getParentPart()->name) : part->name; std::function create_field_ref; /// If there are no monotonic functions, there is no need to save block reference. @@ -1578,7 +1579,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( } } - LOG_TRACE(log, "Used generic exclusion search over index for part {} with {} steps", part->name, steps); + LOG_TRACE(log, "Used generic exclusion search over index for part {} with {} steps", part_name, steps); } else { @@ -1586,7 +1587,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( /// we can use binary search algorithm to find the left and right endpoint key marks of such interval. /// The returned value is the minimum range of marks, containing all keys for which KeyCondition holds - LOG_TRACE(log, "Running binary search on index range for part {} ({} marks)", part->name, marks_count); + LOG_TRACE(log, "Running binary search on index range for part {} ({} marks)", part_name, marks_count); size_t steps = 0; From 2720cddeecfe0fb9517e29f64304df8d674f0886 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 9 Nov 2022 04:28:02 +0000 Subject: [PATCH 259/526] Add support for interactive parameters in INSERT VALUES queries --- src/Client/ClientBase.cpp | 8 ++++++++ .../0_stateless/02476_query_parameters_insert.reference | 1 + .../queries/0_stateless/02476_query_parameters_insert.sql | 8 ++++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02476_query_parameters_insert.reference create mode 100644 tests/queries/0_stateless/02476_query_parameters_insert.sql diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5d7de8ec799..87c92b91d31 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1542,6 +1542,14 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin global_context->applySettingChange(change); } global_context->resetSettingsToDefaultValue(set_query->default_settings); + + /// Query parameters inside SET queries should be also saved on the client side + /// to override their previous definitions set with --param_* arguments + /// and for substitutions to work inside INSERT ... VALUES queries + for (const auto & [name, value] : set_query->query_parameters) + query_parameters.insert_or_assign(name, value); + + global_context->addQueryParameters(set_query->query_parameters); } if (const auto * use_query = parsed_query->as()) { diff --git a/tests/queries/0_stateless/02476_query_parameters_insert.reference b/tests/queries/0_stateless/02476_query_parameters_insert.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02476_query_parameters_insert.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02476_query_parameters_insert.sql b/tests/queries/0_stateless/02476_query_parameters_insert.sql new file mode 100644 index 00000000000..de866ccbc4d --- /dev/null +++ b/tests/queries/0_stateless/02476_query_parameters_insert.sql @@ -0,0 +1,8 @@ +DROP TABLE IF EXISTS 02476_query_parameters_insert; +CREATE TABLE 02476_query_parameters_insert (x Int32) ENGINE=MergeTree() ORDER BY tuple(); + +SET param_x = 1; +INSERT INTO 02476_query_parameters_insert VALUES ({x: Int32}); +SELECT * FROM 02476_query_parameters_insert; + +DROP TABLE 02476_query_parameters_insert; From b9a3d090ed557a2b5cc58c010a9796f071d6eccb Mon Sep 17 00:00:00 2001 From: Duc Canh Le Date: Wed, 9 Nov 2022 13:07:24 +0800 Subject: [PATCH 260/526] use const string & --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9abea8c226d..59171165704 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1466,7 +1466,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( } size_t used_key_size = key_condition.getMaxKeyColumn() + 1; - String part_name = part->isProjectionPart() ? fmt::format("{}.{}", part->name, part->getParentPart()->name) : part->name; + const String & part_name = part->isProjectionPart() ? fmt::format("{}.{}", part->name, part->getParentPart()->name) : part->name; std::function create_field_ref; /// If there are no monotonic functions, there is no need to save block reference. From ee856fc4582c67dc39d234d43b2d854ff6778e21 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 08:01:05 +0000 Subject: [PATCH 261/526] Add table function hudi and deltaLake --- src/TableFunctions/TableFunctionDelta.cpp | 166 ++++++++++++++++++++ src/TableFunctions/TableFunctionDelta.h | 44 ++++++ src/TableFunctions/TableFunctionHudi.cpp | 166 ++++++++++++++++++++ src/TableFunctions/TableFunctionHudi.h | 44 ++++++ src/TableFunctions/registerTableFunctions.h | 2 + 5 files changed, 422 insertions(+) create mode 100644 src/TableFunctions/TableFunctionDelta.cpp create mode 100644 src/TableFunctions/TableFunctionDelta.h create mode 100644 src/TableFunctions/TableFunctionHudi.cpp create mode 100644 src/TableFunctions/TableFunctionHudi.h diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp new file mode 100644 index 00000000000..4190a9e9a93 --- /dev/null +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -0,0 +1,166 @@ +#include "config.h" + +#if USE_AWS_S3 + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include "registerTableFunctions.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +void TableFunctionDelta::parseArgumentsImpl( + const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & base_configuration) +{ + if (args.empty() || args.size() > 6) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); + + auto header_it = StorageURL::collectHeaders(args, base_configuration, context); + if (header_it != args.end()) + args.erase(header_it); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + /// Size -> argument indexes + static auto size_to_args = std::map>{ + {1, {{}}}, + {2, {{"format", 1}}}, + {5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}}, + {6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}}; + + std::map args_to_idx; + /// For 4 arguments we support 2 possible variants: + /// hudi(source, format, structure, compression_method) and hudi(source, access_key_id, access_key_id, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + if (args.size() == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } + /// For 3 arguments we support 2 possible variants: + /// hudi(source, format, structure) and hudi(source, access_key_id, access_key_id) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + else if (args.size() == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + args_to_idx = {{"format", 1}, {"structure", 2}}; + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; + } + else + { + args_to_idx = size_to_args[args.size()]; + } + + /// This argument is always the first + base_configuration.url = checkAndGetLiteralArgument(args[0], "url"); + + if (args_to_idx.contains("format")) + base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + + if (args_to_idx.contains("structure")) + base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); + + if (args_to_idx.contains("compression_method")) + base_configuration.compression_method + = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); + + if (args_to_idx.contains("access_key_id")) + base_configuration.auth_settings.access_key_id + = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); + + if (args_to_idx.contains("secret_access_key")) + base_configuration.auth_settings.secret_access_key + = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); + + if (base_configuration.format == "auto") + base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); +} + +void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Parse args + ASTs & args_func = ast_function->children; + + const auto message = fmt::format( + "The signature of table function {} could be the following:\n" \ + " - url\n" \ + " - url, format\n" \ + " - url, format, structure\n" \ + " - url, access_key_id, secret_access_key\n" \ + " - url, format, structure, compression_method\n" \ + " - url, access_key_id, secret_access_key, format\n" \ + " - url, access_key_id, secret_access_key, format, structure\n" \ + " - url, access_key_id, secret_access_key, format, structure, compression_method", + getName()); + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + auto & args = args_func.at(0)->children; + + parseArgumentsImpl(message, args, context, configuration); +} + +ColumnsDescription TableFunctionDelta::getActualTableStructure(ContextPtr context) const +{ + if (configuration.structure == "auto") + { + context->checkAccess(getSourceAccessType()); + return StorageS3::getTableStructureFromData(configuration, false, std::nullopt, context); + } + + return parseColumnsListFromString(configuration.structure, context); +} + +StoragePtr TableFunctionDelta::executeImpl( + const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + Poco::URI uri (configuration.url); + S3::URI s3_uri (uri); + + ColumnsDescription columns; + if (configuration.structure != "auto") + columns = parseColumnsListFromString(configuration.structure, context); + + StoragePtr storage = std::make_shared( + configuration, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, context, std::nullopt); + + storage->startup(); + + return storage; +} + + +void registerTableFunctionDelta(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/TableFunctions/TableFunctionDelta.h b/src/TableFunctions/TableFunctionDelta.h new file mode 100644 index 00000000000..badfd63f431 --- /dev/null +++ b/src/TableFunctions/TableFunctionDelta.h @@ -0,0 +1,44 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include + + +namespace DB +{ + +class Context; +class TableFunctionS3Cluster; + +/* deltaLake(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary DeltaLake table on S3. + */ +class TableFunctionDelta : public ITableFunction +{ +public: + static constexpr auto name = "deltaLake"; + std::string getName() const override + { + return name; + } + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return name; } + + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + static void parseArgumentsImpl(const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & configuration); + + StorageS3Configuration configuration; +}; + +} + +#endif diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp new file mode 100644 index 00000000000..9b9a27aa296 --- /dev/null +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -0,0 +1,166 @@ +#include "config.h" + +#if USE_AWS_S3 + +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include +# include "registerTableFunctions.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +void TableFunctionHudi::parseArgumentsImpl( + const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & base_configuration) +{ + if (args.empty() || args.size() > 6) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, error_message); + + auto header_it = StorageURL::collectHeaders(args, base_configuration, context); + if (header_it != args.end()) + args.erase(header_it); + + for (auto & arg : args) + arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); + + /// Size -> argument indexes + static auto size_to_args = std::map>{ + {1, {{}}}, + {2, {{"format", 1}}}, + {5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}}, + {6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}}; + + std::map args_to_idx; + /// For 4 arguments we support 2 possible variants: + /// hudi(source, format, structure, compression_method) and hudi(source, access_key_id, access_key_id, format) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + if (args.size() == 4) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; + + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; + } + /// For 3 arguments we support 2 possible variants: + /// hudi(source, format, structure) and hudi(source, access_key_id, access_key_id) + /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. + else if (args.size() == 3) + { + auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) + args_to_idx = {{"format", 1}, {"structure", 2}}; + else + args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; + } + else + { + args_to_idx = size_to_args[args.size()]; + } + + /// This argument is always the first + base_configuration.url = checkAndGetLiteralArgument(args[0], "url"); + + if (args_to_idx.contains("format")) + base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + + if (args_to_idx.contains("structure")) + base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); + + if (args_to_idx.contains("compression_method")) + base_configuration.compression_method + = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); + + if (args_to_idx.contains("access_key_id")) + base_configuration.auth_settings.access_key_id + = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); + + if (args_to_idx.contains("secret_access_key")) + base_configuration.auth_settings.secret_access_key + = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); + + if (base_configuration.format == "auto") + base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); +} + +void TableFunctionHudi::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + /// Parse args + ASTs & args_func = ast_function->children; + + const auto message = fmt::format( + "The signature of table function {} could be the following:\n" \ + " - url\n" \ + " - url, format\n" \ + " - url, format, structure\n" \ + " - url, access_key_id, secret_access_key\n" \ + " - url, format, structure, compression_method\n" \ + " - url, access_key_id, secret_access_key, format\n" \ + " - url, access_key_id, secret_access_key, format, structure\n" \ + " - url, access_key_id, secret_access_key, format, structure, compression_method", + getName()); + + if (args_func.size() != 1) + throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + auto & args = args_func.at(0)->children; + + parseArgumentsImpl(message, args, context, configuration); +} + +ColumnsDescription TableFunctionHudi::getActualTableStructure(ContextPtr context) const +{ + if (configuration.structure == "auto") + { + context->checkAccess(getSourceAccessType()); + return StorageS3::getTableStructureFromData(configuration, false, std::nullopt, context); + } + + return parseColumnsListFromString(configuration.structure, context); +} + +StoragePtr TableFunctionHudi::executeImpl( + const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const +{ + Poco::URI uri (configuration.url); + S3::URI s3_uri (uri); + + ColumnsDescription columns; + if (configuration.structure != "auto") + columns = parseColumnsListFromString(configuration.structure, context); + + StoragePtr storage = std::make_shared( + configuration, StorageID(getDatabaseName(), table_name), columns, ConstraintsDescription{}, String{}, context, std::nullopt); + + storage->startup(); + + return storage; +} + + +void registerTableFunctionHudi(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + +} + +#endif diff --git a/src/TableFunctions/TableFunctionHudi.h b/src/TableFunctions/TableFunctionHudi.h new file mode 100644 index 00000000000..a370bca8c45 --- /dev/null +++ b/src/TableFunctions/TableFunctionHudi.h @@ -0,0 +1,44 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include + + +namespace DB +{ + +class Context; +class TableFunctionS3Cluster; + +/* hudi(source, [access_key_id, secret_access_key,] format, structure[, compression]) - creates a temporary Hudi table on S3. + */ +class TableFunctionHudi : public ITableFunction +{ +public: + static constexpr auto name = "hudi"; + std::string getName() const override + { + return name; + } + +protected: + StoragePtr executeImpl( + const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns) const override; + + const char * getStorageTypeName() const override { return name; } + + ColumnsDescription getActualTableStructure(ContextPtr context) const override; + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + static void parseArgumentsImpl(const String & error_message, ASTs & args, ContextPtr context, StorageS3Configuration & configuration); + + StorageS3Configuration configuration; +}; + +} + +#endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 3bc9e3a85da..e1d93cfa0f5 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -24,6 +24,8 @@ void registerTableFunctionMeiliSearch(TableFunctionFactory & factory); void registerTableFunctionS3(TableFunctionFactory & factory); void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory); +void registerTableFunctionHudi(TableFunctionFactory & factory); +void registerTableFunctionDelta(TableFunctionFactory & factory); #endif #if USE_HDFS From 5eafe3fc46d38c8568c7c5f69ed689f2fb646c10 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 08:03:32 +0000 Subject: [PATCH 262/526] fix comment --- src/TableFunctions/TableFunctionDelta.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 4190a9e9a93..6153754170a 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -49,7 +49,7 @@ void TableFunctionDelta::parseArgumentsImpl( std::map args_to_idx; /// For 4 arguments we support 2 possible variants: - /// hudi(source, format, structure, compression_method) and hudi(source, access_key_id, access_key_id, format) + /// deltaLake(source, format, structure, compression_method) and deltaLake(source, access_key_id, access_key_id, format) /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. if (args.size() == 4) { @@ -61,7 +61,7 @@ void TableFunctionDelta::parseArgumentsImpl( args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}}; } /// For 3 arguments we support 2 possible variants: - /// hudi(source, format, structure) and hudi(source, access_key_id, access_key_id) + /// deltaLake(source, format, structure) and deltaLake(source, access_key_id, access_key_id) /// We can distinguish them by looking at the 2-nd argument: check if it's a format name or not. else if (args.size() == 3) { From 5ac2414b442294d26567a4d6d6e58d1a97e994a2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 09:15:11 +0100 Subject: [PATCH 263/526] Disable server test --- .github/workflows/jepsen.yml | 64 +++++++++++++++++------------------- 1 file changed, 31 insertions(+), 33 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index 16433e2baed..f70b11cec0e 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -31,7 +31,6 @@ jobs: sudo rm -fr "$TEMP_PATH" mkdir -p "$TEMP_PATH" cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - docker build -f "$REPO_COPY/docker/test/keeper-jepsen/Dockerfile" -t clickhouse/keeper-jepsen-test "$REPO_COPY/docker/test/keeper-jepsen" cd "$REPO_COPY/tests/ci" python3 jepsen_check.py keeper - name: Cleanup @@ -40,35 +39,34 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - ServerJepsenRelease: - runs-on: [self-hosted, style-checker] - if: ${{ always() }} - needs: [KeeperJepsenRelease] - steps: - - name: Set envs - run: | - cat >> "$GITHUB_ENV" << 'EOF' - TEMP_PATH=${{runner.temp}}/server_jepsen - REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse - EOF - - name: Clear repository - run: | - sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - - name: Check out repository code - uses: actions/checkout@v2 - with: - fetch-depth: 0 - - name: Jepsen Test - run: | - sudo rm -fr "$TEMP_PATH" - mkdir -p "$TEMP_PATH" - cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - docker build -f "$REPO_COPY/docker/test/server-jepsen/Dockerfile" -t clickhouse/server-jepsen-test "$REPO_COPY/docker/test/server-jepsen" - cd "$REPO_COPY/tests/ci" - python3 jepsen_check.py server - - name: Cleanup - if: always() - run: | - docker ps --quiet | xargs --no-run-if-empty docker kill ||: - docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - sudo rm -fr "$TEMP_PATH" + #ServerJepsenRelease: + # runs-on: [self-hosted, style-checker] + # if: ${{ always() }} + # needs: [KeeperJepsenRelease] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/server_jepsen + # REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse + # EOF + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # with: + # fetch-depth: 0 + # - name: Jepsen Test + # run: | + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" + # python3 jepsen_check.py server + # - name: Cleanup + # if: always() + # run: | + # docker ps --quiet | xargs --no-run-if-empty docker kill ||: + # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + # sudo rm -fr "$TEMP_PATH" From e0adcd79c6b1d2ca3a0b75acacf72c8f1ebe30e2 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 9 Nov 2022 09:45:32 +0100 Subject: [PATCH 264/526] Add space after comment --- .github/workflows/jepsen.yml | 62 ++++++++++++++++++------------------ 1 file changed, 31 insertions(+), 31 deletions(-) diff --git a/.github/workflows/jepsen.yml b/.github/workflows/jepsen.yml index f70b11cec0e..5afc066065e 100644 --- a/.github/workflows/jepsen.yml +++ b/.github/workflows/jepsen.yml @@ -39,34 +39,34 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" - #ServerJepsenRelease: - # runs-on: [self-hosted, style-checker] - # if: ${{ always() }} - # needs: [KeeperJepsenRelease] - # steps: - # - name: Set envs - # run: | - # cat >> "$GITHUB_ENV" << 'EOF' - # TEMP_PATH=${{runner.temp}}/server_jepsen - # REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse - # EOF - # - name: Clear repository - # run: | - # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" - # - name: Check out repository code - # uses: actions/checkout@v2 - # with: - # fetch-depth: 0 - # - name: Jepsen Test - # run: | - # sudo rm -fr "$TEMP_PATH" - # mkdir -p "$TEMP_PATH" - # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" - # cd "$REPO_COPY/tests/ci" - # python3 jepsen_check.py server - # - name: Cleanup - # if: always() - # run: | - # docker ps --quiet | xargs --no-run-if-empty docker kill ||: - # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: - # sudo rm -fr "$TEMP_PATH" + # ServerJepsenRelease: + # runs-on: [self-hosted, style-checker] + # if: ${{ always() }} + # needs: [KeeperJepsenRelease] + # steps: + # - name: Set envs + # run: | + # cat >> "$GITHUB_ENV" << 'EOF' + # TEMP_PATH=${{runner.temp}}/server_jepsen + # REPO_COPY=${{runner.temp}}/server_jepsen/ClickHouse + # EOF + # - name: Clear repository + # run: | + # sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + # - name: Check out repository code + # uses: actions/checkout@v2 + # with: + # fetch-depth: 0 + # - name: Jepsen Test + # run: | + # sudo rm -fr "$TEMP_PATH" + # mkdir -p "$TEMP_PATH" + # cp -r "$GITHUB_WORKSPACE" "$TEMP_PATH" + # cd "$REPO_COPY/tests/ci" + # python3 jepsen_check.py server + # - name: Cleanup + # if: always() + # run: | + # docker ps --quiet | xargs --no-run-if-empty docker kill ||: + # docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: + # sudo rm -fr "$TEMP_PATH" From 75d3907e79a099a089fad30704a745f9f0dec981 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 17:10:57 +0800 Subject: [PATCH 265/526] Update src/TableFunctions/TableFunctionDelta.cpp Co-authored-by: Antonio Andelic --- src/TableFunctions/TableFunctionDelta.cpp | 22 ++++++---------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 6153754170a..5049242d6ec 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -79,23 +79,13 @@ void TableFunctionDelta::parseArgumentsImpl( /// This argument is always the first base_configuration.url = checkAndGetLiteralArgument(args[0], "url"); - if (args_to_idx.contains("format")) - base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + static constexpr std::array possible_args{"format", "structure", "compression_method", "access_key_id", "secret_access_key"}; - if (args_to_idx.contains("structure")) - base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); - - if (args_to_idx.contains("compression_method")) - base_configuration.compression_method - = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); - - if (args_to_idx.contains("access_key_id")) - base_configuration.auth_settings.access_key_id - = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); - - if (args_to_idx.contains("secret_access_key")) - base_configuration.auth_settings.secret_access_key - = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); + for (const auto * possible_arg : possible_args) + { + if (args_to_idx.contains(possible_arg)) + base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx[possible_arg]], possible_arg); + } if (base_configuration.format == "auto") base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); From b7cb7ab6db0d123663cfd87de70b87d9a3f5247d Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 17:11:11 +0800 Subject: [PATCH 266/526] Update src/TableFunctions/TableFunctionDelta.cpp Co-authored-by: Antonio Andelic --- src/TableFunctions/TableFunctionDelta.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 5049242d6ec..03efd4f04df 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -130,8 +130,8 @@ ColumnsDescription TableFunctionDelta::getActualTableStructure(ContextPtr contex StoragePtr TableFunctionDelta::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - Poco::URI uri (configuration.url); - S3::URI s3_uri (uri); + Poco::URI uri(configuration.url); + S3::URI s3_uri(uri); ColumnsDescription columns; if (configuration.structure != "auto") From 1a8dd85a4892d08596e4b6a53b34d63cf35d6106 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 17:11:20 +0800 Subject: [PATCH 267/526] Update src/TableFunctions/TableFunctionDelta.cpp Co-authored-by: Antonio Andelic --- src/TableFunctions/TableFunctionDelta.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 03efd4f04df..c088c713f1b 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -109,7 +109,7 @@ void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr getName()); if (args_func.size() != 1) - throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); auto & args = args_func.at(0)->children; From 592f6b31d973785668d43911a93f37c932e4ccc6 Mon Sep 17 00:00:00 2001 From: flynn Date: Wed, 9 Nov 2022 10:04:53 +0000 Subject: [PATCH 268/526] fix and add test --- src/TableFunctions/TableFunctionDelta.cpp | 24 ++++++++++++++------ src/TableFunctions/TableFunctionHudi.cpp | 6 ++--- src/TableFunctions/TableFunctionS3.cpp | 4 ++-- tests/integration/test_storage_delta/test.py | 5 ++++ tests/integration/test_storage_hudi/test.py | 11 +++++++++ 5 files changed, 38 insertions(+), 12 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index c088c713f1b..8456878be76 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -79,13 +79,23 @@ void TableFunctionDelta::parseArgumentsImpl( /// This argument is always the first base_configuration.url = checkAndGetLiteralArgument(args[0], "url"); - static constexpr std::array possible_args{"format", "structure", "compression_method", "access_key_id", "secret_access_key"}; + if (args_to_idx.contains("format")) + base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); - for (const auto * possible_arg : possible_args) - { - if (args_to_idx.contains(possible_arg)) - base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx[possible_arg]], possible_arg); - } + if (args_to_idx.contains("structure")) + base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); + + if (args_to_idx.contains("compression_method")) + base_configuration.compression_method + = checkAndGetLiteralArgument(args[args_to_idx["compression_method"]], "compression_method"); + + if (args_to_idx.contains("access_key_id")) + base_configuration.auth_settings.access_key_id + = checkAndGetLiteralArgument(args[args_to_idx["access_key_id"]], "access_key_id"); + + if (args_to_idx.contains("secret_access_key")) + base_configuration.auth_settings.secret_access_key + = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); if (base_configuration.format == "auto") base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); @@ -109,7 +119,7 @@ void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr getName()); if (args_func.size() != 1) - throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments.", getName()); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); auto & args = args_func.at(0)->children; diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index 9b9a27aa296..6be5dc2d089 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -119,7 +119,7 @@ void TableFunctionHudi::parseArguments(const ASTPtr & ast_function, ContextPtr c getName()); if (args_func.size() != 1) - throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); auto & args = args_func.at(0)->children; @@ -140,8 +140,8 @@ ColumnsDescription TableFunctionHudi::getActualTableStructure(ContextPtr context StoragePtr TableFunctionHudi::executeImpl( const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - Poco::URI uri (configuration.url); - S3::URI s3_uri (uri); + Poco::URI uri(configuration.url); + S3::URI s3_uri(uri); ColumnsDescription columns; if (configuration.structure != "auto") diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 4c0b5352545..ee1460a283c 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -148,8 +148,8 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) StoragePtr TableFunctionS3::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const { - Poco::URI uri (configuration.url); - S3::URI s3_uri (uri); + Poco::URI uri(configuration.url); + S3::URI s3_uri(uri); ColumnsDescription columns; if (configuration.structure != "auto") diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 76dab554a57..89a29a2afe9 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -126,7 +126,12 @@ def test_select_query(started_cluster): run_query(instance, create_query) select_query = "SELECT {} FROM deltalake FORMAT TSV" + select_table_function_query = f"""SELECT {} FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() assert len(result) > 0 + + for column_name in columns: + result = run_query(instance, select_table_function_query.format(column_name)).splitlines() + assert len(result) > 0 diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index 549421afd89..bf0955b2c25 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -133,15 +133,25 @@ def test_select_query(started_cluster): select_query = "SELECT {} FROM hudi FORMAT TSV" + select_table_function_query = f"""SELECT {} FROM hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')""" + for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() assert len(result) > 0 + for column_name in columns: + result = run_query(instance, select_table_function_query.format(column_name)).splitlines() + assert len(result) > 0 + # test if all partition paths is presented in result distinct_select_query = ( "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" ) + + distinct_select_table_function_query = f"""SELECT DISTINCT partitionpath FROM hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123') ORDER BY partitionpath FORMAT TSV""" + result = run_query(instance, distinct_select_query) + result_table_function = run_query(instance, distinct_select_query) expected = [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", @@ -149,3 +159,4 @@ def test_select_query(started_cluster): ] assert TSV(result) == TSV(expected) + assert TSV(result_table_function) == TSV(expected) From 768ada118b12a25bc6ae3e531b597b67a6e56628 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Nov 2022 21:37:31 +0100 Subject: [PATCH 269/526] Analyzer AST key condition crash fix --- src/Storages/MergeTree/KeyCondition.cpp | 2 +- ...477_analyzer_ast_key_condition_crash.reference | 2 ++ .../02477_analyzer_ast_key_condition_crash.sql | 15 +++++++++++++++ 3 files changed, 18 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.reference create mode 100644 tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 8ffd1059a58..1d688427a57 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -783,7 +783,7 @@ KeyCondition::KeyCondition( context, key_column_names, key_expr_, - query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(), + query_info.syntax_analyzer_result ? query_info.syntax_analyzer_result->getArrayJoinSourceNameSet() : NameSet{}, single_point_, strict_) { diff --git a/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.reference b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql new file mode 100644 index 00000000000..53f3a9b23ec --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_ast_key_condition_crash.sql @@ -0,0 +1,15 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64 +) ENGINE = MergeTree ORDER BY id; + +INSERT INTO test_table VALUES (1); + +SELECT * FROM test_table WHERE id = 1; + +SELECT * FROM test_table WHERE id = 1 SETTINGS query_plan_optimize_primary_key = 0; + +DROP TABLE test_table; From 9edb4d779b142309579d2b8f45ce81d6d01ed951 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 9 Nov 2022 12:18:29 +0100 Subject: [PATCH 270/526] fix --- docker/test/fuzzer/run-fuzzer.sh | 1 + tests/ci/ast_fuzzer_check.py | 6 +++--- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docker/test/fuzzer/run-fuzzer.sh b/docker/test/fuzzer/run-fuzzer.sh index e57b671342c..dbb56b258ed 100755 --- a/docker/test/fuzzer/run-fuzzer.sh +++ b/docker/test/fuzzer/run-fuzzer.sh @@ -391,6 +391,7 @@ th { cursor: pointer; }

AST Fuzzer for PR #${PR_TO_TEST} @ ${SHA_TO_TEST}

+ 🌚🌞
@@ -681,6 +682,25 @@ document.getElementById('add').addEventListener('click', e => { resize(); }); +document.getElementById('reload').addEventListener('click', e => { + reloadAll(); +}); + +function showReloadIfNeeded() { + const is_any_field_changed = (host != document.getElementById('url').value + || user != document.getElementById('user').value + || password != document.getElementById('password').value); + if (is_any_field_changed) { + document.getElementById('reload').style.display = ''; + } else { + document.getElementById('reload').style.display = 'none'; + } +} + +document.getElementById('password').addEventListener('input', e => { showReloadIfNeeded(); }) +document.getElementById('user').addEventListener('input', e => { showReloadIfNeeded(); }) +document.getElementById('url').addEventListener('input', e => { showReloadIfNeeded(); }) + function legendAsTooltipPlugin({ className, style = { background: "var(--legend-background)" } } = {}) { let legendEl; @@ -843,10 +863,15 @@ function resize() { new ResizeObserver(resize).observe(document.body); -document.getElementById('params').onsubmit = function(event) { +function reloadAll() { updateParams(); drawAll(); saveState(); + document.getElementById('reload').style.display = 'none'; +} + +document.getElementById('params').onsubmit = function(event) { + reloadAll(); event.preventDefault(); } From a977ea307133e896383047bf10a00ac1631a4fca Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Nov 2022 17:19:56 +0100 Subject: [PATCH 285/526] dashboard.html: scroll and focus to new chart --- programs/server/dashboard.html | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index 97cb8f8db12..b15bced5028 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -673,11 +673,16 @@ function insertChart(i) { chart.addEventListener('mouseleave', e => { edit_buttons.style.display = 'none'; }); charts.appendChild(chart); + return {chart: chart, textarea: query_editor_textarea}; }; document.getElementById('add').addEventListener('click', e => { queries.push({ title: '', query: '' }); - insertChart(plots.length); + + const {chart, textarea} = insertChart(plots.length); + chart.scrollIntoView(); + textarea.focus(); + plots.push(null); resize(); }); From e3f211a9437d272786230ac54939016d72e114f5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Nov 2022 17:20:26 +0100 Subject: [PATCH 286/526] dashboard.html: add debug mode to add add_http_cors_header --- programs/server/dashboard.html | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/programs/server/dashboard.html b/programs/server/dashboard.html index b15bced5028..859ce78068c 100644 --- a/programs/server/dashboard.html +++ b/programs/server/dashboard.html @@ -756,6 +756,8 @@ function legendAsTooltipPlugin({ className, style = { background: "var(--legend- }; } +let add_http_cors_header = false; + async function draw(idx, chart, url_params, query) { if (plots[idx]) { plots[idx].destroy(); @@ -767,6 +769,12 @@ async function draw(idx, chart, url_params, query) { password = document.getElementById('password').value; let url = `${host}?default_format=JSONCompactColumns` + + if (add_http_cors_header) { + // For debug purposes, you may set add_http_cors_header from a browser console + url += '&add_http_cors_header=1'; + } + if (user) { url += `&user=${encodeURIComponent(user)}`; } From 2ad8c5829fa1381214858cf20de98fcc33d96fac Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Nov 2022 14:18:56 +0000 Subject: [PATCH 287/526] add test --- .../02477_s3_request_throttler.reference | 1 + .../0_stateless/02477_s3_request_throttler.sql | 15 +++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02477_s3_request_throttler.reference create mode 100644 tests/queries/0_stateless/02477_s3_request_throttler.sql diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.reference b/tests/queries/0_stateless/02477_s3_request_throttler.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02477_s3_request_throttler.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.sql b/tests/queries/0_stateless/02477_s3_request_throttler.sql new file mode 100644 index 00000000000..7311c8ac9bc --- /dev/null +++ b/tests/queries/0_stateless/02477_s3_request_throttler.sql @@ -0,0 +1,15 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: needs s3 + +-- Limit S3 PUT request per second rate +SET s3_max_put_rps = 2; +SET s3_max_put_burst = 1; + +CREATE TEMPORARY TABLE times (t DateTime); + +-- INSERT query requires 3 PUT requests (CreateMultipartUpload, UploadPart, CompleteMultipartUpload) and 1/rps = 0.5 second in between, the first query is not throttled due to burst +INSERT INTO times SELECT now(); +INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/request-throttler.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10000, s3_truncate_on_insert = 1; +INSERT INTO times SELECT now(); + +SELECT max(t) - min(t) >= 1 FROM times; From 6ec2074d8a296b171b6cc61355fa4f3f31d0523e Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 9 Nov 2022 14:25:43 +0000 Subject: [PATCH 288/526] Fix check for `count` arguments number in FuseFunctionsPass --- src/Analyzer/Passes/FuseFunctionsPass.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/FuseFunctionsPass.cpp b/src/Analyzer/Passes/FuseFunctionsPass.cpp index e69c7627401..6c6613e7598 100644 --- a/src/Analyzer/Passes/FuseFunctionsPass.cpp +++ b/src/Analyzer/Passes/FuseFunctionsPass.cpp @@ -45,7 +45,8 @@ public: const auto & argument_nodes = function_node->getArguments().getNodes(); if (argument_nodes.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Aggregate function '{}' should have exactly one argument", function_node->getFunctionName()); + /// Do not apply for `count()` with without arguments or `count(*)`, only `count(x)` is supported. + return; mapping[QueryTreeNodeWithHash(argument_nodes[0])].push_back(&node); } From c523ffd696d8f0ec417030f6b0c789efacad0ac2 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 9 Nov 2022 14:33:11 +0000 Subject: [PATCH 289/526] Update tests for FuseFunctionsPass --- .../02476_fuse_sum_count.reference | 57 ++++++++++++++++- .../0_stateless/02476_fuse_sum_count.sql | 8 +-- .../02477_fuse_quantiles.reference | 62 +++++++++++++++++++ .../0_stateless/02477_fuse_quantiles.sql | 4 +- 4 files changed, 123 insertions(+), 8 deletions(-) diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.reference b/tests/queries/0_stateless/02476_fuse_sum_count.reference index 5b6936110ba..43a39e8b7e5 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.reference +++ b/tests/queries/0_stateless/02476_fuse_sum_count.reference @@ -2,7 +2,8 @@ \N \N 1.5 3 2 6 -6 10 9 +6 10 9 3 +6 1 5 6 3 2 2 7 2 5 6 3 2 2 7 2 QUERY id: 0 @@ -222,6 +223,60 @@ QUERY id: 0 COLUMN id: 39, column_name: b, result_type: Int8, source_id: 40 JOIN TREE TABLE id: 40, table_name: default.fuse_tbl +QUERY id: 0 + PROJECTION COLUMNS + sum(b) Int64 + count(b) UInt64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: tupleElement, function_type: ordinary, result_type: Int64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: b, result_type: Int64, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 9, function_name: tupleElement, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 10, nodes: 2 + FUNCTION id: 4, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: b, result_type: Int64, source_id: 7 + CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 + JOIN TREE + QUERY id: 7, is_subquery: 1 + PROJECTION COLUMNS + b Int64 + PROJECTION + LIST id: 12, nodes: 1 + COLUMN id: 13, column_name: x, result_type: Int64, source_id: 14 + JOIN TREE + QUERY id: 14, is_subquery: 1 + PROJECTION COLUMNS + x Int64 + count(b) UInt64 + PROJECTION + LIST id: 15, nodes: 2 + FUNCTION id: 16, function_name: tupleElement, function_type: ordinary, result_type: Int64 + ARGUMENTS + LIST id: 17, nodes: 2 + FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 19, nodes: 1 + COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 + CONSTANT id: 22, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 23, function_name: tupleElement, function_type: ordinary, result_type: UInt64 + ARGUMENTS + LIST id: 24, nodes: 2 + FUNCTION id: 18, function_name: sumCount, function_type: aggregate, result_type: Tuple(Int64, UInt64) + ARGUMENTS + LIST id: 19, nodes: 1 + COLUMN id: 20, column_name: b, result_type: Int8, source_id: 21 + CONSTANT id: 25, constant_value: UInt64_2, constant_value_type: UInt8 + JOIN TREE + TABLE id: 21, table_name: default.fuse_tbl 0 0 nan 0 0 nan 45 10 4.5 Decimal(38, 0) UInt64 Float64 diff --git a/tests/queries/0_stateless/02476_fuse_sum_count.sql b/tests/queries/0_stateless/02476_fuse_sum_count.sql index ef342c250e8..8ba096013a6 100644 --- a/tests/queries/0_stateless/02476_fuse_sum_count.sql +++ b/tests/queries/0_stateless/02476_fuse_sum_count.sql @@ -12,10 +12,9 @@ SELECT avg(a), sum(a) FROM (SELECT a FROM fuse_tbl WHERE isNull(a)); SELECT avg(a), sum(a) FROM (SELECT a FROM fuse_tbl WHERE isNotNull(a)); SELECT avg(b), sum(b) FROM (SELECT b FROM fuse_tbl); -SELECT avg(b) * 3, sum(b) + 1 + count(b), count(b) * count(b) FROM (SELECT b FROM fuse_tbl); +SELECT avg(b) * 3, sum(b) + 1 + count(b), count(b) * count(b), count() FROM (SELECT b FROM fuse_tbl); --- TODO(@vdimir): uncomment after https://github.com/ClickHouse/ClickHouse/pull/42865 --- SELECT sum(b), count(b) from (SELECT x as b FROM (SELECT sum(b) as x, count(b) FROM fuse_tbl) ); +SELECT sum(b), count(b) from (SELECT x as b FROM (SELECT sum(b) as x, count(b) FROM fuse_tbl) ); SELECT sum(a + 1), sum(b), count(b), avg(b), count(a + 1), sum(a + 2), count(a) from fuse_tbl SETTINGS optimize_syntax_fuse_functions = 0; SELECT sum(a + 1), sum(b), count(b), avg(b), count(a + 1), sum(a + 2), count(a) from fuse_tbl; @@ -25,8 +24,7 @@ EXPLAIN QUERY TREE run_passes = 1 SELECT sum(b), avg(b) from fuse_tbl; EXPLAIN QUERY TREE run_passes = 1 SELECT sum(a + 1), sum(b), count(b), avg(b), count(a + 1), sum(a + 2), count(a) from fuse_tbl; EXPLAIN QUERY TREE run_passes = 1 SELECT avg(b) * 3, sum(b) + 1 + count(b), count(b) * count(b) FROM (SELECT b FROM fuse_tbl); --- TODO(@vdimir): uncomment after https://github.com/ClickHouse/ClickHouse/pull/42865 --- EXPLAIN QUERY TREE run_passes = 1 SELECT sum(b), count(b) from (SELECT x as b FROM (SELECT sum(b) as x, count(b) FROM fuse_tbl) ); +EXPLAIN QUERY TREE run_passes = 1 SELECT sum(b), count(b) from (SELECT x as b FROM (SELECT sum(b) as x, count(b) FROM fuse_tbl) ); SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)) SETTINGS optimize_syntax_fuse_functions = 0; SELECT sum(x), count(x), avg(x) FROM (SELECT number :: Decimal32(0) AS x FROM numbers(0)); diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.reference b/tests/queries/0_stateless/02477_fuse_quantiles.reference index 7f1f1030e97..0938e9f6f6d 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.reference +++ b/tests/queries/0_stateless/02477_fuse_quantiles.reference @@ -84,3 +84,65 @@ QUERY id: 0 CONSTANT id: 38, constant_value: UInt64_1, constant_value_type: UInt8 JOIN TREE TABLE id: 7, table_name: default.fuse_tbl +501.5 501.5 +QUERY id: 0 + PROJECTION COLUMNS + quantile(0.5)(b) Float64 + quantile(0.9)(b) Float64 + PROJECTION + LIST id: 1, nodes: 2 + FUNCTION id: 2, function_name: arrayElement, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 3, nodes: 2 + FUNCTION id: 4, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: b, result_type: Float64, source_id: 7 + CONSTANT id: 8, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 9, function_name: arrayElement, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 10, nodes: 2 + FUNCTION id: 4, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + ARGUMENTS + LIST id: 5, nodes: 1 + COLUMN id: 6, column_name: b, result_type: Float64, source_id: 7 + CONSTANT id: 11, constant_value: UInt64_2, constant_value_type: UInt8 + JOIN TREE + QUERY id: 7, is_subquery: 1 + PROJECTION COLUMNS + b Float64 + PROJECTION + LIST id: 12, nodes: 1 + FUNCTION id: 13, function_name: plus, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 14, nodes: 2 + COLUMN id: 15, column_name: x, result_type: Float64, source_id: 16 + CONSTANT id: 17, constant_value: UInt64_1, constant_value_type: UInt8 + JOIN TREE + QUERY id: 16, is_subquery: 1 + PROJECTION COLUMNS + x Float64 + quantile(0.9)(b) Float64 + PROJECTION + LIST id: 18, nodes: 2 + FUNCTION id: 19, function_name: arrayElement, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 20, nodes: 2 + FUNCTION id: 21, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + ARGUMENTS + LIST id: 22, nodes: 1 + COLUMN id: 23, column_name: b, result_type: Int32, source_id: 24 + CONSTANT id: 25, constant_value: UInt64_1, constant_value_type: UInt8 + FUNCTION id: 26, function_name: arrayElement, function_type: ordinary, result_type: Float64 + ARGUMENTS + LIST id: 27, nodes: 2 + FUNCTION id: 21, function_name: quantiles, function_type: aggregate, result_type: Array(Float64) + ARGUMENTS + LIST id: 22, nodes: 1 + COLUMN id: 23, column_name: b, result_type: Int32, source_id: 24 + CONSTANT id: 28, constant_value: UInt64_2, constant_value_type: UInt8 + JOIN TREE + TABLE id: 24, table_name: default.fuse_tbl + GROUP BY + LIST id: 29, nodes: 1 + COLUMN id: 15, column_name: x, result_type: Float64, source_id: 16 diff --git a/tests/queries/0_stateless/02477_fuse_quantiles.sql b/tests/queries/0_stateless/02477_fuse_quantiles.sql index bc734af4c8e..b08c7da1f04 100644 --- a/tests/queries/0_stateless/02477_fuse_quantiles.sql +++ b/tests/queries/0_stateless/02477_fuse_quantiles.sql @@ -13,7 +13,7 @@ SELECT quantile(a - 1), quantile(b - 1) + 1, quantile(0.8)(b - 1) + 1, quantile( EXPLAIN QUERY TREE run_passes = 1 SELECT quantile(a - 1), quantile(b - 1) + 1, quantile(0.8)(b - 1) + 1, quantile(0.8)(b - 1) + 2, quantile(0.9)(b - 1) + 1 FROM fuse_tbl; --- -- TODO(@vdimir): uncomment after https://github.com/ClickHouse/ClickHouse/pull/42865 --- SELECT quantile(0.5)(b), quantile(0.9)(b) from (SELECT x as b, quantile(0.5)(b) as x, quantile(0.9)(b) FROM (SELECT quantile(0.5)(b) as x, quantile(0.9)(b) FROM fuse_tbl) ); +SELECT quantile(0.5)(b), quantile(0.9)(b) from (SELECT x + 1 as b FROM (SELECT quantile(0.5)(b) as x, quantile(0.9)(b) FROM fuse_tbl) GROUP BY x); +EXPLAIN QUERY TREE run_passes = 1 SELECT quantile(0.5)(b), quantile(0.9)(b) from (SELECT x + 1 as b FROM (SELECT quantile(0.5)(b) as x, quantile(0.9)(b) FROM fuse_tbl) GROUP BY x); DROP TABLE fuse_tbl; From 7f28a8927392c02b157ce98dd43d52877ba17ed3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 9 Nov 2022 09:39:27 -0500 Subject: [PATCH 290/526] fix ub type punning --- src/Common/HashTable/Hash.h | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index f36ab576766..f7c653f50d4 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -224,14 +224,9 @@ template requires (sizeof(T) <= sizeof(UInt64)) inline size_t hashCRC32(T key, DB::UInt64 updated_value = -1) { - union - { - T in; - DB::UInt64 out; - } u; - u.out = 0; - u.in = key; - return intHashCRC32(u.out, updated_value); + DB::UInt64 out {0}; + std::memcpy(&out, &key, sizeof(T)); + return intHashCRC32(out, updated_value); } template From a930ad25e185b69b9c001cc8c0aab4f9d4fa30ec Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 9 Nov 2022 09:44:59 -0500 Subject: [PATCH 291/526] fix ub type punning --- src/Common/HashTable/Hash.h | 22 ++++++---------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/src/Common/HashTable/Hash.h b/src/Common/HashTable/Hash.h index f7c653f50d4..189db68f59a 100644 --- a/src/Common/HashTable/Hash.h +++ b/src/Common/HashTable/Hash.h @@ -161,14 +161,9 @@ template requires (sizeof(T) <= sizeof(UInt64)) inline size_t DefaultHash64(T key) { - union - { - T in; - DB::UInt64 out; - } u; - u.out = 0; - u.in = key; - return intHash64(u.out); + DB::UInt64 out {0}; + std::memcpy(&out, &key, sizeof(T)); + return intHash64(out); } @@ -441,14 +436,9 @@ struct IntHash32 } else if constexpr (sizeof(T) <= sizeof(UInt64)) { - union - { - T in; - DB::UInt64 out; - } u; - u.out = 0; - u.in = key; - return intHash32(u.out); + DB::UInt64 out {0}; + std::memcpy(&out, &key, sizeof(T)); + return intHash32(out); } UNREACHABLE(); From 7448d1bb5d7c89a769c8906a0dd149a1e493c8ff Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 9 Nov 2022 14:58:41 +0000 Subject: [PATCH 292/526] add docs --- docs/en/engines/table-engines/integrations/s3.md | 5 +++++ docs/en/engines/table-engines/mergetree-family/mergetree.md | 4 ++++ src/Core/Settings.h | 4 ++-- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index db983ab9c68..484fd265c3d 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -127,6 +127,10 @@ The following settings can be set before query execution or placed into configur - `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. - `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. +- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited). +- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. +- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). +- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. @@ -142,6 +146,7 @@ The following settings can be specified in configuration file for given endpoint - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. +- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional. **Example:** diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 486baac2310..4fe286a2098 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -940,6 +940,10 @@ Optional parameters: - `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. +- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited). +- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. +- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). +- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. S3 disk can be configured as `main` or `cold` storage: ``` xml diff --git a/src/Core/Settings.h b/src/Core/Settings.h index e36ff1985fb..2f8aca6c628 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -90,9 +90,9 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ - M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate. Zero means unlimited.", 0) \ + M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ - M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate. Zero means unlimited.", 0) \ + M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ From 997881c7f7154bf1b19d56a899cc550e64cd93f7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 16:07:38 +0000 Subject: [PATCH 293/526] Review fixes. --- src/Interpreters/InterpreterSelectQuery.cpp | 27 +-------- src/Planner/Planner.cpp | 18 +----- src/Planner/PlannerJoinTree.cpp | 9 +-- .../QueryPlan/Optimizations/Optimizations.h | 10 +++- .../Optimizations/distinctReadInOrder.cpp | 12 ++-- .../Optimizations/optimizeReadInOrder.cpp | 58 +++++++++---------- .../QueryPlan/ReadFromMergeTree.cpp | 10 +++- src/Processors/QueryPlan/SortingStep.cpp | 20 ++++++- src/Processors/QueryPlan/SortingStep.h | 3 + 9 files changed, 77 insertions(+), 90 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8cfa902ae04..dd3267361e2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1447,14 +1447,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional

getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*context); auto sorting_step = std::make_unique( plan.getCurrentDataStream(), @@ -2619,14 +2612,7 @@ void InterpreterSelectQuery::executeWindow(QueryPlan & query_plan) // happens in case of `over ()`. if (!window.full_sort_description.empty() && (i == 0 || !sortIsPrefix(window, *windows_sorted[i - 1]))) { - SortingStep::Settings sort_settings; - sort_settings.max_block_size = settings.max_block_size; - sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; - sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; - sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; - sort_settings.tmp_data = context->getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*context); auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -2680,14 +2666,7 @@ void InterpreterSelectQuery::executeOrder(QueryPlan & query_plan, InputOrderInfo const Settings & settings = context->getSettingsRef(); - SortingStep::Settings sort_settings; - sort_settings.max_block_size = settings.max_block_size; - sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; - sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; - sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; - sort_settings.tmp_data = context->getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*context); /// Merge the sorted blocks. auto sorting_step = std::make_unique( diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index fc510150fc8..56cc73456ce 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -571,14 +571,7 @@ void Planner::buildQueryPlanIfNeeded() if (!window_description.full_sort_description.empty() && (i == 0 || !sortDescriptionIsPrefix(window_description.full_sort_description, window_descriptions[i - 1].full_sort_description))) { - SortingStep::Settings sort_settings; - sort_settings.max_block_size = settings.max_block_size; - sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; - sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; - sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; - sort_settings.tmp_data = query_context->getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*query_context); auto sorting_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -676,14 +669,7 @@ void Planner::buildQueryPlanIfNeeded() const Settings & settings = query_context->getSettingsRef(); - SortingStep::Settings sort_settings; - sort_settings.max_block_size = settings.max_block_size; - sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; - sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; - sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; - sort_settings.tmp_data = query_context->getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*query_context); /// Merge the sorted blocks auto sorting_step = std::make_unique( diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index ce8dfb56225..4e50a20bbd6 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -528,14 +528,7 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, for (const auto & key_name : key_names) sort_description.emplace_back(key_name); - SortingStep::Settings sort_settings; - sort_settings.max_block_size = settings.max_block_size; - sort_settings.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sort_settings.max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; - sort_settings.remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; - sort_settings.max_bytes_before_external_sort = settings.max_bytes_before_external_sort; - sort_settings.tmp_data = query_context->getTempDataOnDisk(); - sort_settings.min_free_disk_space = settings.min_free_disk_space_for_temporary_data; + SortingStep::Settings sort_settings(*query_context); auto sorting_step = std::make_unique( plan.getCurrentDataStream(), diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index b4b2f0c8c9c..973304b366b 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -30,10 +30,10 @@ struct Optimization const bool QueryPlanOptimizationSettings::* const is_enabled{}; }; -/// Move ARRAY JOIN up if possible. +/// Move ARRAY JOIN up if possible size_t tryLiftUpArrayJoin(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); -/// Move LimitStep down if possible. +/// Move LimitStep down if possible size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes &); /// Split FilterStep into chain `ExpressionStep -> FilterStep`, where FilterStep contains minimal number of nodes. @@ -59,6 +59,10 @@ size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, size_t tryDistinctReadInOrder(QueryPlan::Node * node); /// Put some steps under union, so that plan optimisation could be applied to union parts separately. +/// For example, the plan can be rewritten like: +/// - Something - - Expression - Something - +/// - Expression - Union - Something - => - Union - Expression - Something - +/// - Something - - Expression - Something - size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); inline const auto & getOptimizations() @@ -85,7 +89,7 @@ struct Frame using Stack = std::vector; -/// Second pass optimizations. +/// Second pass optimizations void optimizePrimaryKeyCondition(const Stack & stack); void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes); diff --git a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp index c01dfd0e6e3..3677a1581c4 100644 --- a/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/distinctReadInOrder.cpp @@ -22,7 +22,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) /// walk through the plan /// (1) check if nodes below preliminary distinct preserve sorting /// (2) gather transforming steps to update their sorting properties later - std::vector steps2update; + std::vector steps_to_update; QueryPlan::Node * node = parent_node; while (!node->children.empty()) { @@ -34,7 +34,7 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) if (!traits.preserves_sorting) return 0; - steps2update.push_back(step); + steps_to_update.push_back(step); node = node->children.front(); } @@ -90,11 +90,11 @@ size_t tryDistinctReadInOrder(QueryPlan::Node * parent_node) /// update data stream's sorting properties for found transforms const DataStream * input_stream = &read_from_merge_tree->getOutputStream(); - while (!steps2update.empty()) + while (!steps_to_update.empty()) { - steps2update.back()->updateInputStream(*input_stream); - input_stream = &steps2update.back()->getOutputStream(); - steps2update.pop_back(); + steps_to_update.back()->updateInputStream(*input_stream); + input_stream = &steps_to_update.back()->getOutputStream(); + steps_to_update.pop_back(); } return 0; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index d426c12dbdc..dfbf0b8d3fd 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -87,6 +87,7 @@ QueryPlan::Node * findReadingStep(QueryPlan::Node & node) using FixedColumns = std::unordered_set; /// Right now we find only simple cases like 'and(..., and(..., and(column = value, ...), ...' +/// Injective functions are supported here. For a condition 'injectiveFunction(x) = 5' column 'x' is fixed. void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expression, FixedColumns & fixed_columns) { std::stack stack; @@ -107,23 +108,21 @@ void appendFixedColumnsFromFilterExpression(const ActionsDAG::Node & filter_expr else if (name == "equals") { const ActionsDAG::Node * maybe_fixed_column = nullptr; - bool is_single = true; + size_t num_constant_columns = 0; for (const auto & child : node->children) { - if (!child->column) - { - if (!maybe_fixed_column) - maybe_fixed_column = child; - else - is_single = false; - } + if (child->column) + ++num_constant_columns; + else + maybe_fixed_column = child; } - if (maybe_fixed_column && is_single) + if (maybe_fixed_column && num_constant_columns + 1 == node->children.size()) { //std::cerr << "====== Added fixed column " << maybe_fixed_column->result_name << ' ' << static_cast(maybe_fixed_column) << std::endl; fixed_columns.insert(maybe_fixed_column); + /// Support injective functions chain. const ActionsDAG::Node * maybe_injective = maybe_fixed_column; while (maybe_injective->type == ActionsDAG::ActionType::FUNCTION && maybe_injective->children.size() == 1 @@ -146,6 +145,8 @@ void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) dag = expression->clone(); } +/// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. +/// Additionally, build a set of fixed columns. void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns) { IQueryPlanStep * step = node.step.get(); @@ -274,7 +275,7 @@ void enreachFixedColumns(const ActionsDAG & dag, FixedColumns & fixed_columns) /// * Input nodes are mapped by name. /// * Function is mapped to function if all children are mapped and function names are same. /// * Alias is mapped to it's children mapping. -/// * Monotonic function can be mapped to it's children mapping if dirrect mapping does not exist. +/// * Monotonic function can be mapped to it's children mapping if direct mapping does not exist. /// In this case, information about monotonicity is filled. /// * Mapped node is nullptr if there is no mapping found. /// @@ -345,13 +346,13 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG }; MatchedTrees::Matches matches; + std::stack stack; for (const auto & node : outer_dag.getNodes()) { if (matches.contains(&node)) continue; - std::stack stack; stack.push(Frame{&node, {}}); while (!stack.empty()) { @@ -410,8 +411,9 @@ MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG if (frame.mapped_children.size() > 1) { std::vector other_parents; - other_parents.reserve(frame.mapped_children.size()); - for (size_t i = 1; i < frame.mapped_children.size(); ++i) + size_t mapped_children_size = frame.mapped_children.size(); + other_parents.reserve(mapped_children_size); + for (size_t i = 1; i < mapped_children_size; ++i) other_parents.push_back(&inner_parents[frame.mapped_children[i]]); for (const auto * parent : *intersection) @@ -548,17 +550,17 @@ InputOrderInfoPtr buildInputOrderInfo( /// /// So far, 0 means any direction is possible. It is ok for constant prefix. int read_direction = 0; - size_t next_descr_column = 0; + size_t next_description_column = 0; size_t next_sort_key = 0; - while (next_descr_column < description.size() && next_sort_key < sorting_key_columns.size()) + while (next_description_column < description.size() && next_sort_key < sorting_key_columns.size()) { const auto & sorting_key_column = sorting_key_columns[next_sort_key]; - const auto & descr = description[next_descr_column]; + const auto & sort_column_description = description[next_description_column]; /// If required order depend on collation, it cannot be matched with primary key order. /// Because primary keys cannot have collations. - if (descr.collator) + if (sort_column_description.collator) break; /// Direction for current sort key. @@ -578,20 +580,20 @@ InputOrderInfoPtr buildInputOrderInfo( if (sort_column_node->type != ActionsDAG::ActionType::INPUT) break; - if (descr.column_name != sorting_key_column) + if (sort_column_description.column_name != sorting_key_column) break; - current_direction = descr.direction; + current_direction = sort_column_description.direction; //std::cerr << "====== (no dag) Found direct match" << std::endl; - ++next_descr_column; + ++next_description_column; ++next_sort_key; } else { - const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(descr.column_name); + const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(sort_column_description.column_name); /// It is possible when e.g. sort by array joined column. if (!sort_node) break; @@ -609,14 +611,14 @@ InputOrderInfoPtr buildInputOrderInfo( /// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1' /// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order. - current_direction = descr.direction; + current_direction = sort_column_description.direction; if (match.monotonicity) { current_direction *= match.monotonicity->direction; strict_monotonic = match.monotonicity->strict; } - ++next_descr_column; + ++next_description_column; ++next_sort_key; } else if (fixed_key_columns.contains(sort_column_node)) @@ -632,8 +634,8 @@ InputOrderInfoPtr buildInputOrderInfo( if (!is_fixed_column) break; - order_key_prefix_descr.push_back(descr); - ++next_descr_column; + order_key_prefix_descr.push_back(sort_column_description); + ++next_description_column; } } @@ -646,7 +648,7 @@ InputOrderInfoPtr buildInputOrderInfo( read_direction = current_direction; if (current_direction) - order_key_prefix_descr.push_back(descr); + order_key_prefix_descr.push_back(sort_column_description); if (current_direction && !strict_monotonic) break; @@ -712,9 +714,7 @@ InputOrderInfoPtr buildInputOrderInfo( return order_info; } -InputOrderInfoPtr buildInputOrderInfo( - SortingStep & sorting, - QueryPlan::Node & node) +InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & node) { QueryPlan::Node * reading_node = findReadingStep(node); if (!reading_node) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index d16e0a8904d..59f3e094cb7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -441,7 +441,11 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (info.sum_marks == 0) return {}; - bool added_prewhere_output = false; + /// PREWHERE actions can remove some input columns (which are needed only for prewhere condition). + /// In case of read-in-order, PREWHERE is executed before sorting. But removed columns could be needed for sorting key. + /// To fix this, we prohibit removing any input in prewhere actions. Instead, projection actions will be added after sorting. + /// See 02354_read_in_order_prewhere.sql as an example. + bool have_input_columns_removed_after_prewhere = false; if (prewhere_info && prewhere_info->prewhere_actions) { auto & outputs = prewhere_info->prewhere_actions->getOutputs(); @@ -451,7 +455,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( if (!outputs_set.contains(input)) { outputs.push_back(input); - added_prewhere_output = true; + have_input_columns_removed_after_prewhere = true; } } } @@ -626,7 +630,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder( } } - if (!pipes.empty() && (need_preliminary_merge || added_prewhere_output)) + if (!pipes.empty() && (need_preliminary_merge || have_input_columns_removed_after_prewhere)) /// Drop temporary columns, added by 'sorting_key_prefix_expr' out_projection = createProjection(pipe_header); diff --git a/src/Processors/QueryPlan/SortingStep.cpp b/src/Processors/QueryPlan/SortingStep.cpp index d9d1f08b661..8766c0ba335 100644 --- a/src/Processors/QueryPlan/SortingStep.cpp +++ b/src/Processors/QueryPlan/SortingStep.cpp @@ -23,6 +23,23 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +SortingStep::Settings::Settings(const Context & context) +{ + const auto & settings = context.getSettingsRef(); + max_block_size = settings.max_block_size; + size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + max_bytes_before_remerge = settings.max_bytes_before_remerge_sort; + remerge_lowered_memory_bytes_ratio = settings.remerge_sort_lowered_memory_bytes_ratio; + max_bytes_before_external_sort = settings.max_bytes_before_external_sort; + tmp_data = context.getTempDataOnDisk(); + min_free_disk_space = settings.min_free_disk_space_for_temporary_data; +} + +SortingStep::Settings::Settings(size_t max_block_size_) +{ + max_block_size = max_block_size_; +} + static ITransformingStep::Traits getTraits(size_t limit) { return ITransformingStep::Traits @@ -71,8 +88,8 @@ SortingStep::SortingStep( , prefix_description(std::move(prefix_description_)) , result_description(std::move(result_description_)) , limit(limit_) + , sort_settings(max_block_size_) { - sort_settings.max_block_size = max_block_size_; /// TODO: check input_stream is sorted by prefix_description. output_stream->sort_description = result_description; output_stream->sort_scope = DataStream::SortScope::Global; @@ -87,6 +104,7 @@ SortingStep::SortingStep( , type(Type::MergingSorted) , result_description(std::move(sort_description_)) , limit(limit_) + , sort_settings(max_block_size_) { sort_settings.max_block_size = max_block_size_; /// TODO: check input_stream is partially sorted (each port) by the same description. diff --git a/src/Processors/QueryPlan/SortingStep.h b/src/Processors/QueryPlan/SortingStep.h index a24df8d48f8..c8b890dac90 100644 --- a/src/Processors/QueryPlan/SortingStep.h +++ b/src/Processors/QueryPlan/SortingStep.h @@ -27,6 +27,9 @@ public: size_t max_bytes_before_external_sort = 0; TemporaryDataOnDiskScopePtr tmp_data = nullptr; size_t min_free_disk_space = 0; + + explicit Settings(const Context & context); + explicit Settings(size_t max_block_size_); }; /// Full From 3c3771a8d7e86b4e27028603a0cc9448499c1988 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 16:17:36 +0000 Subject: [PATCH 294/526] Fix typo. --- src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index dfbf0b8d3fd..83753c701c7 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -575,7 +575,7 @@ InputOrderInfoPtr buildInputOrderInfo( if (!dag) { /// This is possible if there were no Expression or Filter steps in Plan. - /// Examlpe: SELECT * FROM tab ORDER BY a, b + /// Example: SELECT * FROM tab ORDER BY a, b if (sort_column_node->type != ActionsDAG::ActionType::INPUT) break; From 5a8c62ee0140b9bed9460bdd5d146116c91ebe02 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Nov 2022 16:26:27 +0100 Subject: [PATCH 295/526] Analyzer improve aliases support for table expressions --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 102 ++++++++++-------- ...nalyzer_table_expression_aliases.reference | 19 ++++ ...2478_analyzer_table_expression_aliases.sql | 50 +++++++++ 3 files changed, 129 insertions(+), 42 deletions(-) create mode 100644 tests/queries/0_stateless/02478_analyzer_table_expression_aliases.reference create mode 100644 tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 4f24051df79..882eed3eaf7 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -347,15 +347,15 @@ struct IdentifierResolveResult struct IdentifierResolveSettings { - /// Allow to check parent scopes during identifier resolution - bool allow_to_check_parent_scopes = true; - /// Allow to check join tree during identifier resolution bool allow_to_check_join_tree = true; /// Allow to check CTEs during table identifier resolution bool allow_to_check_cte = true; + /// Allow to check parent scopes during identifier resolution + bool allow_to_check_parent_scopes = true; + /// Allow to check database catalog during table identifier resolution bool allow_to_check_database_catalog = true; @@ -429,48 +429,53 @@ public: { if (node->hasAlias()) { - expressions.emplace_back(node.get(), node->getAlias()); - ++alias_name_to_expressions_size[expressions.back().second]; - return; + const auto & node_alias = node->getAlias(); + alias_name_to_expressions[node_alias].push_back(node); } - expressions.emplace_back(node.get(), std::string()); + expressions.emplace_back(node); } void popNode() { - const auto & [_, top_expression_alias] = expressions.back(); + const auto & top_expression = expressions.back(); + const auto & top_expression_alias = top_expression->getAlias(); + if (!top_expression_alias.empty()) { - auto it = alias_name_to_expressions_size.find(top_expression_alias); - --it->second; + auto it = alias_name_to_expressions.find(top_expression_alias); + auto & alias_expressions = it->second; + alias_expressions.pop_back(); - if (it->second == 0) - alias_name_to_expressions_size.erase(it); + if (alias_expressions.empty()) + alias_name_to_expressions.erase(it); } expressions.pop_back(); } - const IQueryTreeNode * getRoot() const + [[maybe_unused]] const QueryTreeNodePtr & getRoot() const { - if (expressions.empty()) - return nullptr; - - return expressions.front().first; + return expressions.front(); } - const IQueryTreeNode * getTop() const + const QueryTreeNodePtr & getTop() const { - if (expressions.empty()) - return nullptr; - - return expressions.back().first; + return expressions.back(); } - bool hasExpressionWithAlias(const std::string & alias) const + [[maybe_unused]] bool hasExpressionWithAlias(const std::string & alias) const { - return alias_name_to_expressions_size.find(alias) != alias_name_to_expressions_size.end(); + return alias_name_to_expressions.contains(alias); + } + + QueryTreeNodePtr getExpressionWithAlias(const std::string & alias) const + { + auto expression_it = alias_name_to_expressions.find(alias); + if (expression_it == alias_name_to_expressions.end()) + return {}; + + return expression_it->second.front(); } [[maybe_unused]] size_t size() const @@ -487,11 +492,12 @@ public: { buffer << expressions.size() << '\n'; - for (const auto & [expression, alias] : expressions) + for (const auto & expression : expressions) { buffer << "Expression "; buffer << expression->formatASTForErrorMessage(); + const auto & alias = expression->getAlias(); if (!alias.empty()) buffer << " alias " << alias; @@ -508,8 +514,8 @@ public: } private: - std::vector> expressions; - std::unordered_map alias_name_to_expressions_size; + QueryTreeNodes expressions; + std::unordered_map> alias_name_to_expressions; }; /** Projection names is name of query tree node that is used in projection part of query node. @@ -1775,7 +1781,7 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi /** Resolve identifier from scope aliases. * * Resolve strategy: - * 1. If alias is registered current expressions that are in resolve process and if last expression is not part of first expression subtree + * 1. If alias is registered in current expressions that are in resolve process and if top expression is not part of bottom expression with the same alias subtree * throw cyclic aliases exception. * Otherwise prevent cache usage for identifier lookup and return nullptr. * @@ -1800,24 +1806,22 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi * * 2. Depending on IdentifierLookupContext get alias name to node map from IdentifierResolveScope. * 3. Try to bind identifier to alias name in map. If there are no such binding return nullptr. - * 4. Add node into current expressions to resolve. TODO: Handle lambdas and tables properly. - * - * 5. If node in map is not resolved, resolve it. It is important because for result type of identifier lookup node can depend on it. + * 4. If node in map is not resolved, resolve it. It is important in case of compound expressions. * Example: SELECT value.a, cast('(1)', 'Tuple(a UInt64)') AS value; * - * Special case for IdentifierNode, if node is identifier depending on lookup context we need to erase entry from expression or lambda map. - * Check QueryExpressionsAliasVisitor documentation. + * Special case if node is identifier node. + * Example: SELECT value, id AS value FROM test_table; * - * Special case for QueryNode, if lookup context is expression, evaluate it as scalar subquery. + * Add node in current scope expressions in resolve process stack. + * Try to resolve identifier. + * If identifier is resolved, depending on lookup context, erase entry from expression or lambda map. Check QueryExpressionsAliasVisitor documentation. + * Pop node from current scope expressions in resolve process stack. * - * 6. Pop node from current expressions to resolve. - * 7. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node + * 5. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. * * Example: SELECT value AS alias, alias.nested_path. * Result: SELECT value AS alias, tupleElement(value, 'nested_path') value.nested_path. - * - * 8. If identifier lookup is in expression context, clone result expression. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { @@ -1845,12 +1849,11 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier identifier_bind_part, scope.scope_node->formatASTForErrorMessage()); - if (scope.expressions_in_resolve_process_stack.hasExpressionWithAlias(identifier_bind_part)) + if (auto root_expression_wih_alias = scope.expressions_in_resolve_process_stack.getExpressionWithAlias(identifier_bind_part)) { - const auto * root_expression = scope.expressions_in_resolve_process_stack.getRoot(); - const auto * top_expression = scope.expressions_in_resolve_process_stack.getTop(); + const auto top_expression = scope.expressions_in_resolve_process_stack.getTop(); - if (!isNodePartOfTree(top_expression, root_expression)) + if (!isNodePartOfTree(top_expression.get(), root_expression_wih_alias.get())) throw Exception(ErrorCodes::CYCLIC_ALIASES, "Cyclic aliases for identifier '{}'. In scope {}", identifier_lookup.identifier.getFullName(), @@ -4611,6 +4614,8 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod auto & from_table_identifier = current_join_tree_node->as(); auto table_identifier_lookup = IdentifierLookup{from_table_identifier.getIdentifier(), IdentifierLookupContext::TABLE_EXPRESSION}; + auto from_table_identifier_alias = from_table_identifier.getAlias(); + IdentifierResolveSettings resolve_settings; /// In join tree initialization ignore join tree as identifier lookup source resolve_settings.allow_to_check_join_tree = false; @@ -4621,7 +4626,15 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod */ resolve_settings.allow_to_resolve_subquery_during_identifier_resolution = false; + scope.expressions_in_resolve_process_stack.pushNode(current_join_tree_node); + auto table_identifier_resolve_result = tryResolveIdentifier(table_identifier_lookup, scope, resolve_settings); + + scope.expressions_in_resolve_process_stack.popNode(); + bool expression_was_root = scope.expressions_in_resolve_process_stack.empty(); + if (expression_was_root) + scope.non_cached_identifier_lookups_during_expression_resolve.clear(); + auto resolved_identifier = table_identifier_resolve_result.resolved_identifier; if (!resolved_identifier) @@ -4632,6 +4645,11 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod resolved_identifier = resolved_identifier->clone(); + /// Update alias name to table expression map + auto table_expression_it = scope.alias_name_to_table_expression_node.find(from_table_identifier_alias); + if (table_expression_it != scope.alias_name_to_table_expression_node.end()) + table_expression_it->second = resolved_identifier; + auto table_expression_modifiers = from_table_identifier.getTableExpressionModifiers(); auto * resolved_identifier_query_node = resolved_identifier->as(); diff --git a/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.reference b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.reference new file mode 100644 index 00000000000..1a9d5016345 --- /dev/null +++ b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.reference @@ -0,0 +1,19 @@ +0 Value +-- +0 Value 0 Value +-- +0 Value 0 Value +-- +0 Value 0 Value +-- +1 1 +-- +1 1 +-- +1 +-- +1 1 +-- +0 Value 0 Value 0 Value 0 Value +-- +1 1 1 1 diff --git a/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql new file mode 100644 index 00000000000..66f50f7b26d --- /dev/null +++ b/tests/queries/0_stateless/02478_analyzer_table_expression_aliases.sql @@ -0,0 +1,50 @@ +SET allow_experimental_analyzer = 1; + +DROP TABLE IF EXISTS test_table; +CREATE TABLE test_table +( + id UInt64, + value String +) ENGINE = TinyLog; + +INSERT INTO test_table VALUES (0, 'Value'); + +SELECT * FROM test_table AS test_table; + +SELECT '--'; + +SELECT * FROM test_table AS t1, t1; + +SELECT '--'; + +SELECT * FROM t1, test_table AS t1; + +SELECT '--'; + +SELECT * FROM test_table AS test_table, test_table; + +SELECT '--'; + +SELECT * FROM (SELECT 1) AS test_table, test_table AS subquery; + +SELECT '--'; + +SELECT * FROM test_table AS subquery, (SELECT 1) AS test_table; + +SELECT '--'; + +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery AS cte_subquery; + +SELECT '--'; + +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery AS cte_subquery, cte_subquery AS subquery; + +SELECT '--'; + +SELECT * FROM t3, test_table AS t1, t1 AS t2, t2 AS t3; + +SELECT '--'; + +SELECT * FROM t3 AS t4, (SELECT 1) AS t1, t1 AS t2, t2 AS t3; + +DROP TABLE test_table; From ff65ca42ad389cc9c0833d51fdf93db665f39e43 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 18:12:01 +0000 Subject: [PATCH 296/526] Fix limit. --- .../Optimizations/optimizeReadInOrder.cpp | 27 ++++++++++++++++--- 1 file changed, 23 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 83753c701c7..fb8c4b17a3e 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -147,13 +147,16 @@ void appendExpression(ActionsDAGPtr & dag, const ActionsDAGPtr & expression) /// This function builds a common DAG which is a gerge of DAGs from Filter and Expression steps chain. /// Additionally, build a set of fixed columns. -void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns) +void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & fixed_columns, size_t & limit) { IQueryPlanStep * step = node.step.get(); if (auto * reading = typeid_cast(step)) { if (const auto * prewhere_info = reading->getPrewhereInfo()) { + /// Should ignore limit if there is filtering. + limit = 0; + if (prewhere_info->prewhere_actions) { //std::cerr << "====== Adding prewhere " << std::endl; @@ -168,13 +171,24 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & if (node.children.size() != 1) return; - buildSortingDAG(*node.children.front(), dag, fixed_columns); + buildSortingDAG(*node.children.front(), dag, fixed_columns, limit); if (auto * expression = typeid_cast(step)) - appendExpression(dag, expression->getExpression()); + { + const auto & actions = expression->getExpression(); + + /// Should ignore limit because arrayJoin() can reduce the number of rows in case of empty array. + if (actions->hasArrayJoin()) + limit = 0; + + appendExpression(dag, actions); + } if (auto * filter = typeid_cast(step)) { + /// Should ignore limit if there is filtering. + limit = 0; + appendExpression(dag, filter->getExpression()); if (const auto * filter_expression = dag->tryFindInOutputs(filter->getFilterColumnName())) appendFixedColumnsFromFilterExpression(*filter_expression, fixed_columns); @@ -182,6 +196,11 @@ void buildSortingDAG(QueryPlan::Node & node, ActionsDAGPtr & dag, FixedColumns & if (auto * array_join = typeid_cast(step)) { + /// Should ignore limit because ARRAY JOIN can reduce the number of rows in case of empty array. + /// But in case of LEFT ARRAY JOIN the result number of rows is always bigger. + if (!array_join->arrayJoin()->is_left) + limit = 0; + const auto & array_joined_columns = array_join->arrayJoin()->columns; /// Remove array joined columns from outputs. @@ -725,7 +744,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n ActionsDAGPtr dag; FixedColumns fixed_columns; - buildSortingDAG(node, dag, fixed_columns); + buildSortingDAG(node, dag, fixed_columns, limit); if (dag && !fixed_columns.empty()) enreachFixedColumns(*dag, fixed_columns); From 3eb6c03b471e1e1f5dd883e451762d9b0daaad83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 19:46:47 +0000 Subject: [PATCH 297/526] Disable clickhouse local and clickhouse client non-interactive progress by default. --- programs/client/Client.cpp | 2 +- programs/local/LocalServer.cpp | 2 +- src/Client/ClientBase.cpp | 2 +- ...ient_INSERT_progress_profile_events.expect | 2 +- ...ocal_INSERT_progress_profile_events.expect | 2 +- .../0_stateless/02456_progress_tty.expect | 27 ++++++------------- 6 files changed, 13 insertions(+), 24 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index 93136df2a5b..b262c5a0db1 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -1088,7 +1088,7 @@ void Client::processConfig() } else { - std::string progress = config().getString("progress", "tty"); + std::string progress = config().getString("progress", "off"); need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().getBool("echo", false); ignore_error = config().getBool("ignore-error", false); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3ac9c1e7c37..fea82b456cd 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -489,7 +489,7 @@ void LocalServer::processConfig() } else { - std::string progress = config().getString("progress", "tty"); + std::string progress = config().getString("progress", "off"); need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().hasOption("echo") || config().hasOption("verbose"); ignore_error = config().getBool("ignore-error", false); diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 95018f8a632..c8fcf560859 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2324,7 +2324,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::TTY, "tty"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") diff --git a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect index 20333ae7960..07815e57610 100755 --- a/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_client_INSERT_progress_profile_events.expect @@ -24,7 +24,7 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -send "yes | head -n10000000 | \$CLICKHOUSE_CLIENT --query \"insert into function null('foo String') format TSV\" >/dev/null\r" +send "yes | head -n10000000 | \$CLICKHOUSE_CLIENT --progress --query \"insert into function null('foo String') format TSV\" >/dev/null\r" expect "Progress: " send "\3" diff --git a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect index 5c95e17aefd..3333ee93468 100755 --- a/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect +++ b/tests/queries/0_stateless/02310_clickhouse_local_INSERT_progress_profile_events.expect @@ -24,7 +24,7 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -send "yes | head -n10000000 | \$CLICKHOUSE_LOCAL --query \"insert into function null('foo String') format TSV\" >/dev/null\r" +send "yes | head -n10000000 | \$CLICKHOUSE_LOCAL --progress --query \"insert into function null('foo String') format TSV\" >/dev/null\r" expect "Progress: " send "\3" diff --git a/tests/queries/0_stateless/02456_progress_tty.expect b/tests/queries/0_stateless/02456_progress_tty.expect index 9daa6caa3fa..45156c6b0f5 100755 --- a/tests/queries/0_stateless/02456_progress_tty.expect +++ b/tests/queries/0_stateless/02456_progress_tty.expect @@ -17,32 +17,21 @@ expect_after { spawn bash send "source $basedir/../shell_config.sh\r" -# Progress is displayed by default -send "\$CLICKHOUSE_LOCAL --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null\r" -expect "Progress: " -expect "█" -send "\3" - -# It is true even if we redirect both stdout and stderr to /dev/null -send "\$CLICKHOUSE_LOCAL --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" -expect "Progress: " -expect "█" +# Progress is not displayed by default +send "\$CLICKHOUSE_LOCAL --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" send "\3" # The option --progress has implicit value of true -send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" +send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null\r" expect "Progress: " expect "█" send "\3" -# But we can set it to false -send "\$CLICKHOUSE_LOCAL --progress false --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" -expect -exact "0\tHello\r\n" -send "\3" - -# As well as to 0 for the same effect -send "\$CLICKHOUSE_LOCAL --progress 0 --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" -expect -exact "0\tHello\r\n" +# It works even if we redirect both stdout and stderr to /dev/null +send "\$CLICKHOUSE_LOCAL --progress --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" +expect "Progress: " +expect "█" send "\3" # If we set it to 1, the progress will be displayed as well From 675507594a3defd942cf3894b9153d079e24db5b Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 9 Nov 2022 21:01:05 +0100 Subject: [PATCH 298/526] Make error message after dropping current user more correct. --- src/Access/ContextAccess.cpp | 17 +++++++++++------ 1 file changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index 4e409946666..bf6c32ee4bd 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -465,6 +465,17 @@ std::shared_ptr ContextAccess::getAccessRightsWithImplicit() template bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... args) const { + if (user_was_dropped) + { + /// If the current user has been dropped we always throw an exception (even if `throw_if_denied` is false) + /// because dropping of the current user is considered as a situation which is exceptional enough to stop + /// query execution. + throw Exception(getUserName() + ": User has been dropped", ErrorCodes::UNKNOWN_USER); + } + + if (is_full_access) + return true; + auto access_granted = [&] { if (trace_log) @@ -483,12 +494,6 @@ bool ContextAccess::checkAccessImplHelper(AccessFlags flags, const Args &... arg return false; }; - if (is_full_access) - return true; - - if (user_was_dropped) - return access_denied("User has been dropped", ErrorCodes::UNKNOWN_USER); - if (flags & AccessType::CLUSTER && !access_control->doesOnClusterQueriesRequireClusterGrant()) flags &= ~AccessType::CLUSTER; From c6f0701dd8479b673d94f3c82e1170ccbcac7672 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 20:01:14 +0000 Subject: [PATCH 299/526] Fix test. --- .../02155_read_in_order_max_rows_to_read.reference | 5 ----- .../0_stateless/02155_read_in_order_max_rows_to_read.sql | 2 +- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference index d21747026dc..fc8a8ffc551 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.reference @@ -4,10 +4,5 @@ 2 3 4 -11 -12 -13 -14 -15 10 20 diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql index a893221df0f..0ceec7ceb74 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql @@ -13,10 +13,10 @@ SELECT a FROM t_max_rows_to_read WHERE a = 10 SETTINGS max_rows_to_read = 4; SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; -SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 SETTINGS max_rows_to_read = 12; SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 20 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } +SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; -- { serverError 158 } SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 FORMAT Null SETTINGS max_rows_to_read = 4; -- { serverError 158 } DROP TABLE t_max_rows_to_read; From 0261ff509575433607c7bab06b5bb0301ede4899 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 9 Nov 2022 20:02:32 +0000 Subject: [PATCH 300/526] Fix test. --- .../0_stateless/02155_read_in_order_max_rows_to_read.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql index 0ceec7ceb74..314d0610d12 100644 --- a/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql +++ b/tests/queries/0_stateless/02155_read_in_order_max_rows_to_read.sql @@ -16,7 +16,7 @@ SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 SETTINGS max_rows_to_read = 12; SELECT a FROM t_max_rows_to_read ORDER BY a LIMIT 20 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } -SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 SETTINGS max_rows_to_read = 12; -- { serverError 158 } +SELECT a FROM t_max_rows_to_read WHERE a > 10 ORDER BY a LIMIT 5 FORMAT Null SETTINGS max_rows_to_read = 12; -- { serverError 158 } SELECT a FROM t_max_rows_to_read WHERE a = 10 OR a = 20 FORMAT Null SETTINGS max_rows_to_read = 4; -- { serverError 158 } DROP TABLE t_max_rows_to_read; From a3d121f7eab49dde09f28909ff4ccd3a7d95152a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 9 Nov 2022 22:44:16 +0100 Subject: [PATCH 301/526] More stable test --- .../0_stateless/02477_projection_materialize_and_zero_copy.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql index 922b613888f..afd6bad1708 100644 --- a/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql +++ b/tests/queries/0_stateless/02477_projection_materialize_and_zero_copy.sql @@ -13,6 +13,6 @@ alter table t materialize projection p_norm settings mutations_sync = 1; SYSTEM FLUSH LOGS; -SELECT * FROM system.text_log WHERE event_time >= now() - 30 and message like '%BAD_DATA_PART_NAME%'; +SELECT * FROM system.text_log WHERE event_time >= now() - 30 and message like '%BAD_DATA_PART_NAME%' and message like '%p_norm%'; DROP TABLE IF EXISTS t; From ae19af00156586acb5d1af09d4564f957ed06220 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 9 Nov 2022 23:33:03 +0100 Subject: [PATCH 302/526] Fix backup of Lazy databases. --- src/Backups/BackupEntriesCollector.cpp | 2 +- src/Databases/DatabaseMemory.cpp | 1 + src/Databases/DatabaseReplicated.cpp | 5 +++++ src/Databases/DatabasesCommon.cpp | 5 ++++- 4 files changed, 11 insertions(+), 2 deletions(-) diff --git a/src/Backups/BackupEntriesCollector.cpp b/src/Backups/BackupEntriesCollector.cpp index 73f78a13765..c1f7f14960b 100644 --- a/src/Backups/BackupEntriesCollector.cpp +++ b/src/Backups/BackupEntriesCollector.cpp @@ -442,7 +442,7 @@ void BackupEntriesCollector::gatherTablesMetadata() if (it != database_info.tables.end()) { const auto & partitions = it->second.partitions; - if (partitions && !storage->supportsBackupPartition()) + if (partitions && storage && !storage->supportsBackupPartition()) { throw Exception( ErrorCodes::CANNOT_BACKUP_TABLE, diff --git a/src/Databases/DatabaseMemory.cpp b/src/Databases/DatabaseMemory.cpp index 8540c785419..99d88597385 100644 --- a/src/Databases/DatabaseMemory.cpp +++ b/src/Databases/DatabaseMemory.cpp @@ -177,6 +177,7 @@ std::vector> DatabaseMemory::getTablesForBackup(co if (create.getTable() != table_name) throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected name {} for temporary table {}", backQuoteIfNeed(create.getTable()), backQuoteIfNeed(table_name)); + chassert(storage); storage->adjustCreateQueryForBackup(create_table_query); res.emplace_back(create_table_query, storage); } diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 4b7599affb0..20fa11e90e2 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -1214,6 +1214,7 @@ DatabaseReplicated::getTablesForBackup(const FilterByNameFunction & filter, cons String table_name = unescapeForFileName(escaped_table_name); if (!filter(table_name)) continue; + String zk_metadata; if (!zookeeper->tryGet(zookeeper_path + "/metadata/" + escaped_table_name, zk_metadata)) throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Metadata for table {} was not found in ZooKeeper", table_name); @@ -1233,6 +1234,10 @@ DatabaseReplicated::getTablesForBackup(const FilterByNameFunction & filter, cons if (storage) storage->adjustCreateQueryForBackup(create_table_query); } + + /// `storage` is allowed to be null here. In this case it means that this storage exists on other replicas + /// but it has not been created on this replica yet. + res.emplace_back(create_table_query, storage); } diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index 93a9523d115..37fd055456e 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -329,6 +329,10 @@ std::vector> DatabaseWithOwnTablesBase::getTablesF for (auto it = getTablesIterator(local_context, filter); it->isValid(); it->next()) { + auto storage = it->table(); + if (!storage) + continue; /// Probably the table has been just dropped. + auto create_table_query = tryGetCreateTableQuery(it->name(), local_context); if (!create_table_query) throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Couldn't get a create query for table {}.{}", backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(it->name())); @@ -337,7 +341,6 @@ std::vector> DatabaseWithOwnTablesBase::getTablesF if (create.getTable() != it->name()) throw Exception(ErrorCodes::INCONSISTENT_METADATA_FOR_BACKUP, "Got a create query with unexpected name {} for table {}.{}", backQuoteIfNeed(create.getTable()), backQuoteIfNeed(getDatabaseName()), backQuoteIfNeed(it->name())); - auto storage = it->table(); storage->adjustCreateQueryForBackup(create_table_query); res.emplace_back(create_table_query, storage); } From 981ece08edf71b4bdfd16b5f71a102e0aed4fd24 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 00:55:39 +0000 Subject: [PATCH 303/526] Rewrite tests for memory overcommit --- .../configs/global_overcommit_tracker.xml | 3 - .../test_global_overcommit_tracker/test.py | 52 ------------ .../__init__.py | 0 .../configs/global_overcommit_tracker.xml | 3 + .../test_overcommit_tracker/test.py | 85 +++++++++++++++++++ .../02104_overcommit_memory.reference | 1 - .../0_stateless/02104_overcommit_memory.sh | 49 ----------- 7 files changed, 88 insertions(+), 105 deletions(-) delete mode 100644 tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml delete mode 100644 tests/integration/test_global_overcommit_tracker/test.py rename tests/integration/{test_global_overcommit_tracker => test_overcommit_tracker}/__init__.py (100%) create mode 100644 tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml create mode 100644 tests/integration/test_overcommit_tracker/test.py delete mode 100644 tests/queries/0_stateless/02104_overcommit_memory.reference delete mode 100755 tests/queries/0_stateless/02104_overcommit_memory.sh diff --git a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml deleted file mode 100644 index a05d8865a6b..00000000000 --- a/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml +++ /dev/null @@ -1,3 +0,0 @@ - - 50000000 - \ No newline at end of file diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py deleted file mode 100644 index 871f9ca983e..00000000000 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ /dev/null @@ -1,52 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster - -cluster = ClickHouseCluster(__file__) - -node = cluster.add_instance( - "node", main_configs=["configs/global_overcommit_tracker.xml"] -) - - -@pytest.fixture(scope="module", autouse=True) -def start_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -TEST_QUERY_A = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=1, memory_usage_overcommit_max_wait_microseconds=500" -TEST_QUERY_B = "SELECT number FROM numbers(1000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator_for_user=2, memory_usage_overcommit_max_wait_microseconds=500" - - -def test_overcommited_is_killed(): - node.query("CREATE USER A") - node.query("GRANT ALL ON *.* TO A") - node.query("CREATE USER B") - node.query("GRANT ALL ON *.* TO B") - - responses_A = list() - responses_B = list() - for _ in range(500): - responses_A.append(node.get_query_request(TEST_QUERY_A, user="A")) - responses_B.append(node.get_query_request(TEST_QUERY_B, user="B")) - - overcommited_killed = False - for response in responses_A: - _, err = response.get_answer_and_error() - if "MEMORY_LIMIT_EXCEEDED" in err: - overcommited_killed = True - finished = False - for response in responses_B: - _, err = response.get_answer_and_error() - if err == "": - finished = True - - assert overcommited_killed, "no overcommited task was killed" - assert finished, "all tasks are killed" - - node.query("DROP USER IF EXISTS A") - node.query("DROP USER IF EXISTS B") diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_overcommit_tracker/__init__.py similarity index 100% rename from tests/integration/test_global_overcommit_tracker/__init__.py rename to tests/integration/test_overcommit_tracker/__init__.py diff --git a/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml new file mode 100644 index 00000000000..6f83a570ccc --- /dev/null +++ b/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml @@ -0,0 +1,3 @@ + + 2000000000 + \ No newline at end of file diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py new file mode 100644 index 00000000000..0592915e036 --- /dev/null +++ b/tests/integration/test_overcommit_tracker/test.py @@ -0,0 +1,85 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", main_configs=["configs/global_overcommit_tracker.xml"] +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" +GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" + +USER_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=1" +USER_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=80000000" + + +def test_user_overcommit(): + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(USER_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(USER_TEST_QUERY_B, user="A")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + + +def test_global_overcommit(): + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + node.query("CREATE USER IF NOT EXISTS B") + node.query("GRANT ALL ON *.* TO B") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + node.query("DROP USER IF EXISTS B") diff --git a/tests/queries/0_stateless/02104_overcommit_memory.reference b/tests/queries/0_stateless/02104_overcommit_memory.reference deleted file mode 100644 index b108f48e0fa..00000000000 --- a/tests/queries/0_stateless/02104_overcommit_memory.reference +++ /dev/null @@ -1 +0,0 @@ -OVERCOMMITED WITH USER LIMIT WAS KILLED diff --git a/tests/queries/0_stateless/02104_overcommit_memory.sh b/tests/queries/0_stateless/02104_overcommit_memory.sh deleted file mode 100755 index 384e30b1283..00000000000 --- a/tests/queries/0_stateless/02104_overcommit_memory.sh +++ /dev/null @@ -1,49 +0,0 @@ -#!/usr/bin/env bash -# Tags: no-parallel, no-fasttest - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -$CLICKHOUSE_CLIENT -q 'CREATE USER IF NOT EXISTS u02104 IDENTIFIED WITH no_password' -$CLICKHOUSE_CLIENT -q 'GRANT ALL ON *.* TO u02104' - -function overcommited() -{ - while true; do - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS memory_overcommit_ratio_denominator=1,memory_usage_overcommit_max_wait_microseconds=500' 2>&1 \ - | grep -F -q "MEMORY_LIMIT_EXCEEDED" && echo "OVERCOMMITED WITH USER LIMIT IS KILLED" - done -} - -function expect_execution() -{ - while true; do - $CLICKHOUSE_CLIENT -u u02104 -q 'SELECT number FROM numbers(130000) GROUP BY number SETTINGS max_memory_usage_for_user=5000000,memory_overcommit_ratio_denominator=2,memory_usage_overcommit_max_wait_microseconds=500' >/dev/null 2>/dev/null - done -} - -export -f overcommited -export -f expect_execution - -function user_test() -{ - for _ in {1..10}; - do - timeout 10 bash -c overcommited & - timeout 10 bash -c expect_execution & - done; - - wait -} - -output=$(user_test) - -if test -z "$output" -then - echo "OVERCOMMITED WITH USER LIMIT WAS NOT KILLED" -else - echo "OVERCOMMITED WITH USER LIMIT WAS KILLED" -fi - -$CLICKHOUSE_CLIENT -q 'DROP USER IF EXISTS u02104' From eb612a699d8acdd9f363c321b1d553e1a857d501 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 01:08:12 +0000 Subject: [PATCH 304/526] Automatic style fix --- tests/integration/test_overcommit_tracker/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 0592915e036..5c18ee950d5 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -8,6 +8,7 @@ node = cluster.add_instance( "node", main_configs=["configs/global_overcommit_tracker.xml"] ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: From c6449a1b237d27b747f0c9d4f1ad0efcb46c95af Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 10 Nov 2022 10:46:51 +0800 Subject: [PATCH 305/526] fix bug --- src/Functions/FunctionsStringArray.h | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Functions/FunctionsStringArray.h b/src/Functions/FunctionsStringArray.h index ef0d9ac2896..30a8a80a5b3 100644 --- a/src/Functions/FunctionsStringArray.h +++ b/src/Functions/FunctionsStringArray.h @@ -77,6 +77,9 @@ public: /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) { + if (arguments.empty() || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName()); + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -142,6 +145,9 @@ public: /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) { + if (arguments.empty() || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName()); + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -207,6 +213,9 @@ public: /// Check the type of the function's arguments. static void checkArguments(const DataTypes & arguments) { + if (arguments.empty() || arguments.size() > 2) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {} takes one or two arguments", getName()); + if (!isString(arguments[0])) throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); From 6cbfab4e954f77e94daa6d13ca6f337136addade Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 10 Nov 2022 03:10:25 +0000 Subject: [PATCH 306/526] fix test --- tests/integration/test_storage_delta/test.py | 4 ++-- tests/integration/test_storage_hudi/test.py | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 98e06d0792b..6f3b7648162 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -126,7 +126,7 @@ def test_select_query(started_cluster): run_query(instance, create_query) select_query = "SELECT {} FROM deltalake FORMAT TSV" - select_table_function_query = "SELECT {} FROM deltaLake('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')" + select_table_function_query = "SELECT {col} FROM deltaLake('http://{ip}:{port}/{bucket}/test_table/', 'minio', 'minio123') FORMAT TSV" for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() @@ -134,6 +134,6 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query( - instance, select_table_function_query.format(column_name) + instance, select_table_function_query.format(col=column_name, ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket) ).splitlines() assert len(result) > 0 diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index b328c9f2c3a..e8a720dcc02 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -133,7 +133,7 @@ def test_select_query(started_cluster): select_query = "SELECT {} FROM hudi FORMAT TSV" - select_table_function_query = "SELECT {} FROM hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123')" + select_table_function_query = "SELECT {col} FROM hudi('http://{ip}:{port}/{bucket}/test_table/', 'minio', 'minio123') FORMAT TSV" for column_name in columns: result = run_query(instance, select_query.format(column_name)).splitlines() @@ -141,7 +141,7 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query( - instance, select_table_function_query.format(column_name) + instance, select_table_function_query.format(col=column_name, ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket) ).splitlines() assert len(result) > 0 @@ -150,10 +150,10 @@ def test_select_query(started_cluster): "SELECT DISTINCT partitionpath FROM hudi ORDER BY partitionpath FORMAT TSV" ) - distinct_select_table_function_query = "SELECT DISTINCT partitionpath FROM hudi('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/test_table/', 'minio', 'minio123') ORDER BY partitionpath FORMAT TSV" + distinct_select_table_function_query = "SELECT DISTINCT partitionpath FROM hudi('http://{ip}:{port}/{bucket}/test_table/', 'minio', 'minio123') ORDER BY partitionpath FORMAT TSV" result = run_query(instance, distinct_select_query) - result_table_function = run_query(instance, distinct_select_query) + result_table_function = run_query(instance, distinct_select_query.format(ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket)) expected = [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", From da77199bf577fa90e5bea1b43db3b50a5543b816 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 10 Nov 2022 03:25:40 +0000 Subject: [PATCH 307/526] fix style fix style --- tests/integration/test_storage_delta/test.py | 8 +++++++- tests/integration/test_storage_hudi/test.py | 15 +++++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_delta/test.py b/tests/integration/test_storage_delta/test.py index 6f3b7648162..a63244df814 100644 --- a/tests/integration/test_storage_delta/test.py +++ b/tests/integration/test_storage_delta/test.py @@ -134,6 +134,12 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query( - instance, select_table_function_query.format(col=column_name, ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket) + instance, + select_table_function_query.format( + col=column_name, + ip=started_cluster.minio_ip, + port=started_cluster.minio_port, + bucket=bucket, + ), ).splitlines() assert len(result) > 0 diff --git a/tests/integration/test_storage_hudi/test.py b/tests/integration/test_storage_hudi/test.py index e8a720dcc02..dd870aae42e 100644 --- a/tests/integration/test_storage_hudi/test.py +++ b/tests/integration/test_storage_hudi/test.py @@ -141,7 +141,13 @@ def test_select_query(started_cluster): for column_name in columns: result = run_query( - instance, select_table_function_query.format(col=column_name, ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket) + instance, + select_table_function_query.format( + col=column_name, + ip=started_cluster.minio_ip, + port=started_cluster.minio_port, + bucket=bucket, + ), ).splitlines() assert len(result) > 0 @@ -153,7 +159,12 @@ def test_select_query(started_cluster): distinct_select_table_function_query = "SELECT DISTINCT partitionpath FROM hudi('http://{ip}:{port}/{bucket}/test_table/', 'minio', 'minio123') ORDER BY partitionpath FORMAT TSV" result = run_query(instance, distinct_select_query) - result_table_function = run_query(instance, distinct_select_query.format(ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket)) + result_table_function = run_query( + instance, + distinct_select_query.format( + ip=started_cluster.minio_ip, port=started_cluster.minio_port, bucket=bucket + ), + ) expected = [ "americas/brazil/sao_paulo", "americas/united_states/san_francisco", From a5a1f5abf7ca4084d66ede084cede34362d5f698 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Thu, 10 Nov 2022 07:33:54 +0000 Subject: [PATCH 308/526] fix client.py - should use '--progress', return tests - they are working, fix help string --- src/Client/ClientBase.cpp | 2 +- tests/queries/0_stateless/02456_progress_tty.expect | 10 ++++++++++ tests/queries/0_stateless/helpers/client.py | 2 +- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index c8fcf560859..5bd65ac2334 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2324,7 +2324,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY (default): tty|on|1|true|yes; to STDERR: err; OFF: off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR: err; OFF (default): off|0|false|no") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") diff --git a/tests/queries/0_stateless/02456_progress_tty.expect b/tests/queries/0_stateless/02456_progress_tty.expect index 45156c6b0f5..ba6cc0537eb 100755 --- a/tests/queries/0_stateless/02456_progress_tty.expect +++ b/tests/queries/0_stateless/02456_progress_tty.expect @@ -34,6 +34,16 @@ expect "Progress: " expect "█" send "\3" +# But we can set it to false +send "\$CLICKHOUSE_LOCAL --progress false --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" +send "\3" + +# As well as to 0 for the same effect +send "\$CLICKHOUSE_LOCAL --progress 0 --query 'SELECT sleep(1), \$\$Hello\$\$ FROM numbers(3) SETTINGS max_block_size = 1' 2>/dev/null\r" +expect -exact "0\tHello\r\n" +send "\3" + # If we set it to 1, the progress will be displayed as well send "\$CLICKHOUSE_LOCAL --progress 1 --query 'SELECT sum(sleep(1) = 0) FROM numbers(3) SETTINGS max_block_size = 1' >/dev/null 2>&1\r" expect "Progress: " diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 5c8589dfca1..184b6b22cb6 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,7 +16,7 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" + command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From 055e655145ae9b7f1062cc76dc23b0ebe9684509 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 07:41:13 +0000 Subject: [PATCH 309/526] Automatic style fix --- tests/queries/0_stateless/helpers/client.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 184b6b22cb6..7c9b2c1c83f 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,7 +16,9 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + command = ( + os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + ) self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From 839de0cd092420356846c6a53468e603f4200594 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 10 Nov 2022 16:03:11 +0800 Subject: [PATCH 310/526] add function factorial --- .../sql-reference/functions/math-functions.md | 30 +++++++++++++++++++ .../00087_math_functions.reference | 3 ++ .../0_stateless/00087_math_functions.sql | 7 +++++ 3 files changed, 40 insertions(+) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 430762a1885..bfd2f63dc5b 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -549,3 +549,33 @@ Result: │ 3.141592653589793 │ └───────────────────┘ ``` + + +## factorial(n) + +Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is Int64. + +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20; a value of 21 or greater overflows the range for Int64 and will causes an throw exception. + + +**Syntax** + +``` sql +factorial(n) +``` + +**Example** + +Query: + +``` sql +SELECT factorial(10); +``` + +Result: + +``` text +┌─factorial(10)─┐ +│ 3628800 │ +└───────────────┘ +``` diff --git a/tests/queries/0_stateless/00087_math_functions.reference b/tests/queries/0_stateless/00087_math_functions.reference index e02aac11faf..48cb24098ff 100644 --- a/tests/queries/0_stateless/00087_math_functions.reference +++ b/tests/queries/0_stateless/00087_math_functions.reference @@ -112,3 +112,6 @@ 1 1 1 +1 +1 +1 diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index e40acfb3481..11456944be3 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -137,3 +137,10 @@ select erf(10) = 1; select erfc(0) = 1; select erfc(-10) = 2; select erfc(28) = 0; + +select factorial(-1) = 1; +select factorial(0) = 1; +select factorial(10) = 3628800; +select factorial(100); -- { serverError 36 } +select factorial('100'); -- { serverError 43 } +select factorial(100.1234); -- { serverError 43 } From d76c271242b9f4a5ce025d6096857bb2d84adeff Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 10 Nov 2022 16:08:22 +0800 Subject: [PATCH 311/526] add missed file --- src/Functions/factorial.cpp | 103 ++++++++++++++++++++++++++++++++++++ 1 file changed, 103 insertions(+) create mode 100644 src/Functions/factorial.cpp diff --git a/src/Functions/factorial.cpp b/src/Functions/factorial.cpp new file mode 100644 index 00000000000..d92cb668214 --- /dev/null +++ b/src/Functions/factorial.cpp @@ -0,0 +1,103 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int BAD_ARGUMENTS; +} + +template +struct FactorialImpl +{ + using ResultType = Int64; + static const constexpr bool allow_decimal = false; + static const constexpr bool allow_fixed_string = false; + static const constexpr bool allow_string_integer = false; + + static inline NO_SANITIZE_UNDEFINED ResultType apply(A a) + { + if constexpr (std::is_floating_point_v || is_over_big_int) + throw Exception( + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type of argument of function factorial, should not be floating point or big int"); + + if constexpr (is_integer) + { + if (a > 20) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "The maximum value for the input argument of function factorial is 20"); + + if constexpr (is_unsigned_v) + return factorials[a]; + else if constexpr (is_signed_v) + return a >= 0 ? factorials[a] : 1; + } + } + +#if USE_EMBEDDED_COMPILER + static constexpr bool compilable = false; /// special type handling, some other time +#endif + +private: + static const constexpr ResultType factorials[21] + = {1, + 1, + 2, + 6, + 24, + 120, + 720, + 5040, + 40320, + 362880, + 3628800, + 39916800, + 479001600, + 6227020800L, + 87178291200L, + 1307674368000L, + 20922789888000L, + 355687428096000L, + 6402373705728000L, + 121645100408832000L, + 2432902008176640000L}; +}; + +struct NameFactorial { static constexpr auto name = "factorial"; }; +using FunctionFactorial = FunctionUnaryArithmetic; + +template <> struct FunctionUnaryArithmeticMonotonicity +{ + static bool has() { return true; } + + static IFunction::Monotonicity get(const Field & /*left*/, const Field & /*right*/) + { + return { + .is_monotonic = true, + .is_positive = true, + .is_always_monotonic = true, + .is_strict = false, + }; + } +}; + + +REGISTER_FUNCTION(Factorial) +{ + factory.registerFunction( + { + R"( +Computes the factorial of an integer value. It works with any native integer type. The return type is Int64. + +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20; a value of 21 or greater overflows the range for Int64 and will causes an throw exception. +)", + Documentation::Examples{{"factorial", "SELECT factorial(10)"}}, + Documentation::Categories{"Mathematical"}}, + FunctionFactory::CaseInsensitive); +} + +} From a1fd2752094f9ccdd01f134e46b0cc0ff68b2c23 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Thu, 10 Nov 2022 09:13:52 +0100 Subject: [PATCH 312/526] Fixed duplicated includes --- src/Functions/FunctionsJSON.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index a89a2b24b9e..b4b15a25047 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -21,11 +21,6 @@ #include #include -#include -#include -#include -#include -#include #include #include #include From 961befb1f5433236426af9baf23642d2fc79685b Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 9 Nov 2022 16:27:04 +0000 Subject: [PATCH 313/526] Not implemented for window frame type 'groups' in analyzer --- src/Planner/PlannerWindowFunctions.cpp | 10 ++++++++++ .../02478_window_frame_type_groups.reference | 0 .../0_stateless/02478_window_frame_type_groups.sql | 7 +++++++ 3 files changed, 17 insertions(+) create mode 100644 tests/queries/0_stateless/02478_window_frame_type_groups.reference create mode 100644 tests/queries/0_stateless/02478_window_frame_type_groups.sql diff --git a/src/Planner/PlannerWindowFunctions.cpp b/src/Planner/PlannerWindowFunctions.cpp index 4fe60a18099..5f4427d98d4 100644 --- a/src/Planner/PlannerWindowFunctions.cpp +++ b/src/Planner/PlannerWindowFunctions.cpp @@ -11,6 +11,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + namespace { @@ -65,6 +70,11 @@ std::vector extractWindowDescriptions(const QueryTreeNodes & auto & window_function_node_typed = window_function_node->as(); auto function_window_description = extractWindowDescriptionFromWindowNode(window_function_node_typed.getWindowNode(), planner_context); + + auto frame_type = function_window_description.frame.type; + if (frame_type != WindowFrame::FrameType::ROWS && frame_type != WindowFrame::FrameType::RANGE) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Window frame '{}' is not implemented", frame_type); + auto window_name = function_window_description.window_name; auto [it, _] = window_name_to_description.emplace(window_name, std::move(function_window_description)); diff --git a/tests/queries/0_stateless/02478_window_frame_type_groups.reference b/tests/queries/0_stateless/02478_window_frame_type_groups.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02478_window_frame_type_groups.sql b/tests/queries/0_stateless/02478_window_frame_type_groups.sql new file mode 100644 index 00000000000..4c6d663791b --- /dev/null +++ b/tests/queries/0_stateless/02478_window_frame_type_groups.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer = 0; + +SELECT toUInt64(dense_rank(1) OVER (ORDER BY 100 ASC GROUPS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)) FROM numbers(10); -- { serverError 48 } + +SET allow_experimental_analyzer = 1; + +SELECT toUInt64(dense_rank(1) OVER (ORDER BY 100 ASC GROUPS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)) FROM numbers(10); -- { serverError 48 } From c50e7957886b6fb7c8584a97d3bbd256316473ad Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 10 Nov 2022 11:04:51 +0000 Subject: [PATCH 314/526] Fix expected ZXID --- tests/integration/test_keeper_four_word_command/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_four_word_command/test.py b/tests/integration/test_keeper_four_word_command/test.py index 4559904f8b7..c425c18158b 100644 --- a/tests/integration/test_keeper_four_word_command/test.py +++ b/tests/integration/test_keeper_four_word_command/test.py @@ -367,7 +367,7 @@ def test_cmd_stat(started_cluster): assert result["Received"] == "10" assert result["Sent"] == "10" assert int(result["Connections"]) == 1 - assert int(result["Zxid"]) > 14 + assert int(result["Zxid"]) >= 10 assert result["Mode"] == "leader" assert result["Node count"] == "13" From fb8c527a2f8df7b201ff0e6654e77bee89f21166 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Nov 2022 14:12:24 +0300 Subject: [PATCH 315/526] Revert " Keeper retries during insert (clean)" --- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Core/Settings.h | 5 - .../MergeTree/EphemeralLockInZooKeeper.cpp | 39 +- .../MergeTree/EphemeralLockInZooKeeper.h | 15 +- ...ReplicatedMergeTreeMergeStrategyPicker.cpp | 4 - .../MergeTree/ReplicatedMergeTreeSink.cpp | 247 +++----- .../MergeTree/ReplicatedMergeTreeSink.h | 19 +- src/Storages/MergeTree/ZooKeeperRetries.h | 265 --------- .../MergeTree/ZooKeeperWithFaultInjection.h | 527 ------------------ src/Storages/StorageReplicatedMergeTree.cpp | 93 +--- src/Storages/StorageReplicatedMergeTree.h | 35 +- tests/config/install.sh | 1 - .../config/users.d/insert_keeper_retries.xml | 8 - .../__init__.py | 0 .../test_inserts_with_keeper_retries/test.py | 100 ---- .../integration/test_restore_replica/test.py | 9 +- .../00121_drop_column_zookeeper.sql | 6 +- ...partitioning_replicated_zookeeper_long.sql | 47 +- .../00652_replicated_mutations_zookeeper.sh | 24 +- ...replicated_without_partition_zookeeper.sql | 8 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 9 +- ...ated_minimalistic_part_header_zookeeper.sh | 1 - .../01037_zookeeper_check_table_empty_pk.sql | 9 +- ...eeper_system_mutations_with_parts_names.sh | 3 +- ...eeper_mutations_and_insert_quorum_long.sql | 11 +- .../01135_default_and_alter_zookeeper.sql | 4 +- ...mutation_stuck_after_replace_partition.sql | 2 - .../0_stateless/01158_zookeeper_log_long.sql | 7 +- ...compact_part_replicated_zookeeper_long.sql | 4 - ...18_long_unsuccessful_mutation_zookeeper.sh | 9 +- ...artition_key_replicated_zookeeper_long.sql | 13 +- ...1417_freeze_partition_verbose_zookeeper.sh | 8 +- ...replicated_detach_drop_and_quorum_long.sql | 1 - ...01451_replicated_detach_drop_part_long.sql | 9 +- .../01459_manual_write_to_replicas.sh | 10 +- .../01459_manual_write_to_replicas_quorum.sh | 6 +- ...check_many_parallel_quorum_inserts_long.sh | 4 +- .../01509_parallel_quorum_and_merge_long.sh | 13 +- ...arallel_quorum_insert_no_replicas_long.sql | 13 +- ..._execute_merges_on_single_replica_long.sql | 2 - ...primary_key_without_order_by_zookeeper.sql | 12 +- ...6_replicated_mutations_empty_partition.sql | 7 +- .../01593_concurrent_alter_mutations_kill.sh | 6 +- ..._part_and_deduplication_zookeeper_long.sql | 5 +- ...kground_checker_blather_zookeeper_long.sql | 8 +- .../01810_max_part_removal_threads_long.sh | 3 - .../0_stateless/01825_type_json_3.sql.j2 | 2 - .../01870_modulo_partition_key.sql | 9 +- .../02228_merge_tree_insert_memory_usage.sql | 5 +- .../02340_parts_refcnt_mergetree.sh | 4 +- .../02369_lost_part_intersecting_merges.sh | 16 +- .../02448_clone_replica_lost_part.sql | 2 - ...456_keeper_retries_during_insert.reference | 2 - .../02456_keeper_retries_during_insert.sql | 26 - 55 files changed, 250 insertions(+), 1451 deletions(-) delete mode 100644 src/Storages/MergeTree/ZooKeeperRetries.h delete mode 100644 src/Storages/MergeTree/ZooKeeperWithFaultInjection.h delete mode 100644 tests/config/users.d/insert_keeper_retries.xml delete mode 100644 tests/integration/test_inserts_with_keeper_retries/__init__.py delete mode 100644 tests/integration/test_inserts_with_keeper_retries/test.py delete mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.reference delete mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.sql diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index 62615afb4f7..aad5131fcb5 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -34,7 +34,7 @@ using TestKeeperRequestPtr = std::shared_ptr; class TestKeeper final : public IKeeper { public: - explicit TestKeeper(const zkutil::ZooKeeperArgs & args_); + TestKeeper(const zkutil::ZooKeeperArgs & args_); ~TestKeeper() override; bool isExpired() const override { return expired; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 9de8241cfbe..c9529b78f8a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -156,7 +156,7 @@ public: using Ptr = std::shared_ptr; using ErrorsList = std::initializer_list; - explicit ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); + ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); /** Config of the form: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 45ca9845afb..8c67cf94a61 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -658,11 +658,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ - M(UInt64, insert_keeper_max_retries, 0, "Max retries for keeper operations during insert", 0) \ - M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ - M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ - M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ - M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 6d75f523056..6ddb35d109e 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB @@ -13,22 +12,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_) - : zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_) + : zookeeper(&zookeeper_), path_prefix(path_prefix_), path(path_) { if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); } std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path) + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) { String path; if (deduplication_path.empty()) { String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER; - path = zookeeper_->create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); + path = zookeeper_.create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); } else { @@ -40,15 +39,11 @@ std::optional createEphemeralLockInZooKeeper( ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; - Coordination::Error e = zookeeper_->tryMulti(ops, responses); + Coordination::Error e = zookeeper_.tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { - LOG_DEBUG( - &Poco::Logger::get("createEphemeralLockInZooKeeper"), - "Deduplication path already exists: deduplication_path={}", - deduplication_path); return {}; } else @@ -87,31 +82,9 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { unlock(); } - catch (const zkutil::KeeperException & e) - { - if (Coordination::isHardwareError(e.code)) - LOG_DEBUG( - &Poco::Logger::get("EphemeralLockInZooKeeper"), - "ZooKeeper communication error during unlock: code={} message='{}'", - e.code, - e.message()); - else if (e.code == Coordination::Error::ZNONODE) - /// To avoid additional round-trip for unlocking, - /// ephemeral node can be deleted explicitly as part of another multi op request to ZK - /// and marked as such via assumeUnlocked() if we got successful response. - /// But it's possible that the multi op request can be executed on server side, and client will not get response due to network issue. - /// In such case, assumeUnlocked() will not be called, so we'll get ZNONODE error here since the noded is already deleted - LOG_DEBUG( - &Poco::Logger::get("EphemeralLockInZooKeeper"), - "ZooKeeper node was already deleted: code={} message={}", - e.code, - e.message()); - else - tryLogCurrentException("EphemeralLockInZooKeeper"); - } catch (...) { - tryLogCurrentException("EphemeralLockInZooKeeper"); + tryLogCurrentException("~EphemeralLockInZooKeeper"); } } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index c630bcc4681..5945fa10d91 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -12,8 +12,6 @@ namespace DB { -class ZooKeeperWithFaultInjection; -using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; namespace ErrorCodes { @@ -27,14 +25,13 @@ namespace ErrorCodes class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); protected: - EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_); + EphemeralLockInZooKeeper() = delete; + EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_); public: - EphemeralLockInZooKeeper() = delete; - /// Fake "secondary node" names for blocks with and without "deduplication_path" static constexpr const char * LEGACY_LOCK_INSERT = "abandonable_lock-insert"; static constexpr const char * LEGACY_LOCK_OTHER = "abandonable_lock-other"; @@ -56,7 +53,7 @@ public: bool isLocked() const { - return zookeeper.get(); + return zookeeper; } String getPath() const @@ -94,13 +91,13 @@ public: ~EphemeralLockInZooKeeper(); private: - ZooKeeperWithFaultInjectionPtr zookeeper; + zkutil::ZooKeeper * zookeeper = nullptr; String path_prefix; String path; }; std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); /// Acquires block number locks in all partitions. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 192f0d23f96..626295d7255 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -112,8 +112,6 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() && now - last_refresh_time < REFRESH_STATE_MINIMUM_INTERVAL_SECONDS) return; - LOG_DEBUG(storage.log, "Updating strategy picker state"); - auto zookeeper = storage.getZooKeeper(); auto all_replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas"); @@ -156,8 +154,6 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() last_refresh_time = now; current_replica_index = current_replica_index_tmp; active_replicas = active_replicas_tmp; - - LOG_DEBUG(storage.log, "Strategy picker state updated, current replica: {}, active replicas: [{}]", current_replica_index, fmt::join(active_replicas, ", ")); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index b6a486f14ee..5482052e184 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -9,6 +9,7 @@ #include #include + namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; @@ -31,7 +32,6 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; - extern const int TABLE_IS_READ_ONLY; extern const int QUERY_WAS_CANCELLED; } @@ -84,7 +84,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; /// Allow to verify that the session in ZooKeeper is still alive. -static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) +static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) { if (!zookeeper) throw Exception("No ZooKeeper session.", ErrorCodes::NO_ZOOKEEPER); @@ -93,7 +93,7 @@ static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -103,7 +103,6 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFault Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); Strings exists_paths; - exists_paths.reserve(replicas.size()); for (const auto & replica : replicas) if (replica != storage.replica_name) exists_paths.emplace_back(fs::path(storage.zookeeper_path) / "replicas" / replica / "is_active"); @@ -111,28 +110,20 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFault auto exists_result = zookeeper->exists(exists_paths); auto get_results = zookeeper->get(Strings{storage.replica_path + "/is_active", storage.replica_path + "/host"}); - Coordination::Error keeper_error = Coordination::Error::ZOK; size_t active_replicas = 1; /// Assume current replica is active (will check below) for (size_t i = 0; i < exists_paths.size(); ++i) { - auto error = exists_result[i].error; - if (error == Coordination::Error::ZOK) + auto status = exists_result[i]; + if (status.error == Coordination::Error::ZOK) ++active_replicas; - else if (Coordination::isHardwareError(error)) - keeper_error = error; } size_t replicas_number = replicas.size(); size_t quorum_size = getQuorumSize(replicas_number); if (active_replicas < quorum_size) - { - if (Coordination::isHardwareError(keeper_error)) - throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); - throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); - } /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. @@ -165,34 +156,15 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - const auto & settings = context->getSettingsRef(); - zookeeper_retries_info = ZooKeeperRetriesInfo( - "ReplicatedMergeTreeSink::consume", - settings.insert_keeper_max_retries ? log : nullptr, - settings.insert_keeper_max_retries, - settings.insert_keeper_retry_initial_backoff_ms, - settings.insert_keeper_retry_max_backoff_ms); - - ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( - settings.insert_keeper_fault_injection_probability, - settings.insert_keeper_fault_injection_seed, - storage.getZooKeeper(), - "ReplicatedMergeTreeSink::consume", - log); + auto zookeeper = storage.getZooKeeper(); + assertSessionIsNotExpired(zookeeper); /** If write is with quorum, then we check that the required number of replicas is now live, * and also that for all previous parts for which quorum is required, this quorum is reached. * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - size_t replicas_num = 0; - ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info); - quorum_retries_ctl.retryLoop( - [&]() - { - zookeeper->setKeeper(storage.getZooKeeper()); - replicas_num = checkQuorumPrecondition(zookeeper); - }); + size_t replicas_num = checkQuorumPrecondition(zookeeper); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -204,6 +176,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; + const Settings & settings = context->getSettingsRef(); for (auto & current_block : part_blocks) { @@ -283,7 +256,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) +void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) { if (!delayed_chunk) return; @@ -298,7 +271,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjecti try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); + commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num); last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; @@ -321,9 +294,8 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt { /// NOTE: No delay in this case. That's Ok. - auto origin_zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(origin_zookeeper); - auto zookeeper = std::make_shared(origin_zookeeper); + auto zookeeper = storage.getZooKeeper(); + assertSessionIsNotExpired(zookeeper); size_t replicas_num = checkQuorumPrecondition(zookeeper); @@ -332,7 +304,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", replicas_num, true); + commitPart(zookeeper, part, "", replicas_num); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -343,11 +315,10 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } void ReplicatedMergeTreeSink::commitPart( - const ZooKeeperWithFaultInjectionPtr & zookeeper, + zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num, - bool writing_existing_part) + size_t replicas_num) { /// It is possible that we alter a part with different types of source columns. /// In this case, if column was not altered, the result type will be different with what we have in metadata. @@ -355,6 +326,8 @@ void ReplicatedMergeTreeSink::commitPart( /// /// metadata_snapshot->check(part->getColumns()); + assertSessionIsNotExpired(zookeeper); + String temporary_part_relative_path = part->getDataPartStorage().getPartDirectory(); /// There is one case when we need to retry transaction in a loop. @@ -364,65 +337,14 @@ void ReplicatedMergeTreeSink::commitPart( bool is_already_existing_part = false; - /// for retries due to keeper error - bool part_committed_locally_but_zookeeper = false; - Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; - - ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info); - retries_ctl.retryLoop([&]() + while (true) { - zookeeper->setKeeper(storage.getZooKeeper()); - if (storage.is_readonly) - { - /// stop retries if in shutdown - if (storage.shutdown_called) - throw Exception( - ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); - - /// When we attach existing parts it's okay to be in read-only mode - /// For example during RESTORE REPLICA. - if (!writing_existing_part) - { - retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); - return; - } - } - - if (retries_ctl.isRetry()) - { - /// If we are retrying, check if last iteration was actually successful, - /// we could get network error on committing part to zk - /// but the operation could be completed by zk server - - /// If this flag is true, then part is in Active state, and we'll not retry anymore - /// we only check if part was committed to zk and return success or failure correspondingly - /// Note: if commit to zk failed then cleanup thread will mark the part as Outdated later - if (part_committed_locally_but_zookeeper) - { - /// check that info about the part was actually written in zk - if (zookeeper->exists(fs::path(storage.replica_path) / "parts" / part->name)) - { - LOG_DEBUG(log, "Part was successfully committed on previous iteration: part_id={}", part->name); - } - else - { - retries_ctl.setUserError( - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, - "Insert failed due to zookeeper error. Please retry. Reason: {}", - Coordination::errorMessage(write_part_info_keeper_error)); - } - - retries_ctl.stopRetries(); - return; - } - } - /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. /// Also, make deduplication check. If a duplicate is detected, no nodes are created. /// Allocate new block number and check for duplicates - const bool deduplicate_block = !block_id.empty(); + bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); ThreadFuzzer::maybeInjectSleep(); @@ -546,13 +468,7 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - if (!retries_ctl.callAndCatchAll( - [&]() - { - waitForQuorum( - zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); - })) - return; + waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); } else { @@ -561,7 +477,6 @@ void ReplicatedMergeTreeSink::commitPart( return; } - LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", block_id, existing_part_name); @@ -593,7 +508,8 @@ void ReplicatedMergeTreeSink::commitPart( } catch (const Exception & e) { - if (e.code() != ErrorCodes::DUPLICATE_DATA_PART && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART + && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } @@ -610,26 +526,15 @@ void ReplicatedMergeTreeSink::commitPart( part->name); } - try - { - ThreadFuzzer::maybeInjectSleep(); - storage.lockSharedData(*part, zookeeper, false, {}); - ThreadFuzzer::maybeInjectSleep(); - } - catch (const Exception &) - { - transaction.rollbackPartsToTemporaryState(); + ThreadFuzzer::maybeInjectSleep(); - part->is_temp = true; - part->renameTo(temporary_part_relative_path, false); - - throw; - } + storage.lockSharedData(*part, false, {}); ThreadFuzzer::maybeInjectSleep(); Coordination::Responses responses; Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT + if (multi_code == Coordination::Error::ZOK) { transaction.commit(); @@ -644,32 +549,18 @@ void ReplicatedMergeTreeSink::commitPart( throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Insert query (for block {}) was cancelled by concurrent ALTER PARTITION", block_number_lock->getPath()); } - else if (Coordination::isHardwareError(multi_code)) + else if (multi_code == Coordination::Error::ZCONNECTIONLOSS + || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) { - write_part_info_keeper_error = multi_code; /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ + * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. + */ transaction.commit(); - - /// Setting this flag is point of no return - /// On next retry, we'll just check if actually operation succeed or failed - /// and return ok or error correspondingly - part_committed_locally_but_zookeeper = true; - - /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action - /// here lambda capture part name, it's ok since we'll not generate new one for this insert, - /// see comments around 'part_committed_locally_but_zookeeper' flag - retries_ctl.actionAfterLastFailedRetry( - [&storage = storage, part_name = part->name]() - { storage.enqueuePartForCheck(part_name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); }); + storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); /// We do not know whether or not data has been inserted. - retries_ctl.setUserError( - ErrorCodes::UNKNOWN_STATUS_OF_INSERT, - "Unknown status, client must retry. Reason: {}", - Coordination::errorMessage(multi_code)); - return; + throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), + ErrorCodes::UNKNOWN_STATUS_OF_INSERT); } else if (Coordination::isUserError(multi_code)) { @@ -689,72 +580,62 @@ void ReplicatedMergeTreeSink::commitPart( part->renameTo(temporary_part_relative_path, false); /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part - /// than it will be ignored on the next iteration. + /// than it will be ignored on the next itration. ++loop_counter; if (loop_counter == max_iterations) { part->is_duplicate = true; /// Part is duplicate, just remove it from local FS throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); } - retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout - return; + continue; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { - storage.unlockSharedData(*part, zookeeper); + storage.unlockSharedData(*part); transaction.rollback(); throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else { - storage.unlockSharedData(*part, zookeeper); + storage.unlockSharedData(*part); /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); - throw Exception( - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, - "Unexpected logical error while adding block {} with ID '{}': {}, path {}", - block_number, - block_id, - Coordination::errorMessage(multi_code), - failed_op_path); + throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } } + else if (Coordination::isHardwareError(multi_code)) + { + storage.unlockSharedData(*part); + transaction.rollback(); + throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + } else { - storage.unlockSharedData(*part, zookeeper); + storage.unlockSharedData(*part); transaction.rollback(); - throw Exception( - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, - "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", - block_number, - block_id, - Coordination::errorMessage(multi_code)); + throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " + + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } - }, - [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); + + break; + } if (isQuorumEnabled()) { - ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); - quorum_retries_ctl.retryLoop([&]() + if (is_already_existing_part) { - zookeeper->setKeeper(storage.getZooKeeper()); + /// We get duplicate part without fetch + /// Check if this quorum insert is parallel or not + if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) + storage.updateQuorum(part->name, true); + else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) + storage.updateQuorum(part->name, false); + } - if (is_already_existing_part) - { - /// We get duplicate part without fetch - /// Check if this quorum insert is parallel or not - if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) - storage.updateQuorum(part->name, true); - else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) - storage.updateQuorum(part->name, false); - } - - if (!quorum_retries_ctl.callAndCatchAll( - [&]() - { waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); })) - return; - }); + waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); } } @@ -769,11 +650,11 @@ void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); - finishDelayedChunk(std::make_shared(zookeeper)); + finishDelayedChunk(zookeeper); } void ReplicatedMergeTreeSink::waitForQuorum( - const ZooKeeperWithFaultInjectionPtr & zookeeper, + zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, Int32 is_active_node_version, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index 2154c7693f2..e3ec5d14a52 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -3,8 +3,6 @@ #include #include #include -#include -#include namespace Poco { class Logger; } @@ -62,7 +60,6 @@ public: } private: - ZooKeeperRetriesInfo zookeeper_retries_info; struct QuorumInfo { String status_path; @@ -74,24 +71,20 @@ private: /// Checks active replicas. /// Returns total number of replicas. - size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); + size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( - const ZooKeeperWithFaultInjectionPtr & zookeeper, + zkutil::ZooKeeperPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num, - bool writing_existing_part); + size_t replicas_num); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. void waitForQuorum( - const ZooKeeperWithFaultInjectionPtr & zookeeper, - const std::string & part_name, - const std::string & quorum_path, - int is_active_node_version, - size_t replicas_num) const; + zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, + const std::string & quorum_path, int is_active_node_version, size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; @@ -123,7 +116,7 @@ private: struct DelayedChunk; std::unique_ptr delayed_chunk; - void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); + void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); }; } diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h deleted file mode 100644 index 22ace074245..00000000000 --- a/src/Storages/MergeTree/ZooKeeperRetries.h +++ /dev/null @@ -1,265 +0,0 @@ -#pragma once -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int OK; -} - -struct ZooKeeperRetriesInfo -{ - ZooKeeperRetriesInfo() = default; - ZooKeeperRetriesInfo(std::string name_, Poco::Logger * logger_, UInt64 max_retries_, UInt64 initial_backoff_ms_, UInt64 max_backoff_ms_) - : name(std::move(name_)) - , logger(logger_) - , max_retries(max_retries_) - , curr_backoff_ms(std::min(initial_backoff_ms_, max_backoff_ms_)) - , max_backoff_ms(max_backoff_ms_) - { - } - - std::string name; - Poco::Logger * logger = nullptr; - UInt64 max_retries = 0; - UInt64 curr_backoff_ms = 0; - UInt64 max_backoff_ms = 0; - UInt64 retry_count = 0; -}; - -class ZooKeeperRetriesControl -{ -public: - ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_) : name(std::move(name_)), retries_info(retries_info_) - { - } - - void retryLoop(auto && f) - { - retryLoop(f, []() {}); - } - - void retryLoop(auto && f, auto && iteration_cleanup) - { - while (canTry()) - { - try - { - f(); - iteration_cleanup(); - } - catch (const zkutil::KeeperException & e) - { - iteration_cleanup(); - - if (!Coordination::isHardwareError(e.code)) - throw; - - setKeeperError(e.code, e.message()); - } - catch (...) - { - iteration_cleanup(); - throw; - } - } - } - - bool callAndCatchAll(auto && f) - { - try - { - f(); - return true; - } - catch (const zkutil::KeeperException & e) - { - setKeeperError(e.code, e.message()); - } - catch (const Exception & e) - { - setUserError(e.code(), e.what()); - } - return false; - } - - void setUserError(int code, std::string message) - { - if (retries_info.logger) - LOG_TRACE( - retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setUserError: error={} message={}", retries_info.name, name, code, message); - - /// if current iteration is already failed, keep initial error - if (!iteration_succeeded) - return; - - iteration_succeeded = false; - user_error.code = code; - user_error.message = std::move(message); - keeper_error = KeeperError{}; - } - - template - void setUserError(int code, fmt::format_string fmt, Args &&... args) - { - setUserError(code, fmt::format(fmt, std::forward(args)...)); - } - - void setKeeperError(Coordination::Error code, std::string message) - { - if (retries_info.logger) - LOG_TRACE( - retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setKeeperError: error={} message={}", retries_info.name, name, code, message); - - /// if current iteration is already failed, keep initial error - if (!iteration_succeeded) - return; - - iteration_succeeded = false; - keeper_error.code = code; - keeper_error.message = std::move(message); - user_error = UserError{}; - } - - void stopRetries() { stop_retries = true; } - - void requestUnconditionalRetry() { unconditional_retry = true; } - - bool isLastRetry() const { return retries_info.retry_count >= retries_info.max_retries; } - - bool isRetry() const { return retries_info.retry_count > 0; } - - Coordination::Error getLastKeeperErrorCode() const { return keeper_error.code; } - - /// action will be called only once and only after latest failed retry - void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } - -private: - struct KeeperError - { - using Code = Coordination::Error; - Code code = Code::ZOK; - std::string message; - }; - - struct UserError - { - int code = ErrorCodes::OK; - std::string message; - }; - - bool canTry() - { - ++iteration_count; - /// first iteration is ordinary execution, no further checks needed - if (0 == iteration_count) - return true; - - if (unconditional_retry) - { - unconditional_retry = false; - return true; - } - - /// iteration succeeded -> no need to retry - if (iteration_succeeded) - { - /// avoid unnecessary logs, - print something only in case of retries - if (retries_info.logger && iteration_count > 1) - LOG_DEBUG( - retries_info.logger, - "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", - retries_info.name, - name, - iteration_count, - retries_info.retry_count); - return false; - } - - if (stop_retries) - { - logLastError("stop retries on request"); - action_after_last_failed_retry(); - throwIfError(); - return false; - } - - if (retries_info.retry_count >= retries_info.max_retries) - { - logLastError("retry limit is reached"); - action_after_last_failed_retry(); - throwIfError(); - return false; - } - - /// retries - ++retries_info.retry_count; - logLastError("will retry due to error"); - sleepForMilliseconds(retries_info.curr_backoff_ms); - retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); - - /// reset the flag, it will be set to false in case of error - iteration_succeeded = true; - - return true; - } - - void throwIfError() const - { - if (user_error.code != ErrorCodes::OK) - throw Exception(user_error.code, user_error.message); - - if (keeper_error.code != KeeperError::Code::ZOK) - throw zkutil::KeeperException(keeper_error.code, keeper_error.message); - } - - void logLastError(std::string_view header) - { - if (user_error.code == ErrorCodes::OK) - { - if (retries_info.logger) - LOG_DEBUG( - retries_info.logger, - "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", - retries_info.name, - name, - header, - retries_info.retry_count, - retries_info.curr_backoff_ms, - keeper_error.code, - keeper_error.message); - } - else - { - if (retries_info.logger) - LOG_DEBUG( - retries_info.logger, - "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", - retries_info.name, - name, - header, - retries_info.retry_count, - retries_info.curr_backoff_ms, - user_error.code, - user_error.message); - } - } - - - std::string name; - ZooKeeperRetriesInfo & retries_info; - Int64 iteration_count = -1; - UserError user_error; - KeeperError keeper_error; - std::function action_after_last_failed_retry = []() {}; - bool unconditional_retry = false; - bool iteration_succeeded = true; - bool stop_retries = false; -}; - -} diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h deleted file mode 100644 index 491f97b52bd..00000000000 --- a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h +++ /dev/null @@ -1,527 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -class RandomFaultInjection -{ -public: - RandomFaultInjection(double probability, UInt64 seed_) : rndgen(seed_), distribution(probability) { } - - void beforeOperation() - { - if (distribution(rndgen)) - throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); - } - void afterOperation() - { - if (distribution(rndgen)) - throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); - } - -private: - std::mt19937_64 rndgen; - std::bernoulli_distribution distribution; -}; - -/// -/// ZooKeeperWithFaultInjection mimics ZooKeeper interface and inject failures according to failure policy if set -/// -class ZooKeeperWithFaultInjection -{ - using zk = zkutil::ZooKeeper; - - zk::Ptr keeper; - zk::Ptr keeper_prev; - std::unique_ptr fault_policy; - std::string name; - Poco::Logger * logger = nullptr; - UInt64 calls_total = 0; - UInt64 calls_without_fault_injection = 0; - const UInt64 seed = 0; - - std::vector ephemeral_nodes; - - ZooKeeperWithFaultInjection( - zk::Ptr const & keeper_, - double fault_injection_probability, - UInt64 fault_injection_seed, - std::string name_, - Poco::Logger * logger_) - : keeper(keeper_), name(std::move(name_)), logger(logger_), seed(fault_injection_seed) - { - fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); - - if (unlikely(logger)) - LOG_TRACE( - logger, - "ZooKeeperWithFaultInjection created: name={} seed={} fault_probability={}", - name, - seed, - fault_injection_probability); - } - -public: - using Ptr = std::shared_ptr; - - static ZooKeeperWithFaultInjection::Ptr createInstance( - double fault_injection_probability, UInt64 fault_injection_seed, const zk::Ptr & zookeeper, std::string name, Poco::Logger * logger) - { - /// validate all parameters here, constructor just accept everything - - if (fault_injection_probability < 0.0) - fault_injection_probability = .0; - else if (fault_injection_probability > 1.0) - fault_injection_probability = 1.0; - - if (0 == fault_injection_seed) - fault_injection_seed = randomSeed(); - - if (fault_injection_probability > 0.0) - return std::shared_ptr( - new ZooKeeperWithFaultInjection(zookeeper, fault_injection_probability, fault_injection_seed, std::move(name), logger)); - - /// if no fault injection provided, create instance which will not log anything - return std::make_shared(zookeeper); - } - - explicit ZooKeeperWithFaultInjection(zk::Ptr const & keeper_) : keeper(keeper_) { } - - ~ZooKeeperWithFaultInjection() - { - if (unlikely(logger)) - LOG_TRACE( - logger, - "ZooKeeperWithFaultInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", - name, - seed, - calls_total, - calls_without_fault_injection, - calls_total - calls_without_fault_injection, - float(calls_total - calls_without_fault_injection) / calls_total); - } - - void setKeeper(zk::Ptr const & keeper_) { keeper = keeper_; } - bool isNull() const { return keeper.get() == nullptr; } - - /// - /// mirror ZooKeeper interface - /// - - Strings getChildren( - const std::string & path, - Coordination::Stat * stat = nullptr, - const zkutil::EventPtr & watch = nullptr, - Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) - { - return access("getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }); - } - - Coordination::Error tryGetChildren( - const std::string & path, - Strings & res, - Coordination::Stat * stat = nullptr, - const zkutil::EventPtr & watch = nullptr, - Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) - { - return access("tryGetChildren", path, [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }); - } - - zk::FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}) - { - return access("asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }); - } - - zk::FutureGet asyncTryGet(const std::string & path) - { - return access("asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }); - } - - bool tryGet( - const std::string & path, - std::string & res, - Coordination::Stat * stat = nullptr, - const zkutil::EventPtr & watch = nullptr, - Coordination::Error * code = nullptr) - { - return access("tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }); - } - - Coordination::Error tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses) - { - constexpr auto method = "tryMulti"; - auto error = access( - method, - !requests.empty() ? requests.front()->getPath() : "", - [&]() { return keeper->tryMulti(requests, responses); }, - [&](const Coordination::Error & original_error) - { - if (original_error == Coordination::Error::ZOK) - faultInjectionPostAction(method, requests, responses); - }, - [&]() - { - responses.clear(); - for (size_t i = 0; i < requests.size(); ++i) - responses.emplace_back(std::make_shared()); - }); - - - /// collect ephemeral nodes when no fault was injected (to clean up on demand) - if (unlikely(fault_policy) && Coordination::Error::ZOK == error) - { - doForEachCreatedEphemeralNode( - method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); - } - return error; - } - - Coordination::Error tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses) - { - constexpr auto method = "tryMultiNoThrow"; - constexpr auto no_throw = true; - constexpr auto inject_failure_before_op = false; - auto error = access( - method, - !requests.empty() ? requests.front()->getPath() : "", - [&]() { return keeper->tryMultiNoThrow(requests, responses); }, - [&](const Coordination::Error & original_error) - { - if (original_error == Coordination::Error::ZOK) - faultInjectionPostAction(method, requests, responses); - }, - [&]() - { - responses.clear(); - for (size_t i = 0; i < requests.size(); ++i) - responses.emplace_back(std::make_shared()); - }); - - /// collect ephemeral nodes when no fault was injected (to clean up later) - if (unlikely(fault_policy) && Coordination::Error::ZOK == error) - { - doForEachCreatedEphemeralNode( - method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); - } - return error; - } - - std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) - { - return access("get", path, [&]() { return keeper->get(path, stat, watch); }); - } - - zkutil::ZooKeeper::MultiGetResponse get(const std::vector & paths) - { - return access("get", !paths.empty() ? paths.front() : "", [&]() { return keeper->get(paths); }); - } - - bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) - { - return access("exists", path, [&]() { return keeper->exists(path, stat, watch); }); - } - - zkutil::ZooKeeper::MultiExistsResponse exists(const std::vector & paths) - { - return access("exists", !paths.empty() ? paths.front() : "", [&]() { return keeper->exists(paths); }); - } - - std::string create(const std::string & path, const std::string & data, int32_t mode) - { - auto path_created = access( - "create", - path, - [&]() { return keeper->create(path, data, mode); }, - [&](std::string const & result_path) - { - try - { - if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) - { - keeper->remove(result_path); - if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFaultInjection cleanup: seed={} func={} path={}", seed, "create", result_path); - } - } - catch (const zkutil::KeeperException & e) - { - if (unlikely(logger)) - LOG_TRACE( - logger, - "ZooKeeperWithFaultInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", - seed, - "create", - result_path, - e.code, - e.message()); - } - }); - - /// collect ephemeral nodes when no fault was injected (to clean up later) - if (unlikely(fault_policy)) - { - if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) - ephemeral_nodes.push_back(path_created); - } - - return path_created; - } - - Coordination::Responses multi(const Coordination::Requests & requests) - { - constexpr auto method = "multi"; - auto result = access( - method, - !requests.empty() ? requests.front()->getPath() : "", - [&]() { return keeper->multi(requests); }, - [&](Coordination::Responses & responses) { faultInjectionPostAction(method, requests, responses); }); - - /// collect ephemeral nodes to clean up - if (unlikely(fault_policy)) - { - doForEachCreatedEphemeralNode( - method, requests, result, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); - } - return result; - } - - void createAncestors(const std::string & path) - { - access("createAncestors", path, [&]() { return keeper->createAncestors(path); }); - } - - Coordination::Error tryRemove(const std::string & path, int32_t version = -1) - { - return access("tryRemove", path, [&]() { return keeper->tryRemove(path, version); }); - } - - void cleanupEphemeralNodes() - { - for (const auto & path : ephemeral_nodes) - { - try - { - if (keeper_prev) - keeper_prev->tryRemove(path); - } - catch (...) - { - if (unlikely(logger)) - tryLogCurrentException(logger, "Exception during ephemeral nodes clean up"); - } - } - - ephemeral_nodes.clear(); - } - -private: - void faultInjectionBefore(std::function fault_cleanup) - { - try - { - if (unlikely(fault_policy)) - fault_policy->beforeOperation(); - } - catch (const zkutil::KeeperException &) - { - fault_cleanup(); - throw; - } - } - void faultInjectionAfter(std::function fault_cleanup) - { - try - { - if (unlikely(fault_policy)) - fault_policy->afterOperation(); - } - catch (const zkutil::KeeperException &) - { - fault_cleanup(); - throw; - } - } - - void doForEachCreatedEphemeralNode( - const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses, auto && action) - { - if (responses.empty()) - return; - - if (responses.size() != requests.size()) - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Number of responses doesn't match number of requests: method={} requests={} responses={}", - method, - requests.size(), - responses.size()); - - /// find create request with ephemeral flag - std::vector> create_requests; - for (size_t i = 0; i < requests.size(); ++i) - { - const auto * create_req = dynamic_cast(requests[i].get()); - if (create_req && create_req->is_ephemeral) - create_requests.emplace_back(i, create_req); - } - - for (auto && [i, req] : create_requests) - { - const auto * create_resp = dynamic_cast(responses.at(i).get()); - if (!create_resp) - throw Exception( - ErrorCodes::LOGICAL_ERROR, "Response should be CreateResponse: method={} index={} path={}", method, i, req->path); - - action(create_resp->path_created); - } - } - - void faultInjectionPostAction(const char * method, const Coordination::Requests & requests, Coordination::Responses & responses) - { - doForEachCreatedEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); - } - - template - struct FaultCleanupTypeImpl - { - using Type = std::function; - }; - - template <> - struct FaultCleanupTypeImpl - { - using Type = std::function; - }; - - template - using FaultCleanupType = typename FaultCleanupTypeImpl::Type; - - template < - bool no_throw_access = false, - bool inject_failure_before_op = true, - int inject_failure_after_op = true, - typename Operation, - typename Result = std::invoke_result_t> - Result access( - const char * func_name, - const std::string & path, - Operation operation, - FaultCleanupType fault_after_op_cleanup = {}, - FaultCleanupType fault_before_op_cleanup = {}) - { - try - { - ++calls_total; - - if (!keeper) - throw zkutil::KeeperException( - "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); - - if constexpr (inject_failure_before_op) - { - faultInjectionBefore( - [&] - { - if (fault_before_op_cleanup) - fault_before_op_cleanup(); - }); - } - - if constexpr (!std::is_same_v) - { - Result res = operation(); - - /// if connectivity error occurred w/o fault injection -> just return it - if constexpr (std::is_same_v) - { - if (Coordination::isHardwareError(res)) - return res; - } - - if constexpr (inject_failure_after_op) - { - faultInjectionAfter( - [&] - { - if (fault_after_op_cleanup) - fault_after_op_cleanup(res); - }); - } - - ++calls_without_fault_injection; - - if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); - - return res; - } - else - { - operation(); - - if constexpr (inject_failure_after_op) - { - faultInjectionAfter( - [&fault_after_op_cleanup] - { - if (fault_after_op_cleanup) - fault_after_op_cleanup(); - }); - } - - ++calls_without_fault_injection; - - if (unlikely(logger)) - LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); - } - } - catch (const zkutil::KeeperException & e) - { - if (unlikely(logger)) - LOG_TRACE( - logger, - "ZooKeeperWithFaultInjection call FAILED: seed={} func={} path={} code={} message={} ", - seed, - func_name, - path, - e.code, - e.message()); - - /// save valid pointer to clean up ephemeral nodes later if necessary - if (keeper) - keeper_prev = keeper; - keeper.reset(); - - /// for try*NoThrow() methods - if constexpr (no_throw_access) - return e.code; - - if constexpr (std::is_same_v) - { - /// try*() methods throws at least on hardware error and return only on user errors - /// todo: the methods return only on subset of user errors, and throw on another errors - /// to mimic the methods exactly - we need to specify errors on which to return for each such method - if (Coordination::isHardwareError(e.code)) - throw; - - return e.code; - } - - throw; - } - } -}; - -using ZooKeeperWithFaultInjectionPtr = ZooKeeperWithFaultInjection::Ptr; -} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index dde844c0ee0..3b85581a157 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4479,16 +4479,9 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it - /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk - /// In case of remote storage like s3, it'll generate unnecessary PUT requests - if (is_readonly && (!has_metadata_in_zookeeper.has_value() || false == has_metadata_in_zookeeper.value())) - throw Exception( - ErrorCodes::TABLE_IS_READ_ONLY, - "Table is in readonly mode since table metadata was not found in zookeeper: replica_path={}", - replica_path); - const auto storage_settings_ptr = getSettings(); + assertNotReadonly(); + const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; @@ -5003,7 +4996,8 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St Int64 mutation_version; { - delimiting_block_lock = allocateBlockNumber(partition_id, getZooKeeper()); + auto zookeeper = getZooKeeper(); + delimiting_block_lock = allocateBlockNumber(partition_id, zookeeper); right = delimiting_block_lock->getNumber(); /// Make sure we cover all parts in drop range. /// There might be parts with mutation version greater than current block number @@ -5284,7 +5278,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c } -bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const +bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const { { std::lock_guard lock(existing_nodes_cache_mutex); @@ -5292,7 +5286,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInject return true; } - bool res = zookeeper->exists(path); + bool res = getZooKeeper()->exists(path); if (res) { @@ -5304,22 +5298,9 @@ bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInject } -std::optional StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, - const zkutil::ZooKeeperPtr & zookeeper, - const String & zookeeper_block_id_path, - const String & zookeeper_path_prefix) const -{ - return allocateBlockNumber( - partition_id, std::make_shared(zookeeper), zookeeper_block_id_path, zookeeper_path_prefix); -} - - -std::optional StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - const String & zookeeper_block_id_path, - const String & zookeeper_path_prefix) const +std::optional +StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path, const String & zookeeper_path_prefix) const { String zookeeper_table_path; if (zookeeper_path_prefix.empty()) @@ -5330,7 +5311,7 @@ std::optional StorageReplicatedMergeTree::allocateBloc String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; - if (!existsNodeCached(zookeeper, partition_path)) + if (!existsNodeCached(partition_path)) { Coordination::Requests ops; /// Check that table is not being dropped ("host" is the first node that is removed on replica drop) @@ -5348,9 +5329,10 @@ std::optional StorageReplicatedMergeTree::allocateBloc } return createEphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", zookeeper, zookeeper_block_id_path); + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_path); } + Strings StorageReplicatedMergeTree::tryWaitForAllReplicasToProcessLogEntry( const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout) { @@ -7103,7 +7085,7 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, - const DataPartPtr & part, + MutableDataPartPtr & part, const String & block_id_path) const { const String & part_name = part->name; @@ -7685,28 +7667,11 @@ void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_nam String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; LOG_TRACE(log, "Set zookeeper temporary ephemeral lock {}", zookeeper_node); - createZeroCopyLockNode( - std::make_shared(zookeeper), zookeeper_node, zkutil::CreateMode::Ephemeral, false); + createZeroCopyLockNode(zookeeper, zookeeper_node, zkutil::CreateMode::Ephemeral, false); } } -void StorageReplicatedMergeTree::lockSharedData( - const IMergeTreeDataPart & part, - bool replace_existing_lock, - std::optional hardlinked_files) const -{ - auto zookeeper = tryGetZooKeeper(); - if (zookeeper) - return lockSharedData(part, std::make_shared(zookeeper), replace_existing_lock, hardlinked_files); - else - return lockSharedData(part, std::make_shared(nullptr), replace_existing_lock, hardlinked_files); -} - -void StorageReplicatedMergeTree::lockSharedData( - const IMergeTreeDataPart & part, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - bool replace_existing_lock, - std::optional hardlinked_files) const +void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const { auto settings = getSettings(); @@ -7716,7 +7681,8 @@ void StorageReplicatedMergeTree::lockSharedData( if (!part.getDataPartStorage().supportZeroCopyReplication()) return; - if (zookeeper->isNull()) + zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); + if (!zookeeper) return; String id = part.getUniqueId(); @@ -7750,14 +7716,7 @@ void StorageReplicatedMergeTree::lockSharedData( } } -std::pair -StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const -{ - return unlockSharedData(part, std::make_shared(nullptr)); -} - -std::pair -StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const +std::pair StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const { auto settings = getSettings(); if (!settings->allow_remote_fs_zero_copy_replication) @@ -7803,10 +7762,11 @@ StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, co /// We remove parts during table shutdown. If exception happen, restarting thread will be already turned /// off and nobody will reconnect our zookeeper connection. In this case we use zookeeper connection from /// context. + zkutil::ZooKeeperPtr zookeeper; if (shutdown_called.load()) - zookeeper->setKeeper(getZooKeeperIfTableShutDown()); + zookeeper = getZooKeeperIfTableShutDown(); else - zookeeper->setKeeper(getZooKeeper()); + zookeeper = getZooKeeper(); /// It can happen that we didn't had the connection to zookeeper during table creation, but actually /// table is completely dropped, so we can drop it without any additional checks. @@ -7831,7 +7791,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -7893,7 +7853,7 @@ std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionP std::pair StorageReplicatedMergeTree::unlockSharedDataByID( String part_id, const String & table_uuid, const String & part_name, - const String & replica_name_, const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const MergeTreeSettings & settings, + const String & replica_name_, const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) { boost::replace_all(part_id, "/", "_"); @@ -7912,8 +7872,7 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); - auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( - zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs(zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8428,7 +8387,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP void StorageReplicatedMergeTree::createZeroCopyLockNode( - const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode, + const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, int32_t mode, bool replace_existing_lock, const String & path_to_set_hardlinked_files, const NameSet & hardlinked_files) { /// In rare case other replica can remove path between createAncestors and createIfNotExists @@ -8545,7 +8504,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St id, table_uuid, part_name, detached_replica_name, toString(disk->getDataSourceDescription().type), - std::make_shared(zookeeper), local_context->getReplicatedMergeTreeSettings(), + zookeeper, local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), detached_zookeeper_path, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d767d94889c..fe2105ab4ec 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -83,9 +83,6 @@ namespace DB * as the time will take the time of creation the appropriate part on any of the replicas. */ -class ZooKeeperWithFaultInjection; -using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; - class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -270,11 +267,6 @@ public: /// Lock part in zookeeper for use shared data in several nodes void lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const override; - void lockSharedData( - const IMergeTreeDataPart & part, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - bool replace_existing_lock, - std::optional hardlinked_files) const; void lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const; @@ -282,23 +274,13 @@ public: /// Return true if data unlocked /// Return false if data is still used by another node std::pair unlockSharedData(const IMergeTreeDataPart & part) const override; - std::pair - unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const; /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static std::pair unlockSharedDataByID( - String part_id, - const String & table_uuid, - const String & part_name, - const String & replica_name_, - const std::string & disk_type, - const ZooKeeperWithFaultInjectionPtr & zookeeper_, - const MergeTreeSettings & settings, - Poco::Logger * logger, - const String & zookeeper_path_old, - MergeTreeDataFormatVersion data_format_version); + static std::pair unlockSharedDataByID(String part_id, const String & table_uuid, const String & part_name, const String & replica_name_, + const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, + const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version); /// Fetch part only if some replica has it on shared storage like S3 MutableDataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; @@ -552,7 +534,7 @@ private: bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; - void getCommitPartOps(Coordination::Requests & ops, const DataPartPtr & part, const String & block_id_path = "") const; + void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const; /// Adds actions to `ops` that remove a part from ZooKeeper. /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). @@ -729,11 +711,6 @@ private: std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; - std::optional allocateBlockNumber( - const String & partition_id, - const ZooKeeperWithFaultInjectionPtr & zookeeper, - const String & zookeeper_block_id_path = "", - const String & zookeeper_path_prefix = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. @@ -771,7 +748,7 @@ private: /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. mutable std::unordered_set existing_nodes_cache; mutable std::mutex existing_nodes_cache_mutex; - bool existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const; + bool existsNodeCached(const std::string & path) const; /// Cancels INSERTs in the block range by removing ephemeral block numbers void clearLockedBlockNumbersInPartition(zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); @@ -859,7 +836,7 @@ private: const String & part_name, const String & zookeeper_path_old); static void createZeroCopyLockNode( - const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, + const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false, const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {}); diff --git a/tests/config/install.sh b/tests/config/install.sh index 51865665f59..876d05507fe 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -71,7 +71,6 @@ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/ -ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml deleted file mode 100644 index f7b652bf2ef..00000000000 --- a/tests/config/users.d/insert_keeper_retries.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - 0 - 0.0 - - - diff --git a/tests/integration/test_inserts_with_keeper_retries/__init__.py b/tests/integration/test_inserts_with_keeper_retries/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py deleted file mode 100644 index dbf18365053..00000000000 --- a/tests/integration/test_inserts_with_keeper_retries/test.py +++ /dev/null @@ -1,100 +0,0 @@ -#!/usr/bin/env python3 - -import pytest -import time -import threading -from helpers.cluster import ClickHouseCluster -from multiprocessing.dummy import Pool -from helpers.network import PartitionManager -from helpers.client import QueryRuntimeException -from helpers.test_tools import assert_eq_with_retry - -cluster = ClickHouseCluster(__file__) - -node1 = cluster.add_instance("node1", with_zookeeper=True) - - -@pytest.fixture(scope="module") -def started_cluster(): - global cluster - try: - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - -def test_replica_inserts_with_keeper_restart(started_cluster): - try: - node1.query( - "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" - ) - - p = Pool(1) - zk_stopped_event = threading.Event() - - def zoo_restart(zk_stopped_event): - cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - zk_stopped_event.set() - cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) - - job = p.apply_async(zoo_restart, (zk_stopped_event,)) - - zk_stopped_event.wait(90) - - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" - ) - - job.wait() - p.close() - p.join() - - assert node1.query("SELECT COUNT() FROM r") == "20\n" - - finally: - node1.query("DROP TABLE IF EXISTS r SYNC") - - -def test_replica_inserts_with_keeper_disconnect(started_cluster): - try: - node1.query( - "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" - ) - - p = Pool(1) - disconnect_event = threading.Event() - - def keeper_disconnect(node, event): - with PartitionManager() as pm: - pm.drop_instance_zk_connections(node) - event.set() - - job = p.apply_async( - keeper_disconnect, - ( - node1, - disconnect_event, - ), - ) - disconnect_event.wait(90) - - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" - ) - node1.query( - "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" - ) - - job.wait() - p.close() - p.join() - - assert node1.query("SELECT COUNT() FROM r") == "20\n" - - finally: - node1.query("DROP TABLE IF EXISTS r SYNC") diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py index 31c503f6184..0b11cdf7512 100644 --- a/tests/integration/test_restore_replica/test.py +++ b/tests/integration/test_restore_replica/test.py @@ -7,9 +7,6 @@ from helpers.test_tools import assert_eq_with_retry def fill_nodes(nodes): - for node in nodes: - node.query("DROP TABLE IF EXISTS test SYNC") - for node in nodes: node.query( """ @@ -32,7 +29,11 @@ nodes = [node_1, node_2, node_3] def fill_table(): - fill_nodes(nodes) + node_1.query("TRUNCATE TABLE test") + + for node in nodes: + node.query("SYSTEM SYNC REPLICA test") + check_data(0, 0) # it will create multiple parts in each partition and probably cause merges diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index ed1f654f847..f62f11c60fd 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,14 +1,14 @@ -- Tags: zookeeper, no-replicated-database -- Tag no-replicated-database: Old syntax is not allowed -DROP TABLE IF EXISTS alter_00121 SYNC; +DROP TABLE IF EXISTS alter_00121; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; -DROP TABLE alter_00121 SYNC; +DROP TABLE alter_00121; CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); @@ -23,4 +23,4 @@ SELECT * FROM alter_00121 ORDER BY d; ALTER TABLE alter_00121 DROP COLUMN x; SELECT * FROM alter_00121 ORDER BY d; -DROP TABLE alter_00121 SYNC; +DROP TABLE alter_00121; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 0ee8ba07006..8267a451728 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -1,12 +1,11 @@ -- Tags: long, replica SET replication_alter_partitions_sync = 2; -SET insert_keeper_fault_injection_probability=0; SELECT '*** Not partitioned ***'; -DROP TABLE IF EXISTS not_partitioned_replica1_00502 SYNC; -DROP TABLE IF EXISTS not_partitioned_replica2_00502 SYNC; +DROP TABLE IF EXISTS not_partitioned_replica1_00502; +DROP TABLE IF EXISTS not_partitioned_replica2_00502; CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; @@ -15,7 +14,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA not_partitioned_replica2_00502; +SYSTEM SYNC REPLICA not_partitioned_replica1_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica2_00502' AND active ORDER BY name; @@ -26,13 +25,13 @@ ALTER TABLE not_partitioned_replica1_00502 DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned_replica2_00502; -DROP TABLE not_partitioned_replica1_00502 SYNC; -DROP TABLE not_partitioned_replica2_00502 SYNC; +DROP TABLE not_partitioned_replica1_00502; +DROP TABLE not_partitioned_replica2_00502; SELECT '*** Partitioned by week ***'; -DROP TABLE IF EXISTS partitioned_by_week_replica1 SYNC; -DROP TABLE IF EXISTS partitioned_by_week_replica2 SYNC; +DROP TABLE IF EXISTS partitioned_by_week_replica1; +DROP TABLE IF EXISTS partitioned_by_week_replica2; CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; @@ -42,7 +41,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_week_replica2; +SYSTEM SYNC REPLICA partitioned_by_week_replica1; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica2' AND active ORDER BY name; @@ -53,13 +52,13 @@ ALTER TABLE partitioned_by_week_replica1 DROP PARTITION '1999-12-27'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_week_replica2; -DROP TABLE partitioned_by_week_replica1 SYNC; -DROP TABLE partitioned_by_week_replica2 SYNC; +DROP TABLE partitioned_by_week_replica1; +DROP TABLE partitioned_by_week_replica2; SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502 SYNC; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); @@ -68,7 +67,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -80,13 +79,13 @@ ALTER TABLE partitioned_by_tuple_replica1_00502 DETACH PARTITION ID '20000101-1' SELECT 'Sum after DETACH PARTITION:'; SELECT sum(y) FROM partitioned_by_tuple_replica2_00502; -DROP TABLE partitioned_by_tuple_replica1_00502 SYNC; -DROP TABLE partitioned_by_tuple_replica2_00502 SYNC; +DROP TABLE partitioned_by_tuple_replica1_00502; +DROP TABLE partitioned_by_tuple_replica2_00502; SELECT '*** Partitioned by String ***'; -DROP TABLE IF EXISTS partitioned_by_string_replica1 SYNC; -DROP TABLE IF EXISTS partitioned_by_string_replica2 SYNC; +DROP TABLE IF EXISTS partitioned_by_string_replica1; +DROP TABLE IF EXISTS partitioned_by_string_replica2; CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; @@ -106,13 +105,13 @@ ALTER TABLE partitioned_by_string_replica1 DROP PARTITION 'bbb'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_string_replica2; -DROP TABLE partitioned_by_string_replica1 SYNC; -DROP TABLE partitioned_by_string_replica2 SYNC; +DROP TABLE partitioned_by_string_replica1; +DROP TABLE partitioned_by_string_replica2; SELECT '*** Table without columns with fixed size ***'; -DROP TABLE IF EXISTS without_fixed_size_columns_replica1 SYNC; -DROP TABLE IF EXISTS without_fixed_size_columns_replica2 SYNC; +DROP TABLE IF EXISTS without_fixed_size_columns_replica1; +DROP TABLE IF EXISTS without_fixed_size_columns_replica2; CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; @@ -131,5 +130,5 @@ ALTER TABLE without_fixed_size_columns_replica1 DROP PARTITION 1; SELECT 'After DROP PARTITION:'; SELECT * FROM without_fixed_size_columns_replica2 ORDER BY s; -DROP TABLE without_fixed_size_columns_replica1 SYNC; -DROP TABLE without_fixed_size_columns_replica2 SYNC; +DROP TABLE without_fixed_size_columns_replica1; +DROP TABLE without_fixed_size_columns_replica2; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index a1e7d8727c7..7a6c7609660 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" @@ -19,9 +19,9 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 1, 'a')" -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 2, 'b'), ('2000-01-01', 3, 'c'), ('2000-01-01', 4, 'd') \ ('2000-02-01', 2, 'b'), ('2000-02-01', 3, 'c'), ('2000-02-01', 4, 'd')" @@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE s = 'd' SETT ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTINGS mutations_sync = 2" # Insert more data -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" ${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" @@ -49,8 +49,8 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partiti ${CLICKHOUSE_CLIENT} --query="SELECT '*** Test mutations cleaner ***'" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2" # Create 2 replicas with finished_mutations_to_keep = 2 ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r1') ORDER BY x SETTINGS \ @@ -63,7 +63,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE cleanup_delay_period_random_add = 0" # Insert some data -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" +${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" # Add some mutations and wait for their execution ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" @@ -80,8 +80,8 @@ sleep 1.5 # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner_r2' ORDER BY mutation_id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2" diff --git a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql index 8f694345d93..4cdd75f520c 100644 --- a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql @@ -2,8 +2,8 @@ SET optimize_on_insert = 0; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661 SYNC; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661; CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w); CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w); @@ -21,5 +21,5 @@ OPTIMIZE TABLE partitioned_by_tuple_replica1_00661 FINAL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00661; SELECT * FROM partitioned_by_tuple_replica2_00661 ORDER BY d, x, w, y; -DROP TABLE partitioned_by_tuple_replica1_00661 SYNC; -DROP TABLE partitioned_by_tuple_replica2_00661 SYNC; +DROP TABLE partitioned_by_tuple_replica1_00661; +DROP TABLE partitioned_by_tuple_replica2_00661; diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 3f384380f9b..4637b210194 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE IF EXISTS fetches_r1 SYNC; - DROP TABLE IF EXISTS fetches_r2 SYNC" + DROP TABLE IF EXISTS fetches_r1; + DROP TABLE IF EXISTS fetches_r2" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r1') ORDER BY x" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r2') ORDER BY x \ @@ -18,7 +18,6 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate prefer_fetch_merged_part_size_threshold=0" ${CLICKHOUSE_CLIENT} -n --query=" - SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); INSERT INTO fetches_r1 VALUES (3)" @@ -52,5 +51,5 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutate ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE fetches_r1 SYNC; - DROP TABLE fetches_r2 SYNC" + DROP TABLE fetches_r1; + DROP TABLE fetches_r2" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 5fc3fa460e6..6f609065c01 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -12,7 +12,6 @@ $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index b836f806170..703f8824055 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,10 +1,9 @@ -- Tags: zookeeper -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET check_query_single_value_result = 0; SET send_logs_level = 'fatal'; -DROP TABLE IF EXISTS mt_without_pk SYNC; +DROP TABLE IF EXISTS mt_without_pk; CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeTree() ORDER BY tuple(); @@ -12,9 +11,9 @@ INSERT INTO mt_without_pk VALUES (1, 2); CHECK TABLE mt_without_pk; -DROP TABLE IF EXISTS mt_without_pk SYNC; +DROP TABLE IF EXISTS mt_without_pk; -DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; +DROP TABLE IF EXISTS replicated_mt_without_pk; CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); @@ -22,4 +21,4 @@ INSERT INTO replicated_mt_without_pk VALUES (1, 2); CHECK TABLE replicated_mt_without_pk; -DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; +DROP TABLE IF EXISTS replicated_mt_without_pk; diff --git a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh index 68c511b80ac..a438cf7badc 100755 --- a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh +++ b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh @@ -52,8 +52,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations" -# test relays on part ids, which are non-deterministic with keeper fault injections, so disable it -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" +${CLICKHOUSE_CLIENT} --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" ${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM replicated_table_for_mutations" diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index e4acfed8a0c..43ab053655a 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -1,8 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; -DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum1; +DROP TABLE IF EXISTS mutations_and_quorum2; CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); @@ -10,9 +10,6 @@ CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGI -- Should not be larger then 600e6 (default timeout in clickhouse-test) SET insert_quorum=2, insert_quorum_parallel=0, insert_quorum_timeout=300e3; -SET insert_keeper_max_retries=100; -SET insert_keeper_retry_max_backoff_ms=10; - INSERT INTO mutations_and_quorum1 VALUES ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'), ('2019-04-01', 'test4'), ('2019-05-01', 'test1'), ('2019-06-01', 'test2'), ('2019-07-01', 'test3'), ('2019-08-01', 'test4'), ('2019-09-01', 'test1'), ('2019-10-01', 'test2'), ('2019-11-01', 'test3'), ('2019-12-01', 'test4'); ALTER TABLE mutations_and_quorum1 DELETE WHERE something = 'test1' SETTINGS mutations_sync=2; @@ -22,5 +19,5 @@ SELECT COUNT() FROM mutations_and_quorum2; SELECT COUNT() FROM system.mutations WHERE database = currentDatabase() AND table like 'mutations_and_quorum%' and is_done = 0; -DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; -DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum1; +DROP TABLE IF EXISTS mutations_and_quorum2; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index 2096942630e..c370726c72f 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS default_table SYNC; +DROP TABLE IF EXISTS default_table; CREATE TABLE default_table ( @@ -26,4 +26,4 @@ ALTER TABLE default_table MODIFY COLUMN enum_column Enum8('undefined' = 0, 'fox' SHOW CREATE TABLE default_table; -DROP TABLE IF EXISTS default_table SYNC; +DROP TABLE IF EXISTS default_table; diff --git a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql index bd11b24d568..687c8051eed 100644 --- a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql +++ b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql @@ -1,7 +1,5 @@ -- Tags: zookeeper -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries - set send_logs_level='error'; drop table if exists mt; drop table if exists rmt sync; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index cabb0ffe294..5f6ca762ae7 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -1,16 +1,13 @@ -- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts -- Tag no-replicated-database: Fails due to additional replicas or shards -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries - -drop table if exists rmt sync; +drop table if exists rmt; -- cleanup code will perform extra Exists -- (so the .reference will not match) create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; system sync replica rmt; insert into rmt values (1); insert into rmt values (1); -system sync replica rmt; system flush logs; select 'log'; @@ -33,7 +30,7 @@ from system.zookeeper_log where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks/%' and op_num not in (1, 12, 500)) order by xid, type, request_idx; -drop table rmt sync; +drop table rmt; system flush logs; select 'duration_ms'; diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql index 78adbee612f..875bd1f96de 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql @@ -1,9 +1,5 @@ -- Tags: long, replica --- in case of keeper fault injection on insert, set bigger number of retries because partitions -set insert_keeper_max_retries=100; -set insert_keeper_retry_max_backoff_ms=10; - -- Testing basic functionality with compact parts set replication_alter_partitions_sync = 2; drop table if exists mt_compact; diff --git a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh index f7615974237..34fa822b6ea 100755 --- a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh +++ b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" $CLICKHOUSE_CLIENT --query " CREATE TABLE mutation_table( @@ -17,10 +17,9 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY key % 10 " -# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" +$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table VALUES(0, 'hello')" +$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table VALUES(0, 'hello')" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM mutation_table" @@ -72,4 +71,4 @@ $CLICKHOUSE_CLIENT --query "SELECT is_done, parts_to_do FROM system.mutations wh $CLICKHOUSE_CLIENT --query "SELECT type, new_part_name FROM system.replication_queue WHERE table='mutation_table' and database='$CLICKHOUSE_DATABASE'" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index f20156fd9e3..8bf5d4f2cf8 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -1,10 +1,7 @@ -- Tags: long, replica -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries -SET replication_alter_partitions_sync=2; - -DROP TABLE IF EXISTS test SYNC; -DROP TABLE IF EXISTS test2 SYNC; +DROP TABLE IF EXISTS test; +DROP TABLE IF EXISTS test2; CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; @@ -20,6 +17,7 @@ ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 3); INSERT INTO test VALUES ('goodbye', 'test'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; +SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -33,6 +31,7 @@ ALTER TABLE test MODIFY COLUMN x Int8; INSERT INTO test VALUES (111, 'abc'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; +SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -49,5 +48,5 @@ ALTER TABLE test RENAME COLUMN y TO z; -- { serverError 524 } ALTER TABLE test DROP COLUMN x; -- { serverError 47 } ALTER TABLE test DROP COLUMN y; -- { serverError 47 } -DROP TABLE test SYNC; -DROP TABLE test2 SYNC; +DROP TABLE test; +DROP TABLE test2; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 4629450c1f9..01678d1b500 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -12,9 +12,9 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" +${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \ @@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name FROM table" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated DETACH PARTITION '3';" -${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" +${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated ATTACH PARTITION '3' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $ATTACH_OUT_STRUCTURE" \ @@ -38,4 +38,4 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name, old_part_name FROM table" # teardown -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql index eea231c9f58..0155b83db31 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql @@ -1,6 +1,5 @@ -- Tags: long, replica, no-replicated-database -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index bf7a471fa40..9f9d1db78f1 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,11 +1,10 @@ -- Tags: long, replica, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS replica1 SYNC; -DROP TABLE IF EXISTS replica2 SYNC; +DROP TABLE IF EXISTS replica1; +DROP TABLE IF EXISTS replica2; CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; @@ -49,5 +48,5 @@ SELECT v FROM replica1 ORDER BY v; SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); -DROP TABLE replica1 SYNC; -DROP TABLE replica2 SYNC; +DROP TABLE replica1; +DROP TABLE replica2; diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index c05d813ca7f..13c772e75d2 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -11,7 +11,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i SYNC; + DROP TABLE IF EXISTS r$i; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -20,11 +20,7 @@ function thread { for x in {0..99}; do # sometimes we can try to commit obsolete part if fetches will be quite fast, # so supress warning messages like "Tried to commit obsolete part ... covered by ..." - # (2) keeper fault injection for inserts because - # it can be a cause of deduplicated parts be visible to SELECTs for sometime (until cleanup thread remove them), - # so the same SELECT on different replicas can return different results, i.e. test output will be non-deterministic - # (see #9712) - $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas + $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas done } @@ -41,5 +37,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 209e18e3329..6eabc9ae1b5 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i SYNC; + DROP TABLE IF EXISTS r$i; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -22,7 +22,7 @@ valid_exceptions_to_retry='Quorum for previous write has not been satisfied yet| function thread { for x in {0..99}; do while true; do - $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break + $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break done done } @@ -40,5 +40,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" done diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index a2ecef64bbb..030ae017e71 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i SYNC; + DROP TABLE IF EXISTS r$i; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_quorum_many', 'r$i') ORDER BY x; " done @@ -20,7 +20,7 @@ done function thread { i=0 retries=300 while [[ $i -lt $retries ]]; do # server can be dead - $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --insert_keeper_max_retries=100 --insert_keeper_retry_max_backoff_ms=10 --query "INSERT INTO r$1 SELECT $2" && break + $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break ((++i)) sleep 0.1 done diff --git a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh index 445706e35bf..9325cac0ae6 100755 --- a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh +++ b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0" @@ -19,10 +19,9 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q2 (x UInt64) ENGINE=ReplicatedMerg $CLICKHOUSE_CLIENT -q "SYSTEM STOP REPLICATION QUEUES parallel_q2" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO parallel_q1 VALUES (1)" +$CLICKHOUSE_CLIENT -q "INSERT INTO parallel_q1 VALUES (1)" -# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert -$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --insert_keeper_fault_injection_probability=0 --query="INSERT INTO parallel_q1 VALUES (2)" & +$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --query="INSERT INTO parallel_q1 VALUES (2)" & part_count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM system.parts WHERE table='parallel_q1' and database='${CLICKHOUSE_DATABASE}'") @@ -67,5 +66,5 @@ $CLICKHOUSE_CLIENT --query="SELECT event_type FROM system.part_log WHERE table=' $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q2" $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql index 86bef193f4d..2f0c59ab64b 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql @@ -16,9 +16,6 @@ CREATE TABLE r2 ( ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); -SET insert_keeper_max_retries=100; -SET insert_keeper_retry_max_backoff_ms=10; - SET insert_quorum_parallel=1; SET insert_quorum=3; @@ -82,11 +79,11 @@ SYSTEM STOP FETCHES r2; SET insert_quorum_timeout=0; -INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } -- retry should fail despite the insert_deduplicate enabled -INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } -INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } SELECT * FROM r2 WHERE key=4; SYSTEM START FETCHES r2; @@ -102,5 +99,5 @@ SELECT 'insert happened'; SELECT COUNT() FROM r1; SELECT COUNT() FROM r2; -DROP TABLE IF EXISTS r1 SYNC; -DROP TABLE IF EXISTS r2 SYNC; +DROP TABLE IF EXISTS r1; +DROP TABLE IF EXISTS r2; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index f217b6094b2..cf06af0113d 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -2,8 +2,6 @@ -- Tag no-replicated-database: Fails due to additional replicas or shards -- Tag no-parallel: static zk path -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries - DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql index b55b4871363..af2a46cd77f 100644 --- a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql +++ b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS merge_tree_pk SYNC; +DROP TABLE IF EXISTS merge_tree_pk; CREATE TABLE merge_tree_pk ( @@ -24,9 +24,9 @@ ATTACH TABLE merge_tree_pk; SELECT * FROM merge_tree_pk FINAL ORDER BY key, value; -DROP TABLE IF EXISTS merge_tree_pk SYNC; +DROP TABLE IF EXISTS merge_tree_pk; -DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; +DROP TABLE IF EXISTS merge_tree_pk_sql; CREATE TABLE merge_tree_pk_sql ( @@ -60,9 +60,9 @@ SELECT * FROM merge_tree_pk_sql FINAL ORDER BY key, value; SHOW CREATE TABLE merge_tree_pk_sql; -DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; +DROP TABLE IF EXISTS merge_tree_pk_sql; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; CREATE TABLE replicated_merge_tree_pk_sql ( @@ -99,4 +99,4 @@ ATTACH TABLE replicated_merge_tree_pk_sql; SHOW CREATE TABLE replicated_merge_tree_pk_sql; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index a5f301b1353..e52e0c94c3c 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -1,6 +1,6 @@ -- Tags: replica -DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions; CREATE TABLE replicated_mutations_empty_partitions ( @@ -11,8 +11,7 @@ ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_rep ORDER BY key PARTITION by key; --- insert_keeper* settings are adjusted since several actual inserts are happening behind one statement due to partitioning i.e. inserts in different partitions -INSERT INTO replicated_mutations_empty_partitions SETTINGS insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=10 SELECT number, toString(number) FROM numbers(10); +INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number) FROM numbers(10); SELECT count(distinct value) FROM replicated_mutations_empty_partitions; @@ -32,4 +31,4 @@ SELECT sum(value) FROM replicated_mutations_empty_partitions; SHOW CREATE TABLE replicated_mutations_empty_partitions; -DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index d68f9bc1837..acaa2cfcd25 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_kill (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_kill', '1') ORDER BY key PARTITION BY key % 100 SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" +$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" function alter_thread { @@ -67,4 +67,4 @@ done $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE concurrent_mutate_kill" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_mutate_kill FINAL" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_mutate_kill" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql index e2926d9a8ac..c77f29d89c2 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql @@ -1,8 +1,7 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -SET insert_keeper_fault_injection_probability=0; -DROP TABLE IF EXISTS partitioned_table SYNC; +DROP TABLE IF EXISTS partitioned_table; CREATE TABLE partitioned_table ( key UInt64, @@ -48,4 +47,4 @@ SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AN SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; -DROP TABLE IF EXISTS partitioned_table SYNC; +DROP TABLE IF EXISTS partitioned_table; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index 87e1a039488..dad5cdbf1f3 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -1,7 +1,7 @@ -- Tags: long, zookeeper -DROP TABLE IF EXISTS i20203_1 SYNC; -DROP TABLE IF EXISTS i20203_2 SYNC; +DROP TABLE IF EXISTS i20203_1; +DROP TABLE IF EXISTS i20203_2; CREATE TABLE i20203_1 (a Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') @@ -26,5 +26,5 @@ WHERE table = 'i20203_2' AND database = currentDatabase(); ATTACH TABLE i20203_1; -DROP TABLE i20203_1 SYNC; -DROP TABLE i20203_2 SYNC; +DROP TABLE IF EXISTS i20203_1; +DROP TABLE IF EXISTS i20203_2; diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index 4e4255fe9c2..85662438f33 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -51,9 +51,6 @@ $CLICKHOUSE_CLIENT -nm -q """ partition by key%100 settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; - SET insert_keeper_max_retries=1000; - SET insert_keeper_retry_max_backoff_ms=10; - insert into rep_data_01810 select * from numbers(100); drop table rep_data_01810 settings log_queries=1; system flush logs; diff --git a/tests/queries/0_stateless/01825_type_json_3.sql.j2 b/tests/queries/0_stateless/01825_type_json_3.sql.j2 index 0fbf7a936d8..62d86c3efd4 100644 --- a/tests/queries/0_stateless/01825_type_json_3.sql.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.sql.j2 @@ -2,8 +2,6 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries - SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_json_3; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 46cd09622b1..1de23c13a65 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -1,6 +1,3 @@ -SET insert_keeper_max_retries=100; -SET insert_keeper_retry_max_backoff_ms=10; - SELECT 'simple partition key:'; DROP TABLE IF EXISTS table1 SYNC; CREATE TABLE table1 (id Int64, v UInt64) @@ -18,7 +15,7 @@ select 'where id % 200 < 0:'; select id from table1 where id % 200 < 0 order by id; SELECT 'tuple as partition key:'; -DROP TABLE IF EXISTS table2 SYNC; +DROP TABLE IF EXISTS table2; CREATE TABLE table2 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id; @@ -27,7 +24,7 @@ INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10); SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p; SELECT 'recursive modulo partition key:'; -DROP TABLE IF EXISTS table3 SYNC; +DROP TABLE IF EXISTS table3; CREATE TABLE table3 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id; @@ -41,7 +38,7 @@ SELECT 'After detach:'; SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; SELECT 'Indexes:'; -DROP TABLE IF EXISTS table4 SYNC; +DROP TABLE IF EXISTS table4; CREATE TABLE table4 (id Int64, v UInt64, s String, INDEX a (id * 2, s) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v; diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 8924627a717..635da491aa0 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,5 +1,4 @@ -- Tags: long, no-parallel -SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush @@ -9,8 +8,8 @@ insert into data_02228 select number, 1, number from numbers_mt(100e3) settings insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; -drop table if exists data_rep_02228 SYNC; +drop table if exists data_rep_02228; create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } -drop table data_rep_02228 SYNC; +drop table data_rep_02228; diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index 9ea924377b2..e8111bf7a03 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -10,7 +10,7 @@ function check_refcnt_for_table() local table=$1 && shift $CLICKHOUSE_CLIENT -q "system stop merges $table" - $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" + $CLICKHOUSE_CLIENT -q "insert into $table select number, number%4 from numbers(200)" local query_id query_id="$table-$(random_str 10)" @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nmq " check_refcnt_for_table data_02340 $CLICKHOUSE_CLIENT -nmq " - drop table if exists data_02340_rep sync; + drop table if exists data_02340_rep; create table data_02340_rep (key Int, part Int) engine=ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340_rep diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index 88ff1f5b7c6..11caf1e45de 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -1,18 +1,18 @@ #!/usr/bin/env bash -# Tags: long, zookeeper +# Tags: zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" -$CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt1;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt2;" $CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n;" $CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n;" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" +$CLICKHOUSE_CLIENT -q "insert into rmt1 values (1);" +$CLICKHOUSE_CLIENT -q "insert into rmt1 values (2);" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" @@ -32,7 +32,7 @@ $CLICKHOUSE_CLIENT -q "select * from rmt1;" 2>/dev/null $CLICKHOUSE_CLIENT -q "detach table rmt1;" $CLICKHOUSE_CLIENT -q "attach table rmt1;" -$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" +$CLICKHOUSE_CLIENT -q "insert into rmt1 values (3);" $CLICKHOUSE_CLIENT -q "system start merges rmt2;" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" @@ -42,5 +42,5 @@ $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" $CLICKHOUSE_CLIENT -q "select 3, *, _part from rmt1 order by n;" $CLICKHOUSE_CLIENT -q "select 4, *, _part from rmt2 order by n;" -$CLICKHOUSE_CLIENT -q "drop table rmt1 sync;" -$CLICKHOUSE_CLIENT -q "drop table rmt2 sync;" +$CLICKHOUSE_CLIENT -q "drop table rmt1;" +$CLICKHOUSE_CLIENT -q "drop table rmt2;" diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 0db816332a1..371f7389837 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -1,7 +1,5 @@ -- Tags: long -SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries - drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference deleted file mode 100644 index f5f1cf8ac64..00000000000 --- a/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -11 diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql deleted file mode 100644 index 774370bc132..00000000000 --- a/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql +++ /dev/null @@ -1,26 +0,0 @@ --- Tags: replica - -DROP TABLE IF EXISTS keeper_retries_r1 SYNC; -DROP TABLE IF EXISTS keeper_retries_r2 SYNC; - -CREATE TABLE keeper_retries_r1(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r1') ORDER BY tuple (); -CREATE TABLE keeper_retries_r2(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r2') ORDER BY tuple(); - -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (1); -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (2); -- { serverError KEEPER_EXCEPTION } -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=10 VALUES (3); -- { serverError KEEPER_EXCEPTION } - -SET insert_quorum=2; -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (11); -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (12); -- { serverError KEEPER_EXCEPTION } -INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=1 VALUES (13); -- { serverError KEEPER_EXCEPTION } - --- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.05, insert_keeper_fault_injection_seed=1 VALUES (21); --- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.2, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=2 VALUES (22); --- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.3, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=3 VALUES (23); --- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.4, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=4 VALUES (24); - -SELECT * FROM keeper_retries_r1 order by a; - -DROP TABLE keeper_retries_r1 SYNC; -DROP TABLE keeper_retries_r2 SYNC; From c12c1cdf9b1dafbb117c008a6e7fb4ee8e789087 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 7 Nov 2022 21:30:41 +0100 Subject: [PATCH 316/526] Allow release only from ready commits --- tests/ci/release.py | 49 +++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 47 insertions(+), 2 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index fd4bda3eae4..8024091e300 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -1,5 +1,14 @@ #!/usr/bin/env python +""" +script to create releases for ClickHouse + +The `gh` CLI prefered over the PyGithub to have an easy way to rollback bad +release in command line by simple execution giving rollback commands + +On another hand, PyGithub is used for convenient getting commit's status from API +""" + from contextlib import contextmanager from typing import List, Optional @@ -8,6 +17,8 @@ import logging import subprocess from git_helper import commit, release_branch +from github_helper import GitHub +from mark_release_ready import RELEASE_READY_STATUS from version_helper import ( FILE_WITH_VERSION_PATH, GENERATED_CONTRIBUTORS, @@ -67,12 +78,12 @@ class Release: self._release_branch = "" self._rollback_stack = [] # type: List[str] - def run(self, cmd: str, cwd: Optional[str] = None) -> str: + def run(self, cmd: str, cwd: Optional[str] = None, **kwargs) -> str: cwd_text = "" if cwd: cwd_text = f" (CWD='{cwd}')" logging.info("Running command%s:\n %s", cwd_text, cmd) - return self._git.run(cmd, cwd) + return self._git.run(cmd, cwd, **kwargs) def set_release_branch(self): # Fetch release commit in case it does not exist locally @@ -94,6 +105,38 @@ class Release: return VersionType.LTS return VersionType.STABLE + def check_commit_release_ready(self): + # First, get the auth token from gh cli + auth_status = self.run( + "gh auth status -t", stderr=subprocess.STDOUT + ).splitlines() + token = "" + for line in auth_status: + if "✓ Token:" in line: + token = line.split()[-1] + if not token: + logging.error("Can not extract token from `gh auth`") + raise subprocess.SubprocessError("Can not extract token from `gh auth`") + gh = GitHub(token, per_page=100) + repo = gh.get_repo(str(self.repo)) + + # Statuses are ordered by descending updated_at, so the first necessary + # status in the list is the most recent + statuses = repo.get_commit(self.release_commit).get_statuses() + for status in statuses: + if status.context == RELEASE_READY_STATUS: + if status.state == "success": + return + + raise Exception( + f"the status {RELEASE_READY_STATUS} is {status.state}, not success" + ) + + raise Exception( + f"the status {RELEASE_READY_STATUS} " + f"is not found for commit {self.release_commit}" + ) + def check_prerequisites(self): """ Check tooling installed in the system, `git` is checked by Git() init @@ -108,6 +151,8 @@ class Release: ) raise + self.check_commit_release_ready() + def do(self, check_dirty: bool, check_branch: bool, with_release_branch: bool): self.check_prerequisites() From 71437851b18f155b5e2620ab31480649427ab5fc Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Nov 2022 12:56:00 +0100 Subject: [PATCH 317/526] Update client.py --- tests/queries/0_stateless/helpers/client.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 7c9b2c1c83f..26c82cad0d1 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -17,7 +17,7 @@ class client(object): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: command = ( - os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client --progress" + os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" ) self.client.command = command self.client.eol("\r") From 77c0728e779ed4024336c58d49fc22a22df24ce6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 10 Nov 2022 11:59:25 +0000 Subject: [PATCH 318/526] Fix aarch build. --- src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index fb8c4b17a3e..ac131a1b67c 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -740,7 +740,7 @@ InputOrderInfoPtr buildInputOrderInfo(SortingStep & sorting, QueryPlan::Node & n return nullptr; const auto & description = sorting.getSortDescription(); - auto limit = sorting.getLimit(); + size_t limit = sorting.getLimit(); ActionsDAGPtr dag; FixedColumns fixed_columns; From 98bb346a29af47aa323eec4788575dae4957893b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 12:05:41 +0000 Subject: [PATCH 319/526] Automatic style fix --- tests/queries/0_stateless/helpers/client.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/helpers/client.py b/tests/queries/0_stateless/helpers/client.py index 26c82cad0d1..5c8589dfca1 100644 --- a/tests/queries/0_stateless/helpers/client.py +++ b/tests/queries/0_stateless/helpers/client.py @@ -16,9 +16,7 @@ class client(object): def __init__(self, command=None, name="", log=None): self.client = uexpect.spawn(["/bin/bash", "--noediting"]) if command is None: - command = ( - os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" - ) + command = os.environ.get("CLICKHOUSE_BINARY", "clickhouse") + " client" self.client.command = command self.client.eol("\r") self.client.logger(log, prefix=name) From e2de80953285c0ec93da2eb4a8ed349c9be6f432 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 14 Sep 2022 15:40:45 +0200 Subject: [PATCH 320/526] Add typing to commit_status_helper.py --- tests/ci/commit_status_helper.py | 33 ++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 8b9d28502c1..cbd0fd4b3ce 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -3,19 +3,19 @@ import csv import os import time -from typing import Optional +from typing import List import logging from ci_config import CI_CONFIG, REQUIRED_CHECKS from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from github import Github from github.Commit import Commit -from pr_info import SKIP_MERGEABLE_CHECK_LABEL +from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL RETRY = 5 -def override_status(status, check_name, invert=False): +def override_status(status: str, check_name: str, invert=False) -> str: if CI_CONFIG["tests_config"].get(check_name, {}).get("force_tests", False): return "success" @@ -27,24 +27,23 @@ def override_status(status, check_name, invert=False): return status -def get_commit( - gh: Github, commit_sha: str, retry_count: int = RETRY -) -> Optional[Commit]: +def get_commit(gh: Github, commit_sha: str, retry_count: int = RETRY) -> Commit: for i in range(retry_count): try: repo = gh.get_repo(GITHUB_REPOSITORY) commit = repo.get_commit(commit_sha) - return commit + break except Exception as ex: if i == retry_count - 1: raise ex time.sleep(i) - # just suppress warning - return None + return commit -def post_commit_status(gh, sha, check_name, description, state, report_url): +def post_commit_status( + gh: Github, sha: str, check_name: str, description: str, state: str, report_url: str +): for i in range(RETRY): try: commit = get_commit(gh, sha, 1) @@ -61,7 +60,9 @@ def post_commit_status(gh, sha, check_name, description, state, report_url): time.sleep(i) -def post_commit_status_to_file(file_path, description, state, report_url): +def post_commit_status_to_file( + file_path: str, description: str, state: str, report_url: str +): if os.path.exists(file_path): raise Exception(f'File "{file_path}" already exists!') with open(file_path, "w", encoding="utf-8") as f: @@ -69,21 +70,21 @@ def post_commit_status_to_file(file_path, description, state, report_url): out.writerow([state, report_url, description]) -def remove_labels(gh, pr_info, labels_names): +def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.remove_from_labels(label) -def post_labels(gh, pr_info, labels_names): +def post_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) for label in labels_names: pull_request.add_to_labels(label) -def fail_mergeable_check(commit, description): +def fail_mergeable_check(commit: Commit, description: str): commit.create_status( context="Mergeable Check", description=description, @@ -92,7 +93,7 @@ def fail_mergeable_check(commit, description): ) -def reset_mergeable_check(commit, description=""): +def reset_mergeable_check(commit: Commit, description: str = ""): commit.create_status( context="Mergeable Check", description=description, @@ -101,7 +102,7 @@ def reset_mergeable_check(commit, description=""): ) -def update_mergeable_check(gh, pr_info, check_name): +def update_mergeable_check(gh: Github, pr_info: PRInfo, check_name: str): if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels: return From eec74e46e42a41b11eb80555acb2cb2a0aaaf8eb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 14 Sep 2022 16:00:46 +0200 Subject: [PATCH 321/526] Move get_commit_filtered_statuses to commit_status_helper --- tests/ci/commit_status_helper.py | 18 ++++++++++++++++++ tests/ci/rerun_helper.py | 26 ++++---------------------- 2 files changed, 22 insertions(+), 22 deletions(-) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index cbd0fd4b3ce..185dc64daa9 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -10,9 +10,11 @@ from ci_config import CI_CONFIG, REQUIRED_CHECKS from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from github import Github from github.Commit import Commit +from github.CommitStatus import CommitStatus from pr_info import PRInfo, SKIP_MERGEABLE_CHECK_LABEL RETRY = 5 +CommitStatuses = List[CommitStatus] def override_status(status: str, check_name: str, invert=False) -> str: @@ -70,6 +72,22 @@ def post_commit_status_to_file( out.writerow([state, report_url, description]) +def get_commit_filtered_statuses(commit: Commit) -> CommitStatuses: + """ + Squash statuses to latest state + 1. context="first", state="success", update_time=1 + 2. context="second", state="success", update_time=2 + 3. context="first", stat="failure", update_time=3 + =========> + 1. context="second", state="success" + 2. context="first", stat="failure" + """ + filtered = {} + for status in sorted(commit.get_statuses(), key=lambda x: x.updated_at): + filtered[status.context] = status + return list(filtered.values()) + + def remove_labels(gh: Github, pr_info: PRInfo, labels_names: List[str]): repo = gh.get_repo(GITHUB_REPOSITORY) pull_request = repo.get_pull(pr_info.number) diff --git a/tests/ci/rerun_helper.py b/tests/ci/rerun_helper.py index c4ae70eadb9..fa73256d759 100644 --- a/tests/ci/rerun_helper.py +++ b/tests/ci/rerun_helper.py @@ -1,14 +1,13 @@ #!/usr/bin/env python3 -from typing import List, Optional +from typing import Optional -from commit_status_helper import get_commit +from commit_status_helper import get_commit, get_commit_filtered_statuses from github import Github from github.CommitStatus import CommitStatus from pr_info import PRInfo -CommitStatuses = List[CommitStatus] - +# TODO: move it to commit_status_helper class RerunHelper: def __init__(self, gh: Github, pr_info: PRInfo, check_name: str): self.gh = gh @@ -18,7 +17,7 @@ class RerunHelper: if commit is None: raise ValueError(f"unable to receive commit for {pr_info.sha}") self.pygh_commit = commit - self.statuses = self.ger_filtered_statuses() + self.statuses = get_commit_filtered_statuses(commit) def is_already_finished_by_status(self) -> bool: # currently we agree even for failed statuses @@ -35,20 +34,3 @@ class RerunHelper: if self.check_name in status.context: return status return None - - def ger_filtered_statuses(self) -> CommitStatuses: - """ - Squash statuses to latest state - 1. context="first", state="success", update_time=1 - 2. context="second", state="success", update_time=2 - 3. context="first", stat="failure", update_time=3 - =========> - 1. context="second", state="success" - 2. context="first", stat="failure" - """ - filt = {} - for status in sorted( - self.pygh_commit.get_statuses(), key=lambda x: x.updated_at - ): - filt[status.context] = status - return list(filt.values()) From f8c907c463f5a53c79589d129a6e9a32f8b8fc6e Mon Sep 17 00:00:00 2001 From: Igor Nikonov <954088+devcrafter@users.noreply.github.com> Date: Thu, 10 Nov 2022 13:14:04 +0100 Subject: [PATCH 322/526] Revert "Revert " Keeper retries during insert (clean)"" --- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.h | 2 +- src/Core/Settings.h | 5 + .../MergeTree/EphemeralLockInZooKeeper.cpp | 39 +- .../MergeTree/EphemeralLockInZooKeeper.h | 15 +- ...ReplicatedMergeTreeMergeStrategyPicker.cpp | 4 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 247 +++++--- .../MergeTree/ReplicatedMergeTreeSink.h | 19 +- src/Storages/MergeTree/ZooKeeperRetries.h | 265 +++++++++ .../MergeTree/ZooKeeperWithFaultInjection.h | 527 ++++++++++++++++++ src/Storages/StorageReplicatedMergeTree.cpp | 93 +++- src/Storages/StorageReplicatedMergeTree.h | 35 +- tests/config/install.sh | 1 + .../config/users.d/insert_keeper_retries.xml | 8 + .../__init__.py | 0 .../test_inserts_with_keeper_retries/test.py | 100 ++++ .../integration/test_restore_replica/test.py | 9 +- .../00121_drop_column_zookeeper.sql | 6 +- ...partitioning_replicated_zookeeper_long.sql | 47 +- .../00652_replicated_mutations_zookeeper.sh | 24 +- ...replicated_without_partition_zookeeper.sql | 8 +- ..._fetch_merged_or_mutated_part_zookeeper.sh | 9 +- ...ated_minimalistic_part_header_zookeeper.sh | 1 + .../01037_zookeeper_check_table_empty_pk.sql | 9 +- ...eeper_system_mutations_with_parts_names.sh | 3 +- ...eeper_mutations_and_insert_quorum_long.sql | 11 +- .../01135_default_and_alter_zookeeper.sql | 4 +- ...mutation_stuck_after_replace_partition.sql | 2 + .../0_stateless/01158_zookeeper_log_long.sql | 7 +- ...compact_part_replicated_zookeeper_long.sql | 4 + ...18_long_unsuccessful_mutation_zookeeper.sh | 9 +- ...artition_key_replicated_zookeeper_long.sql | 13 +- ...1417_freeze_partition_verbose_zookeeper.sh | 8 +- ...replicated_detach_drop_and_quorum_long.sql | 1 + ...01451_replicated_detach_drop_part_long.sql | 9 +- .../01459_manual_write_to_replicas.sh | 10 +- .../01459_manual_write_to_replicas_quorum.sh | 6 +- ...check_many_parallel_quorum_inserts_long.sh | 4 +- .../01509_parallel_quorum_and_merge_long.sh | 13 +- ...arallel_quorum_insert_no_replicas_long.sql | 13 +- ..._execute_merges_on_single_replica_long.sql | 2 + ...primary_key_without_order_by_zookeeper.sql | 12 +- ...6_replicated_mutations_empty_partition.sql | 7 +- .../01593_concurrent_alter_mutations_kill.sh | 6 +- ..._part_and_deduplication_zookeeper_long.sql | 5 +- ...kground_checker_blather_zookeeper_long.sql | 8 +- .../01810_max_part_removal_threads_long.sh | 3 + .../0_stateless/01825_type_json_3.sql.j2 | 2 + .../01870_modulo_partition_key.sql | 9 +- .../02228_merge_tree_insert_memory_usage.sql | 5 +- .../02340_parts_refcnt_mergetree.sh | 4 +- .../02369_lost_part_intersecting_merges.sh | 16 +- .../02448_clone_replica_lost_part.sql | 2 + ...456_keeper_retries_during_insert.reference | 2 + .../02456_keeper_retries_during_insert.sql | 26 + 55 files changed, 1451 insertions(+), 250 deletions(-) create mode 100644 src/Storages/MergeTree/ZooKeeperRetries.h create mode 100644 src/Storages/MergeTree/ZooKeeperWithFaultInjection.h create mode 100644 tests/config/users.d/insert_keeper_retries.xml create mode 100644 tests/integration/test_inserts_with_keeper_retries/__init__.py create mode 100644 tests/integration/test_inserts_with_keeper_retries/test.py create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.reference create mode 100644 tests/queries/0_stateless/02456_keeper_retries_during_insert.sql diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index aad5131fcb5..62615afb4f7 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -34,7 +34,7 @@ using TestKeeperRequestPtr = std::shared_ptr; class TestKeeper final : public IKeeper { public: - TestKeeper(const zkutil::ZooKeeperArgs & args_); + explicit TestKeeper(const zkutil::ZooKeeperArgs & args_); ~TestKeeper() override; bool isExpired() const override { return expired; } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index c9529b78f8a..9de8241cfbe 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -156,7 +156,7 @@ public: using Ptr = std::shared_ptr; using ErrorsList = std::initializer_list; - ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); + explicit ZooKeeper(const ZooKeeperArgs & args_, std::shared_ptr zk_log_ = nullptr); /** Config of the form: diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 8c67cf94a61..45ca9845afb 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -658,6 +658,11 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(Bool, multiple_joins_try_to_keep_original_names, false, "Do not add aliases to top level expression list on multiple joins rewrite", 0) \ M(Bool, optimize_distinct_in_order, true, "Enable DISTINCT optimization if some columns in DISTINCT form a prefix of sorting. For example, prefix of sorting key in merge tree or ORDER BY statement", 0) \ M(Bool, optimize_sorting_by_input_stream_properties, true, "Optimize sorting by sorting properties of input stream", 0) \ + M(UInt64, insert_keeper_max_retries, 0, "Max retries for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_initial_backoff_ms, 100, "Initial backoff timeout for keeper operations during insert", 0) \ + M(UInt64, insert_keeper_retry_max_backoff_ms, 10000, "Max backoff timeout for keeper operations during insert", 0) \ + M(Float, insert_keeper_fault_injection_probability, 0.0f, "Approximate probability of failure for a keeper request during insert. Valid value is in interval [0.0f, 1.0f]", 0) \ + M(UInt64, insert_keeper_fault_injection_seed, 0, "0 - random seed, otherwise the setting value", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS and move obsolete settings to OBSOLETE_SETTINGS. diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp index 6ddb35d109e..6d75f523056 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -12,22 +13,22 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_) - : zookeeper(&zookeeper_), path_prefix(path_prefix_), path(path_) +EphemeralLockInZooKeeper::EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_) + : zookeeper(zookeeper_), path_prefix(path_prefix_), path(path_) { if (path.size() <= path_prefix.size()) throw Exception("Logical error: name of the main node is shorter than prefix.", ErrorCodes::LOGICAL_ERROR); } std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path) + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path) { String path; if (deduplication_path.empty()) { String holder_path = temp_path + "/" + EphemeralLockInZooKeeper::LEGACY_LOCK_OTHER; - path = zookeeper_.create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); + path = zookeeper_->create(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential); } else { @@ -39,11 +40,15 @@ std::optional createEphemeralLockInZooKeeper( ops.emplace_back(zkutil::makeRemoveRequest(deduplication_path, -1)); ops.emplace_back(zkutil::makeCreateRequest(path_prefix_, holder_path, zkutil::CreateMode::EphemeralSequential)); Coordination::Responses responses; - Coordination::Error e = zookeeper_.tryMulti(ops, responses); + Coordination::Error e = zookeeper_->tryMulti(ops, responses); if (e != Coordination::Error::ZOK) { if (responses[0]->error == Coordination::Error::ZNODEEXISTS) { + LOG_DEBUG( + &Poco::Logger::get("createEphemeralLockInZooKeeper"), + "Deduplication path already exists: deduplication_path={}", + deduplication_path); return {}; } else @@ -82,9 +87,31 @@ EphemeralLockInZooKeeper::~EphemeralLockInZooKeeper() { unlock(); } + catch (const zkutil::KeeperException & e) + { + if (Coordination::isHardwareError(e.code)) + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper communication error during unlock: code={} message='{}'", + e.code, + e.message()); + else if (e.code == Coordination::Error::ZNONODE) + /// To avoid additional round-trip for unlocking, + /// ephemeral node can be deleted explicitly as part of another multi op request to ZK + /// and marked as such via assumeUnlocked() if we got successful response. + /// But it's possible that the multi op request can be executed on server side, and client will not get response due to network issue. + /// In such case, assumeUnlocked() will not be called, so we'll get ZNONODE error here since the noded is already deleted + LOG_DEBUG( + &Poco::Logger::get("EphemeralLockInZooKeeper"), + "ZooKeeper node was already deleted: code={} message={}", + e.code, + e.message()); + else + tryLogCurrentException("EphemeralLockInZooKeeper"); + } catch (...) { - tryLogCurrentException("~EphemeralLockInZooKeeper"); + tryLogCurrentException("EphemeralLockInZooKeeper"); } } diff --git a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h index 5945fa10d91..c630bcc4681 100644 --- a/src/Storages/MergeTree/EphemeralLockInZooKeeper.h +++ b/src/Storages/MergeTree/EphemeralLockInZooKeeper.h @@ -12,6 +12,8 @@ namespace DB { +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; namespace ErrorCodes { @@ -25,13 +27,14 @@ namespace ErrorCodes class EphemeralLockInZooKeeper : public boost::noncopyable { friend std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); protected: - EphemeralLockInZooKeeper() = delete; - EphemeralLockInZooKeeper(const String & path_prefix_, zkutil::ZooKeeper & zookeeper_, const String & path_); + EphemeralLockInZooKeeper(const String & path_prefix_, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & path_); public: + EphemeralLockInZooKeeper() = delete; + /// Fake "secondary node" names for blocks with and without "deduplication_path" static constexpr const char * LEGACY_LOCK_INSERT = "abandonable_lock-insert"; static constexpr const char * LEGACY_LOCK_OTHER = "abandonable_lock-other"; @@ -53,7 +56,7 @@ public: bool isLocked() const { - return zookeeper; + return zookeeper.get(); } String getPath() const @@ -91,13 +94,13 @@ public: ~EphemeralLockInZooKeeper(); private: - zkutil::ZooKeeper * zookeeper = nullptr; + ZooKeeperWithFaultInjectionPtr zookeeper; String path_prefix; String path; }; std::optional createEphemeralLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, const String & deduplication_path); + const String & path_prefix_, const String & temp_path, const ZooKeeperWithFaultInjectionPtr & zookeeper_, const String & deduplication_path); /// Acquires block number locks in all partitions. diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp index 626295d7255..192f0d23f96 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeMergeStrategyPicker.cpp @@ -112,6 +112,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() && now - last_refresh_time < REFRESH_STATE_MINIMUM_INTERVAL_SECONDS) return; + LOG_DEBUG(storage.log, "Updating strategy picker state"); + auto zookeeper = storage.getZooKeeper(); auto all_replicas = zookeeper->getChildren(storage.zookeeper_path + "/replicas"); @@ -154,6 +156,8 @@ void ReplicatedMergeTreeMergeStrategyPicker::refreshState() last_refresh_time = now; current_replica_index = current_replica_index_tmp; active_replicas = active_replicas_tmp; + + LOG_DEBUG(storage.log, "Strategy picker state updated, current replica: {}, active replicas: [{}]", current_replica_index, fmt::join(active_replicas, ", ")); } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 5482052e184..b6a486f14ee 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -9,7 +9,6 @@ #include #include - namespace ProfileEvents { extern const Event DuplicatedInsertedBlocks; @@ -32,6 +31,7 @@ namespace ErrorCodes extern const int DUPLICATE_DATA_PART; extern const int PART_IS_TEMPORARILY_LOCKED; extern const int LOGICAL_ERROR; + extern const int TABLE_IS_READ_ONLY; extern const int QUERY_WAS_CANCELLED; } @@ -84,7 +84,7 @@ ReplicatedMergeTreeSink::ReplicatedMergeTreeSink( ReplicatedMergeTreeSink::~ReplicatedMergeTreeSink() = default; /// Allow to verify that the session in ZooKeeper is still alive. -static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) +static void assertSessionIsNotExpired(const zkutil::ZooKeeperPtr & zookeeper) { if (!zookeeper) throw Exception("No ZooKeeper session.", ErrorCodes::NO_ZOOKEEPER); @@ -93,7 +93,7 @@ static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) throw Exception("ZooKeeper session has been expired.", ErrorCodes::NO_ZOOKEEPER); } -size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper) +size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!isQuorumEnabled()) return 0; @@ -103,6 +103,7 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z Strings replicas = zookeeper->getChildren(fs::path(storage.zookeeper_path) / "replicas"); Strings exists_paths; + exists_paths.reserve(replicas.size()); for (const auto & replica : replicas) if (replica != storage.replica_name) exists_paths.emplace_back(fs::path(storage.zookeeper_path) / "replicas" / replica / "is_active"); @@ -110,20 +111,28 @@ size_t ReplicatedMergeTreeSink::checkQuorumPrecondition(zkutil::ZooKeeperPtr & z auto exists_result = zookeeper->exists(exists_paths); auto get_results = zookeeper->get(Strings{storage.replica_path + "/is_active", storage.replica_path + "/host"}); + Coordination::Error keeper_error = Coordination::Error::ZOK; size_t active_replicas = 1; /// Assume current replica is active (will check below) for (size_t i = 0; i < exists_paths.size(); ++i) { - auto status = exists_result[i]; - if (status.error == Coordination::Error::ZOK) + auto error = exists_result[i].error; + if (error == Coordination::Error::ZOK) ++active_replicas; + else if (Coordination::isHardwareError(error)) + keeper_error = error; } size_t replicas_number = replicas.size(); size_t quorum_size = getQuorumSize(replicas_number); if (active_replicas < quorum_size) + { + if (Coordination::isHardwareError(keeper_error)) + throw Coordination::Exception("Failed to check number of alive replicas", keeper_error); + throw Exception(ErrorCodes::TOO_FEW_LIVE_REPLICAS, "Number of alive replicas ({}) is less than requested quorum ({}/{}).", active_replicas, quorum_size, replicas_number); + } /** Is there a quorum for the last part for which a quorum is needed? * Write of all the parts with the included quorum is linearly ordered. @@ -156,15 +165,34 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + const auto & settings = context->getSettingsRef(); + zookeeper_retries_info = ZooKeeperRetriesInfo( + "ReplicatedMergeTreeSink::consume", + settings.insert_keeper_max_retries ? log : nullptr, + settings.insert_keeper_max_retries, + settings.insert_keeper_retry_initial_backoff_ms, + settings.insert_keeper_retry_max_backoff_ms); + + ZooKeeperWithFaultInjectionPtr zookeeper = ZooKeeperWithFaultInjection::createInstance( + settings.insert_keeper_fault_injection_probability, + settings.insert_keeper_fault_injection_seed, + storage.getZooKeeper(), + "ReplicatedMergeTreeSink::consume", + log); /** If write is with quorum, then we check that the required number of replicas is now live, * and also that for all previous parts for which quorum is required, this quorum is reached. * And also check that during the insertion, the replica was not reinitialized or disabled (by the value of `is_active` node). * TODO Too complex logic, you can do better. */ - size_t replicas_num = checkQuorumPrecondition(zookeeper); + size_t replicas_num = 0; + ZooKeeperRetriesControl quorum_retries_ctl("checkQuorumPrecondition", zookeeper_retries_info); + quorum_retries_ctl.retryLoop( + [&]() + { + zookeeper->setKeeper(storage.getZooKeeper()); + replicas_num = checkQuorumPrecondition(zookeeper); + }); if (!storage_snapshot->object_columns.empty()) convertDynamicColumnsToTuples(block, storage_snapshot); @@ -176,7 +204,6 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) size_t streams = 0; bool support_parallel_write = false; - const Settings & settings = context->getSettingsRef(); for (auto & current_block : part_blocks) { @@ -256,7 +283,7 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) finishDelayedChunk(zookeeper); } -void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper) +void ReplicatedMergeTreeSink::finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper) { if (!delayed_chunk) return; @@ -271,7 +298,7 @@ void ReplicatedMergeTreeSink::finishDelayedChunk(zkutil::ZooKeeperPtr & zookeepe try { - commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num); + commitPart(zookeeper, part, partition.block_id, delayed_chunk->replicas_num, false); last_block_is_duplicate = last_block_is_duplicate || part->is_duplicate; @@ -294,8 +321,9 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt { /// NOTE: No delay in this case. That's Ok. - auto zookeeper = storage.getZooKeeper(); - assertSessionIsNotExpired(zookeeper); + auto origin_zookeeper = storage.getZooKeeper(); + assertSessionIsNotExpired(origin_zookeeper); + auto zookeeper = std::make_shared(origin_zookeeper); size_t replicas_num = checkQuorumPrecondition(zookeeper); @@ -304,7 +332,7 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt try { part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - commitPart(zookeeper, part, "", replicas_num); + commitPart(zookeeper, part, "", replicas_num, true); PartLog::addNewPart(storage.getContext(), part, watch.elapsed()); } catch (...) @@ -315,10 +343,11 @@ void ReplicatedMergeTreeSink::writeExistingPart(MergeTreeData::MutableDataPartPt } void ReplicatedMergeTreeSink::commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num) + size_t replicas_num, + bool writing_existing_part) { /// It is possible that we alter a part with different types of source columns. /// In this case, if column was not altered, the result type will be different with what we have in metadata. @@ -326,8 +355,6 @@ void ReplicatedMergeTreeSink::commitPart( /// /// metadata_snapshot->check(part->getColumns()); - assertSessionIsNotExpired(zookeeper); - String temporary_part_relative_path = part->getDataPartStorage().getPartDirectory(); /// There is one case when we need to retry transaction in a loop. @@ -337,14 +364,65 @@ void ReplicatedMergeTreeSink::commitPart( bool is_already_existing_part = false; - while (true) + /// for retries due to keeper error + bool part_committed_locally_but_zookeeper = false; + Coordination::Error write_part_info_keeper_error = Coordination::Error::ZOK; + + ZooKeeperRetriesControl retries_ctl("commitPart", zookeeper_retries_info); + retries_ctl.retryLoop([&]() { + zookeeper->setKeeper(storage.getZooKeeper()); + if (storage.is_readonly) + { + /// stop retries if in shutdown + if (storage.shutdown_called) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode due to shutdown: replica_path={}", storage.replica_path); + + /// When we attach existing parts it's okay to be in read-only mode + /// For example during RESTORE REPLICA. + if (!writing_existing_part) + { + retries_ctl.setUserError(ErrorCodes::TABLE_IS_READ_ONLY, "Table is in readonly mode: replica_path={}", storage.replica_path); + return; + } + } + + if (retries_ctl.isRetry()) + { + /// If we are retrying, check if last iteration was actually successful, + /// we could get network error on committing part to zk + /// but the operation could be completed by zk server + + /// If this flag is true, then part is in Active state, and we'll not retry anymore + /// we only check if part was committed to zk and return success or failure correspondingly + /// Note: if commit to zk failed then cleanup thread will mark the part as Outdated later + if (part_committed_locally_but_zookeeper) + { + /// check that info about the part was actually written in zk + if (zookeeper->exists(fs::path(storage.replica_path) / "parts" / part->name)) + { + LOG_DEBUG(log, "Part was successfully committed on previous iteration: part_id={}", part->name); + } + else + { + retries_ctl.setUserError( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Insert failed due to zookeeper error. Please retry. Reason: {}", + Coordination::errorMessage(write_part_info_keeper_error)); + } + + retries_ctl.stopRetries(); + return; + } + } + /// Obtain incremental block number and lock it. The lock holds our intention to add the block to the filesystem. /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. /// Also, make deduplication check. If a duplicate is detected, no nodes are created. /// Allocate new block number and check for duplicates - bool deduplicate_block = !block_id.empty(); + const bool deduplicate_block = !block_id.empty(); String block_id_path = deduplicate_block ? storage.zookeeper_path + "/blocks/" + block_id : ""; auto block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, block_id_path); ThreadFuzzer::maybeInjectSleep(); @@ -468,7 +546,13 @@ void ReplicatedMergeTreeSink::commitPart( else quorum_path = storage.zookeeper_path + "/quorum/status"; - waitForQuorum(zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); + if (!retries_ctl.callAndCatchAll( + [&]() + { + waitForQuorum( + zookeeper, existing_part_name, quorum_path, quorum_info.is_active_node_version, replicas_num); + })) + return; } else { @@ -477,6 +561,7 @@ void ReplicatedMergeTreeSink::commitPart( return; } + LOG_INFO(log, "Block with ID {} already exists on other replicas as part {}; will write it locally with that name.", block_id, existing_part_name); @@ -508,8 +593,7 @@ void ReplicatedMergeTreeSink::commitPart( } catch (const Exception & e) { - if (e.code() != ErrorCodes::DUPLICATE_DATA_PART - && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) + if (e.code() != ErrorCodes::DUPLICATE_DATA_PART && e.code() != ErrorCodes::PART_IS_TEMPORARILY_LOCKED) throw; } @@ -526,15 +610,26 @@ void ReplicatedMergeTreeSink::commitPart( part->name); } - ThreadFuzzer::maybeInjectSleep(); + try + { + ThreadFuzzer::maybeInjectSleep(); + storage.lockSharedData(*part, zookeeper, false, {}); + ThreadFuzzer::maybeInjectSleep(); + } + catch (const Exception &) + { + transaction.rollbackPartsToTemporaryState(); - storage.lockSharedData(*part, false, {}); + part->is_temp = true; + part->renameTo(temporary_part_relative_path, false); + + throw; + } ThreadFuzzer::maybeInjectSleep(); Coordination::Responses responses; Coordination::Error multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT - if (multi_code == Coordination::Error::ZOK) { transaction.commit(); @@ -549,18 +644,32 @@ void ReplicatedMergeTreeSink::commitPart( throw Exception(ErrorCodes::QUERY_WAS_CANCELLED, "Insert query (for block {}) was cancelled by concurrent ALTER PARTITION", block_number_lock->getPath()); } - else if (multi_code == Coordination::Error::ZCONNECTIONLOSS - || multi_code == Coordination::Error::ZOPERATIONTIMEOUT) + else if (Coordination::isHardwareError(multi_code)) { + write_part_info_keeper_error = multi_code; /** If the connection is lost, and we do not know if the changes were applied, we can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ + * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. + */ transaction.commit(); - storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); + + /// Setting this flag is point of no return + /// On next retry, we'll just check if actually operation succeed or failed + /// and return ok or error correspondingly + part_committed_locally_but_zookeeper = true; + + /// if all retries will be exhausted by accessing zookeeper on fresh retry -> we'll add committed part to queue in the action + /// here lambda capture part name, it's ok since we'll not generate new one for this insert, + /// see comments around 'part_committed_locally_but_zookeeper' flag + retries_ctl.actionAfterLastFailedRetry( + [&storage = storage, part_name = part->name]() + { storage.enqueuePartForCheck(part_name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); }); /// We do not know whether or not data has been inserted. - throw Exception("Unknown status, client must retry. Reason: " + String(Coordination::errorMessage(multi_code)), - ErrorCodes::UNKNOWN_STATUS_OF_INSERT); + retries_ctl.setUserError( + ErrorCodes::UNKNOWN_STATUS_OF_INSERT, + "Unknown status, client must retry. Reason: {}", + Coordination::errorMessage(multi_code)); + return; } else if (Coordination::isUserError(multi_code)) { @@ -580,62 +689,72 @@ void ReplicatedMergeTreeSink::commitPart( part->renameTo(temporary_part_relative_path, false); /// If this part appeared on other replica than it's better to try to write it locally one more time. If it's our part - /// than it will be ignored on the next itration. + /// than it will be ignored on the next iteration. ++loop_counter; if (loop_counter == max_iterations) { part->is_duplicate = true; /// Part is duplicate, just remove it from local FS throw Exception("Too many transaction retries - it may indicate an error", ErrorCodes::DUPLICATE_DATA_PART); } - continue; + retries_ctl.requestUnconditionalRetry(); /// we want one more iteration w/o counting it as a try and timeout + return; } else if (multi_code == Coordination::Error::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); throw Exception("Another quorum insert has been already started", ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); - throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code) + ", path " + failed_op_path, - ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected logical error while adding block {} with ID '{}': {}, path {}", + block_number, + block_id, + Coordination::errorMessage(multi_code), + failed_op_path); } } - else if (Coordination::isHardwareError(multi_code)) - { - storage.unlockSharedData(*part); - transaction.rollback(); - throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); - } else { - storage.unlockSharedData(*part); + storage.unlockSharedData(*part, zookeeper); transaction.rollback(); - throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + Coordination::errorMessage(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + throw Exception( + ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR, + "Unexpected ZooKeeper error while adding block {} with ID '{}': {}", + block_number, + block_id, + Coordination::errorMessage(multi_code)); } - - break; - } + }, + [&zookeeper]() { zookeeper->cleanupEphemeralNodes(); }); if (isQuorumEnabled()) { - if (is_already_existing_part) + ZooKeeperRetriesControl quorum_retries_ctl("waitForQuorum", zookeeper_retries_info); + quorum_retries_ctl.retryLoop([&]() { - /// We get duplicate part without fetch - /// Check if this quorum insert is parallel or not - if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) - storage.updateQuorum(part->name, true); - else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) - storage.updateQuorum(part->name, false); - } + zookeeper->setKeeper(storage.getZooKeeper()); - waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); + if (is_already_existing_part) + { + /// We get duplicate part without fetch + /// Check if this quorum insert is parallel or not + if (zookeeper->exists(storage.zookeeper_path + "/quorum/parallel/" + part->name)) + storage.updateQuorum(part->name, true); + else if (zookeeper->exists(storage.zookeeper_path + "/quorum/status")) + storage.updateQuorum(part->name, false); + } + + if (!quorum_retries_ctl.callAndCatchAll( + [&]() + { waitForQuorum(zookeeper, part->name, quorum_info.status_path, quorum_info.is_active_node_version, replicas_num); })) + return; + }); } } @@ -650,11 +769,11 @@ void ReplicatedMergeTreeSink::onFinish() { auto zookeeper = storage.getZooKeeper(); assertSessionIsNotExpired(zookeeper); - finishDelayedChunk(zookeeper); + finishDelayedChunk(std::make_shared(zookeeper)); } void ReplicatedMergeTreeSink::waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & part_name, const std::string & quorum_path, Int32 is_active_node_version, diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h index e3ec5d14a52..2154c7693f2 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.h @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace Poco { class Logger; } @@ -60,6 +62,7 @@ public: } private: + ZooKeeperRetriesInfo zookeeper_retries_info; struct QuorumInfo { String status_path; @@ -71,20 +74,24 @@ private: /// Checks active replicas. /// Returns total number of replicas. - size_t checkQuorumPrecondition(zkutil::ZooKeeperPtr & zookeeper); + size_t checkQuorumPrecondition(const ZooKeeperWithFaultInjectionPtr & zookeeper); /// Rename temporary part and commit to ZooKeeper. void commitPart( - zkutil::ZooKeeperPtr & zookeeper, + const ZooKeeperWithFaultInjectionPtr & zookeeper, MergeTreeData::MutableDataPartPtr & part, const String & block_id, - size_t replicas_num); + size_t replicas_num, + bool writing_existing_part); /// Wait for quorum to be satisfied on path (quorum_path) form part (part_name) /// Also checks that replica still alive. void waitForQuorum( - zkutil::ZooKeeperPtr & zookeeper, const std::string & part_name, - const std::string & quorum_path, int is_active_node_version, size_t replicas_num) const; + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const std::string & part_name, + const std::string & quorum_path, + int is_active_node_version, + size_t replicas_num) const; StorageReplicatedMergeTree & storage; StorageMetadataPtr metadata_snapshot; @@ -116,7 +123,7 @@ private: struct DelayedChunk; std::unique_ptr delayed_chunk; - void finishDelayedChunk(zkutil::ZooKeeperPtr & zookeeper); + void finishDelayedChunk(const ZooKeeperWithFaultInjectionPtr & zookeeper); }; } diff --git a/src/Storages/MergeTree/ZooKeeperRetries.h b/src/Storages/MergeTree/ZooKeeperRetries.h new file mode 100644 index 00000000000..22ace074245 --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperRetries.h @@ -0,0 +1,265 @@ +#pragma once +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int OK; +} + +struct ZooKeeperRetriesInfo +{ + ZooKeeperRetriesInfo() = default; + ZooKeeperRetriesInfo(std::string name_, Poco::Logger * logger_, UInt64 max_retries_, UInt64 initial_backoff_ms_, UInt64 max_backoff_ms_) + : name(std::move(name_)) + , logger(logger_) + , max_retries(max_retries_) + , curr_backoff_ms(std::min(initial_backoff_ms_, max_backoff_ms_)) + , max_backoff_ms(max_backoff_ms_) + { + } + + std::string name; + Poco::Logger * logger = nullptr; + UInt64 max_retries = 0; + UInt64 curr_backoff_ms = 0; + UInt64 max_backoff_ms = 0; + UInt64 retry_count = 0; +}; + +class ZooKeeperRetriesControl +{ +public: + ZooKeeperRetriesControl(std::string name_, ZooKeeperRetriesInfo & retries_info_) : name(std::move(name_)), retries_info(retries_info_) + { + } + + void retryLoop(auto && f) + { + retryLoop(f, []() {}); + } + + void retryLoop(auto && f, auto && iteration_cleanup) + { + while (canTry()) + { + try + { + f(); + iteration_cleanup(); + } + catch (const zkutil::KeeperException & e) + { + iteration_cleanup(); + + if (!Coordination::isHardwareError(e.code)) + throw; + + setKeeperError(e.code, e.message()); + } + catch (...) + { + iteration_cleanup(); + throw; + } + } + } + + bool callAndCatchAll(auto && f) + { + try + { + f(); + return true; + } + catch (const zkutil::KeeperException & e) + { + setKeeperError(e.code, e.message()); + } + catch (const Exception & e) + { + setUserError(e.code(), e.what()); + } + return false; + } + + void setUserError(int code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setUserError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + user_error.code = code; + user_error.message = std::move(message); + keeper_error = KeeperError{}; + } + + template + void setUserError(int code, fmt::format_string fmt, Args &&... args) + { + setUserError(code, fmt::format(fmt, std::forward(args)...)); + } + + void setKeeperError(Coordination::Error code, std::string message) + { + if (retries_info.logger) + LOG_TRACE( + retries_info.logger, "ZooKeeperRetriesControl: {}/{}: setKeeperError: error={} message={}", retries_info.name, name, code, message); + + /// if current iteration is already failed, keep initial error + if (!iteration_succeeded) + return; + + iteration_succeeded = false; + keeper_error.code = code; + keeper_error.message = std::move(message); + user_error = UserError{}; + } + + void stopRetries() { stop_retries = true; } + + void requestUnconditionalRetry() { unconditional_retry = true; } + + bool isLastRetry() const { return retries_info.retry_count >= retries_info.max_retries; } + + bool isRetry() const { return retries_info.retry_count > 0; } + + Coordination::Error getLastKeeperErrorCode() const { return keeper_error.code; } + + /// action will be called only once and only after latest failed retry + void actionAfterLastFailedRetry(std::function f) { action_after_last_failed_retry = std::move(f); } + +private: + struct KeeperError + { + using Code = Coordination::Error; + Code code = Code::ZOK; + std::string message; + }; + + struct UserError + { + int code = ErrorCodes::OK; + std::string message; + }; + + bool canTry() + { + ++iteration_count; + /// first iteration is ordinary execution, no further checks needed + if (0 == iteration_count) + return true; + + if (unconditional_retry) + { + unconditional_retry = false; + return true; + } + + /// iteration succeeded -> no need to retry + if (iteration_succeeded) + { + /// avoid unnecessary logs, - print something only in case of retries + if (retries_info.logger && iteration_count > 1) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: succeeded after: iterations={} total_retries={}", + retries_info.name, + name, + iteration_count, + retries_info.retry_count); + return false; + } + + if (stop_retries) + { + logLastError("stop retries on request"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + if (retries_info.retry_count >= retries_info.max_retries) + { + logLastError("retry limit is reached"); + action_after_last_failed_retry(); + throwIfError(); + return false; + } + + /// retries + ++retries_info.retry_count; + logLastError("will retry due to error"); + sleepForMilliseconds(retries_info.curr_backoff_ms); + retries_info.curr_backoff_ms = std::min(retries_info.curr_backoff_ms * 2, retries_info.max_backoff_ms); + + /// reset the flag, it will be set to false in case of error + iteration_succeeded = true; + + return true; + } + + void throwIfError() const + { + if (user_error.code != ErrorCodes::OK) + throw Exception(user_error.code, user_error.message); + + if (keeper_error.code != KeeperError::Code::ZOK) + throw zkutil::KeeperException(keeper_error.code, keeper_error.message); + } + + void logLastError(std::string_view header) + { + if (user_error.code == ErrorCodes::OK) + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + keeper_error.code, + keeper_error.message); + } + else + { + if (retries_info.logger) + LOG_DEBUG( + retries_info.logger, + "ZooKeeperRetriesControl: {}/{}: {}: retry_count={} timeout={}ms error={} message={}", + retries_info.name, + name, + header, + retries_info.retry_count, + retries_info.curr_backoff_ms, + user_error.code, + user_error.message); + } + } + + + std::string name; + ZooKeeperRetriesInfo & retries_info; + Int64 iteration_count = -1; + UserError user_error; + KeeperError keeper_error; + std::function action_after_last_failed_retry = []() {}; + bool unconditional_retry = false; + bool iteration_succeeded = true; + bool stop_retries = false; +}; + +} diff --git a/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h new file mode 100644 index 00000000000..491f97b52bd --- /dev/null +++ b/src/Storages/MergeTree/ZooKeeperWithFaultInjection.h @@ -0,0 +1,527 @@ +#pragma once +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +class RandomFaultInjection +{ +public: + RandomFaultInjection(double probability, UInt64 seed_) : rndgen(seed_), distribution(probability) { } + + void beforeOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection before operation", Coordination::Error::ZSESSIONEXPIRED); + } + void afterOperation() + { + if (distribution(rndgen)) + throw zkutil::KeeperException("Fault injection after operation", Coordination::Error::ZOPERATIONTIMEOUT); + } + +private: + std::mt19937_64 rndgen; + std::bernoulli_distribution distribution; +}; + +/// +/// ZooKeeperWithFaultInjection mimics ZooKeeper interface and inject failures according to failure policy if set +/// +class ZooKeeperWithFaultInjection +{ + using zk = zkutil::ZooKeeper; + + zk::Ptr keeper; + zk::Ptr keeper_prev; + std::unique_ptr fault_policy; + std::string name; + Poco::Logger * logger = nullptr; + UInt64 calls_total = 0; + UInt64 calls_without_fault_injection = 0; + const UInt64 seed = 0; + + std::vector ephemeral_nodes; + + ZooKeeperWithFaultInjection( + zk::Ptr const & keeper_, + double fault_injection_probability, + UInt64 fault_injection_seed, + std::string name_, + Poco::Logger * logger_) + : keeper(keeper_), name(std::move(name_)), logger(logger_), seed(fault_injection_seed) + { + fault_policy = std::make_unique(fault_injection_probability, fault_injection_seed); + + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection created: name={} seed={} fault_probability={}", + name, + seed, + fault_injection_probability); + } + +public: + using Ptr = std::shared_ptr; + + static ZooKeeperWithFaultInjection::Ptr createInstance( + double fault_injection_probability, UInt64 fault_injection_seed, const zk::Ptr & zookeeper, std::string name, Poco::Logger * logger) + { + /// validate all parameters here, constructor just accept everything + + if (fault_injection_probability < 0.0) + fault_injection_probability = .0; + else if (fault_injection_probability > 1.0) + fault_injection_probability = 1.0; + + if (0 == fault_injection_seed) + fault_injection_seed = randomSeed(); + + if (fault_injection_probability > 0.0) + return std::shared_ptr( + new ZooKeeperWithFaultInjection(zookeeper, fault_injection_probability, fault_injection_seed, std::move(name), logger)); + + /// if no fault injection provided, create instance which will not log anything + return std::make_shared(zookeeper); + } + + explicit ZooKeeperWithFaultInjection(zk::Ptr const & keeper_) : keeper(keeper_) { } + + ~ZooKeeperWithFaultInjection() + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection report: name={} seed={} calls_total={} calls_succeeded={} calls_failed={} failure_rate={}", + name, + seed, + calls_total, + calls_without_fault_injection, + calls_total - calls_without_fault_injection, + float(calls_total - calls_without_fault_injection) / calls_total); + } + + void setKeeper(zk::Ptr const & keeper_) { keeper = keeper_; } + bool isNull() const { return keeper.get() == nullptr; } + + /// + /// mirror ZooKeeper interface + /// + + Strings getChildren( + const std::string & path, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access("getChildren", path, [&]() { return keeper->getChildren(path, stat, watch, list_request_type); }); + } + + Coordination::Error tryGetChildren( + const std::string & path, + Strings & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::ListRequestType list_request_type = Coordination::ListRequestType::ALL) + { + return access("tryGetChildren", path, [&]() { return keeper->tryGetChildren(path, res, stat, watch, list_request_type); }); + } + + zk::FutureExists asyncExists(const std::string & path, Coordination::WatchCallback watch_callback = {}) + { + return access("asyncExists", path, [&]() { return keeper->asyncExists(path, watch_callback); }); + } + + zk::FutureGet asyncTryGet(const std::string & path) + { + return access("asyncTryGet", path, [&]() { return keeper->asyncTryGet(path); }); + } + + bool tryGet( + const std::string & path, + std::string & res, + Coordination::Stat * stat = nullptr, + const zkutil::EventPtr & watch = nullptr, + Coordination::Error * code = nullptr) + { + return access("tryGet", path, [&]() { return keeper->tryGet(path, res, stat, watch, code); }); + } + + Coordination::Error tryMulti(const Coordination::Requests & requests, Coordination::Responses & responses) + { + constexpr auto method = "tryMulti"; + auto error = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMulti(requests, responses); }, + [&](const Coordination::Error & original_error) + { + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); + }); + + + /// collect ephemeral nodes when no fault was injected (to clean up on demand) + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachCreatedEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; + } + + Coordination::Error tryMultiNoThrow(const Coordination::Requests & requests, Coordination::Responses & responses) + { + constexpr auto method = "tryMultiNoThrow"; + constexpr auto no_throw = true; + constexpr auto inject_failure_before_op = false; + auto error = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->tryMultiNoThrow(requests, responses); }, + [&](const Coordination::Error & original_error) + { + if (original_error == Coordination::Error::ZOK) + faultInjectionPostAction(method, requests, responses); + }, + [&]() + { + responses.clear(); + for (size_t i = 0; i < requests.size(); ++i) + responses.emplace_back(std::make_shared()); + }); + + /// collect ephemeral nodes when no fault was injected (to clean up later) + if (unlikely(fault_policy) && Coordination::Error::ZOK == error) + { + doForEachCreatedEphemeralNode( + method, requests, responses, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return error; + } + + std::string get(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access("get", path, [&]() { return keeper->get(path, stat, watch); }); + } + + zkutil::ZooKeeper::MultiGetResponse get(const std::vector & paths) + { + return access("get", !paths.empty() ? paths.front() : "", [&]() { return keeper->get(paths); }); + } + + bool exists(const std::string & path, Coordination::Stat * stat = nullptr, const zkutil::EventPtr & watch = nullptr) + { + return access("exists", path, [&]() { return keeper->exists(path, stat, watch); }); + } + + zkutil::ZooKeeper::MultiExistsResponse exists(const std::vector & paths) + { + return access("exists", !paths.empty() ? paths.front() : "", [&]() { return keeper->exists(paths); }); + } + + std::string create(const std::string & path, const std::string & data, int32_t mode) + { + auto path_created = access( + "create", + path, + [&]() { return keeper->create(path, data, mode); }, + [&](std::string const & result_path) + { + try + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + { + keeper->remove(result_path); + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection cleanup: seed={} func={} path={}", seed, "create", result_path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection cleanup FAILED: seed={} func={} path={} code={} message={} ", + seed, + "create", + result_path, + e.code, + e.message()); + } + }); + + /// collect ephemeral nodes when no fault was injected (to clean up later) + if (unlikely(fault_policy)) + { + if (mode == zkutil::CreateMode::EphemeralSequential || mode == zkutil::CreateMode::Ephemeral) + ephemeral_nodes.push_back(path_created); + } + + return path_created; + } + + Coordination::Responses multi(const Coordination::Requests & requests) + { + constexpr auto method = "multi"; + auto result = access( + method, + !requests.empty() ? requests.front()->getPath() : "", + [&]() { return keeper->multi(requests); }, + [&](Coordination::Responses & responses) { faultInjectionPostAction(method, requests, responses); }); + + /// collect ephemeral nodes to clean up + if (unlikely(fault_policy)) + { + doForEachCreatedEphemeralNode( + method, requests, result, [&](const String & path_created) { ephemeral_nodes.push_back(path_created); }); + } + return result; + } + + void createAncestors(const std::string & path) + { + access("createAncestors", path, [&]() { return keeper->createAncestors(path); }); + } + + Coordination::Error tryRemove(const std::string & path, int32_t version = -1) + { + return access("tryRemove", path, [&]() { return keeper->tryRemove(path, version); }); + } + + void cleanupEphemeralNodes() + { + for (const auto & path : ephemeral_nodes) + { + try + { + if (keeper_prev) + keeper_prev->tryRemove(path); + } + catch (...) + { + if (unlikely(logger)) + tryLogCurrentException(logger, "Exception during ephemeral nodes clean up"); + } + } + + ephemeral_nodes.clear(); + } + +private: + void faultInjectionBefore(std::function fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->beforeOperation(); + } + catch (const zkutil::KeeperException &) + { + fault_cleanup(); + throw; + } + } + void faultInjectionAfter(std::function fault_cleanup) + { + try + { + if (unlikely(fault_policy)) + fault_policy->afterOperation(); + } + catch (const zkutil::KeeperException &) + { + fault_cleanup(); + throw; + } + } + + void doForEachCreatedEphemeralNode( + const char * method, const Coordination::Requests & requests, const Coordination::Responses & responses, auto && action) + { + if (responses.empty()) + return; + + if (responses.size() != requests.size()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Number of responses doesn't match number of requests: method={} requests={} responses={}", + method, + requests.size(), + responses.size()); + + /// find create request with ephemeral flag + std::vector> create_requests; + for (size_t i = 0; i < requests.size(); ++i) + { + const auto * create_req = dynamic_cast(requests[i].get()); + if (create_req && create_req->is_ephemeral) + create_requests.emplace_back(i, create_req); + } + + for (auto && [i, req] : create_requests) + { + const auto * create_resp = dynamic_cast(responses.at(i).get()); + if (!create_resp) + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Response should be CreateResponse: method={} index={} path={}", method, i, req->path); + + action(create_resp->path_created); + } + } + + void faultInjectionPostAction(const char * method, const Coordination::Requests & requests, Coordination::Responses & responses) + { + doForEachCreatedEphemeralNode(method, requests, responses, [&](const String & path_created) { keeper->remove(path_created); }); + } + + template + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template <> + struct FaultCleanupTypeImpl + { + using Type = std::function; + }; + + template + using FaultCleanupType = typename FaultCleanupTypeImpl::Type; + + template < + bool no_throw_access = false, + bool inject_failure_before_op = true, + int inject_failure_after_op = true, + typename Operation, + typename Result = std::invoke_result_t> + Result access( + const char * func_name, + const std::string & path, + Operation operation, + FaultCleanupType fault_after_op_cleanup = {}, + FaultCleanupType fault_before_op_cleanup = {}) + { + try + { + ++calls_total; + + if (!keeper) + throw zkutil::KeeperException( + "Session is considered to be expired due to fault injection", Coordination::Error::ZSESSIONEXPIRED); + + if constexpr (inject_failure_before_op) + { + faultInjectionBefore( + [&] + { + if (fault_before_op_cleanup) + fault_before_op_cleanup(); + }); + } + + if constexpr (!std::is_same_v) + { + Result res = operation(); + + /// if connectivity error occurred w/o fault injection -> just return it + if constexpr (std::is_same_v) + { + if (Coordination::isHardwareError(res)) + return res; + } + + if constexpr (inject_failure_after_op) + { + faultInjectionAfter( + [&] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(res); + }); + } + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + + return res; + } + else + { + operation(); + + if constexpr (inject_failure_after_op) + { + faultInjectionAfter( + [&fault_after_op_cleanup] + { + if (fault_after_op_cleanup) + fault_after_op_cleanup(); + }); + } + + ++calls_without_fault_injection; + + if (unlikely(logger)) + LOG_TRACE(logger, "ZooKeeperWithFaultInjection call SUCCEEDED: seed={} func={} path={}", seed, func_name, path); + } + } + catch (const zkutil::KeeperException & e) + { + if (unlikely(logger)) + LOG_TRACE( + logger, + "ZooKeeperWithFaultInjection call FAILED: seed={} func={} path={} code={} message={} ", + seed, + func_name, + path, + e.code, + e.message()); + + /// save valid pointer to clean up ephemeral nodes later if necessary + if (keeper) + keeper_prev = keeper; + keeper.reset(); + + /// for try*NoThrow() methods + if constexpr (no_throw_access) + return e.code; + + if constexpr (std::is_same_v) + { + /// try*() methods throws at least on hardware error and return only on user errors + /// todo: the methods return only on subset of user errors, and throw on another errors + /// to mimic the methods exactly - we need to specify errors on which to return for each such method + if (Coordination::isHardwareError(e.code)) + throw; + + return e.code; + } + + throw; + } + } +}; + +using ZooKeeperWithFaultInjectionPtr = ZooKeeperWithFaultInjection::Ptr; +} diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 3b85581a157..dde844c0ee0 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4479,9 +4479,16 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { - const auto storage_settings_ptr = getSettings(); - assertNotReadonly(); + /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it + /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk + /// In case of remote storage like s3, it'll generate unnecessary PUT requests + if (is_readonly && (!has_metadata_in_zookeeper.has_value() || false == has_metadata_in_zookeeper.value())) + throw Exception( + ErrorCodes::TABLE_IS_READ_ONLY, + "Table is in readonly mode since table metadata was not found in zookeeper: replica_path={}", + replica_path); + const auto storage_settings_ptr = getSettings(); const Settings & query_settings = local_context->getSettingsRef(); bool deduplicate = storage_settings_ptr->replicated_deduplication_window != 0 && query_settings.insert_deduplicate; @@ -4996,8 +5003,7 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St Int64 mutation_version; { - auto zookeeper = getZooKeeper(); - delimiting_block_lock = allocateBlockNumber(partition_id, zookeeper); + delimiting_block_lock = allocateBlockNumber(partition_id, getZooKeeper()); right = delimiting_block_lock->getNumber(); /// Make sure we cover all parts in drop range. /// There might be parts with mutation version greater than current block number @@ -5278,7 +5284,7 @@ void StorageReplicatedMergeTree::rename(const String & new_path_to_table_data, c } -bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) const +bool StorageReplicatedMergeTree::existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const { { std::lock_guard lock(existing_nodes_cache_mutex); @@ -5286,7 +5292,7 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons return true; } - bool res = getZooKeeper()->exists(path); + bool res = zookeeper->exists(path); if (res) { @@ -5298,9 +5304,22 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) cons } -std::optional -StorageReplicatedMergeTree::allocateBlockNumber( - const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path, const String & zookeeper_path_prefix) const +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const zkutil::ZooKeeperPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const +{ + return allocateBlockNumber( + partition_id, std::make_shared(zookeeper), zookeeper_block_id_path, zookeeper_path_prefix); +} + + +std::optional StorageReplicatedMergeTree::allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path, + const String & zookeeper_path_prefix) const { String zookeeper_table_path; if (zookeeper_path_prefix.empty()) @@ -5311,7 +5330,7 @@ StorageReplicatedMergeTree::allocateBlockNumber( String block_numbers_path = fs::path(zookeeper_table_path) / "block_numbers"; String partition_path = fs::path(block_numbers_path) / partition_id; - if (!existsNodeCached(partition_path)) + if (!existsNodeCached(zookeeper, partition_path)) { Coordination::Requests ops; /// Check that table is not being dropped ("host" is the first node that is removed on replica drop) @@ -5329,10 +5348,9 @@ StorageReplicatedMergeTree::allocateBlockNumber( } return createEphemeralLockInZooKeeper( - fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", *zookeeper, zookeeper_block_id_path); + fs::path(partition_path) / "block-", fs::path(zookeeper_table_path) / "temp", zookeeper, zookeeper_block_id_path); } - Strings StorageReplicatedMergeTree::tryWaitForAllReplicasToProcessLogEntry( const String & table_zookeeper_path, const ReplicatedMergeTreeLogEntryData & entry, Int64 wait_for_inactive_timeout) { @@ -7085,7 +7103,7 @@ CancellationCode StorageReplicatedMergeTree::killPartMoveToShard(const UUID & ta void StorageReplicatedMergeTree::getCommitPartOps( Coordination::Requests & ops, - MutableDataPartPtr & part, + const DataPartPtr & part, const String & block_id_path) const { const String & part_name = part->name; @@ -7667,11 +7685,28 @@ void StorageReplicatedMergeTree::lockSharedDataTemporary(const String & part_nam String zookeeper_node = fs::path(zc_zookeeper_path) / id / replica_name; LOG_TRACE(log, "Set zookeeper temporary ephemeral lock {}", zookeeper_node); - createZeroCopyLockNode(zookeeper, zookeeper_node, zkutil::CreateMode::Ephemeral, false); + createZeroCopyLockNode( + std::make_shared(zookeeper), zookeeper_node, zkutil::CreateMode::Ephemeral, false); } } -void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + bool replace_existing_lock, + std::optional hardlinked_files) const +{ + auto zookeeper = tryGetZooKeeper(); + if (zookeeper) + return lockSharedData(part, std::make_shared(zookeeper), replace_existing_lock, hardlinked_files); + else + return lockSharedData(part, std::make_shared(nullptr), replace_existing_lock, hardlinked_files); +} + +void StorageReplicatedMergeTree::lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const { auto settings = getSettings(); @@ -7681,8 +7716,7 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, if (!part.getDataPartStorage().supportZeroCopyReplication()) return; - zkutil::ZooKeeperPtr zookeeper = tryGetZooKeeper(); - if (!zookeeper) + if (zookeeper->isNull()) return; String id = part.getUniqueId(); @@ -7716,7 +7750,14 @@ void StorageReplicatedMergeTree::lockSharedData(const IMergeTreeDataPart & part, } } -std::pair StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part) const +{ + return unlockSharedData(part, std::make_shared(nullptr)); +} + +std::pair +StorageReplicatedMergeTree::unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const { auto settings = getSettings(); if (!settings->allow_remote_fs_zero_copy_replication) @@ -7762,11 +7803,10 @@ std::pair StorageReplicatedMergeTree::unlockSharedData(const IMer /// We remove parts during table shutdown. If exception happen, restarting thread will be already turned /// off and nobody will reconnect our zookeeper connection. In this case we use zookeeper connection from /// context. - zkutil::ZooKeeperPtr zookeeper; if (shutdown_called.load()) - zookeeper = getZooKeeperIfTableShutDown(); + zookeeper->setKeeper(getZooKeeperIfTableShutDown()); else - zookeeper = getZooKeeper(); + zookeeper->setKeeper(getZooKeeper()); /// It can happen that we didn't had the connection to zookeeper during table creation, but actually /// table is completely dropped, so we can drop it without any additional checks. @@ -7791,7 +7831,7 @@ namespace /// But sometimes we need an opposite. When we deleting all_0_0_0_1 it can be non replicated to other replicas, so we are the only owner of this part. /// In this case when we will drop all_0_0_0_1 we will drop blobs for all_0_0_0. But it will lead to dataloss. For such case we need to check that other replicas /// still need parent part. -std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) +std::pair getParentLockedBlobs(const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const std::string & zero_copy_part_path_prefix, const std::string & part_info_str, MergeTreeDataFormatVersion format_version, Poco::Logger * log) { NameSet files_not_to_remove; @@ -7853,7 +7893,7 @@ std::pair getParentLockedBlobs(zkutil::ZooKeeperPtr zookeeper_ptr std::pair StorageReplicatedMergeTree::unlockSharedDataByID( String part_id, const String & table_uuid, const String & part_name, - const String & replica_name_, const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_ptr, const MergeTreeSettings & settings, + const String & replica_name_, const std::string & disk_type, const ZooKeeperWithFaultInjectionPtr & zookeeper_ptr, const MergeTreeSettings & settings, Poco::Logger * logger, const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version) { boost::replace_all(part_id, "/", "_"); @@ -7872,7 +7912,8 @@ std::pair StorageReplicatedMergeTree::unlockSharedDataByID( if (!files_not_to_remove_str.empty()) boost::split(files_not_to_remove, files_not_to_remove_str, boost::is_any_of("\n ")); - auto [has_parent, parent_not_to_remove] = getParentLockedBlobs(zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); + auto [has_parent, parent_not_to_remove] = getParentLockedBlobs( + zookeeper_ptr, fs::path(zc_zookeeper_path).parent_path(), part_name, data_format_version, logger); files_not_to_remove.insert(parent_not_to_remove.begin(), parent_not_to_remove.end()); String zookeeper_part_uniq_node = fs::path(zc_zookeeper_path) / part_id; @@ -8387,7 +8428,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP void StorageReplicatedMergeTree::createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, int32_t mode, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode, bool replace_existing_lock, const String & path_to_set_hardlinked_files, const NameSet & hardlinked_files) { /// In rare case other replica can remove path between createAncestors and createIfNotExists @@ -8504,7 +8545,7 @@ bool StorageReplicatedMergeTree::removeSharedDetachedPart(DiskPtr disk, const St id, table_uuid, part_name, detached_replica_name, toString(disk->getDataSourceDescription().type), - zookeeper, local_context->getReplicatedMergeTreeSettings(), + std::make_shared(zookeeper), local_context->getReplicatedMergeTreeSettings(), &Poco::Logger::get("StorageReplicatedMergeTree"), detached_zookeeper_path, MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING); diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index fe2105ab4ec..d767d94889c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -83,6 +83,9 @@ namespace DB * as the time will take the time of creation the appropriate part on any of the replicas. */ +class ZooKeeperWithFaultInjection; +using ZooKeeperWithFaultInjectionPtr = std::shared_ptr; + class StorageReplicatedMergeTree final : public MergeTreeData { public: @@ -267,6 +270,11 @@ public: /// Lock part in zookeeper for use shared data in several nodes void lockSharedData(const IMergeTreeDataPart & part, bool replace_existing_lock, std::optional hardlinked_files) const override; + void lockSharedData( + const IMergeTreeDataPart & part, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + bool replace_existing_lock, + std::optional hardlinked_files) const; void lockSharedDataTemporary(const String & part_name, const String & part_id, const DiskPtr & disk) const; @@ -274,13 +282,23 @@ public: /// Return true if data unlocked /// Return false if data is still used by another node std::pair unlockSharedData(const IMergeTreeDataPart & part) const override; + std::pair + unlockSharedData(const IMergeTreeDataPart & part, const ZooKeeperWithFaultInjectionPtr & zookeeper) const; /// Unlock shared data part in zookeeper by part id /// Return true if data unlocked /// Return false if data is still used by another node - static std::pair unlockSharedDataByID(String part_id, const String & table_uuid, const String & part_name, const String & replica_name_, - const std::string & disk_type, zkutil::ZooKeeperPtr zookeeper_, const MergeTreeSettings & settings, Poco::Logger * logger, - const String & zookeeper_path_old, MergeTreeDataFormatVersion data_format_version); + static std::pair unlockSharedDataByID( + String part_id, + const String & table_uuid, + const String & part_name, + const String & replica_name_, + const std::string & disk_type, + const ZooKeeperWithFaultInjectionPtr & zookeeper_, + const MergeTreeSettings & settings, + Poco::Logger * logger, + const String & zookeeper_path_old, + MergeTreeDataFormatVersion data_format_version); /// Fetch part only if some replica has it on shared storage like S3 MutableDataPartStoragePtr tryToFetchIfShared(const IMergeTreeDataPart & part, const DiskPtr & disk, const String & path) override; @@ -534,7 +552,7 @@ private: bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const override; - void getCommitPartOps(Coordination::Requests & ops, MutableDataPartPtr & part, const String & block_id_path = "") const; + void getCommitPartOps(Coordination::Requests & ops, const DataPartPtr & part, const String & block_id_path = "") const; /// Adds actions to `ops` that remove a part from ZooKeeper. /// Set has_children to true for "old-style" parts (those with /columns and /checksums child znodes). @@ -711,6 +729,11 @@ private: std::optional allocateBlockNumber( const String & partition_id, const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_block_id_path = "", const String & zookeeper_path_prefix = "") const; + std::optional allocateBlockNumber( + const String & partition_id, + const ZooKeeperWithFaultInjectionPtr & zookeeper, + const String & zookeeper_block_id_path = "", + const String & zookeeper_path_prefix = "") const; /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica. @@ -748,7 +771,7 @@ private: /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. mutable std::unordered_set existing_nodes_cache; mutable std::mutex existing_nodes_cache_mutex; - bool existsNodeCached(const std::string & path) const; + bool existsNodeCached(const ZooKeeperWithFaultInjectionPtr & zookeeper, const std::string & path) const; /// Cancels INSERTs in the block range by removing ephemeral block numbers void clearLockedBlockNumbersInPartition(zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num); @@ -836,7 +859,7 @@ private: const String & part_name, const String & zookeeper_path_old); static void createZeroCopyLockNode( - const zkutil::ZooKeeperPtr & zookeeper, const String & zookeeper_node, + const ZooKeeperWithFaultInjectionPtr & zookeeper, const String & zookeeper_node, int32_t mode = zkutil::CreateMode::Persistent, bool replace_existing_lock = false, const String & path_to_set_hardlinked_files = "", const NameSet & hardlinked_files = {}); diff --git a/tests/config/install.sh b/tests/config/install.sh index 876d05507fe..51865665f59 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -71,6 +71,7 @@ ln -sf $SRC_PATH/users.d/no_fsync_metadata.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/filelog.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/enable_blobs_check.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/users.d/marks.xml $DEST_SERVER_PATH/users.d/ +ln -sf $SRC_PATH/users.d/insert_keeper_retries.xml $DEST_SERVER_PATH/users.d/ # FIXME DataPartsExchange may hang for http_send_timeout seconds # when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"), diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml new file mode 100644 index 00000000000..f7b652bf2ef --- /dev/null +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -0,0 +1,8 @@ + + + + 0 + 0.0 + + + diff --git a/tests/integration/test_inserts_with_keeper_retries/__init__.py b/tests/integration/test_inserts_with_keeper_retries/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_inserts_with_keeper_retries/test.py b/tests/integration/test_inserts_with_keeper_retries/test.py new file mode 100644 index 00000000000..dbf18365053 --- /dev/null +++ b/tests/integration/test_inserts_with_keeper_retries/test.py @@ -0,0 +1,100 @@ +#!/usr/bin/env python3 + +import pytest +import time +import threading +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from helpers.network import PartitionManager +from helpers.client import QueryRuntimeException +from helpers.test_tools import assert_eq_with_retry + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance("node1", with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + global cluster + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_replica_inserts_with_keeper_restart(started_cluster): + try: + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + + p = Pool(1) + zk_stopped_event = threading.Event() + + def zoo_restart(zk_stopped_event): + cluster.stop_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + zk_stopped_event.set() + cluster.start_zookeeper_nodes(["zoo1", "zoo2", "zoo3"]) + + job = p.apply_async(zoo_restart, (zk_stopped_event,)) + + zk_stopped_event.wait(90) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" + ) + + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "20\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") + + +def test_replica_inserts_with_keeper_disconnect(started_cluster): + try: + node1.query( + "CREATE TABLE r (a UInt64, b String) ENGINE=ReplicatedMergeTree('/test/r', '0') ORDER BY tuple()" + ) + + p = Pool(1) + disconnect_event = threading.Event() + + def keeper_disconnect(node, event): + with PartitionManager() as pm: + pm.drop_instance_zk_connections(node) + event.set() + + job = p.apply_async( + keeper_disconnect, + ( + node1, + disconnect_event, + ), + ) + disconnect_event.wait(90) + + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10) SETTINGS insert_keeper_max_retries=20" + ) + node1.query( + "INSERT INTO r SELECT number, toString(number) FROM numbers(10, 10) SETTINGS insert_keeper_max_retries=20" + ) + + job.wait() + p.close() + p.join() + + assert node1.query("SELECT COUNT() FROM r") == "20\n" + + finally: + node1.query("DROP TABLE IF EXISTS r SYNC") diff --git a/tests/integration/test_restore_replica/test.py b/tests/integration/test_restore_replica/test.py index 0b11cdf7512..31c503f6184 100644 --- a/tests/integration/test_restore_replica/test.py +++ b/tests/integration/test_restore_replica/test.py @@ -7,6 +7,9 @@ from helpers.test_tools import assert_eq_with_retry def fill_nodes(nodes): + for node in nodes: + node.query("DROP TABLE IF EXISTS test SYNC") + for node in nodes: node.query( """ @@ -29,11 +32,7 @@ nodes = [node_1, node_2, node_3] def fill_table(): - node_1.query("TRUNCATE TABLE test") - - for node in nodes: - node.query("SYSTEM SYNC REPLICA test") - + fill_nodes(nodes) check_data(0, 0) # it will create multiple parts in each partition and probably cause merges diff --git a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql index f62f11c60fd..ed1f654f847 100644 --- a/tests/queries/0_stateless/00121_drop_column_zookeeper.sql +++ b/tests/queries/0_stateless/00121_drop_column_zookeeper.sql @@ -1,14 +1,14 @@ -- Tags: zookeeper, no-replicated-database -- Tag no-replicated-database: Old syntax is not allowed -DROP TABLE IF EXISTS alter_00121; +DROP TABLE IF EXISTS alter_00121 SYNC; set allow_deprecated_syntax_for_merge_tree=1; CREATE TABLE alter_00121 (d Date, x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t1', 'r1', d, (d), 8192); INSERT INTO alter_00121 VALUES ('2014-01-01', 1); ALTER TABLE alter_00121 DROP COLUMN x; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; CREATE TABLE alter_00121 (d Date) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/alter_00121/t2', 'r1', d, (d), 8192); @@ -23,4 +23,4 @@ SELECT * FROM alter_00121 ORDER BY d; ALTER TABLE alter_00121 DROP COLUMN x; SELECT * FROM alter_00121 ORDER BY d; -DROP TABLE alter_00121; +DROP TABLE alter_00121 SYNC; diff --git a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql index 8267a451728..0ee8ba07006 100644 --- a/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper_long.sql @@ -1,11 +1,12 @@ -- Tags: long, replica SET replication_alter_partitions_sync = 2; +SET insert_keeper_fault_injection_probability=0; SELECT '*** Not partitioned ***'; -DROP TABLE IF EXISTS not_partitioned_replica1_00502; -DROP TABLE IF EXISTS not_partitioned_replica2_00502; +DROP TABLE IF EXISTS not_partitioned_replica1_00502 SYNC; +DROP TABLE IF EXISTS not_partitioned_replica2_00502 SYNC; CREATE TABLE not_partitioned_replica1_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '1') ORDER BY x; CREATE TABLE not_partitioned_replica2_00502(x UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/not_partitioned_00502', '2') ORDER BY x; @@ -14,7 +15,7 @@ INSERT INTO not_partitioned_replica1_00502 VALUES (4), (5); SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA not_partitioned_replica1_00502; +SYSTEM SYNC REPLICA not_partitioned_replica2_00502; OPTIMIZE TABLE not_partitioned_replica1_00502 PARTITION tuple() FINAL; SELECT 'Parts after OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'not_partitioned_replica2_00502' AND active ORDER BY name; @@ -25,13 +26,13 @@ ALTER TABLE not_partitioned_replica1_00502 DETACH PARTITION ID 'all'; SELECT 'Sum after DETACH PARTITION:'; SELECT sum(x) FROM not_partitioned_replica2_00502; -DROP TABLE not_partitioned_replica1_00502; -DROP TABLE not_partitioned_replica2_00502; +DROP TABLE not_partitioned_replica1_00502 SYNC; +DROP TABLE not_partitioned_replica2_00502 SYNC; SELECT '*** Partitioned by week ***'; -DROP TABLE IF EXISTS partitioned_by_week_replica1; -DROP TABLE IF EXISTS partitioned_by_week_replica2; +DROP TABLE IF EXISTS partitioned_by_week_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_week_replica2 SYNC; CREATE TABLE partitioned_by_week_replica1(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '1') PARTITION BY toMonday(d) ORDER BY x; CREATE TABLE partitioned_by_week_replica2(d Date, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_week_00502', '2') PARTITION BY toMonday(d) ORDER BY x; @@ -41,7 +42,7 @@ INSERT INTO partitioned_by_week_replica1 VALUES ('2000-01-03', 4), ('2000-01-03' SELECT 'Parts before OPTIMIZE:'; -- Select parts on the first replica to avoid waiting for replication. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica1' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_week_replica1; +SYSTEM SYNC REPLICA partitioned_by_week_replica2; OPTIMIZE TABLE partitioned_by_week_replica1 PARTITION '2000-01-03' FINAL; SELECT 'Parts after OPTIMIZE:'; -- After OPTIMIZE with replication_alter_partitions_sync=2 replicas must be in sync. SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_week_replica2' AND active ORDER BY name; @@ -52,13 +53,13 @@ ALTER TABLE partitioned_by_week_replica1 DROP PARTITION '1999-12-27'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_week_replica2; -DROP TABLE partitioned_by_week_replica1; -DROP TABLE partitioned_by_week_replica2; +DROP TABLE partitioned_by_week_replica1 SYNC; +DROP TABLE partitioned_by_week_replica2 SYNC; SELECT '*** Partitioned by a (Date, UInt8) tuple ***'; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00502 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '1') ORDER BY x PARTITION BY (d, x); CREATE TABLE partitioned_by_tuple_replica2_00502(d Date, x UInt8, y UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00502', '2') ORDER BY x PARTITION BY (d, x); @@ -67,7 +68,7 @@ INSERT INTO partitioned_by_tuple_replica1_00502 VALUES ('2000-01-02', 1, 4), ('2 SELECT 'Parts before OPTIMIZE:'; SELECT partition, name FROM system.parts WHERE database = currentDatabase() AND table = 'partitioned_by_tuple_replica1_00502' AND active ORDER BY name; -SYSTEM SYNC REPLICA partitioned_by_tuple_replica1_00502; +SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00502; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-01', 1) FINAL; OPTIMIZE TABLE partitioned_by_tuple_replica1_00502 PARTITION ('2000-01-02', 1) FINAL; SELECT 'Parts after OPTIMIZE:'; @@ -79,13 +80,13 @@ ALTER TABLE partitioned_by_tuple_replica1_00502 DETACH PARTITION ID '20000101-1' SELECT 'Sum after DETACH PARTITION:'; SELECT sum(y) FROM partitioned_by_tuple_replica2_00502; -DROP TABLE partitioned_by_tuple_replica1_00502; -DROP TABLE partitioned_by_tuple_replica2_00502; +DROP TABLE partitioned_by_tuple_replica1_00502 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00502 SYNC; SELECT '*** Partitioned by String ***'; -DROP TABLE IF EXISTS partitioned_by_string_replica1; -DROP TABLE IF EXISTS partitioned_by_string_replica2; +DROP TABLE IF EXISTS partitioned_by_string_replica1 SYNC; +DROP TABLE IF EXISTS partitioned_by_string_replica2 SYNC; CREATE TABLE partitioned_by_string_replica1(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '1') PARTITION BY s ORDER BY x; CREATE TABLE partitioned_by_string_replica2(s String, x UInt8) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/partitioned_by_string_00502', '2') PARTITION BY s ORDER BY x; @@ -105,13 +106,13 @@ ALTER TABLE partitioned_by_string_replica1 DROP PARTITION 'bbb'; SELECT 'Sum after DROP PARTITION:'; SELECT sum(x) FROM partitioned_by_string_replica2; -DROP TABLE partitioned_by_string_replica1; -DROP TABLE partitioned_by_string_replica2; +DROP TABLE partitioned_by_string_replica1 SYNC; +DROP TABLE partitioned_by_string_replica2 SYNC; SELECT '*** Table without columns with fixed size ***'; -DROP TABLE IF EXISTS without_fixed_size_columns_replica1; -DROP TABLE IF EXISTS without_fixed_size_columns_replica2; +DROP TABLE IF EXISTS without_fixed_size_columns_replica1 SYNC; +DROP TABLE IF EXISTS without_fixed_size_columns_replica2 SYNC; CREATE TABLE without_fixed_size_columns_replica1(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '1') PARTITION BY length(s) ORDER BY s; CREATE TABLE without_fixed_size_columns_replica2(s String) ENGINE ReplicatedMergeTree('/clickhouse/tables/{database}/test/without_fixed_size_columns_00502', '2') PARTITION BY length(s) ORDER BY s; @@ -130,5 +131,5 @@ ALTER TABLE without_fixed_size_columns_replica1 DROP PARTITION 1; SELECT 'After DROP PARTITION:'; SELECT * FROM without_fixed_size_columns_replica2 ORDER BY s; -DROP TABLE without_fixed_size_columns_replica1; -DROP TABLE without_fixed_size_columns_replica2; +DROP TABLE without_fixed_size_columns_replica1 SYNC; +DROP TABLE without_fixed_size_columns_replica2 SYNC; diff --git a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh index 7a6c7609660..a1e7d8727c7 100755 --- a/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh +++ b/tests/queries/0_stateless/00652_replicated_mutations_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=./mergetree_mutations.lib . "$CURDIR"/mergetree_mutations.lib -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_r2 SYNC" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r1(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r1', d, intDiv(x, 10), 8192)" ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE TABLE mutations_r2(d Date, x UInt32, s String, m MATERIALIZED x + 2) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations', 'r2', d, intDiv(x, 10), 8192)" @@ -19,9 +19,9 @@ ${CLICKHOUSE_CLIENT} --allow_deprecated_syntax_for_merge_tree=1 --query="CREATE ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 1, 'a')" -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 2, 'b'), ('2000-01-01', 3, 'c'), ('2000-01-01', 4, 'd') \ ('2000-02-01', 2, 'b'), ('2000-02-01', 3, 'c'), ('2000-02-01', 4, 'd')" @@ -35,7 +35,7 @@ ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE s = 'd' SETT ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_r1 DELETE WHERE m = 3 SETTINGS mutations_sync = 2" # Insert more data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_r1(d, x, s) VALUES \ +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_r1(d, x, s) VALUES \ ('2000-01-01', 5, 'e'), ('2000-02-01', 5, 'e')" ${CLICKHOUSE_CLIENT} --query "SYSTEM SYNC REPLICA mutations_r2" @@ -49,8 +49,8 @@ ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, block_numbers.partiti ${CLICKHOUSE_CLIENT} --query="SELECT '*** Test mutations cleaner ***'" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS mutations_cleaner_r2 SYNC" # Create 2 replicas with finished_mutations_to_keep = 2 ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/mutations_cleaner', 'r1') ORDER BY x SETTINGS \ @@ -63,7 +63,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE mutations_cleaner_r2(x UInt32) ENGINE cleanup_delay_period_random_add = 0" # Insert some data -${CLICKHOUSE_CLIENT} --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO mutations_cleaner_r1(x) VALUES (1), (2), (3), (4), (5)" # Add some mutations and wait for their execution ${CLICKHOUSE_CLIENT} --query="ALTER TABLE mutations_cleaner_r1 DELETE WHERE x = 1 SETTINGS mutations_sync = 2" @@ -80,8 +80,8 @@ sleep 1.5 # Check that the first mutation is cleaned ${CLICKHOUSE_CLIENT} --query="SELECT mutation_id, command, is_done FROM system.mutations WHERE database = '$CLICKHOUSE_DATABASE' and table = 'mutations_cleaner_r2' ORDER BY mutation_id" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_r2 SYNC" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1" -${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r1 SYNC" +${CLICKHOUSE_CLIENT} --query="DROP TABLE mutations_cleaner_r2 SYNC" diff --git a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql index 4cdd75f520c..8f694345d93 100644 --- a/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql +++ b/tests/queries/0_stateless/00661_optimize_final_replicated_without_partition_zookeeper.sql @@ -2,8 +2,8 @@ SET optimize_on_insert = 0; -DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661; -DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661; +DROP TABLE IF EXISTS partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE IF EXISTS partitioned_by_tuple_replica2_00661 SYNC; CREATE TABLE partitioned_by_tuple_replica1_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '1') PARTITION BY (d, x) ORDER BY (d, x, w); CREATE TABLE partitioned_by_tuple_replica2_00661(d Date, x UInt8, w String, y UInt8) ENGINE = ReplicatedSummingMergeTree('/clickhouse/tables/{database}/test/partitioned_by_tuple_00661', '2') PARTITION BY (d, x) ORDER BY (d, x, w); @@ -21,5 +21,5 @@ OPTIMIZE TABLE partitioned_by_tuple_replica1_00661 FINAL; SYSTEM SYNC REPLICA partitioned_by_tuple_replica2_00661; SELECT * FROM partitioned_by_tuple_replica2_00661 ORDER BY d, x, w, y; -DROP TABLE partitioned_by_tuple_replica1_00661; -DROP TABLE partitioned_by_tuple_replica2_00661; +DROP TABLE partitioned_by_tuple_replica1_00661 SYNC; +DROP TABLE partitioned_by_tuple_replica2_00661 SYNC; diff --git a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh index 4637b210194..3f384380f9b 100755 --- a/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh +++ b/tests/queries/0_stateless/00715_fetch_merged_or_mutated_part_zookeeper.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE IF EXISTS fetches_r1; - DROP TABLE IF EXISTS fetches_r2" + DROP TABLE IF EXISTS fetches_r1 SYNC; + DROP TABLE IF EXISTS fetches_r2 SYNC" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r1(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r1') ORDER BY x" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/fetches', 'r2') ORDER BY x \ @@ -18,6 +18,7 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE fetches_r2(x UInt32) ENGINE Replicate prefer_fetch_merged_part_size_threshold=0" ${CLICKHOUSE_CLIENT} -n --query=" + SET insert_keeper_fault_injection_probability=0; INSERT INTO fetches_r1 VALUES (1); INSERT INTO fetches_r1 VALUES (2); INSERT INTO fetches_r1 VALUES (3)" @@ -51,5 +52,5 @@ ${CLICKHOUSE_CLIENT} --query="SELECT '*** Check data after fetch/clone of mutate ${CLICKHOUSE_CLIENT} --query="SELECT _part, * FROM fetches_r2 ORDER BY x" ${CLICKHOUSE_CLIENT} -n --query=" - DROP TABLE fetches_r1; - DROP TABLE fetches_r2" + DROP TABLE fetches_r1 SYNC; + DROP TABLE fetches_r2 SYNC" diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh index 6f609065c01..5fc3fa460e6 100755 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sh @@ -12,6 +12,7 @@ $CLICKHOUSE_CLIENT -nm -q " DROP TABLE IF EXISTS part_header_r1; DROP TABLE IF EXISTS part_header_r2; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) diff --git a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql index 703f8824055..b836f806170 100644 --- a/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql +++ b/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -1,9 +1,10 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET check_query_single_value_result = 0; SET send_logs_level = 'fatal'; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeTree() ORDER BY tuple(); @@ -11,9 +12,9 @@ INSERT INTO mt_without_pk VALUES (1, 2); CHECK TABLE mt_without_pk; -DROP TABLE IF EXISTS mt_without_pk; +DROP TABLE IF EXISTS mt_without_pk SYNC; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01037/replicated_mt_without_pk', '1') ORDER BY tuple(); @@ -21,4 +22,4 @@ INSERT INTO replicated_mt_without_pk VALUES (1, 2); CHECK TABLE replicated_mt_without_pk; -DROP TABLE IF EXISTS replicated_mt_without_pk; +DROP TABLE IF EXISTS replicated_mt_without_pk SYNC; diff --git a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh index a438cf7badc..68c511b80ac 100755 --- a/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh +++ b/tests/queries/0_stateless/01045_zookeeper_system_mutations_with_parts_names.sh @@ -52,7 +52,8 @@ ${CLICKHOUSE_CLIENT} --query="CREATE TABLE replicated_table_for_mutations(k UInt ${CLICKHOUSE_CLIENT} --query="SYSTEM STOP MERGES replicated_table_for_mutations" -${CLICKHOUSE_CLIENT} --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" +# test relays on part ids, which are non-deterministic with keeper fault injections, so disable it +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query="INSERT INTO replicated_table_for_mutations select number, number from numbers(100000)" ${CLICKHOUSE_CLIENT} --query="SELECT sum(v1) FROM replicated_table_for_mutations" diff --git a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql index 43ab053655a..e4acfed8a0c 100644 --- a/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql +++ b/tests/queries/0_stateless/01090_zookeeper_mutations_and_insert_quorum_long.sql @@ -1,8 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; CREATE TABLE mutations_and_quorum1 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '1') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test_01090/mutations_and_quorum', '2') PARTITION BY toYYYYMM(server_date) ORDER BY (server_date, something); @@ -10,6 +10,9 @@ CREATE TABLE mutations_and_quorum2 (`server_date` Date, `something` String) ENGI -- Should not be larger then 600e6 (default timeout in clickhouse-test) SET insert_quorum=2, insert_quorum_parallel=0, insert_quorum_timeout=300e3; +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + INSERT INTO mutations_and_quorum1 VALUES ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'), ('2019-04-01', 'test4'), ('2019-05-01', 'test1'), ('2019-06-01', 'test2'), ('2019-07-01', 'test3'), ('2019-08-01', 'test4'), ('2019-09-01', 'test1'), ('2019-10-01', 'test2'), ('2019-11-01', 'test3'), ('2019-12-01', 'test4'); ALTER TABLE mutations_and_quorum1 DELETE WHERE something = 'test1' SETTINGS mutations_sync=2; @@ -19,5 +22,5 @@ SELECT COUNT() FROM mutations_and_quorum2; SELECT COUNT() FROM system.mutations WHERE database = currentDatabase() AND table like 'mutations_and_quorum%' and is_done = 0; -DROP TABLE IF EXISTS mutations_and_quorum1; -DROP TABLE IF EXISTS mutations_and_quorum2; +DROP TABLE IF EXISTS mutations_and_quorum1 SYNC; +DROP TABLE IF EXISTS mutations_and_quorum2 SYNC; diff --git a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql index c370726c72f..2096942630e 100644 --- a/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql +++ b/tests/queries/0_stateless/01135_default_and_alter_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; CREATE TABLE default_table ( @@ -26,4 +26,4 @@ ALTER TABLE default_table MODIFY COLUMN enum_column Enum8('undefined' = 0, 'fox' SHOW CREATE TABLE default_table; -DROP TABLE IF EXISTS default_table; +DROP TABLE IF EXISTS default_table SYNC; diff --git a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql index 687c8051eed..bd11b24d568 100644 --- a/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql +++ b/tests/queries/0_stateless/01149_zookeeper_mutation_stuck_after_replace_partition.sql @@ -1,5 +1,7 @@ -- Tags: zookeeper +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + set send_logs_level='error'; drop table if exists mt; drop table if exists rmt sync; diff --git a/tests/queries/0_stateless/01158_zookeeper_log_long.sql b/tests/queries/0_stateless/01158_zookeeper_log_long.sql index 5f6ca762ae7..cabb0ffe294 100644 --- a/tests/queries/0_stateless/01158_zookeeper_log_long.sql +++ b/tests/queries/0_stateless/01158_zookeeper_log_long.sql @@ -1,13 +1,16 @@ -- Tags: long, zookeeper, no-replicated-database, no-polymorphic-parts -- Tag no-replicated-database: Fails due to additional replicas or shards -drop table if exists rmt; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + +drop table if exists rmt sync; -- cleanup code will perform extra Exists -- (so the .reference will not match) create table rmt (n int) engine=ReplicatedMergeTree('/test/01158/{database}/rmt', '1') order by n settings cleanup_delay_period=86400, replicated_can_become_leader=0; system sync replica rmt; insert into rmt values (1); insert into rmt values (1); +system sync replica rmt; system flush logs; select 'log'; @@ -30,7 +33,7 @@ from system.zookeeper_log where (session_id, xid) in (select session_id, xid from system.zookeeper_log where path like '/test/01158/' || currentDatabase() || '/rmt/blocks/%' and op_num not in (1, 12, 500)) order by xid, type, request_idx; -drop table rmt; +drop table rmt sync; system flush logs; select 'duration_ms'; diff --git a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql index 875bd1f96de..78adbee612f 100644 --- a/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01201_drop_column_compact_part_replicated_zookeeper_long.sql @@ -1,5 +1,9 @@ -- Tags: long, replica +-- in case of keeper fault injection on insert, set bigger number of retries because partitions +set insert_keeper_max_retries=100; +set insert_keeper_retry_max_backoff_ms=10; + -- Testing basic functionality with compact parts set replication_alter_partitions_sync = 2; drop table if exists mt_compact; diff --git a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh index 34fa822b6ea..f7615974237 100755 --- a/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh +++ b/tests/queries/0_stateless/01318_long_unsuccessful_mutation_zookeeper.sh @@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" $CLICKHOUSE_CLIENT --query " CREATE TABLE mutation_table( @@ -17,9 +17,10 @@ $CLICKHOUSE_CLIENT --query " PARTITION BY key % 10 " -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table select number, toString(number) from numbers(100000) where number % 10 != 0" -$CLICKHOUSE_CLIENT --query "INSERT INTO mutation_table VALUES(0, 'hello')" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO mutation_table VALUES(0, 'hello')" $CLICKHOUSE_CLIENT --query "SELECT COUNT() FROM mutation_table" @@ -71,4 +72,4 @@ $CLICKHOUSE_CLIENT --query "SELECT is_done, parts_to_do FROM system.mutations wh $CLICKHOUSE_CLIENT --query "SELECT type, new_part_name FROM system.replication_queue WHERE table='mutation_table' and database='$CLICKHOUSE_DATABASE'" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS mutation_table SYNC" diff --git a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql index 8bf5d4f2cf8..f20156fd9e3 100644 --- a/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql +++ b/tests/queries/0_stateless/01346_alter_enum_partition_key_replicated_zookeeper_long.sql @@ -1,7 +1,10 @@ -- Tags: long, replica -DROP TABLE IF EXISTS test; -DROP TABLE IF EXISTS test2; +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries +SET replication_alter_partitions_sync=2; + +DROP TABLE IF EXISTS test SYNC; +DROP TABLE IF EXISTS test2 SYNC; CREATE TABLE test (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r1') PARTITION BY x ORDER BY y; CREATE TABLE test2 (x Enum('hello' = 1, 'world' = 2), y String) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/test_01346/table', 'r2') PARTITION BY x ORDER BY y; @@ -17,7 +20,6 @@ ALTER TABLE test MODIFY COLUMN x Enum('hello' = 1, 'world' = 2, 'goodbye' = 3); INSERT INTO test VALUES ('goodbye', 'test'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -31,7 +33,6 @@ ALTER TABLE test MODIFY COLUMN x Int8; INSERT INTO test VALUES (111, 'abc'); OPTIMIZE TABLE test FINAL; SELECT * FROM test ORDER BY x; -SYSTEM SYNC REPLICA test2; SELECT * FROM test2 ORDER BY x; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test' AND active ORDER BY partition; SELECT name, partition, partition_id FROM system.parts WHERE database = currentDatabase() AND table = 'test2' AND active ORDER BY partition; @@ -48,5 +49,5 @@ ALTER TABLE test RENAME COLUMN y TO z; -- { serverError 524 } ALTER TABLE test DROP COLUMN x; -- { serverError 47 } ALTER TABLE test DROP COLUMN y; -- { serverError 47 } -DROP TABLE test; -DROP TABLE test2; +DROP TABLE test SYNC; +DROP TABLE test2 SYNC; diff --git a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh index 01678d1b500..4629450c1f9 100755 --- a/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh +++ b/tests/queries/0_stateless/01417_freeze_partition_verbose_zookeeper.sh @@ -12,9 +12,9 @@ FREEZE_OUT_STRUCTURE='backup_name String, backup_path String , part_backup_path # setup -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" ${CLICKHOUSE_CLIENT} --query "CREATE TABLE table_for_freeze_replicated (key UInt64, value String) ENGINE = ReplicatedMergeTree('/test/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/table_for_freeze_replicated', '1') ORDER BY key PARTITION BY key % 10;" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated SELECT number, toString(number) from numbers(10);" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE WITH NAME 'test_01417' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $FREEZE_OUT_STRUCTURE" \ @@ -25,7 +25,7 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name FROM table" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated DETACH PARTITION '3';" -${CLICKHOUSE_CLIENT} --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" +${CLICKHOUSE_CLIENT} --insert_keeper_fault_injection_probability=0 --query "INSERT INTO table_for_freeze_replicated VALUES (3, '3');" ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated ATTACH PARTITION '3' FORMAT TSVWithNames SETTINGS alter_partition_verbose_result = 1;" \ | ${CLICKHOUSE_LOCAL} --structure "$ALTER_OUT_STRUCTURE, $ATTACH_OUT_STRUCTURE" \ @@ -38,4 +38,4 @@ ${CLICKHOUSE_CLIENT} --query "ALTER TABLE table_for_freeze_replicated FREEZE PAR --query "SELECT command_type, partition_id, part_name, backup_name, old_part_name FROM table" # teardown -${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated;" +${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS table_for_freeze_replicated SYNC;" diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql index 0155b83db31..eea231c9f58 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_and_quorum_long.sql @@ -1,5 +1,6 @@ -- Tags: long, replica, no-replicated-database +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; diff --git a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql index 9f9d1db78f1..bf7a471fa40 100644 --- a/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql +++ b/tests/queries/0_stateless/01451_replicated_detach_drop_part_long.sql @@ -1,10 +1,11 @@ -- Tags: long, replica, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries SET replication_alter_partitions_sync = 2; -DROP TABLE IF EXISTS replica1; -DROP TABLE IF EXISTS replica2; +DROP TABLE IF EXISTS replica1 SYNC; +DROP TABLE IF EXISTS replica2 SYNC; CREATE TABLE replica1 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r1') order by tuple() settings max_replicated_merges_in_queue = 0; CREATE TABLE replica2 (v UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/'||currentDatabase()||'test/01451/attach', 'r2') order by tuple() settings max_replicated_merges_in_queue = 0; @@ -48,5 +49,5 @@ SELECT v FROM replica1 ORDER BY v; SELECT name FROM system.parts WHERE table = 'replica2' AND active AND database = currentDatabase(); -DROP TABLE replica1; -DROP TABLE replica2; +DROP TABLE replica1 SYNC; +DROP TABLE replica2 SYNC; diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh index 13c772e75d2..c05d813ca7f 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas.sh @@ -11,7 +11,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -20,7 +20,11 @@ function thread { for x in {0..99}; do # sometimes we can try to commit obsolete part if fetches will be quite fast, # so supress warning messages like "Tried to commit obsolete part ... covered by ..." - $CLICKHOUSE_CLIENT --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas + # (2) keeper fault injection for inserts because + # it can be a cause of deduplicated parts be visible to SELECTs for sometime (until cleanup thread remove them), + # so the same SELECT on different replicas can return different results, i.e. test output will be non-deterministic + # (see #9712) + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x % $NUM_REPLICAS = $1 ? $x - 1 : $x" 2>/dev/null # Replace some records as duplicates so they will be written by other replicas done } @@ -37,5 +41,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh index 6eabc9ae1b5..209e18e3329 100755 --- a/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh +++ b/tests/queries/0_stateless/01459_manual_write_to_replicas_quorum.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=10 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', 'r$i') ORDER BY x; " done @@ -22,7 +22,7 @@ valid_exceptions_to_retry='Quorum for previous write has not been satisfied yet| function thread { for x in {0..99}; do while true; do - $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break + $CLICKHOUSE_CLIENT --insert_quorum 5 --insert_quorum_parallel 0 --insert_keeper_fault_injection_probability=0 --query "INSERT INTO r$1 SELECT $x" 2>&1 | grep -qE "$valid_exceptions_to_retry" || break done done } @@ -40,5 +40,5 @@ for i in $(seq 1 $NUM_REPLICAS); do done for i in $(seq 1 $NUM_REPLICAS); do - $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i;" + $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS r$i SYNC;" done diff --git a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh index 030ae017e71..a2ecef64bbb 100755 --- a/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh +++ b/tests/queries/0_stateless/01509_check_many_parallel_quorum_inserts_long.sh @@ -12,7 +12,7 @@ NUM_REPLICAS=6 for i in $(seq 1 $NUM_REPLICAS); do $CLICKHOUSE_CLIENT -n -q " - DROP TABLE IF EXISTS r$i; + DROP TABLE IF EXISTS r$i SYNC; CREATE TABLE r$i (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_quorum_many', 'r$i') ORDER BY x; " done @@ -20,7 +20,7 @@ done function thread { i=0 retries=300 while [[ $i -lt $retries ]]; do # server can be dead - $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --query "INSERT INTO r$1 SELECT $2" && break + $CLICKHOUSE_CLIENT --insert_quorum 3 --insert_quorum_parallel 1 --insert_keeper_max_retries=100 --insert_keeper_retry_max_backoff_ms=10 --query "INSERT INTO r$1 SELECT $2" && break ((++i)) sleep 0.1 done diff --git a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh index 9325cac0ae6..445706e35bf 100755 --- a/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh +++ b/tests/queries/0_stateless/01509_parallel_quorum_and_merge_long.sh @@ -9,8 +9,8 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q1 (x UInt64) ENGINE=ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/parallel_q', 'r1') ORDER BY tuple() SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0" @@ -19,9 +19,10 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE parallel_q2 (x UInt64) ENGINE=ReplicatedMerg $CLICKHOUSE_CLIENT -q "SYSTEM STOP REPLICATION QUEUES parallel_q2" -$CLICKHOUSE_CLIENT -q "INSERT INTO parallel_q1 VALUES (1)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO parallel_q1 VALUES (1)" -$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --query="INSERT INTO parallel_q1 VALUES (2)" & +# disable keeper fault injection during insert since test checks part names. Part names can differ in case of retries during insert +$CLICKHOUSE_CLIENT --insert_quorum 2 --insert_quorum_parallel 1 --insert_keeper_fault_injection_probability=0 --query="INSERT INTO parallel_q1 VALUES (2)" & part_count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM system.parts WHERE table='parallel_q1' and database='${CLICKHOUSE_DATABASE}'") @@ -66,5 +67,5 @@ $CLICKHOUSE_CLIENT --query="SELECT event_type FROM system.part_log WHERE table=' $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q2" $CLICKHOUSE_CLIENT --query="SELECT COUNT() FROM parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1" -$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q1 SYNC" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS parallel_q2 SYNC" diff --git a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql index 2f0c59ab64b..86bef193f4d 100644 --- a/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql +++ b/tests/queries/0_stateless/01509_parallel_quorum_insert_no_replicas_long.sql @@ -16,6 +16,9 @@ CREATE TABLE r2 ( ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01509_parallel_quorum_insert_no_replicas', '2') ORDER BY tuple(); +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SET insert_quorum_parallel=1; SET insert_quorum=3; @@ -79,11 +82,11 @@ SYSTEM STOP FETCHES r2; SET insert_quorum_timeout=0; -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } -- retry should fail despite the insert_deduplicate enabled -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } -INSERT INTO r1 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } +INSERT INTO r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (4, '4'); -- { serverError 319 } SELECT * FROM r2 WHERE key=4; SYSTEM START FETCHES r2; @@ -99,5 +102,5 @@ SELECT 'insert happened'; SELECT COUNT() FROM r1; SELECT COUNT() FROM r2; -DROP TABLE IF EXISTS r1; -DROP TABLE IF EXISTS r2; +DROP TABLE IF EXISTS r1 SYNC; +DROP TABLE IF EXISTS r2 SYNC; diff --git a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql index cf06af0113d..f217b6094b2 100644 --- a/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql +++ b/tests/queries/0_stateless/01532_execute_merges_on_single_replica_long.sql @@ -2,6 +2,8 @@ -- Tag no-replicated-database: Fails due to additional replicas or shards -- Tag no-parallel: static zk path +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + DROP TABLE IF EXISTS execute_on_single_replica_r1 NO DELAY; DROP TABLE IF EXISTS execute_on_single_replica_r2 NO DELAY; diff --git a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql index af2a46cd77f..b55b4871363 100644 --- a/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql +++ b/tests/queries/0_stateless/01532_primary_key_without_order_by_zookeeper.sql @@ -1,6 +1,6 @@ -- Tags: zookeeper -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; CREATE TABLE merge_tree_pk ( @@ -24,9 +24,9 @@ ATTACH TABLE merge_tree_pk; SELECT * FROM merge_tree_pk FINAL ORDER BY key, value; -DROP TABLE IF EXISTS merge_tree_pk; +DROP TABLE IF EXISTS merge_tree_pk SYNC; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; CREATE TABLE merge_tree_pk_sql ( @@ -60,9 +60,9 @@ SELECT * FROM merge_tree_pk_sql FINAL ORDER BY key, value; SHOW CREATE TABLE merge_tree_pk_sql; -DROP TABLE IF EXISTS merge_tree_pk_sql; +DROP TABLE IF EXISTS merge_tree_pk_sql SYNC; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; CREATE TABLE replicated_merge_tree_pk_sql ( @@ -99,4 +99,4 @@ ATTACH TABLE replicated_merge_tree_pk_sql; SHOW CREATE TABLE replicated_merge_tree_pk_sql; -DROP TABLE IF EXISTS replicated_merge_tree_pk_sql; +DROP TABLE IF EXISTS replicated_merge_tree_pk_sql SYNC; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index e52e0c94c3c..a5f301b1353 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -1,6 +1,6 @@ -- Tags: replica -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; CREATE TABLE replicated_mutations_empty_partitions ( @@ -11,7 +11,8 @@ ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_rep ORDER BY key PARTITION by key; -INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number) FROM numbers(10); +-- insert_keeper* settings are adjusted since several actual inserts are happening behind one statement due to partitioning i.e. inserts in different partitions +INSERT INTO replicated_mutations_empty_partitions SETTINGS insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=10 SELECT number, toString(number) FROM numbers(10); SELECT count(distinct value) FROM replicated_mutations_empty_partitions; @@ -31,4 +32,4 @@ SELECT sum(value) FROM replicated_mutations_empty_partitions; SHOW CREATE TABLE replicated_mutations_empty_partitions; -DROP TABLE IF EXISTS replicated_mutations_empty_partitions; +DROP TABLE IF EXISTS replicated_mutations_empty_partitions SYNC; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh index acaa2cfcd25..d68f9bc1837 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill.sh @@ -5,11 +5,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_mutate_kill (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/concurrent_mutate_kill', '1') ORDER BY key PARTITION BY key % 100 SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" -$CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 --query "INSERT INTO concurrent_mutate_kill SELECT number, toString(number) FROM numbers(1000000)" function alter_thread { @@ -67,4 +67,4 @@ done $CLICKHOUSE_CLIENT --query "SHOW CREATE TABLE concurrent_mutate_kill" $CLICKHOUSE_CLIENT --query "OPTIMIZE TABLE concurrent_mutate_kill FINAL" $CLICKHOUSE_CLIENT --query "SELECT sum(value) FROM concurrent_mutate_kill" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS concurrent_mutate_kill SYNC" diff --git a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql index c77f29d89c2..e2926d9a8ac 100644 --- a/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql +++ b/tests/queries/0_stateless/01650_drop_part_and_deduplication_zookeeper_long.sql @@ -1,7 +1,8 @@ -- Tags: long, zookeeper, no-replicated-database -- Tag no-replicated-database: Fails due to additional replicas or shards -DROP TABLE IF EXISTS partitioned_table; +SET insert_keeper_fault_injection_probability=0; +DROP TABLE IF EXISTS partitioned_table SYNC; CREATE TABLE partitioned_table ( key UInt64, @@ -47,4 +48,4 @@ SELECT partition_id, name FROM system.parts WHERE table = 'partitioned_table' AN SELECT substring(name, 1, 2), value FROM system.zookeeper WHERE path='/clickhouse/' || currentDatabase() || '/01650_drop_part_and_deduplication_partitioned_table/blocks/' ORDER BY value; -DROP TABLE IF EXISTS partitioned_table; +DROP TABLE IF EXISTS partitioned_table SYNC; diff --git a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql index dad5cdbf1f3..87e1a039488 100644 --- a/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql +++ b/tests/queries/0_stateless/01715_background_checker_blather_zookeeper_long.sql @@ -1,7 +1,7 @@ -- Tags: long, zookeeper -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE IF EXISTS i20203_1 SYNC; +DROP TABLE IF EXISTS i20203_2 SYNC; CREATE TABLE i20203_1 (a Int8) ENGINE = ReplicatedMergeTree('/clickhouse/{database}/01715_background_checker_i20203', 'r1') @@ -26,5 +26,5 @@ WHERE table = 'i20203_2' AND database = currentDatabase(); ATTACH TABLE i20203_1; -DROP TABLE IF EXISTS i20203_1; -DROP TABLE IF EXISTS i20203_2; +DROP TABLE i20203_1 SYNC; +DROP TABLE i20203_2 SYNC; diff --git a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh index 85662438f33..4e4255fe9c2 100755 --- a/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh +++ b/tests/queries/0_stateless/01810_max_part_removal_threads_long.sh @@ -51,6 +51,9 @@ $CLICKHOUSE_CLIENT -nm -q """ partition by key%100 settings max_part_removal_threads=10, concurrent_part_removal_threshold=99, min_bytes_for_wide_part=0; + SET insert_keeper_max_retries=1000; + SET insert_keeper_retry_max_backoff_ms=10; + insert into rep_data_01810 select * from numbers(100); drop table rep_data_01810 settings log_queries=1; system flush logs; diff --git a/tests/queries/0_stateless/01825_type_json_3.sql.j2 b/tests/queries/0_stateless/01825_type_json_3.sql.j2 index 62d86c3efd4..0fbf7a936d8 100644 --- a/tests/queries/0_stateless/01825_type_json_3.sql.j2 +++ b/tests/queries/0_stateless/01825_type_json_3.sql.j2 @@ -2,6 +2,8 @@ {% for engine in ["ReplicatedMergeTree('/clickhouse/tables/{database}/test_01825_3/t_json_3', 'r1') ORDER BY tuple()", "Memory"] -%} +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + SET allow_experimental_object_type = 1; DROP TABLE IF EXISTS t_json_3; diff --git a/tests/queries/0_stateless/01870_modulo_partition_key.sql b/tests/queries/0_stateless/01870_modulo_partition_key.sql index 1de23c13a65..46cd09622b1 100644 --- a/tests/queries/0_stateless/01870_modulo_partition_key.sql +++ b/tests/queries/0_stateless/01870_modulo_partition_key.sql @@ -1,3 +1,6 @@ +SET insert_keeper_max_retries=100; +SET insert_keeper_retry_max_backoff_ms=10; + SELECT 'simple partition key:'; DROP TABLE IF EXISTS table1 SYNC; CREATE TABLE table1 (id Int64, v UInt64) @@ -15,7 +18,7 @@ select 'where id % 200 < 0:'; select id from table1 where id % 200 < 0 order by id; SELECT 'tuple as partition key:'; -DROP TABLE IF EXISTS table2; +DROP TABLE IF EXISTS table2 SYNC; CREATE TABLE table2 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (toInt32(id / 2) % 3, id % 200) ORDER BY id; @@ -24,7 +27,7 @@ INSERT INTO table2 SELECT number-205, number FROM numbers(400, 10); SELECT partition as p FROM system.parts WHERE table='table2' and database=currentDatabase() ORDER BY p; SELECT 'recursive modulo partition key:'; -DROP TABLE IF EXISTS table3; +DROP TABLE IF EXISTS table3 SYNC; CREATE TABLE table3 (id Int64, v UInt64) ENGINE = MergeTree() PARTITION BY (id % 200, (id % 200) % 10, toInt32(round((id % 200) / 2, 0))) ORDER BY id; @@ -38,7 +41,7 @@ SELECT 'After detach:'; SELECT partition as p FROM system.parts WHERE table='table3' and database=currentDatabase() ORDER BY p; SELECT 'Indexes:'; -DROP TABLE IF EXISTS table4; +DROP TABLE IF EXISTS table4 SYNC; CREATE TABLE table4 (id Int64, v UInt64, s String, INDEX a (id * 2, s) TYPE minmax GRANULARITY 3 ) ENGINE = MergeTree() PARTITION BY id % 10 ORDER BY v; diff --git a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql index 635da491aa0..8924627a717 100644 --- a/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql +++ b/tests/queries/0_stateless/02228_merge_tree_insert_memory_usage.sql @@ -1,4 +1,5 @@ -- Tags: long, no-parallel +SET insert_keeper_fault_injection_probability=0; -- to succeed this test can require too many retries due to 1024 partitions, so disable fault injections -- regression for MEMORY_LIMIT_EXCEEDED error because of deferred final part flush @@ -8,8 +9,8 @@ insert into data_02228 select number, 1, number from numbers_mt(100e3) settings insert into data_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } drop table data_02228; -drop table if exists data_rep_02228; +drop table if exists data_rep_02228 SYNC; create table data_rep_02228 (key1 UInt32, sign Int8, s UInt64) engine = ReplicatedCollapsingMergeTree('/clickhouse/{database}', 'r1', sign) order by (key1) partition by key1 % 1024; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=0; insert into data_rep_02228 select number, 1, number from numbers_mt(100e3) settings max_memory_usage='300Mi', max_partitions_per_insert_block=1024, max_insert_delayed_streams_for_parallel_write=10000000; -- { serverError MEMORY_LIMIT_EXCEEDED } -drop table data_rep_02228; +drop table data_rep_02228 SYNC; diff --git a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh index e8111bf7a03..9ea924377b2 100755 --- a/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh +++ b/tests/queries/0_stateless/02340_parts_refcnt_mergetree.sh @@ -10,7 +10,7 @@ function check_refcnt_for_table() local table=$1 && shift $CLICKHOUSE_CLIENT -q "system stop merges $table" - $CLICKHOUSE_CLIENT -q "insert into $table select number, number%4 from numbers(200)" + $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into $table select number, number%4 from numbers(200)" local query_id query_id="$table-$(random_str 10)" @@ -52,7 +52,7 @@ $CLICKHOUSE_CLIENT -nmq " check_refcnt_for_table data_02340 $CLICKHOUSE_CLIENT -nmq " - drop table if exists data_02340_rep; + drop table if exists data_02340_rep sync; create table data_02340_rep (key Int, part Int) engine=ReplicatedMergeTree('/clickhouse/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX', '1') partition by part order by key settings index_granularity=1; " || exit 1 check_refcnt_for_table data_02340_rep diff --git a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh index 11caf1e45de..88ff1f5b7c6 100755 --- a/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh +++ b/tests/queries/0_stateless/02369_lost_part_intersecting_merges.sh @@ -1,18 +1,18 @@ #!/usr/bin/env bash -# Tags: zookeeper +# Tags: long, zookeeper CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "drop table if exists rmt1;" -$CLICKHOUSE_CLIENT -q "drop table if exists rmt2;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table if exists rmt2 sync;" $CLICKHOUSE_CLIENT -q "create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '1') order by n;" $CLICKHOUSE_CLIENT -q "create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02369/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{database}', '2') order by n;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (1);" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (2);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (1);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (2);" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" @@ -32,7 +32,7 @@ $CLICKHOUSE_CLIENT -q "select * from rmt1;" 2>/dev/null $CLICKHOUSE_CLIENT -q "detach table rmt1;" $CLICKHOUSE_CLIENT -q "attach table rmt1;" -$CLICKHOUSE_CLIENT -q "insert into rmt1 values (3);" +$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "insert into rmt1 values (3);" $CLICKHOUSE_CLIENT -q "system start merges rmt2;" $CLICKHOUSE_CLIENT -q "system sync replica rmt1;" $CLICKHOUSE_CLIENT -q "optimize table rmt1 final;" @@ -42,5 +42,5 @@ $CLICKHOUSE_CLIENT -q "system sync replica rmt2;" $CLICKHOUSE_CLIENT -q "select 3, *, _part from rmt1 order by n;" $CLICKHOUSE_CLIENT -q "select 4, *, _part from rmt2 order by n;" -$CLICKHOUSE_CLIENT -q "drop table rmt1;" -$CLICKHOUSE_CLIENT -q "drop table rmt2;" +$CLICKHOUSE_CLIENT -q "drop table rmt1 sync;" +$CLICKHOUSE_CLIENT -q "drop table rmt2 sync;" diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 371f7389837..0db816332a1 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -1,5 +1,7 @@ -- Tags: long +SET insert_keeper_fault_injection_probability=0; -- disable fault injection; part ids are non-deterministic in case of insert retries + drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference new file mode 100644 index 00000000000..f5f1cf8ac64 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.reference @@ -0,0 +1,2 @@ +1 +11 diff --git a/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql new file mode 100644 index 00000000000..774370bc132 --- /dev/null +++ b/tests/queries/0_stateless/02456_keeper_retries_during_insert.sql @@ -0,0 +1,26 @@ +-- Tags: replica + +DROP TABLE IF EXISTS keeper_retries_r1 SYNC; +DROP TABLE IF EXISTS keeper_retries_r2 SYNC; + +CREATE TABLE keeper_retries_r1(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r1') ORDER BY tuple (); +CREATE TABLE keeper_retries_r2(a UInt8) ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/test/02456_keeper_retries_during_insert', 'r2') ORDER BY tuple(); + +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (1); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (2); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=10 VALUES (3); -- { serverError KEEPER_EXCEPTION } + +SET insert_quorum=2; +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=0 VALUES (11); +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_max_retries=0 VALUES (12); -- { serverError KEEPER_EXCEPTION } +INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_probability=1, insert_keeper_retry_max_backoff_ms=1 VALUES (13); -- { serverError KEEPER_EXCEPTION } + +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.05, insert_keeper_fault_injection_seed=1 VALUES (21); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.2, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=2 VALUES (22); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.3, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=3 VALUES (23); +-- INSERT INTO keeper_retries_r1 SETTINGS insert_keeper_fault_injection_mode=1, insert_keeper_fault_injection_probability=0.4, insert_keeper_max_retries=100, insert_keeper_retry_max_backoff_ms=1, insert_keeper_fault_injection_seed=4 VALUES (24); + +SELECT * FROM keeper_retries_r1 order by a; + +DROP TABLE keeper_retries_r1 SYNC; +DROP TABLE keeper_retries_r2 SYNC; From cbdb2b0be471cba9722bc22efcca04bcac922fd4 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 10 Nov 2022 12:19:06 +0000 Subject: [PATCH 323/526] add more asserts in test --- .../02477_s3_request_throttler.reference | 1 + .../0_stateless/02477_s3_request_throttler.sql | 13 ++++++++++++- 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.reference b/tests/queries/0_stateless/02477_s3_request_throttler.reference index d00491fd7e5..9315e86b328 100644 --- a/tests/queries/0_stateless/02477_s3_request_throttler.reference +++ b/tests/queries/0_stateless/02477_s3_request_throttler.reference @@ -1 +1,2 @@ 1 +1 1 1 diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.sql b/tests/queries/0_stateless/02477_s3_request_throttler.sql index 7311c8ac9bc..25653b1bab0 100644 --- a/tests/queries/0_stateless/02477_s3_request_throttler.sql +++ b/tests/queries/0_stateless/02477_s3_request_throttler.sql @@ -7,9 +7,20 @@ SET s3_max_put_burst = 1; CREATE TEMPORARY TABLE times (t DateTime); --- INSERT query requires 3 PUT requests (CreateMultipartUpload, UploadPart, CompleteMultipartUpload) and 1/rps = 0.5 second in between, the first query is not throttled due to burst +-- INSERT query requires 3 PUT requests and 1/rps = 0.5 second in between, the first query is not throttled due to burst INSERT INTO times SELECT now(); INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/request-throttler.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10000, s3_truncate_on_insert = 1; INSERT INTO times SELECT now(); SELECT max(t) - min(t) >= 1 FROM times; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['S3CreateMultipartUpload'] == 1, + ProfileEvents['S3UploadPart'] == 1, + ProfileEvents['S3CompleteMultipartUpload'] == 1 +FROM system.query_log +WHERE query LIKE '%request-throttler.csv%' +AND type = 'QueryFinish' +AND current_database = currentDatabase() +ORDER BY query_start_time DESC +LIMIT 1; From c00f71327a3eb7f90a58b42adaaa980a6b731f73 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 10 Nov 2022 12:29:24 +0000 Subject: [PATCH 324/526] Fix: backward-compatility check - remove unused config for now --- tests/config/users.d/insert_keeper_retries.xml | 8 -------- 1 file changed, 8 deletions(-) delete mode 100644 tests/config/users.d/insert_keeper_retries.xml diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml deleted file mode 100644 index f7b652bf2ef..00000000000 --- a/tests/config/users.d/insert_keeper_retries.xml +++ /dev/null @@ -1,8 +0,0 @@ - - - - 0 - 0.0 - - - From f58e960a7dbe71b54d69ec06d9d54967d0d97ea8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 10 Nov 2022 21:07:44 +0800 Subject: [PATCH 325/526] update as request --- .../sql-reference/functions/math-functions.md | 2 +- src/Functions/factorial.cpp | 18 ++++++++++++++---- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index bfd2f63dc5b..7f349686d8b 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -555,7 +555,7 @@ Result: Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is Int64. -The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20; a value of 21 or greater overflows the range for Int64 and will causes an throw exception. +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. **Syntax** diff --git a/src/Functions/factorial.cpp b/src/Functions/factorial.cpp index d92cb668214..5c46b97c193 100644 --- a/src/Functions/factorial.cpp +++ b/src/Functions/factorial.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB { @@ -74,13 +75,22 @@ template <> struct FunctionUnaryArithmeticMonotonicity { static bool has() { return true; } - static IFunction::Monotonicity get(const Field & /*left*/, const Field & /*right*/) + static IFunction::Monotonicity get(const Field & left, const Field & right) { + bool is_strict = false; + if (!left.isNull() && !right.isNull()) + { + auto left_value = applyVisitor(FieldVisitorConvertToNumber(), left); + auto right_value = applyVisitor(FieldVisitorConvertToNumber(), left); + if (1 <= left_value && left_value <= right_value && right_value <= 20) + is_strict = true; + } + return { .is_monotonic = true, .is_positive = true, .is_always_monotonic = true, - .is_strict = false, + .is_strict = is_strict, }; } }; @@ -91,9 +101,9 @@ REGISTER_FUNCTION(Factorial) factory.registerFunction( { R"( -Computes the factorial of an integer value. It works with any native integer type. The return type is Int64. +Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is Int64. -The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20; a value of 21 or greater overflows the range for Int64 and will causes an throw exception. +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. )", Documentation::Examples{{"factorial", "SELECT factorial(10)"}}, Documentation::Categories{"Mathematical"}}, From 8a7c7bfb3df3084d01c621e34338b7363d029cc8 Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Thu, 10 Nov 2022 21:12:55 +0800 Subject: [PATCH 326/526] extract test of function factorial --- tests/queries/0_stateless/00087_math_functions.reference | 3 --- tests/queries/0_stateless/00087_math_functions.sql | 7 ------- tests/queries/0_stateless/02478_factorial.reference | 3 +++ tests/queries/0_stateless/02478_factorial.sql | 7 +++++++ 4 files changed, 10 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/02478_factorial.reference create mode 100644 tests/queries/0_stateless/02478_factorial.sql diff --git a/tests/queries/0_stateless/00087_math_functions.reference b/tests/queries/0_stateless/00087_math_functions.reference index 48cb24098ff..e02aac11faf 100644 --- a/tests/queries/0_stateless/00087_math_functions.reference +++ b/tests/queries/0_stateless/00087_math_functions.reference @@ -112,6 +112,3 @@ 1 1 1 -1 -1 -1 diff --git a/tests/queries/0_stateless/00087_math_functions.sql b/tests/queries/0_stateless/00087_math_functions.sql index 11456944be3..e40acfb3481 100644 --- a/tests/queries/0_stateless/00087_math_functions.sql +++ b/tests/queries/0_stateless/00087_math_functions.sql @@ -137,10 +137,3 @@ select erf(10) = 1; select erfc(0) = 1; select erfc(-10) = 2; select erfc(28) = 0; - -select factorial(-1) = 1; -select factorial(0) = 1; -select factorial(10) = 3628800; -select factorial(100); -- { serverError 36 } -select factorial('100'); -- { serverError 43 } -select factorial(100.1234); -- { serverError 43 } diff --git a/tests/queries/0_stateless/02478_factorial.reference b/tests/queries/0_stateless/02478_factorial.reference new file mode 100644 index 00000000000..e8183f05f5d --- /dev/null +++ b/tests/queries/0_stateless/02478_factorial.reference @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/tests/queries/0_stateless/02478_factorial.sql b/tests/queries/0_stateless/02478_factorial.sql new file mode 100644 index 00000000000..e1a0f7d60e5 --- /dev/null +++ b/tests/queries/0_stateless/02478_factorial.sql @@ -0,0 +1,7 @@ +select factorial(-1) = 1; +select factorial(0) = 1; +select factorial(10) = 3628800; + +select factorial(100); -- { serverError 36 } +select factorial('100'); -- { serverError 43 } +select factorial(100.1234); -- { serverError 43 } From fc7311aab7070153a27bdbc6bd10a13c979809c5 Mon Sep 17 00:00:00 2001 From: flynn Date: Thu, 10 Nov 2022 15:09:55 +0000 Subject: [PATCH 327/526] fix --- src/TableFunctions/registerTableFunctions.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 9328c12c122..e27ec0a74ec 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -27,6 +27,8 @@ void registerTableFunctions() registerTableFunctionS3(factory); registerTableFunctionS3Cluster(factory); registerTableFunctionCOS(factory); + registerTableFunctionHudi(factory); + registerTableFunctionDelta(factory); #endif #if USE_HDFS From 49e53e48f91c5a1f49ec2c6d5cf15e14bf1d75f8 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Thu, 10 Nov 2022 15:32:06 +0000 Subject: [PATCH 328/526] initial analysis --- .../example-datasets/github.md | 2247 +++++++++++++++++ .../images/superset-commits-authors.png | Bin 0 -> 252198 bytes .../superset-github-lines-added-deleted.png | Bin 0 -> 70603 bytes 3 files changed, 2247 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/github.md create mode 100644 docs/en/getting-started/example-datasets/images/superset-commits-authors.png create mode 100644 docs/en/getting-started/example-datasets/images/superset-github-lines-added-deleted.png diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md new file mode 100644 index 00000000000..6a5c9eab2f5 --- /dev/null +++ b/docs/en/getting-started/example-datasets/github.md @@ -0,0 +1,2247 @@ +--- +slug: /en/getting-started/example-datasets/github +sidebar_label: Github Repo Analysis +description: Analyze the ClickHouse Github repo or any repository of your choosing +--- + +# ClickHouse GitHub data + +This dataset contains all of the commits and changes for the ClickHouse repository. It can be generated using the native `git-import` tool distributed with ClickHouse. + +The generated data provides a `tsv` file for each of the following tables: + +- `commits` - commits with statistics; +- `file_changes` - files changed in every commit with the info about the change and statistics; +- `line_changes` - every changed line in every changed file in every commit with full info about the line and the information about the previous change of this line. + +As of November 8th, 2022, each tsv is approximately the following size and number of rows: + +- `commits` - 7.8M - 266,051 rows +- `file_changes` - 53M - 266,051 rows +- `line_changes` - 2.7G - 7,535,157 rows + +# Generating the data + +```bash +git clone git@github.com:ClickHouse/ClickHouse.git +cd ClickHouse +clickhouse git-import --skip-paths 'generated\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch ' +``` + +This will take around 3 minutes (as of November 8th 2022) to complete for the ClickHouse repository. + +A full list of available options can be obtained from the tools native help. + +```bash +clickhouse git-import -h +``` + +This help also provides the DDL for each of the above tables e.g. + +``` +CREATE TABLE git.commits +( + hash String, + author LowCardinality(String), + time DateTime, + message String, + files_added UInt32, + files_deleted UInt32, + files_renamed UInt32, + files_modified UInt32, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; +``` + +**These queries should work on any repository. Feel free to explore and report your findings** Some guidelines with respect to execution times (as of November 2022): + +- Linux - `~/clickhouse git-import` - 160 mins + +# Downloading and inserting the data + +Generated files for the following repositories can be found below: + +- ClickHouse (Nov 8th 2022) + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz - 2.5 MB + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz - 4.5MB + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz - 127.4 MB +- Linux (Nov 8th 2022) + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/commits.tsv.xz - 2.5 MB + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/file_changes.tsv.xz - 4.5MB + - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/line_changes.tsv.xz - 127.4 MB + +To insert this data, prepare the database by executing the following queries: + +```sql +DROP DATABASE IF EXISTS git; +CREATE DATABASE git; + +CREATE TABLE git.commits +( + hash String, + author LowCardinality(String), + time DateTime, + message String, + files_added UInt32, + files_deleted UInt32, + files_renamed UInt32, + files_modified UInt32, + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.file_changes +( + change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), + lines_added UInt32, + lines_deleted UInt32, + hunks_added UInt32, + hunks_removed UInt32, + hunks_changed UInt32, + + commit_hash String, + author LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; + +CREATE TABLE git.line_changes +( + sign Int8, + line_number_old UInt32, + line_number_new UInt32, + hunk_num UInt32, + hunk_start_line_number_old UInt32, + hunk_start_line_number_new UInt32, + hunk_lines_added UInt32, + hunk_lines_deleted UInt32, + hunk_context LowCardinality(String), + line LowCardinality(String), + indent UInt8, + line_type Enum('Empty' = 0, 'Comment' = 1, 'Punct' = 2, 'Code' = 3), + + prev_commit_hash String, + prev_author LowCardinality(String), + prev_time DateTime, + + file_change_type Enum('Add' = 1, 'Delete' = 2, 'Modify' = 3, 'Rename' = 4, 'Copy' = 5, 'Type' = 6), + path LowCardinality(String), + old_path LowCardinality(String), + file_extension LowCardinality(String), + file_lines_added UInt32, + file_lines_deleted UInt32, + file_hunks_added UInt32, + file_hunks_removed UInt32, + file_hunks_changed UInt32, + + commit_hash String, + author LowCardinality(String), + time DateTime, + commit_message String, + commit_files_added UInt32, + commit_files_deleted UInt32, + commit_files_renamed UInt32, + commit_files_modified UInt32, + commit_lines_added UInt32, + commit_lines_deleted UInt32, + commit_hunks_added UInt32, + commit_hunks_removed UInt32, + commit_hunks_changed UInt32 +) ENGINE = MergeTree ORDER BY time; +``` + +Insert the data using `INSERT INTO SELECT` and the [s3 function](https://clickhouse.com/docs/en/integrations/s3/s3-table-functions/). For example, below, we insert the ClickHouse files into each of their respective tables: + +*commits* + +```sql +INSERT INTO git.commits SELECT * +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz', 'TSV', 'hash String,author LowCardinality(String), time DateTime, message String, files_added UInt32, files_deleted UInt32, files_renamed UInt32, files_modified UInt32, lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32') + +0 rows in set. Elapsed: 1.826 sec. Processed 62.78 thousand rows, 8.50 MB (34.39 thousand rows/s., 4.66 MB/s.) +``` + +*file_changes* + +```sql +INSERT INTO git.file_changes SELECT * +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz', 'TSV', 'change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6), path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32, commit_hash String, author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') + +0 rows in set. Elapsed: 2.688 sec. Processed 266.05 thousand rows, 48.30 MB (98.97 thousand rows/s., 17.97 MB/s.) +``` + +*line_changes* + +```sql +INSERT INTO git.line_changes SELECT * +FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz', 'TSV', ' sign Int8, line_number_old UInt32, line_number_new UInt32, hunk_num UInt32, hunk_start_line_number_old UInt32, hunk_start_line_number_new UInt32, hunk_lines_added UInt32,\n hunk_lines_deleted UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, line_type Enum(\'Empty\' = 0, \'Comment\' = 1, \'Punct\' = 2, \'Code\' = 3), prev_commit_hash String, prev_author LowCardinality(String), prev_time DateTime, file_change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6),\n path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), file_lines_added UInt32, file_lines_deleted UInt32, file_hunks_added UInt32, file_hunks_removed UInt32, file_hunks_changed UInt32, commit_hash String,\n author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') + +0 rows in set. Elapsed: 50.535 sec. Processed 7.54 million rows, 2.09 GB (149.11 thousand rows/s., 41.40 MB/s.) +``` + +# Queries + +The tool suggests several queries via its help output. We have answered these in addition to some additional supplementary questions of interest: + +- [History of a single file](#history-of-a-single-file) +- [Find the current active files](#find-the-current-active-files) +- [List files with most modifications](#list-files-with-most-modifications) +- [What day of the week do commits usually occur?](#what-day-of-the-week-do-commits-usually-occur) +- [List files with maximum number of authors](#list-files-with-maximum-number-of-authors) +- [Oldest lines of code in the repository](#oldest-lines-of-code-in-the-repository) +- [Files with longest history](#files-with-longest-history) +- [Distribution of contributors with respect to docs and code over the month](#distribution-of-contributors-with-respect-to-docs-and-code-over-the-month) +- [Authors with the most diverse impact](#authors-with-the-most-diverse-impact) +- [Favorite files for an author](#favorite-files-for-an-author) +- [Largest files with lowest number of authors](#largest-files-with-lowest-number-of-authors) +- [Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories](#commits-and-lines-of-code-distribution-by-time-by-weekday-by-author-for-specific-subdirectories) +- [Matrix of authors that shows what authors tends to rewrite another authors code](#matrix-of-authors-that-shows-what-authors-tends-to-rewrite-another-authors-code) +- [Who is the highest percentage contributor per day of week?](#who-is-the-highest-percentage-contributor-per-day-of-week) +- [Distribution of code age across repository](#distribution-of-code-age-across-repository) +- [What percentage of code for an author has been removed by other authors?](#what-percentage-of-code-for-an-author-has-been-removed-by-other-authors) +- [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) +- [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) +- [History of subdirectory/file - number of lines, commits and contributors over time](#history-of-subdirectoryfile---number-of-lines-commits-and-contributors-over-time) +- [Files sorted by average code age](#files-sorted-by-average-code-age) +- [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) +- [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) +- [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) +- [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) +- [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) +- [Most consecutive days of commits by an author](#most-consecutive-days-of-commits-by-an-author) +- [Line by line commit history of a file](#line-by-line-commit-history-of-a-file) +- [Show Git blame for a file](#git-blame) + +These queries are of approximately increasing complexity vs. the tool's arbitrary order. + +## History of a single file + +The simplest of queries. Here we look at all commit messages for the `StorageReplicatedMergeTree.cpp`. Since these are likely more interesting, we sort by the most recent messages first. + +```sql +SELECT + time, + substring(commit_hash, 1, 11) AS commit, + change_type, + author, + path, + old_path, + lines_added, + lines_deleted, + commit_message +FROM git.file_changes +WHERE path = 'src/Storages/StorageReplicatedMergeTree.cpp' +ORDER BY time DESC +LIMIT 10 + +┌────────────────time─┬─commit──────┬─change_type─┬─author─────────────┬─path────────────────────────────────────────┬─old_path─┬─lines_added─┬─lines_deleted─┬─commit_message───────────────────────────────────┐ +│ 2022-10-30 16:30:51 │ c68ab231f91 │ Modify │ Alexander Tokmakov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 13 │ 10 │ fix accessing part in Deleting state │ +│ 2022-10-23 16:24:20 │ b40d9200d20 │ Modify │ Anton Popov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 28 │ 30 │ better semantic of constsness of DataPartStorage │ +│ 2022-10-23 01:23:15 │ 56e5daba0c9 │ Modify │ Anton Popov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 28 │ 44 │ remove DataPartStorageBuilder │ +│ 2022-10-21 13:35:37 │ 851f556d65a │ Modify │ Igor Nikonov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 3 │ 2 │ Remove unused parameter │ +│ 2022-10-21 13:02:52 │ 13d31eefbc3 │ Modify │ Igor Nikonov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 4 │ 4 │ Replicated merge tree polishing │ +│ 2022-10-21 12:25:19 │ 4e76629aafc │ Modify │ Azat Khuzhin │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 3 │ 2 │ Fixes for -Wshorten-64-to-32 │ +│ 2022-10-19 13:59:28 │ 05e6b94b541 │ Modify │ Antonio Andelic │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 4 │ 0 │ Polishing │ +│ 2022-10-19 13:34:20 │ e5408aac991 │ Modify │ Antonio Andelic │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 3 │ 53 │ Simplify logic │ +│ 2022-10-18 15:36:11 │ 7befe2825c9 │ Modify │ Alexey Milovidov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 2 │ 2 │ Update StorageReplicatedMergeTree.cpp │ +│ 2022-10-18 15:35:44 │ 0623ad4e374 │ Modify │ Alexey Milovidov │ src/Storages/StorageReplicatedMergeTree.cpp │ │ 1 │ 1 │ Update StorageReplicatedMergeTree.cpp │ +└─────────────────────┴─────────────┴─────────────┴────────────────────┴─────────────────────────────────────────────┴──────────┴─────────────┴───────────────┴──────────────────────────────────────────────────┘ + +10 rows in set. Elapsed: 0.006 sec. Processed 12.10 thousand rows, 1.60 MB (1.93 million rows/s., 255.40 MB/s.) +``` + +We can also review the line changes, excluding renames i.e. we won't show changes before a rename event when the file existed under a different name: + +```sql +SELECT + time, + substring(commit_hash, 1, 11) AS commit, + sign, + line_number_old, + line_number_new, + author, + line +FROM git.line_changes +WHERE path = 'src/Storages/StorageReplicatedMergeTree.cpp' +ORDER BY line_number_new ASC +LIMIT 10 + +┌────────────────time─┬─commit──────┬─sign─┬─line_number_old─┬─line_number_new─┬─author───────────┬─line──────────────────────────────────────────────────┐ +│ 2020-04-16 02:06:10 │ cdeda4ab915 │ -1 │ 1 │ 1 │ Alexey Milovidov │ #include │ +│ 2020-04-16 02:06:10 │ cdeda4ab915 │ 1 │ 2 │ 1 │ Alexey Milovidov │ #include │ +│ 2020-04-16 02:06:10 │ cdeda4ab915 │ 1 │ 2 │ 2 │ Alexey Milovidov │ │ +│ 2021-05-03 23:46:51 │ 02ce9cc7254 │ -1 │ 3 │ 2 │ Alexey Milovidov │ #include │ +│ 2021-05-27 22:21:02 │ e2f29b9df02 │ -1 │ 3 │ 2 │ s-kat │ #include │ +│ 2022-10-03 22:30:50 │ 210882b9c4d │ 1 │ 2 │ 3 │ alesapin │ #include │ +│ 2022-10-23 16:24:20 │ b40d9200d20 │ 1 │ 2 │ 3 │ Anton Popov │ #include │ +│ 2021-06-20 09:24:43 │ 4c391f8e994 │ 1 │ 2 │ 3 │ Mike Kot │ #include "Common/hex.h" │ +│ 2021-12-29 09:18:56 │ 8112a712336 │ -1 │ 6 │ 5 │ avogar │ #include │ +│ 2022-04-21 20:19:13 │ 9133e398b8c │ 1 │ 11 │ 12 │ Nikolai Kochetov │ #include │ +└─────────────────────┴─────────────┴──────┴─────────────────┴─────────────────┴──────────────────┴───────────────────────────────────────────────────────┘ + +10 rows in set. Elapsed: 0.258 sec. Processed 7.54 million rows, 654.92 MB (29.24 million rows/s., 2.54 GB/s.) +``` + +Note a more complex variant of this query exists where we find the [line-by-line commit history of a file](#line-by-line-commit-history-of-a-file) considering renames. + +## Find the current active files + +This is important for later analysis when we only want to consider the current files in the repository. We estimate this set as the files which haven't been renamed or deleted (and then re-added/re-named). + +**Note there appears to have been a broken commit history in relation to files under the `dbms`, `libs`, `tests/testflows/` directories during their renames. We also thus exclude these.** + +```sql +SELECT path +FROM +( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path +) +GROUP BY path +HAVING (argMax(change_type, last_time) != 2) AND NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)') ORDER BY path +LIMIT 10 + +┌─path────────────────────────────────────────────────────────────┐ +│ tests/queries/0_stateless/01054_random_printable_ascii_ubsan.sh │ +│ tests/queries/0_stateless/02247_read_bools_as_numbers_json.sh │ +│ tests/performance/file_table_function.xml │ +│ tests/queries/0_stateless/01902_self_aliases_in_columns.sql │ +│ tests/queries/0_stateless/01070_h3_get_base_cell.reference │ +│ src/Functions/ztest.cpp │ +│ src/Interpreters/InterpreterShowTablesQuery.h │ +│ src/Parsers/Kusto/ParserKQLStatement.h │ +│ tests/queries/0_stateless/00938_dataset_test.sql │ +│ src/Dictionaries/Embedded/GeodataProviders/Types.h │ +└─────────────────────────────────────────────────────────────────┘ + +10 rows in set. Elapsed: 0.085 sec. Processed 532.10 thousand rows, 8.68 MB (6.30 million rows/s., 102.64 MB/s.) +``` + +Note that this allows for files to be renamed and then re-renamed to their original values. First we aggregare `old_path` for a list of deleted files as a result of renaming. We union this with the last operation for every `path`. Finally, we filter this list to those where the final event is not a `Delete`. + +```sql +SELECT uniq(path) +FROM +( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)') ORDER BY path +) + +┌─uniq(path)─┐ +│ 18559 │ +└────────────┘ +1 row in set. Elapsed: 0.089 sec. Processed 532.10 thousand rows, 8.68 MB (6.01 million rows/s., 97.99 MB/s.) +``` + +Note that we skipped import of several directories during import i.e. + +`--skip-paths 'generated\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/'` + +Applying this pattern to `git list-files`, reports 18155. + +```bash +git ls-files | grep -v -E 'generated\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' | wc -l + 18155 +``` + +**Our current solution is therefore an estimate of the current files** + +The difference here is caused by a few factors: + +- A rename can occur alongside other modifications to the file. These are listed as separate events in file_changes but with the same time. The `argMax` function has no way of distinguishing these - it picks the first value. The natural ordering of the inserts (the only means of knowing the correct order) is not maintained across the union so modified events can be selected. For example, below the `src/Functions/geometryFromColumn.h` file has several modifications before being renamed to `src/Functions/geometryConverters.h`. Our current solution may pick a Modify event as the latest change causing `src/Functions/geometryFromColumn.h` to be retained. + + + ```sql + SELECT + change_type, + path, + old_path, + time, + commit_hash + FROM git.file_changes + WHERE (path = 'src/Functions/geometryFromColumn.h') OR (old_path = 'src/Functions/geometryFromColumn.h') + + ┌─change_type─┬─path───────────────────────────────┬─old_path───────────────────────────┬────────────────time─┬─commit_hash──────────────────────────────┐ + │ Add │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 9376b676e9a9bb8911b872e1887da85a45f7479d │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 6d59be5ea4768034f6526f7f9813062e0c369f7b │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 33acc2aa5dc091a7cb948f78c558529789b2bad8 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 78e0db268ceadc42f82bc63a77ee1a4da6002463 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 14a891057d292a164c4179bfddaef45a74eaf83a │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ d0d6e6953c2a2af9fb2300921ff96b9362f22edb │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ fe8382521139a58c0ba277eb848e88894658db66 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 3be3d5cde8788165bc0558f1e2a22568311c3103 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ afad9bf4d0a55ed52a3f55483bc0973456e10a56 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ e3290ecc78ca3ea82b49ebcda22b5d3a4df154e6 │ + │ Rename │ src/Functions/geometryConverters.h │ src/Functions/geometryFromColumn.h │ 2021-03-11 12:08:16 │ 125945769586baf6ffd15919b29565b1b2a63218 │ + └─────────────┴────────────────────────────────────┴────────────────────────────────────┴─────────────────────┴──────────────────────────────────────────┘ + 11 rows in set. Elapsed: 0.030 sec. Processed 266.05 thousand rows, 6.61 MB (8.89 million rows/s., 220.82 MB/s.) + ``` +- Broken commit history - missing delete events. Source and cause TBD. + +These differences shouldn't meaningfully impact our analysis. **We welcome improved versions of this query**. + +## List files with most modifications + +Limiting to current files, we consider the number of modifications to be the sum of deletes and additions. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + sum(lines_added) + sum(lines_deleted) AS modifications +FROM git.file_changes +WHERE (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) +GROUP BY path +ORDER BY modifications DESC +LIMIT 10 + +┌─path───────────────────────────────────────────────────┬─modifications─┐ +│ src/Storages/StorageReplicatedMergeTree.cpp │ 21871 │ +│ src/Storages/MergeTree/MergeTreeData.cpp │ 17709 │ +│ programs/client/Client.cpp │ 15882 │ +│ src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp │ 14249 │ +│ src/Interpreters/InterpreterSelectQuery.cpp │ 12636 │ +│ src/Parsers/ExpressionListParsers.cpp │ 11794 │ +│ src/Analyzer/QueryAnalysisPass.cpp │ 11760 │ +│ src/Coordination/KeeperStorage.cpp │ 10225 │ +│ src/Functions/FunctionsConversion.h │ 9247 │ +│ src/Parsers/ExpressionElementParsers.cpp │ 8197 │ +└────────────────────────────────────────────────────────┴───────────────┘ + +10 rows in set. Elapsed: 0.134 sec. Processed 798.15 thousand rows, 16.46 MB (5.95 million rows/s., 122.62 MB/s.) +``` + +## What day of the week do commits usually occur? + +```sql +SELECT day_of_week, count() as c FROM git.commits GROUP BY dayOfWeek(time) as day_of_week + +┌─day_of_week─┬─────c─┐ +│ 1 │ 10291 │ +│ 2 │ 10519 │ +│ 3 │ 10598 │ +│ 4 │ 10812 │ +│ 5 │ 9964 │ +│ 6 │ 4583 │ +│ 7 │ 5134 │ +└─────────────┴───────┘ +7 rows in set. Elapsed: 0.262 sec. Processed 62.78 thousand rows, 251.14 KB (239.73 thousand rows/s., 958.93 KB/s.) +``` + +This makes sense with some productivity drop-off on Fridays. Great to see people committing code at weekends! Big thanks to our contributors! + +## History of subdirectory/file - number of lines, commits and contributors over time + +This would produce a large query result that is unrealistic to show or visualize if unfiltered. We, therefore, allow a file or subdirectory to be filtered in the following example. Here we group by week using the `toStartOfWeek` function - adapt as required. + +```sql +SELECT + week, + sum(lines_added) AS lines_added, + sum(lines_deleted) AS lines_deleted, + uniq(commit_hash) AS num_commits, + uniq(author) AS authors +FROM git.file_changes +WHERE path LIKE 'src/Storages%' +GROUP BY toStartOfWeek(time) AS week +ORDER BY week ASC +LIMIT 10 + +┌───────week─┬─lines_added─┬─lines_deleted─┬─num_commits─┬─authors─┐ +│ 2020-03-29 │ 49 │ 35 │ 4 │ 3 │ +│ 2020-04-05 │ 940 │ 601 │ 55 │ 14 │ +│ 2020-04-12 │ 1472 │ 607 │ 32 │ 11 │ +│ 2020-04-19 │ 917 │ 841 │ 39 │ 12 │ +│ 2020-04-26 │ 1067 │ 626 │ 36 │ 10 │ +│ 2020-05-03 │ 514 │ 435 │ 27 │ 10 │ +│ 2020-05-10 │ 2552 │ 537 │ 48 │ 12 │ +│ 2020-05-17 │ 3585 │ 1913 │ 83 │ 9 │ +│ 2020-05-24 │ 2851 │ 1812 │ 74 │ 18 │ +│ 2020-05-31 │ 2771 │ 2077 │ 77 │ 16 │ +└────────────┴─────────────┴───────────────┴─────────────┴─────────┘ +10 rows in set. Elapsed: 0.043 sec. Processed 266.05 thousand rows, 15.85 MB (6.12 million rows/s., 364.61 MB/s.) +``` + +This data visualizes well. Below we use Superset. + +**For lines added and deleted:** + +image + +**For commits and authors:** + +image + +## List files with maximum number of authors + +Limit to current files only. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + uniq(author) AS num_authors +FROM git.file_changes +WHERE path IN (current_files) +GROUP BY path +ORDER BY num_authors DESC +LIMIT 10 + +┌─path────────────────────────────────────────┬─num_authors─┐ +│ src/Core/Settings.h │ 127 │ +│ CMakeLists.txt │ 96 │ +│ .gitmodules │ 85 │ +│ src/Storages/MergeTree/MergeTreeData.cpp │ 72 │ +│ src/CMakeLists.txt │ 71 │ +│ programs/server/Server.cpp │ 70 │ +│ src/Interpreters/Context.cpp │ 64 │ +│ src/Storages/StorageReplicatedMergeTree.cpp │ 63 │ +│ src/Common/ErrorCodes.cpp │ 61 │ +│ src/Interpreters/InterpreterSelectQuery.cpp │ 59 │ +└─────────────────────────────────────────────┴─────────────┘ + +10 rows in set. Elapsed: 0.239 sec. Processed 798.15 thousand rows, 14.13 MB (3.35 million rows/s., 59.22 MB/s.) +``` + +## Oldest lines of code in the repository + +Limited to current files only. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + any(path) AS file_path, + line, + max(time) AS latest_change, + any(file_change_type) +FROM git.line_changes +WHERE path IN (current_files) +GROUP BY line +ORDER BY latest_change ASC +LIMIT 10 + +┌─file_path───────────────────────────────────┬─line────────────────────────────────────────────────────────┬───────latest_change─┬─any(file_change_type)─┐ +│ utils/compressor/test.sh │ ./compressor -d < compressor.snp > compressor2 │ 2011-06-17 22:19:39 │ Modify │ +│ utils/compressor/test.sh │ ./compressor < compressor > compressor.snp │ 2011-06-17 22:19:39 │ Modify │ +│ utils/compressor/test.sh │ ./compressor -d < compressor.qlz > compressor2 │ 2014-02-24 03:14:30 │ Add │ +│ utils/compressor/test.sh │ ./compressor < compressor > compressor.qlz │ 2014-02-24 03:14:30 │ Add │ +│ utils/config-processor/config-processor.cpp │ if (argc != 2) │ 2014-02-26 19:10:00 │ Add │ +│ utils/config-processor/config-processor.cpp │ std::cerr << "std::exception: " << e.what() << std::endl; │ 2014-02-26 19:10:00 │ Add │ +│ utils/config-processor/config-processor.cpp │ std::cerr << "Exception: " << e.displayText() << std::endl; │ 2014-02-26 19:10:00 │ Add │ +│ utils/config-processor/config-processor.cpp │ Poco::XML::DOMWriter().writeNode(std::cout, document); │ 2014-02-26 19:10:00 │ Add │ +│ utils/config-processor/config-processor.cpp │ std::cerr << "Some exception" << std::endl; │ 2014-02-26 19:10:00 │ Add │ +│ utils/config-processor/config-processor.cpp │ std::cerr << "usage: " << argv[0] << " path" << std::endl; │ 2014-02-26 19:10:00 │ Add │ +└─────────────────────────────────────────────┴─────────────────────────────────────────────────────────────┴─────────────────────┴───────────────────────┘ + +10 rows in set. Elapsed: 1.101 sec. Processed 8.07 million rows, 905.86 MB (7.33 million rows/s., 823.13 MB/s.) +``` + +## Files with longest history + +Limited to current files only. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + count() AS c, + path, + max(time) AS latest_change +FROM git.file_changes +WHERE path IN (current_files) +GROUP BY path +ORDER BY c DESC +LIMIT 10 + +┌───c─┬─path────────────────────────────────────────┬───────latest_change─┐ +│ 790 │ src/Storages/StorageReplicatedMergeTree.cpp │ 2022-10-30 16:30:51 │ +│ 788 │ src/Storages/MergeTree/MergeTreeData.cpp │ 2022-11-04 09:26:44 │ +│ 752 │ src/Core/Settings.h │ 2022-10-25 11:35:25 │ +│ 749 │ CMakeLists.txt │ 2022-10-05 21:00:49 │ +│ 575 │ src/Interpreters/InterpreterSelectQuery.cpp │ 2022-11-01 10:20:10 │ +│ 563 │ CHANGELOG.md │ 2022-10-27 08:19:50 │ +│ 491 │ src/Interpreters/Context.cpp │ 2022-10-25 12:26:29 │ +│ 437 │ programs/server/Server.cpp │ 2022-10-21 12:25:19 │ +│ 375 │ programs/client/Client.cpp │ 2022-11-03 03:16:55 │ +│ 350 │ src/CMakeLists.txt │ 2022-10-24 09:22:37 │ +└─────┴─────────────────────────────────────────────┴─────────────────────┘ + +10 rows in set. Elapsed: 0.124 sec. Processed 798.15 thousand rows, 14.71 MB (6.44 million rows/s., 118.61 MB/s.) +``` + +Our core data structure, the Merge Tree, is obviously under constant evolution with a long history of edits! + +## Distribution of contributors with respect to docs and code over the month + +**During data capture the changes on the `docs/` folder have been filtered out so this is an estimate only** + +Do we write more docs at certain times of the month e.g., around release dates? We can use the `countIf` function to compute a simple ratio, visualizing the result using the `bar` function. + +```sql +SELECT + day, + bar(docs_ratio * 1000, 0, 100, 100) AS bar +FROM +( + SELECT + day, + countIf(file_extension IN ('h', 'cpp', 'sql', 'md')) AS code, + countIf(file_extension = 'md') AS docs, + docs / (code + docs) AS docs_ratio + FROM git.line_changes + WHERE (sign = 1) AND (file_extension IN ('h', 'cpp', 'sql', 'md')) + GROUP BY dayOfMonth(time) AS day +) + +┌─day─┬─bar──────────────────────────────────────────────────────────┐ +│ 1 │ ██████████████████████████████████▎ │ +│ 2 │ ███████████████████████▏ │ +│ 3 │ ███████████████████████████████▋ │ +│ 4 │ ████████████▊ │ +│ 5 │ ████████████████████▊ │ +│ 6 │ ███████▊ │ +│ 7 │ ███▋ │ +│ 8 │ ████████▍ │ +│ 9 │ ██████████████ │ +│ 10 │ ████████████████▋ │ +│ 11 │ █████████████▏ │ +│ 12 │ ██████████████████████████████████▌ │ +│ 13 │ ████████████████████████████▌ │ +│ 14 │ ██████▋ │ +│ 15 │ ████████████████████████████████████████▎ │ +│ 16 │ ██████████▏ │ +│ 17 │ █████████████████████████████████████▏ │ +│ 18 │ ████████████████████████████████▍ │ +│ 19 │ ██████████▊ │ +│ 20 │ ████████████████████████████████▋ │ +│ 21 │ █████ │ +│ 22 │ ███████████████████████▏ │ +│ 23 │ ██████████████████████████▋ │ +│ 24 │ ███████▌ │ +│ 25 │ █████████████████████████████████▏ │ +│ 26 │ ███████████ │ +│ 27 │ ███████████████████████████████████████████████████████████▎ │ +│ 28 │ █████████████████████████████████████████████████▌ │ +│ 29 │ ███▌ │ +│ 30 │ ██████████████████████████████████████▋ │ +│ 31 │ ████████████████████████████████▏ │ +└─────┴──────────────────────────────────────────────────────────────┘ + +31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.56 million rows/s., 949.58 MB/s.) +``` + +Maybe a little more near the end of the month, but overall we keep a good even distribution. + +## Authors with the most diverse impact + +We consider diversity here to be the number of unique files an author has contributed to. + +```sql +SELECT + author, + uniq(path) AS num_files +FROM git.file_changes +WHERE (change_type IN ('Add', 'Modify')) AND (file_extension IN ('h', 'cpp', 'sql')) +GROUP BY author +ORDER BY num_files DESC +LIMIT 10 + +┌─author─────────────┬─num_files─┐ +│ Alexey Milovidov │ 8433 │ +│ Nikolai Kochetov │ 3257 │ +│ Vitaly Baranov │ 2316 │ +│ Maksim Kita │ 2172 │ +│ Azat Khuzhin │ 1988 │ +│ alesapin │ 1818 │ +│ Alexander Tokmakov │ 1751 │ +│ Amos Bird │ 1641 │ +│ Ivan │ 1629 │ +│ alexey-milovidov │ 1581 │ +└────────────────────┴───────────┘ + +10 rows in set. Elapsed: 0.041 sec. Processed 266.05 thousand rows, 4.92 MB (6.56 million rows/s., 121.21 MB/s.) +``` + +## Favorite files for an author + +Here we select our founder [Alexey Milovidov](https://github.com/alexey-milovidov) and limit our analysis to current files. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + count() AS c +FROM git.file_changes +WHERE (author = 'alexey-milovidov') AND (path IN (current_files)) +GROUP BY path +ORDER BY c DESC +LIMIT 10 + +┌─path────────────────────────────────────────────┬───c─┐ +│ CHANGELOG.md │ 174 │ +│ CMakeLists.txt │ 22 │ +│ src/Common/HashTable/HashTable.h │ 8 │ +│ .github/PULL_REQUEST_TEMPLATE.md │ 8 │ +│ src/Core/Settings.h │ 8 │ +│ src/Storages/StorageReplicatedMergeTree.cpp │ 7 │ +│ README.md │ 7 │ +│ docker/test/fuzzer/run-fuzzer.sh │ 7 │ +│ programs/install/Install.cpp │ 7 │ +│ src/Dictionaries/ExecutableDictionarySource.cpp │ 6 │ +└─────────────────────────────────────────────────┴─────┘ + +10 rows in set. Elapsed: 0.106 sec. Processed 798.15 thousand rows, 13.97 MB (7.51 million rows/s., 131.41 MB/s.) +``` + +This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the basename of the file to identify his popular files - this allows for renames and should focus on code contributions. + + +```sql +SELECT + base, + count() AS c +FROM git.file_changes +WHERE (author = 'alexey-milovidov') AND (file_extension IN ('h', 'cpp', 'sql')) +GROUP BY basename(path) AS base +ORDER BY c DESC +LIMIT 10 + +┌─base───────────────────────────┬──c─┐ +│ StorageReplicatedMergeTree.cpp │ 22 │ +│ Settings.h │ 22 │ +│ InterpreterSelectQuery.cpp │ 19 │ +│ MergeTreeData.cpp │ 18 │ +│ Client.cpp │ 17 │ +│ Context.cpp │ 17 │ +│ Server.cpp │ 12 │ +│ ExecutableDictionarySource.cpp │ 12 │ +│ ExpressionAnalyzer.cpp │ 12 │ +│ PODArray.h │ 12 │ +└────────────────────────────────┴────┘ +10 rows in set. Elapsed: 0.032 sec. Processed 266.05 thousand rows, 5.68 MB (8.22 million rows/s., 175.50 MB/s.) +``` + +This is maybe more reflective of his areas of interest. + +## Largest files with lowest number of authors + +For this, we first need to identify the largest files. Estimating this via a full file reconstruction, for every file, from the history of commits will be very expensive! + +To estimate, assuming we restrict to current files, we sum line additions and subtract deletions. We can then compute a ratio of length to the number of authors. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + sum(lines_added) - sum(lines_deleted) AS num_lines, + uniqExact(author) AS num_authors, + num_lines / num_authors AS lines_author_ratio +FROM git.file_changes +WHERE path IN (current_files) +GROUP BY path +ORDER BY lines_author_ratio DESC +LIMIT 10 + +┌─path──────────────────────────────────────────────────────────────────┬─num_lines─┬─num_authors─┬─lines_author_ratio─┐ +│ src/Common/ClassificationDictionaries/emotional_dictionary_rus.txt │ 148590 │ 1 │ 148590 │ +│ src/Functions/ClassificationDictionaries/emotional_dictionary_rus.txt │ 55533 │ 1 │ 55533 │ +│ src/Functions/ClassificationDictionaries/charset_freq.txt │ 35722 │ 1 │ 35722 │ +│ src/Common/ClassificationDictionaries/charset_freq.txt │ 35722 │ 1 │ 35722 │ +│ tests/integration/test_storage_meilisearch/movies.json │ 19549 │ 1 │ 19549 │ +│ tests/queries/0_stateless/02364_multiSearch_function_family.reference │ 12874 │ 1 │ 12874 │ +│ src/Functions/ClassificationDictionaries/programming_freq.txt │ 9434 │ 1 │ 9434 │ +│ src/Common/ClassificationDictionaries/programming_freq.txt │ 9434 │ 1 │ 9434 │ +│ tests/performance/explain_ast.xml │ 5911 │ 1 │ 5911 │ +│ src/Analyzer/QueryAnalysisPass.cpp │ 5686 │ 1 │ 5686 │ +└───────────────────────────────────────────────────────────────────────┴───────────┴─────────────┴────────────────────┘ + +10 rows in set. Elapsed: 0.138 sec. Processed 798.15 thousand rows, 16.57 MB (5.79 million rows/s., 120.11 MB/s.) +``` + +Text dictionaries aren't maybe realistic, so lets restrict to code only via a file extension filter! + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + sum(lines_added) - sum(lines_deleted) AS num_lines, + uniqExact(author) AS num_authors, + num_lines / num_authors AS lines_author_ratio +FROM git.file_changes +WHERE (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) +GROUP BY path +ORDER BY lines_author_ratio DESC +LIMIT 10 + +┌─path──────────────────────────────────┬─num_lines─┬─num_authors─┬─lines_author_ratio─┐ +│ src/Analyzer/QueryAnalysisPass.cpp │ 5686 │ 1 │ 5686 │ +│ src/Analyzer/QueryTreeBuilder.cpp │ 880 │ 1 │ 880 │ +│ src/Planner/Planner.cpp │ 873 │ 1 │ 873 │ +│ src/Backups/RestorerFromBackup.cpp │ 869 │ 1 │ 869 │ +│ utils/memcpy-bench/FastMemcpy.h │ 770 │ 1 │ 770 │ +│ src/Planner/PlannerActionsVisitor.cpp │ 765 │ 1 │ 765 │ +│ src/Functions/sphinxstemen.cpp │ 728 │ 1 │ 728 │ +│ src/Planner/PlannerJoinTree.cpp │ 708 │ 1 │ 708 │ +│ src/Planner/PlannerJoins.cpp │ 695 │ 1 │ 695 │ +│ src/Analyzer/QueryNode.h │ 607 │ 1 │ 607 │ +└───────────────────────────────────────┴───────────┴─────────────┴────────────────────┘ +10 rows in set. Elapsed: 0.140 sec. Processed 798.15 thousand rows, 16.84 MB (5.70 million rows/s., 120.32 MB/s.) +``` + +There is some recency bias in this - newer files have fewer opportunities for commits. What about if we restrict to files at least 1 yr old? + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + min(time) AS min_date, + path, + sum(lines_added) - sum(lines_deleted) AS num_lines, + uniqExact(author) AS num_authors, + num_lines / num_authors AS lines_author_ratio +FROM git.file_changes +WHERE (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) +GROUP BY path +HAVING min_date <= (now() - toIntervalYear(1)) +ORDER BY lines_author_ratio DESC +LIMIT 10 + +┌────────────min_date─┬─path───────────────────────────────────────────────────────────┬─num_lines─┬─num_authors─┬─lines_author_ratio─┐ +│ 2021-03-08 07:00:54 │ utils/memcpy-bench/FastMemcpy.h │ 770 │ 1 │ 770 │ +│ 2021-05-04 13:47:34 │ src/Functions/sphinxstemen.cpp │ 728 │ 1 │ 728 │ +│ 2021-03-14 16:52:51 │ utils/memcpy-bench/glibc/dwarf2.h │ 592 │ 1 │ 592 │ +│ 2021-03-08 09:04:52 │ utils/memcpy-bench/FastMemcpy_Avx.h │ 496 │ 1 │ 496 │ +│ 2020-10-19 01:10:50 │ tests/queries/0_stateless/01518_nullable_aggregate_states2.sql │ 411 │ 1 │ 411 │ +│ 2020-11-24 14:53:34 │ programs/server/GRPCHandler.cpp │ 399 │ 1 │ 399 │ +│ 2021-03-09 14:10:28 │ src/DataTypes/Serializations/SerializationSparse.cpp │ 363 │ 1 │ 363 │ +│ 2021-08-20 15:06:57 │ src/Functions/vectorFunctions.cpp │ 1327 │ 4 │ 331.75 │ +│ 2020-08-04 03:26:23 │ src/Interpreters/MySQL/CreateQueryConvertVisitor.cpp │ 311 │ 1 │ 311 │ +│ 2020-11-06 15:45:13 │ src/Storages/Rocksdb/StorageEmbeddedRocksdb.cpp │ 611 │ 2 │ 305.5 │ +└─────────────────────┴────────────────────────────────────────────────────────────────┴───────────┴─────────────┴────────────────────┘ + +10 rows in set. Elapsed: 0.143 sec. Processed 798.15 thousand rows, 18.00 MB (5.58 million rows/s., 125.87 MB/s.) +``` + +## Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories + +We interpret this as the number of lines added and removed by the day of the week. In this case, we focus on the [Functions directory](https://github.com/ClickHouse/ClickHouse/tree/master/src/Functions) + +```sql +SELECT + dayOfWeek, + uniq(commit_hash) AS commits, + sum(lines_added) AS lines_added, + sum(lines_deleted) AS lines_deleted +FROM git.file_changes +WHERE path LIKE 'src/Functions%' +GROUP BY toDayOfWeek(time) AS dayOfWeek + +┌─dayOfWeek─┬─commits─┬─lines_added─┬─lines_deleted─┐ +│ 1 │ 476 │ 24619 │ 15782 │ +│ 2 │ 434 │ 18098 │ 9938 │ +│ 3 │ 496 │ 26562 │ 20883 │ +│ 4 │ 587 │ 65674 │ 18862 │ +│ 5 │ 504 │ 85917 │ 14518 │ +│ 6 │ 314 │ 13604 │ 10144 │ +│ 7 │ 294 │ 11938 │ 6451 │ +└───────────┴─────────┴─────────────┴───────────────┘ + +7 rows in set. Elapsed: 0.034 sec. Processed 266.05 thousand rows, 14.66 MB (7.73 million rows/s., 425.56 MB/s.) +``` + +And by time of day, + +```sql +SELECT + hourOfDay, + uniq(commit_hash) AS commits, + sum(lines_added) AS lines_added, + sum(lines_deleted) AS lines_deleted +FROM git.file_changes +WHERE path LIKE 'src/Functions%' +GROUP BY toHour(time) AS hourOfDay + +┌─hourOfDay─┬─commits─┬─lines_added─┬─lines_deleted─┐ +│ 0 │ 71 │ 4169 │ 3404 │ +│ 1 │ 90 │ 2174 │ 1927 │ +│ 2 │ 65 │ 2343 │ 1515 │ +│ 3 │ 76 │ 2552 │ 493 │ +│ 4 │ 62 │ 1480 │ 1304 │ +│ 5 │ 38 │ 1644 │ 253 │ +│ 6 │ 104 │ 4434 │ 2979 │ +│ 7 │ 117 │ 4171 │ 1678 │ +│ 8 │ 106 │ 4604 │ 4673 │ +│ 9 │ 135 │ 60550 │ 2678 │ +│ 10 │ 149 │ 6133 │ 3482 │ +│ 11 │ 182 │ 8040 │ 3833 │ +│ 12 │ 209 │ 29428 │ 15040 │ +│ 13 │ 187 │ 10204 │ 5491 │ +│ 14 │ 204 │ 9028 │ 6060 │ +│ 15 │ 231 │ 15179 │ 10077 │ +│ 16 │ 196 │ 9568 │ 5925 │ +│ 17 │ 138 │ 4941 │ 3849 │ +│ 18 │ 123 │ 4193 │ 3036 │ +│ 19 │ 165 │ 8817 │ 6646 │ +│ 20 │ 140 │ 3749 │ 2379 │ +│ 21 │ 132 │ 41585 │ 4182 │ +│ 22 │ 85 │ 4094 │ 3955 │ +│ 23 │ 100 │ 3332 │ 1719 │ +└───────────┴─────────┴─────────────┴───────────────┘ + +24 rows in set. Elapsed: 0.039 sec. Processed 266.05 thousand rows, 14.66 MB (6.77 million rows/s., 372.89 MB/s.) +``` + +This distribution makes sense given most of our development team is in Amsterdam. The `bar` functions helps us visualize these distributions: + +```sql +SELECT + hourOfDay, + bar(commits, 0, 400, 50) AS commits, + bar(lines_added, 0, 30000, 50) AS lines_added, + bar(lines_deleted, 0, 15000, 50) AS lines_deleted +FROM +( + SELECT + hourOfDay, + uniq(commit_hash) AS commits, + sum(lines_added) AS lines_added, + sum(lines_deleted) AS lines_deleted + FROM git.file_changes + WHERE path LIKE 'src/Functions%' + GROUP BY toHour(time) AS hourOfDay +) + +┌─hourOfDay─┬─commits───────────────────────┬─lines_added────────────────────────────────────────┬─lines_deleted──────────────────────────────────────┐ +│ 0 │ ████████▊ │ ██████▊ │ ███████████▎ │ +│ 1 │ ███████████▎ │ ███▌ │ ██████▍ │ +│ 2 │ ████████ │ ███▊ │ █████ │ +│ 3 │ █████████▌ │ ████▎ │ █▋ │ +│ 4 │ ███████▋ │ ██▍ │ ████▎ │ +│ 5 │ ████▋ │ ██▋ │ ▋ │ +│ 6 │ █████████████ │ ███████▍ │ █████████▊ │ +│ 7 │ ██████████████▋ │ ██████▊ │ █████▌ │ +│ 8 │ █████████████▎ │ ███████▋ │ ███████████████▌ │ +│ 9 │ ████████████████▊ │ ██████████████████████████████████████████████████ │ ████████▊ │ +│ 10 │ ██████████████████▋ │ ██████████▏ │ ███████████▌ │ +│ 11 │ ██████████████████████▋ │ █████████████▍ │ ████████████▋ │ +│ 12 │ ██████████████████████████ │ █████████████████████████████████████████████████ │ ██████████████████████████████████████████████████ │ +│ 13 │ ███████████████████████▍ │ █████████████████ │ ██████████████████▎ │ +│ 14 │ █████████████████████████▌ │ ███████████████ │ ████████████████████▏ │ +│ 15 │ ████████████████████████████▊ │ █████████████████████████▎ │ █████████████████████████████████▌ │ +│ 16 │ ████████████████████████▌ │ ███████████████▊ │ ███████████████████▋ │ +│ 17 │ █████████████████▎ │ ████████▏ │ ████████████▋ │ +│ 18 │ ███████████████▍ │ ██████▊ │ ██████████ │ +│ 19 │ ████████████████████▋ │ ██████████████▋ │ ██████████████████████▏ │ +│ 20 │ █████████████████▌ │ ██████▏ │ ███████▊ │ +│ 21 │ ████████████████▌ │ ██████████████████████████████████████████████████ │ █████████████▊ │ +│ 22 │ ██████████▋ │ ██████▋ │ █████████████▏ │ +│ 23 │ ████████████▌ │ █████▌ │ █████▋ │ +└───────────┴───────────────────────────────┴────────────────────────────────────────────────────┴────────────────────────────────────────────────────┘ + +24 rows in set. Elapsed: 0.038 sec. Processed 266.05 thousand rows, 14.66 MB (7.09 million rows/s., 390.69 MB/s.) +``` + +## Matrix of authors that shows what authors tends to rewrite another authors code + +The `sign = -1` indicates a code deletion. We exclude punctuation and the insertion of empty lines. + +```sql +SELECT + prev_author, + author, + count() AS c +FROM git.line_changes +WHERE (sign = -1) AND (file_extension IN ('h', 'cpp')) AND (line_type NOT IN ('Punct', 'Empty')) AND (author != prev_author) AND (prev_author != '') +GROUP BY + prev_author, + author +ORDER BY c DESC +LIMIT 1 BY prev_author +LIMIT 20 + +┌─prev_author──────────┬─author───────────┬─────c─┐ +│ Ivan │ Alexey Milovidov │ 18554 │ +│ Alexey Arno │ Alexey Milovidov │ 18475 │ +│ Michael Kolupaev │ Alexey Milovidov │ 14135 │ +│ Alexey Milovidov │ Nikolai Kochetov │ 13435 │ +│ Andrey Mironov │ Alexey Milovidov │ 10418 │ +│ proller │ Alexey Milovidov │ 7280 │ +│ Nikolai Kochetov │ Alexey Milovidov │ 6806 │ +│ alexey-milovidov │ Alexey Milovidov │ 5027 │ +│ Vitaliy Lyudvichenko │ Alexey Milovidov │ 4390 │ +│ Amos Bird │ Ivan Lezhankin │ 3125 │ +│ f1yegor │ Alexey Milovidov │ 3119 │ +│ Pavel Kartavyy │ Alexey Milovidov │ 3087 │ +│ Alexey Zatelepin │ Alexey Milovidov │ 2978 │ +│ alesapin │ Alexey Milovidov │ 2949 │ +│ Sergey Fedorov │ Alexey Milovidov │ 2727 │ +│ Ivan Lezhankin │ Alexey Milovidov │ 2618 │ +│ Vasily Nemkov │ Alexey Milovidov │ 2547 │ +│ Alexander Tokmakov │ Alexey Milovidov │ 2493 │ +│ Nikita Vasilev │ Maksim Kita │ 2420 │ +│ Anton Popov │ Amos Bird │ 2127 │ +└──────────────────────┴──────────────────┴───────┘ + +20 rows in set. Elapsed: 0.098 sec. Processed 7.54 million rows, 42.16 MB (76.67 million rows/s., 428.99 MB/s.) +``` + +## Who is the highest percentage contributor per day of week? + +If we consider by just number of commits: + +```sql +SELECT + day_of_week, + author, + count() AS c +FROM git.commits +GROUP BY + dayOfWeek(time) AS day_of_week, + author +ORDER BY + day_of_week ASC, + c DESC +LIMIT 1 BY day_of_week + +┌─day_of_week─┬─author───────────┬────c─┐ +│ 1 │ Alexey Milovidov │ 2204 │ +│ 2 │ Alexey Milovidov │ 1588 │ +│ 3 │ Alexey Milovidov │ 1725 │ +│ 4 │ Alexey Milovidov │ 1915 │ +│ 5 │ Alexey Milovidov │ 1940 │ +│ 6 │ Alexey Milovidov │ 1851 │ +│ 7 │ Alexey Milovidov │ 2400 │ +└─────────────┴──────────────────┴──────┘ + +7 rows in set. Elapsed: 0.012 sec. Processed 62.78 thousand rows, 395.47 KB (5.44 million rows/s., 34.27 MB/s.) +``` + +Ok, some possible advantages here to the longest contributor - our founder Alexey. Lets limit our analysis to the last year. + + +```sql +SELECT + day_of_week, + author, + count() AS c +FROM git.commits +WHERE time > (now() - toIntervalYear(1)) +GROUP BY + dayOfWeek(time) AS day_of_week, + author +ORDER BY + day_of_week ASC, + c DESC +LIMIT 1 BY day_of_week + +┌─day_of_week─┬─author───────────┬───c─┐ +│ 1 │ Alexey Milovidov │ 198 │ +│ 2 │ alesapin │ 162 │ +│ 3 │ alesapin │ 163 │ +│ 4 │ Azat Khuzhin │ 166 │ +│ 5 │ alesapin │ 191 │ +│ 6 │ Alexey Milovidov │ 179 │ +│ 7 │ Alexey Milovidov │ 243 │ +└─────────────┴──────────────────┴─────┘ + +7 rows in set. Elapsed: 0.004 sec. Processed 21.82 thousand rows, 140.02 KB (4.88 million rows/s., 31.29 MB/s.) +``` + +This is still alittle simple and doesn't reflect people's work. + +A better metric might be who is the top contributor each day as a fraction of the total work performed in the last year. Note that we treat the deletion and adding code equally. + +```sql +SELECT + top_author.day_of_week, + top_author.author, + top_author.author_work / all_work.total_work AS top_author_percent +FROM +( + SELECT + day_of_week, + author, + sum(lines_added) + sum(lines_deleted) AS author_work + FROM git.file_changes + WHERE time > (now() - toIntervalYear(1)) + GROUP BY + author, + dayOfWeek(time) AS day_of_week + ORDER BY + day_of_week ASC, + author_work DESC + LIMIT 1 BY day_of_week +) AS top_author +INNER JOIN +( + SELECT + day_of_week, + sum(lines_added) + sum(lines_deleted) AS total_work + FROM git.file_changes + WHERE time > (now() - toIntervalYear(1)) + GROUP BY dayOfWeek(time) AS day_of_week +) AS all_work USING (day_of_week) + +┌─day_of_week─┬─author──────────────┬──top_author_percent─┐ +│ 1 │ Alexey Milovidov │ 0.3168282877768332 │ +│ 2 │ Mikhail f. Shiryaev │ 0.3523434231193969 │ +│ 3 │ vdimir │ 0.11859742484577324 │ +│ 4 │ Nikolay Degterinsky │ 0.34577318920318467 │ +│ 5 │ Alexey Milovidov │ 0.13208704423684223 │ +│ 6 │ Alexey Milovidov │ 0.18895257783624633 │ +│ 7 │ Robert Schulze │ 0.3617405888930302 │ +└─────────────┴─────────────────────┴─────────────────────┘ + +7 rows in set. Elapsed: 0.014 sec. Processed 106.12 thousand rows, 1.38 MB (7.61 million rows/s., 98.65 MB/s.) +``` + +## Distribution of code age across repository + +We limit the analysis to the current files. For brevity, we restrict the results to a depth of 2 with 5 files per root folder. Adjust as required. + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + concat(root, '/', sub_folder) AS folder, + round(avg(days_present)) AS avg_age_of_files, + min(days_present) AS min_age_files, + max(days_present) AS max_age_files, + count() AS c +FROM +( + SELECT + path, + dateDiff('day', min(time), toDate('2022-11-03')) AS days_present + FROM git.file_changes + WHERE (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY path +) +GROUP BY + splitByChar('/', path)[1] AS root, + splitByChar('/', path)[2] AS sub_folder +ORDER BY + root ASC, + c DESC +LIMIT 5 BY root + +┌─folder───────────────────────────┬─avg_age_of_files─┬─min_age_files─┬─max_age_files─┬────c─┐ +│ base/base │ 387 │ 201 │ 397 │ 84 │ +│ base/glibc-compatibility │ 887 │ 59 │ 993 │ 19 │ +│ base/consistent-hashing │ 993 │ 993 │ 993 │ 5 │ +│ base/widechar_width │ 993 │ 993 │ 993 │ 2 │ +│ base/consistent-hashing-sumbur │ 993 │ 993 │ 993 │ 2 │ +│ docker/test │ 1043 │ 1043 │ 1043 │ 1 │ +│ programs/odbc-bridge │ 835 │ 91 │ 945 │ 25 │ +│ programs/copier │ 587 │ 14 │ 945 │ 22 │ +│ programs/library-bridge │ 155 │ 47 │ 608 │ 21 │ +│ programs/disks │ 144 │ 62 │ 150 │ 14 │ +│ programs/server │ 874 │ 709 │ 945 │ 10 │ +│ rust/BLAKE3 │ 52 │ 52 │ 52 │ 1 │ +│ src/Functions │ 752 │ 0 │ 944 │ 809 │ +│ src/Storages │ 700 │ 8 │ 944 │ 736 │ +│ src/Interpreters │ 684 │ 3 │ 944 │ 490 │ +│ src/Processors │ 703 │ 44 │ 944 │ 482 │ +│ src/Common │ 673 │ 7 │ 944 │ 473 │ +│ tests/queries │ 674 │ -5 │ 945 │ 3777 │ +│ tests/integration │ 656 │ 132 │ 945 │ 4 │ +│ utils/memcpy-bench │ 601 │ 599 │ 605 │ 10 │ +│ utils/keeper-bench │ 570 │ 569 │ 570 │ 7 │ +│ utils/durability-test │ 793 │ 793 │ 793 │ 4 │ +│ utils/self-extracting-executable │ 143 │ 143 │ 143 │ 3 │ +│ utils/self-extr-exec │ 224 │ 224 │ 224 │ 2 │ +└──────────────────────────────────┴──────────────────┴───────────────┴───────────────┴──────┘ + +24 rows in set. Elapsed: 0.129 sec. Processed 798.15 thousand rows, 15.11 MB (6.19 million rows/s., 117.08 MB/s.) +``` + +## What percentage of code for an author has been removed by other authors? + +For this question, we need the number of lines written by an author divided by the total number of lines they have had removed by another contributor. + +```sql +SELECT + k, + written_code.c, + removed_code.c, + removed_code.c / written_code.c AS remove_ratio +FROM +( + SELECT + author AS k, + count() AS c + FROM git.line_changes + WHERE (sign = 1) AND (file_extension IN ('h', 'cpp')) AND (line_type NOT IN ('Punct', 'Empty')) + GROUP BY k +) AS written_code +INNER JOIN +( + SELECT + prev_author AS k, + count() AS c + FROM git.line_changes + WHERE (sign = -1) AND (file_extension IN ('h', 'cpp')) AND (line_type NOT IN ('Punct', 'Empty')) AND (author != prev_author) + GROUP BY k +) AS removed_code USING (k) +WHERE written_code.c > 1000 +ORDER BY remove_ratio DESC +LIMIT 10 + +┌─k──────────────────┬─────c─┬─removed_code.c─┬───────remove_ratio─┐ +│ Marek Vavruša │ 1458 │ 1318 │ 0.9039780521262003 │ +│ Ivan │ 32715 │ 27500 │ 0.8405930001528351 │ +│ artpaul │ 3450 │ 2840 │ 0.8231884057971014 │ +│ Silviu Caragea │ 1542 │ 1209 │ 0.7840466926070039 │ +│ Ruslan │ 1027 │ 802 │ 0.7809152872444012 │ +│ Tsarkova Anastasia │ 1755 │ 1364 │ 0.7772079772079772 │ +│ Vyacheslav Alipov │ 3526 │ 2727 │ 0.7733976176971072 │ +│ Marek Vavruša │ 1467 │ 1124 │ 0.7661895023858214 │ +│ f1yegor │ 7194 │ 5213 │ 0.7246316374756742 │ +│ kreuzerkrieg │ 3406 │ 2468 │ 0.724603640634175 │ +└────────────────────┴───────┴────────────────┴────────────────────┘ + +10 rows in set. Elapsed: 0.126 sec. Processed 15.07 million rows, 73.51 MB (119.97 million rows/s., 585.16 MB/s.) +``` + +## List files that were rewritten most number of times? + +We consider a rewrite to be when over 50% of the file are deleted, and 50% added. Adjust the query to your own interpretation of what constitutes this. + +The query is limited to the current files only. We list all file changes by grouping by `path` and `commit_hash`, returning the number of lines added and removed. Using a window function, we estimate the file's total size at any moment in time by performing a cumulative sum and estimating the impact of any change on file size as `lines added - lines removed`. Using this statistic, we can calculate the percentage of the file that has been added or removed for each change. Finally, we count the number of file changes that constitute a rewrite per file i.e. `(percent_add >= 0.5) AND (percent_delete >= 0.5) AND current_size > 50`. Note we require files to be more than 50 lines to avoid early contributions to a file being counted as a rewrite. This also avoids a bias to very small files, which may be more likely to be rewritten. + + +```sql +WITH + current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ), + file_changes AS + ( + SELECT + path, + max(time) AS max_time, + commit_hash, + any(lines_added) AS num_added, + any(lines_deleted) AS num_deleted, + any(change_type) AS type + FROM git.file_changes + WHERE (change_type IN ('Add', 'Modify')) AND (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY + path, + commit_hash + ORDER BY + path ASC, + max_time ASC + ), + rewrites AS + ( + SELECT + path, + commit_hash, + max_time, + type, + num_added, + num_deleted, + sum(num_added - num_deleted) OVER (PARTITION BY path ORDER BY max_time ASC) AS current_size, + if(current_size > 0, num_added / current_size, 0) AS percent_add, + if(current_size > 0, num_deleted / current_size, 0) AS percent_delete + FROM file_changes + ) +SELECT + path, + count() AS num_rewrites +FROM rewrites +WHERE (type = 'Modify') AND (percent_add >= 0.5) AND (percent_delete >= 0.5) AND (current_size > 50) +GROUP BY path +ORDER BY num_rewrites DESC +LIMIT 10 + +┌─path──────────────────────────────────────────────────┬─num_rewrites─┐ +│ src/Storages/WindowView/StorageWindowView.cpp │ 8 │ +│ src/Functions/array/arrayIndex.h │ 7 │ +│ src/Dictionaries/CacheDictionary.cpp │ 6 │ +│ src/Dictionaries/RangeHashedDictionary.cpp │ 5 │ +│ programs/client/Client.cpp │ 4 │ +│ src/Functions/polygonPerimeter.cpp │ 4 │ +│ src/Functions/polygonsEquals.cpp │ 4 │ +│ src/Functions/polygonsWithin.cpp │ 4 │ +│ src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp │ 4 │ +│ src/Functions/polygonsSymDifference.cpp │ 4 │ +└───────────────────────────────────────────────────────┴──────────────┘ + +10 rows in set. Elapsed: 0.299 sec. Processed 798.15 thousand rows, 31.52 MB (2.67 million rows/s., 105.29 MB/s.) +``` + +## What weekday does the code have the highest chance to stay in the repository? + +For this, we need to identify a line of code uniquely. We estimate this(as the same line may appear multiple times in a file) using the path and line contents. + +We query for lines added, joining this with the lines removed - filtering to cases where the latter occurs more recently than the former. This gives us the deleted lines from which we can compute the time between these two events. + +Finally, we aggregate across this dataset to compute the average number of days lines stay in the repository by the day of the week. + +```sql +SELECT + day_of_week_added, + count() AS num, + avg(days_present) AS avg_days_present +FROM +( + SELECT + added_code.line, + added_code.time AS added_day, + dateDiff('day', added_code.time, removed_code.time) AS days_present + FROM + ( + SELECT + path, + line, + max(time) AS time + FROM git.line_changes + WHERE (sign = 1) AND (line_type NOT IN ('Punct', 'Empty')) + GROUP BY + path, + line + ) AS added_code + INNER JOIN + ( + SELECT + path, + line, + max(time) AS time + FROM git.line_changes + WHERE (sign = -1) AND (line_type NOT IN ('Punct', 'Empty')) + GROUP BY + path, + line + ) AS removed_code USING (path, line) + WHERE removed_code.time > added_code.time +) +GROUP BY dayOfWeek(added_day) AS day_of_week_added + +┌─day_of_week_added─┬────num─┬───avg_days_present─┐ +│ 1 │ 171879 │ 193.81759260875384 │ +│ 2 │ 141448 │ 153.0931013517335 │ +│ 3 │ 161230 │ 137.61553681076722 │ +│ 4 │ 255728 │ 121.14149799787273 │ +│ 5 │ 203907 │ 141.60181847606998 │ +│ 6 │ 62305 │ 202.43449161383518 │ +│ 7 │ 70904 │ 220.0266134491707 │ +└───────────────────┴────────┴────────────────────┘ + +7 rows in set. Elapsed: 3.965 sec. Processed 15.07 million rows, 1.92 GB (3.80 million rows/s., 483.50 MB/s.) +``` + +## Files sorted by average code age + +This query uses the same principle as [What weekday does the code have the highest chance to stay in the repository](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - by aiming to uniquely identify a line of code using the path and line contents. +This allows us to identify the time between when a line was added and removed. We filter to current files and code only, however, and average the time for each file across lines. + +```sql +WITH + current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ), + lines_removed AS + ( + SELECT + added_code.path AS path, + added_code.line, + added_code.time AS added_day, + dateDiff('day', added_code.time, removed_code.time) AS days_present + FROM + ( + SELECT + path, + line, + max(time) AS time, + any(file_extension) AS file_extension + FROM git.line_changes + WHERE (sign = 1) AND (line_type NOT IN ('Punct', 'Empty')) + GROUP BY + path, + line + ) AS added_code + INNER JOIN + ( + SELECT + path, + line, + max(time) AS time + FROM git.line_changes + WHERE (sign = -1) AND (line_type NOT IN ('Punct', 'Empty')) + GROUP BY + path, + line + ) AS removed_code USING (path, line) + WHERE (removed_code.time > added_code.time) AND (path IN (current_files)) AND (file_extension IN ('h', 'cpp', 'sql')) + ) +SELECT + path, + avg(days_present) AS avg_code_age +FROM lines_removed +GROUP BY path +ORDER BY avg_code_age DESC +LIMIT 10 + +┌─path────────────────────────────────────────────────────────────┬──────avg_code_age─┐ +│ utils/corrector_utf8/corrector_utf8.cpp │ 1353.888888888889 │ +│ tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql │ 881 │ +│ src/Functions/replaceRegexpOne.cpp │ 861 │ +│ src/Functions/replaceRegexpAll.cpp │ 861 │ +│ src/Functions/replaceOne.cpp │ 861 │ +│ utils/zookeeper-remove-by-list/main.cpp │ 838.25 │ +│ tests/queries/0_stateless/01356_state_resample.sql │ 819 │ +│ tests/queries/0_stateless/01293_create_role.sql │ 819 │ +│ src/Functions/ReplaceStringImpl.h │ 810 │ +│ src/Interpreters/createBlockSelector.cpp │ 795 │ +└─────────────────────────────────────────────────────────────────┴───────────────────┘ + +10 rows in set. Elapsed: 3.134 sec. Processed 16.13 million rows, 1.83 GB (5.15 million rows/s., 582.99 MB/s.) +``` + +## Who tends to write more tests / cpp code / comments? + +There are a few ways we can address this question. Focusing on the code to test ratio, this query is relatively simple - count the number of contributions to folders containing `tests` and compute the ratio to total contributions. + +Note we limit to users with more than 20 changes to focus on regular committers and avoid a bias to one-off contributions. + +```sql +SELECT + author, + countIf((file_extension NOT IN ('h', 'cpp')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, + code / (code + test) AS ratio_code +FROM git.file_changes +GROUP BY author +HAVING code > 20 +ORDER BY code DESC +LIMIT 20 + +┌─author───────────────┬─test─┬──code─┬─────────ratio_code─┐ +│ Alexey Milovidov │ 9016 │ 41799 │ 0.8225720751746531 │ +│ Nikolai Kochetov │ 1376 │ 13361 │ 0.9066295718260161 │ +│ alesapin │ 3704 │ 8796 │ 0.70368 │ +│ kssenii │ 1257 │ 6769 │ 0.843384001993521 │ +│ Maksim Kita │ 1352 │ 5862 │ 0.8125866370945384 │ +│ Alexander Tokmakov │ 2011 │ 5727 │ 0.740113724476609 │ +│ Vitaly Baranov │ 2245 │ 5521 │ 0.7109193922225083 │ +│ Ivan Lezhankin │ 803 │ 4698 │ 0.8540265406289765 │ +│ Anton Popov │ 1056 │ 4346 │ 0.8045168456127361 │ +│ Ivan │ 4937 │ 4269 │ 0.4637193134912014 │ +│ Azat Khuzhin │ 2651 │ 3697 │ 0.5823881537492124 │ +│ Amos Bird │ 702 │ 2901 │ 0.8051623646960866 │ +│ proller │ 1662 │ 2377 │ 0.5885120079227532 │ +│ chertus │ 706 │ 2359 │ 0.7696574225122349 │ +│ alexey-milovidov │ 330 │ 2321 │ 0.8755186721991701 │ +│ Alexey Arno │ 207 │ 2310 │ 0.9177592371871275 │ +│ Vitaliy Lyudvichenko │ 479 │ 2283 │ 0.8265749456915279 │ +│ Robert Schulze │ 328 │ 2196 │ 0.8700475435816165 │ +│ CurtizJ │ 812 │ 2158 │ 0.7265993265993266 │ +│ Alexander Kuzmenkov │ 1198 │ 2092 │ 0.6358662613981763 │ +└──────────────────────┴──────┴───────┴────────────────────┘ + +20 rows in set. Elapsed: 0.034 sec. Processed 266.05 thousand rows, 4.65 MB (7.93 million rows/s., 138.76 MB/s.) +``` + +We can plot this distribution as a histogram. + +```sql +WITH ( + SELECT histogram(10)(ratio_code) AS hist + FROM + ( + SELECT + author, + countIf((file_extension NOT IN ('h', 'cpp')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, + code / (code + test) AS ratio_code + FROM git.file_changes + GROUP BY author + HAVING code > 20 + ) + ) AS hist +SELECT + arrayJoin(hist).1 AS lower, + arrayJoin(hist).2 AS upper, + bar(arrayJoin(hist).3, 0, 100, 20) AS bar + +┌────────────────lower─┬───────────────upper─┬─bar───────────┐ +│ 0.033562166285278416 │ 0.08337307389808846 │ ▏ │ +│ 0.08337307389808846 │ 0.17470067710547066 │ ▍ │ +│ 0.17470067710547066 │ 0.25909878535992237 │ ▍ │ +│ 0.25909878535992237 │ 0.3775444108257119 │ ▋ │ +│ 0.3775444108257119 │ 0.5108436376911997 │ ███▏ │ +│ 0.5108436376911997 │ 0.627700343453621 │ █████▋ │ +│ 0.627700343453621 │ 0.7417374581723406 │ ███████████▊ │ +│ 0.7417374581723406 │ 0.8467725898688147 │ ████████████▏ │ +│ 0.8467725898688147 │ 0.9427852671078976 │ ██████████▌ │ +│ 0.9427852671078976 │ 1 │ █████████▊ │ +└──────────────────────┴─────────────────────┴───────────────┘ + +10 rows in set. Elapsed: 0.053 sec. Processed 266.05 thousand rows, 4.65 MB (5.01 million rows/s., 87.61 MB/s.) +``` + +Most contributors write more code than tests, as you'd expect. + +What about who adds the most comments when contributing code? + +```sql +SELECT + author, + countIf((line_type = 'Comment') OR (line_type = 'Punct')) AS comments, + countIf(line_type = 'Code') AS code, + comments / (comments + code) AS ratio_comments +FROM git.line_changes +WHERE (file_extension IN ('h', 'cpp', 'sql')) AND (sign = 1) +GROUP BY author +HAVING code > 20 +ORDER BY code DESC +LIMIT 10 + +┌─author─────────────┬─comments─┬───code─┬──────ratio_comments─┐ +│ Alexey Milovidov │ 100869 │ 356978 │ 0.22031158880586746 │ +│ Nikolai Kochetov │ 34057 │ 113261 │ 0.2311801680717903 │ +│ Vitaly Baranov │ 24994 │ 84504 │ 0.22825987689272864 │ +│ Maksim Kita │ 23338 │ 78778 │ 0.22854400877433506 │ +│ alesapin │ 21976 │ 72279 │ 0.23315473980160203 │ +│ kssenii │ 19465 │ 61852 │ 0.23937184106644366 │ +│ Alexey Arno │ 16469 │ 61674 │ 0.21075464213045314 │ +│ Alexander Tokmakov │ 10760 │ 41964 │ 0.20408163265306123 │ +│ Anton Popov │ 9158 │ 38448 │ 0.1923707095744234 │ +│ Ivan │ 6480 │ 33711 │ 0.161230126147645 │ +└────────────────────┴──────────┴────────┴─────────────────────┘ + +10 rows in set. Elapsed: 0.136 sec. Processed 7.54 million rows, 31.57 MB (55.33 million rows/s., 231.83 MB/s.) +``` + +Surprisingly high % for all our contributors and part of what makes our code soo readable. + +## How does an authors commits change over time with respect to code/comments percentage? + +To compute this by author is trivial, + +```sql +SELECT + author, + countIf(line_type = 'Code') AS code_lines, + countIf((line_type = 'Comment') OR (line_type = 'Punct')) AS comments, + code_lines / (comments + code_lines) AS ratio_code, + toStartOfWeek(time) AS week +FROM git.line_changes +GROUP BY + time, + author +ORDER BY + author ASC, + time ASC +LIMIT 10 + +┌─author──────────────────────┬─code_lines─┬─comments─┬─────────ratio_code─┬───────week─┐ +│ 1lann │ 8 │ 0 │ 1 │ 2022-03-06 │ +│ 20018712 │ 2 │ 0 │ 1 │ 2020-09-13 │ +│ 243f6a8885a308d313198a2e037 │ 0 │ 2 │ 0 │ 2020-12-06 │ +│ 243f6a8885a308d313198a2e037 │ 0 │ 112 │ 0 │ 2020-12-06 │ +│ 243f6a8885a308d313198a2e037 │ 0 │ 14 │ 0 │ 2020-12-06 │ +│ 3ldar-nasyrov │ 2 │ 0 │ 1 │ 2021-03-14 │ +│ 821008736@qq.com │ 27 │ 2 │ 0.9310344827586207 │ 2019-04-21 │ +│ ANDREI STAROVEROV │ 182 │ 60 │ 0.7520661157024794 │ 2021-05-09 │ +│ ANDREI STAROVEROV │ 7 │ 0 │ 1 │ 2021-05-09 │ +│ ANDREI STAROVEROV │ 32 │ 12 │ 0.7272727272727273 │ 2021-05-09 │ +└─────────────────────────────┴────────────┴──────────┴────────────────────┴────────────┘ + +10 rows in set. Elapsed: 0.145 sec. Processed 7.54 million rows, 51.09 MB (51.83 million rows/s., 351.44 MB/s.) +``` + +Ideally, however, we want to see how this changes in aggregate across all authors from the first day they start committing. Do they slowly reduce the number of comments they write? + +To compute this, we first work out each author's comments ratio over time - similar to [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments). This is joined against each author's start date, allowing us to calculate the comment ratio by week offset. + +After calculating the average by-week offset across all authors, we sample these results by selecting every 10th week. + +```sql +WITH author_ratios_by_offset AS + ( + SELECT + author, + dateDiff('week', start_dates.start_date, contributions.week) AS week_offset, + ratio_code + FROM + ( + SELECT + author, + toStartOfWeek(min(time)) AS start_date + FROM git.line_changes + WHERE file_extension IN ('h', 'cpp', 'sql') + GROUP BY author AS start_dates + ) AS start_dates + INNER JOIN + ( + SELECT + author, + countIf(line_type = 'Code') AS code, + countIf((line_type = 'Comment') OR (line_type = 'Punct')) AS comments, + comments / (comments + code) AS ratio_code, + toStartOfWeek(time) AS week + FROM git.line_changes + WHERE (file_extension IN ('h', 'cpp', 'sql')) AND (sign = 1) + GROUP BY + time, + author + HAVING code > 20 + ORDER BY + author ASC, + time ASC + ) AS contributions USING (author) + ) +SELECT + week_offset, + avg(ratio_code) AS avg_code_ratio +FROM author_ratios_by_offset +GROUP BY week_offset +HAVING (week_offset % 10) = 0 +ORDER BY week_offset ASC +LIMIT 20 + +┌─week_offset─┬──────avg_code_ratio─┐ +│ 0 │ 0.21626798253005078 │ +│ 10 │ 0.18299433892099454 │ +│ 20 │ 0.22847255749045017 │ +│ 30 │ 0.2037816688365288 │ +│ 40 │ 0.1987063517030308 │ +│ 50 │ 0.17341406302829748 │ +│ 60 │ 0.1808884776496144 │ +│ 70 │ 0.18711773536450496 │ +│ 80 │ 0.18905573684766458 │ +│ 90 │ 0.2505147771581594 │ +│ 100 │ 0.2427673990917429 │ +│ 110 │ 0.19088569009169926 │ +│ 120 │ 0.14218574654598348 │ +│ 130 │ 0.20894252550489317 │ +│ 140 │ 0.22316626978848397 │ +│ 150 │ 0.1859507592277053 │ +│ 160 │ 0.22007759757363546 │ +│ 170 │ 0.20406936638195144 │ +│ 180 │ 0.1412102467834332 │ +│ 190 │ 0.20677550885049117 │ +└─────────────┴─────────────────────┘ + +20 rows in set. Elapsed: 0.167 sec. Processed 15.07 million rows, 101.74 MB (90.51 million rows/s., 610.98 MB/s.) +``` + +Encouragingly, our comment % is pretty constant and doesn't degrade the longer authors contribute. + +## What is the average time before code will be rewritten and the median (half-life of code decay)? + +We can use the same principle as [List files that were rewritten most number of time or by most of authors](#list-files-that-were-rewritten-most-number-of-time-or-by-most-of-authors) to identify rewrites but consider all files. A window function is used to compute the time between rewrites for each file. From this, we can calculate an average and median across all files. + +```sql + +WITH + changes AS + ( + SELECT + path, + commit_hash, + max_time, + type, + num_added, + num_deleted, + sum(num_added - num_deleted) OVER (PARTITION BY path ORDER BY max_time ASC) AS current_size, + if(current_size > 0, num_added / current_size, 0) AS percent_add, + if(current_size > 0, num_deleted / current_size, 0) AS percent_delete + FROM + ( + SELECT + path, + max(time) AS max_time, + commit_hash, + any(lines_added) AS num_added, + any(lines_deleted) AS num_deleted, + any(change_type) AS type + FROM git.file_changes + WHERE (change_type IN ('Add', 'Modify')) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY + path, + commit_hash + ORDER BY + path ASC, + max_time ASC + ) + ), + rewrites AS + ( + SELECT + *, + any(max_time) OVER (PARTITION BY path ORDER BY max_time ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS previous_rewrite, + dateDiff('day', previous_rewrite, max_time) AS rewrite_days + FROM changes + WHERE (type = 'Modify') AND (percent_add >= 0.5) AND (percent_delete >= 0.5) AND (current_size > 50) + ) +SELECT + avgIf(rewrite_days, rewrite_days > 0) AS avg_rewrite_time, + quantilesTimingIf(0.5)(rewrite_days, rewrite_days > 0) AS half_life +FROM rewrites + +┌─avg_rewrite_time─┬─half_life─┐ +│ 122.2890625 │ [23] │ +└──────────────────┴───────────┘ + +1 row in set. Elapsed: 0.388 sec. Processed 266.05 thousand rows, 22.85 MB (685.82 thousand rows/s., 58.89 MB/s.) +``` + +## What is the worst time to write code in sense that the code has highest chance to be re-written? + +Similar to [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) and [List files that were rewritten most number of time or by most of authors](#list-files-that-were-rewritten-most-number-of-time-or-by-most-of-authors), except we aggregate by day of week. Adjust as required e.g. month of year. + +```sql +WITH + changes AS + ( + SELECT + path, + commit_hash, + max_time, + type, + num_added, + num_deleted, + sum(num_added - num_deleted) OVER (PARTITION BY path ORDER BY max_time ASC) AS current_size, + if(current_size > 0, num_added / current_size, 0) AS percent_add, + if(current_size > 0, num_deleted / current_size, 0) AS percent_delete + FROM + ( + SELECT + path, + max(time) AS max_time, + commit_hash, + any(file_lines_added) AS num_added, + any(file_lines_deleted) AS num_deleted, + any(file_change_type) AS type + FROM git.line_changes + WHERE (file_change_type IN ('Add', 'Modify')) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY + path, + commit_hash + ORDER BY + path ASC, + max_time ASC + ) + ), + rewrites AS + ( + SELECT any(max_time) OVER (PARTITION BY path ORDER BY max_time ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS previous_rewrite + FROM changes + WHERE (type = 'Modify') AND (percent_add >= 0.5) AND (percent_delete >= 0.5) AND (current_size > 50) + ) +SELECT + dayOfWeek(previous_rewrite) AS dayOfWeek, + count() AS num_re_writes +FROM rewrites +GROUP BY dayOfWeek + +┌─dayOfWeek─┬─num_re_writes─┐ +│ 1 │ 111 │ +│ 2 │ 121 │ +│ 3 │ 91 │ +│ 4 │ 111 │ +│ 5 │ 90 │ +│ 6 │ 64 │ +│ 7 │ 46 │ +└───────────┴───────────────┘ + +7 rows in set. Elapsed: 0.466 sec. Processed 7.54 million rows, 701.52 MB (16.15 million rows/s., 1.50 GB/s.) +``` + +## Which authors code is the most sticky? + +We define "sticky" as how long does an author's code stay before its rewritten. Similar to the previous question [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - using the same metric for rewrites i.e. 50% additions and 50% deletions to the file. We compute the average rewrite time per author and only consider contributors with more than two files. + +```sql +WITH + changes AS + ( + SELECT + path, + author, + commit_hash, + max_time, + type, + num_added, + num_deleted, + sum(num_added - num_deleted) OVER (PARTITION BY path ORDER BY max_time ASC) AS current_size, + if(current_size > 0, num_added / current_size, 0) AS percent_add, + if(current_size > 0, num_deleted / current_size, 0) AS percent_delete + FROM + ( + SELECT + path, + any(author) AS author, + max(time) AS max_time, + commit_hash, + any(file_lines_added) AS num_added, + any(file_lines_deleted) AS num_deleted, + any(file_change_type) AS type + FROM git.line_changes + WHERE (file_change_type IN ('Add', 'Modify')) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY + path, + commit_hash + ORDER BY + path ASC, + max_time ASC + ) + ), + rewrites AS + ( + SELECT + *, + any(max_time) OVER (PARTITION BY path ORDER BY max_time ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS previous_rewrite, + dateDiff('day', previous_rewrite, max_time) AS rewrite_days, + any(author) OVER (PARTITION BY path ORDER BY max_time ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS prev_author + FROM changes + WHERE (type = 'Modify') AND (percent_add >= 0.5) AND (percent_delete >= 0.5) AND (current_size > 50) + ) +SELECT + prev_author, + avg(rewrite_days) AS c, + uniq(path) AS num_files +FROM rewrites +GROUP BY prev_author +HAVING num_files > 2 +ORDER BY c DESC +LIMIT 10 + +┌─prev_author─────────┬──────────────────c─┬─num_files─┐ +│ Michael Kolupaev │ 304.6 │ 4 │ +│ alexey-milovidov │ 81.83333333333333 │ 4 │ +│ Alexander Kuzmenkov │ 64.5 │ 5 │ +│ Pavel Kruglov │ 55.8 │ 6 │ +│ Alexey Milovidov │ 48.416666666666664 │ 90 │ +│ Amos Bird │ 42.8 │ 4 │ +│ alesapin │ 38.083333333333336 │ 12 │ +│ Nikolai Kochetov │ 33.18421052631579 │ 26 │ +│ Alexander Tokmakov │ 31.866666666666667 │ 12 │ +│ Alexey Zatelepin │ 22.5 │ 4 │ +└─────────────────────┴────────────────────┴───────────┘ + +10 rows in set. Elapsed: 0.555 sec. Processed 7.54 million rows, 720.60 MB (13.58 million rows/s., 1.30 GB/s.) +``` + +## Most consecutive days of commits by an author + +This query first requires us to calculate the days when an author has committed. Using a window function, partitioning by author, we can compute the days between their commits. For each commit, if the time since the last commit was 1 day we mark it as consecutive (1) and 0 otherwise - storing this result in `consecutive_day`. + +Our subsequent array functions compute each author's longest sequence of consecutive ones. First, the `groupArray` function is used to collate all `consecutive_day` values for an author. This array of 1s and 0s, is then split on 0 values into subarrays. Finally, we calculate the longest subarray. + +```sql +WITH commit_days AS + ( + SELECT + author, + day, + any(day) OVER (PARTITION BY author ORDER BY day ASC ROWS BETWEEN 1 PRECEDING AND CURRENT ROW) AS previous_commit, + dateDiff('day', previous_commit, day) AS days_since_last, + if(days_since_last = 1, 1, 0) AS consecutive_day + FROM + ( + SELECT + author, + toStartOfDay(time) AS day + FROM git.commits + GROUP BY + author, + day + ORDER BY + author ASC, + day ASC + ) + ) +SELECT + author, + arrayMax(arrayMap(x -> length(x), arraySplit(x -> (x = 0), groupArray(consecutive_day)))) AS max_consecutive_days +FROM commit_days +GROUP BY author +ORDER BY max_consecutive_days DESC +LIMIT 10 + +┌─author───────────┬─max_consecutive_days─┐ +│ kssenii │ 33 │ +│ Alexey Milovidov │ 31 │ +│ alesapin │ 27 │ +│ Azat Khuzhin │ 24 │ +│ Nikolai Kochetov │ 16 │ +│ Igor Nikonov │ 12 │ +│ feng lv │ 12 │ +│ alexey-milovidov │ 12 │ +│ Maksim Kita │ 12 │ +│ Nikita Vasilev │ 12 │ +└──────────────────┴──────────────────────┘ + +10 rows in set. Elapsed: 0.025 sec. Processed 62.78 thousand rows, 395.47 KB (2.54 million rows/s., 16.02 MB/s.) +``` + +## Line by line commit history of a file + +Files can be renamed. When this occurs, we get a rename event, where the `path` column is set to the new path of the file and the `old_path` represents the previous location e.g. + + +```sql +SELECT + time, + path, + old_path, + commit_hash, + commit_message +FROM git.file_changes +WHERE (path = 'src/Storages/StorageReplicatedMergeTree.cpp') AND (change_type = 'Rename') + +┌────────────────time─┬─path────────────────────────────────────────┬─old_path─────────────────────────────────────┬─commit_hash──────────────────────────────┬─commit_message─┐ +│ 2020-04-03 16:14:31 │ src/Storages/StorageReplicatedMergeTree.cpp │ dbms/Storages/StorageReplicatedMergeTree.cpp │ 06446b4f08a142d6f1bc30664c47ded88ab51782 │ dbms/ → src/ │ +└─────────────────────┴─────────────────────────────────────────────┴──────────────────────────────────────────────┴──────────────────────────────────────────┴────────────────┘ + +1 row in set. Elapsed: 0.135 sec. Processed 266.05 thousand rows, 20.73 MB (1.98 million rows/s., 154.04 MB/s.) +``` + +This makes viewing the full history of a file challenging since we don't have a single value connecting all line or file changes. + +To address this, we can use User Defined Functions (UDFs). These cannot, currently, be recursive, so to identify the history of a file we must define a series of UDFs which call each other explicitly. + +This means we can only track renames to a maximum depth - the below example is 5 deep. It is unlikely a file will be renamed more times than this, so for now, this is sufficient. + +```sql +CREATE FUNCTION file_path_history AS (n) -> if(empty(n), [], arrayConcat([n], file_history_01((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history_01 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_02((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history_02 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_03((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history_03 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_04((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history_04 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_05((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history_05 AS (n) -> if(isNull(n), [], [n]); +``` + +By calling `file_path_history('src/Storages/StorageReplicatedMergeTree.cpp')` we recurse through the rename history, with each function calling the next level with the `old_path`. The results are combined using `arrayConcat`. + +For example, + +```sql +SELECT file_path_history('src/Storages/StorageReplicatedMergeTree.cpp') AS paths + +┌─paths─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ ['src/Storages/StorageReplicatedMergeTree.cpp','dbms/Storages/StorageReplicatedMergeTree.cpp','dbms/src/Storages/StorageReplicatedMergeTree.cpp'] │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ + +1 row in set. Elapsed: 0.074 sec. Processed 344.06 thousand rows, 6.27 MB (4.65 million rows/s., 84.71 MB/s.) +``` + +We can use this capability to now assemble the commits for the entire history of a file. In this example, we show one commit for each of the `path` values. + +```sql +SELECT + time, + substring(commit_hash, 1, 11) AS commit, + change_type, + author, + path, + commit_message +FROM git.file_changes +WHERE path IN file_path_history('src/Storages/StorageReplicatedMergeTree.cpp') +ORDER BY time DESC +LIMIT 1 BY path +FORMAT PrettyCompactMonoBlock + +┌────────────────time─┬─commit──────┬─change_type─┬─author─────────────┬─path─────────────────────────────────────────────┬─commit_message──────────────────────────────────────────────────────────────────┐ +│ 2022-10-30 16:30:51 │ c68ab231f91 │ Modify │ Alexander Tokmakov │ src/Storages/StorageReplicatedMergeTree.cpp │ fix accessing part in Deleting state │ +│ 2020-04-03 15:21:24 │ 38a50f44d34 │ Modify │ alesapin │ dbms/Storages/StorageReplicatedMergeTree.cpp │ Remove empty line │ +│ 2020-04-01 19:21:27 │ 1d5a77c1132 │ Modify │ alesapin │ dbms/src/Storages/StorageReplicatedMergeTree.cpp │ Tried to add ability to rename primary key columns but just banned this ability │ +└─────────────────────┴─────────────┴─────────────┴────────────────────┴──────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────┘ + +3 rows in set. Elapsed: 0.170 sec. Processed 611.53 thousand rows, 41.76 MB (3.60 million rows/s., 246.07 MB/s.) +``` + +# Unsolved Questions + +## Git blame + +This is particularly difficult to get an exact result due to the inability to currently keep state in array functions. This will be possible with an `arrayFold` or `arrayReduce`, which allows state to be held on each iteration. + +We welcome solutions here. \ No newline at end of file diff --git a/docs/en/getting-started/example-datasets/images/superset-commits-authors.png b/docs/en/getting-started/example-datasets/images/superset-commits-authors.png new file mode 100644 index 0000000000000000000000000000000000000000..7be831467cfb69ccc095c5d2c43a8db68bcfb9b5 GIT binary patch literal 252198 zcmeFZWk6Kz7B&nch@hYdh@ym~G?LPz(p}OejijXHFd`y?q;yJ(QqnyrEuBLMNHfyy z0K3diO-8T@xKP1exWi1pHu~>lb&tYK)T4Uj0E&={f0DrKs&ZJ>u zodN#F#{4V|^z(o5kZEUreh(VLTu2ilfNh8u z=eU;cE`NgI6q0(?ieWXxLaN`2FJbDoJD>A<)#Bt~UE|u)aSxJOdXn@FjUN{1+~2-5 zyd-xl_|7wjLkz~k#<}#DFAW5j<`(_`m-~W~&s~};jJn=zP5VDw4h~Q4`5zXG1#+bY zhr^WYNpph!m&@<^ikDyf_m&g}RR(f2eKeOxEylSX#g`AF{+^TuA zI9KHu7Q)yV*~Qr>Taad34RQ zB>IrA5dD2;_r24gZ3#H2kb`c&Dym!VqLS!e z0~8DT8OH?&{pkG0a`ZAEqAZ$K=qar4%+aIFfy;0GJ=q@IY=db;miqs7&o@YWNm27J zyYs~O`T5`La>u`(oHS(atWJ^3GIMcp8CH`G_tDIdKgy1@86ZVbeC5zmvL0uv?qtsz zCjW*spc1Uv)*c-z%`WVOjR;+a4%VXy!=MFaz&ve$BJ04O4gFH zt4dfF%Uaf>7jA#cKp+YvcoktB+tKy)V#pOsaaPXSji*O&2r4$;b2-1_GrWF}Z?y7p zMo36Vn*a={I9lgbVW4GY(~}~)6^8yE=Jc64hg6gOu;_v^Uu9&w-KiJZZMd;a-UFn!l8vPd2@GKmV=^>ubD!)ISEcQ*jFvnU)Y zrp#gBULBIfI6sld8KW$^xy~J-)Ycm2nNjP;Ia2gu>L6LLPb=banez`flac4&ZX5vaepnB*M#<#8*eQ_D5W5!Y%%VcgQv z(zYO`lsh)C!?haaz14B1iZ4#V2I7K(*1|+0-@p3vm-*hg5raqdq`^M8b#@?`z$Jj{ ze#iIkl}B5#u-DR59uiJfNd^K_)Y}HL>(7aBKgyRq)@*efy}I;TS^RPS(XJ_)aT&Cl zzATpyC*Cs>T1Ayt!5!{MKaV2W(z*zduHXCAwe_-$^8?+Ad_L?8&bGM4(@h~Kqh(*4 z#r&-)YE;S&gOucc-)<|P!|V4c2NC!OMy#8*YD#wF{660n<4naVCt2?=#PM`wEh|M4 zDO)YP^Dxqy;dVHKCvbVLP$Ed^uc!kK07YjGpm%b_*~#@CKjvL2tUq{U?cflbl0xON z@}bpic{snMe7Wsd{2)U-vWjlW)eOBTwlA324>2*hH;F#p@5l%fV-Y`^zBvDxwUD$B z&gv;b2`&713_~Bn$h+u^Y~UQ@P;k$_=(* zcHN4M#gj5DIuzqBA6Uzv+YsORDxTPfQcWaG>l!eVJ|MrqE8lQ&Z;~!vzr5)4uvcSC z$$a49T8g38nj{0q{8;bmnjDKN-e2(voW>26llm0<2$ABo_5txdF)`76y!?JfJD(z2 zU0KxJAKAQz8q5)6&?~o%ST2Mac8l)oBfqzbb0ED9Cz0LmpGQXGZa|JUx4zGYb=2)o zvbIur<$ZymQI}sy&8G)*M4E9kBV2U5g_haI?}GPPwH9hxR7om1wGs@luf3$ExR%

!4InnP^Zrnxc?1;(qu`|&zr zZeOmOYq@|^xRUV}Sl1^G2>L?tfn=}KGZHfzLfA!DDrluF;?f}(KRzV+YuPJG0+Abe ze>egFCrYdC+k%z9AkcTyH1ik5WI|2@qwnoA?hy=gA&B%norp|8j zH56V0i5>KXjTyt-ZuQbYxUQCF92VB?>3S^dO|iI786cNjywY%wZYL4Q*Q~3&^P>D% z2!w&d?)UkpYq{10QVMDHjl!;V36Ef1*CX#n$-xE>@-1GYgZ0A7U3Eo1B1$Gmmx>Qb zHNCgK3j}pcm#Slt4wD^v+*?*^g$RyE76_f>#x1If4(WBstt#kd<@3esd}{PR%wO(HJ1&q(ov`#DU0h8xWc;9@ zwkH)RR{)u>j&0N9PqtakRQ!w*i1$0fXTLsQx|i=pEjoJ90rDzuzoJ9SsM6!$%Xpol z6QX(`dy3qUzH@Vp!uwt^GUOSN1?XHAkt@}$!NkLo%du65q+ZI^$0jU!_y0--uiX0T zRApB0ve%s8S~KBBiXzj}e<(fdRcDBtA%|JL_XV6=Y`s%9;WQs9&p}N22E8Bv8%=4% z6aRjfIb>^nZ7t?JM`Z*5<1y=;AgcTFUS2gP_&nhXZ^vM4v(Zt2fj9uJ$sLc`6U}aI z?s(U9gv!8#uUTW#M!xE~__M%<*d!>5)Pl!1El112p z0lF(0Ryw|WG5@fji@e?ZdUs@wf}F3Njk9`1lC>Jrrt07|>q?K_AywpYQ)TUt3vH#> z5_961cx=z8T}_66dhnH1myP}c)A^}ArDT5totcZSIxlCqwwYh2syt*4mAL+Qh&^*| z>8*G2y6;WN#)=?`dmv~o5<%Yod@aMy>Jp@$c;9(^xDz|!Zd#;ILPCO^66MB58xgW! z9o}PCKRpGR$rNZuB&#RmU-4J zv>J11tNOvd>{`!TE&I3Y--vjVpf;GeRQG~aNIcgeN88$r$-FdBmYAa!j{ld| zsMH!kI{uZ3suY5dwPqyv?Jen%%9av0h#nuHX6zqb4)ijA#^_v?bSGKbjm>59j$lZa z+G`0~<_K*C?)W^r?)7asb#)byC!DwrLv~AbL&X~V6(jh0A!sHtVV>cef5|zp0ltD5 zGV{V07yl5{ny>5yq8E)g1BuE{KID1qzRx-&i+x#BR+hf(akSZLjy^_hz0<0b`-0~` zB!_?T=xRoi0mp?E$$=mN6O-L|tiF@1Bb380U%gCjmc57xkvnHc1-A(hQyCjnr4l6)E zlo=Tr=SED!%2LZ-T^FEo9BWkH-?TCK&z96#DclUkyeZK;QI5v5b~aa_fz-a*Yn8L1Y7ba+ zVa`3i$dy>wP4_Q{e)>*p@lNy7;xy>l#`UP-g>dgfJn?AsVHYIIjOBr<9&2+=dX(tX z=ugu{7cN|oad0R^v5^Ur#da`MOt>YAYQHiJ6P=2vFwyl2MTmCHMaTIT=OL|`2%C7kv-gys8%N(l>Wa)rX zR9~O!^$iDW>xg?oJ3uGd6gmi39TK_+Iow+ zdVsMCk)3EuuuGzJ=PuEWJ2dEnLiD3?5%kgaJYe&j3q(Ypci~2yki!ZHtB}dz6dE

66<v ze=9x=otxy|Vm-rSeNd*qRa`QcI0Z;`m8(jE$Y5ptYEm@bam85ouqmi*$iUbhtp`5w462EPKkQs-`T&>y?v zqdEY{8eMxl+sGb>Aum^9D!xHZGAl1uVJefI7JRu*Xd%vY&XoEv4kglAVNfEtbjznM z7Fc(V_hztB>8kJH_*8jngg_laajU$dPLWr==_2I&{Y2GtwqpkZ%x zg1N492YIsosX-Fiv3*mMbbSWGW+=T$mSl1RS|ykf6GKHX9A`Pq7fl^Jj_}dCMz_NH9c1-Rn&g>A7#g*1 z7mk+X^o&Sy9oCrD3dy zyR*Nq=l|x-o5oszPhlG2JwtuBQS(r=2&zlbC6(HO-;3t|3#Hc10)3$5Y=bWQ znmyf(K(H$8tiN{t!{zZmh9bBLuYkNbBA+y(~tV(#Tu3**SQT4pB={raeGxA?gfZpZdcX4+g$MPL?k?6}i zFP5rI<+G#eb#ce=QR{iawq;ar=Y=srQ+R;^e~o{+mVJ$H46o@k-{I2PAUC;QX8N5G zK#cHOwNwhrrBvGa;(}5m2e7>C*6Lq)Kv4p2=PgglE^(XN(4Aa0#-q1~7ThI!e5%NF zT_KRZbnKM(c2{~vy|Nf`4&muO!xRp+U)Xi!2VcyOm;To zq544o@?MmZQ2W_@zO33%3wYm=F0Xdp$FEM3J0P_&%;8!3yu3C^ahgFgH5JWM5(#G2>RVs=S)_ z2K&i`bl8KD0(H?P8-twjo%(#e3VY9*6#1&v02}6`LH<&nr*2ez3Rs?yvNJEs3+)g{ zc>0ZRwCcgb)+GPQivPQN(u@W>j=PN>d{FKtAPt*zj$JwGM@JBEmkecUrs}C#%(>Z4 z8EzI%-D*AVd2jJ*&pqyI-Z+j$aCAwJ*m!m$oaS9<96UW^jW4Y*W3i0H?+W1s8#y;X zIRIqChR~~U!4bZ%@2pW34pSV<@WP6Z5X$L%y3MKG0{8YSQJN?%*g-TQ|8koX;hH#5 z64OV|l%`x+l}KZrKL63AX6q}p5V!um$5uPj>g`ek#cPF&IlHb#l$(VgCog(`J+l?S z>HRU38}plzIpn<&U*|uUcS*N1iR~D+cb)4L-`F5=8bTjtf7#zKbvl+;7S9VRYA-G_ zT^F-8EAGeIexJ ze!ZnQ`Pduq`l@wKvW5B*Y}L*o3c&SGld;ILf!SEr0I&DzvR|A!*!Qj`Oy3obvHA>Q zr&xk_UM#z1Ufn-96JGk)`;J-fnZj$#tq#204K^KYd{g1an>A8&>+M97Sk?^e4ER%Qpl%Yo#z z-`}rW_i@^6Chr)|*RylOn$VK=q{gVIC8;QdNz|JAxO1O19`B_*=!~3NRm}v4H9nuL zp0kx_)ke~Qc@6&Yl*{3S6C8C>uHWTeZ+b>wE90lFT^nDv`*fyTYXUG$A%!%^Lj4+1Qw%2L5Gt2^Vm_p z`m>|EDM_B+a<`86VMg@_LxrC)!q#$LvdB(JI^99pQqQO5PrF*h2Fu?(Y_9mh4r^dh zcH`ykv9KfazMg~v@c|bn1Oo9P)n88N!bnu^*2O^R9TD>T(XUh}|CVPIdd4%;fo84b z1u?cFj5ZuT(Kwwi(6zEW;+~5R^tMvf?#Yk{TiDj;M$kd?ATN8(Ss7R6J>C0B6qh(i zeF}qzQ)}{z&~XUcjdo>UHL=}X<@PH)PmQK*q(OuuHoe+sY7?XkrAL-+?2O2h@eJYQ zpq_x)M>Pik%g`GOJ?poaQj!uD`hB<)YwR(&qYQ3nBh0e2bQ}<~=)M}k3j6+Gz+T_b zV_LA-u0xeEAELB&wCcarrrz^eC#PQfcoBkrfg=|(mg@24Y2O!T&b(Bcck8aXgI?PU zUFG&~>Z$aMY_k2UCIj5cl%~H2Y=2u_jm}B+)fG@ACme)*Z{{qgWGSVSnx?W67E1P? z)LUgs-_LQJ^qkr$i@7&`pdMUMXa*1fU;Z8E+x9|5OUGj$!Em2>R-*`>?mLYZuZv!B zY*OPOW3Rqm__MA3Z#cVTC4)`)PHgY(g)bY<*St3F$`W3D*@&Zlzcrc0Ahup@xc1x7 z%}l8y&S>DBwOBhiITn8}$2d->^nS~8!t1uf0rAUWgEn2a9-~qF9SpJh zv~N`2GUnV3hcV{;?YI_vh9IuNxg|uqCoW<+a+q|>g17z}`}%F>*<`A9V!S@peW89` z^%m;f^o#*lm#XYHoFP<3OI22B;#ZX0n#`IEW2a-%s>(H^h^Cfz*-A$l#vK@*aE>-> zjD#DOgl@r!kAv%ASwynou-6xs|3aKj4z#r&f&v-eX+LCYLHwnuI&Ie*Y_Xpnm54qG z{C5t1^1lr)$!OYOCJuy5&;OUJ|KFYeP(=PO?d(`M!jt&e1a{md+W+N7Bu~$su7Zg& zo3Jk)`wk3=?Vpj0sjRKk6$YQCWvdViM(uNRnMuywI!;@`%9nDY#4hk$_4sSe0!Ahl zL99XqRAH_2W9UjH`x+Fyf847!{;@HDWqm(CTIN-F)Zq@}tl3|Q zHs-n|UU$rR?dsKc+j@-%x<9u0PjSJeXDo?_-1hiz_fX{){ef-=v|da$?kn1BYMaPo zJ}Xv+BNkJVb)}?Q=aj`918Pxr4&m&2p^;PK-`fU>j{b+u_%*N)Yxf}z{6G5DPVW{* zSh4YUB`aCdlRml(Xvjo?GB)xhJG&QU_!Fbo);RQf*0(!)H#T;}YX65p5U?@dxxA%_ zBZ3tdAw{k!iS3Iqwb%Mn<#HGtj!7QB=vXb!K)OnxT|Rt7#~>>HZ|>OaH^KOp`c(^k zGB?zhWFt%>-Klf9tAD^5e9-mccz9SdsCCBa6C#Ine4^lbs)7t#1anm~A>cY{P{6Tx z07B&3Y@VPz&ZRikGy`OkQlObL-r_>M9&{hLL;z$MtFAVBE!kmtp;J&5xQO|yYa*J{ z@;iD&3;&R58le9j1FpO52zw`#Eh7UlRv zjduw$9M2hX484o&3oDFOl(yr|J&-ov?vsvgvLi%h@k!9Up}%zf(pz%u^rKy;oIA>x zi4*8^;$C}U?;iWO@veZAai%;+=sgRCV>`oXftLgKv7l!!U8UHP6N$JK#>%-|6Y6dJ z7Q3xV0{GoDH&DQc#%idFYKJTFocMCcK-28z$a&og`&asV)mooJDQhxIG7P%KmM@GJ zJxs1WhSf=>cq|CYHjyj<8e~>h6@9~NpK0>s$RW%T!Y*ymz2kfNwBlH_>P;|-&` zs4AavO&&E!ltB(1(sknbu8p~(d+eN(&iKtV70B&7E5ZMiK$sk3lcGtH8Jl40+-aO{ ze^FXd&Xz5>CwWp!!QW`x#088C*D<^&f>#{( zga<*J3BpM%K0ICd*AbA{m)aI`K!*qg&Hi|WAl*EvlO?1%0hJ_LPxW6r7wvE`Jw>z( zZ+V)}DE{dq zuZsou~px^CXuNW`^!939t}Xr;IyY2V_?%Q^Os+ z(Lz;-P~qi$+B?pt(eU5N1;ssIsTdBc%a=SWEyWPk#a2n_c6#gy29~rf2u^DOkwg(A z+u18R!W_OocoinG{g3-(m8FT6lb_67+V`lN;X;q4WGh}cV@?#msrk`B3?})N zs#9=Um_9`+S(=&Irl%++I-jD!{|~*^&IC_=Ez0D%Hd<_*T>31*%-iH%@#qSzp1C__ zKTs)hnzpH>RICEn0%&$x_Atxt(OOmlb?BLYWRZE7pxzvoJx!~7GPg)ye{hs2#)IdF ztIzSCSmv7wutds9UZtvfoSC3!bKmWJ`D;R)z2!@y}L5v$WalX8A zZ`Z#K>R{y(5#qSk&&kdyB>hjL5}JMuX1tH2*YJ+SHY*J*bm5QtcMJUvKQUvLzsu39 zLWiVVpdOH+f3zrFxPQGB2@eAcB@-(Sr7Gl@J2)7jlI~dk<78&hEoj6he!E$5vB!X? zHNn};OFKpy87m^Mwh z?#3x<{Wr*I6N7}X6oLml96K3u544(mrs*;M@Fcf-OM}*%+Vb>*>2qk+O$l_vLUCyO zu6{%bRU0L-`~#q(8Z1^b`l)d-)O-Q;bvo9EVg3FeNE zl;EyXQwrSvNA{yF1^qJ&qM>*t-^3OWnGt9KeVY0YDF?ofzx4W(VM&Ox0{@_^^M{kmQb_e!7if%0~IU^^d{ms*H6pia{HO;0m?J}e{ zFc5%^bAPL2P$y8|TiI5#n6e3jau%b>&p{vlNo>)k1hs3ZUCw>Y89Q~rt`K#2Wwm4w?cLYwV@GoIT{6b>(GGE77w~|+%eXP8QMNrGkEB}V* zlMM_?0KlZEZB2?D0YoC5>1==hjmxke8-EnSsW z5y?w;^mD@+#x3IP=k1=N0Mv>D1C5Lf8zWHPhF#s|7~kJT2bnz3`N!_M7Yk&^t~lc& zn3dyDgk2CL{~xhy-+10kF&x5uIog62(n%Kd;>?FRvQMBGuqwjBgdy0J9EXLt&5WB{!qef_oH zl%=N~qg_we1IVU6&Cd`odWze^_^E%q=O~u-`<2I!PSS=z-TO z{i~SY0k2{>!eksB?_0mnIk<*`SRt~1!BNiLGEibhc6*+EGL2TB|JU8QaeQ4G^S9dH z9~Z9rRdIK}DESjKp+mPW1vfK1O2HElz|G<8Ssh+%CJG{%OXZ)&J$cwJkf-kjOfTZF zT<(oH0*Wg>u0KCzRO@VvHwRNyl5C&E?w4gC!b?Hs$6 zmwN9S%a-IJzQxslEePdLEP%kpe5vE5emral^DstJJRBC1HVbEN;!qjqTUGe&mXPu|bNyh_7>kF7fHfAI zM6P~WU%h^Zo9UzKpA64<==_pPK?VuQ9@6pQPQ9UvmrY5f0S8b(40q3-h6;EUhk~W8 z@Z(UNM_NvX`>k_>x} zyRTQr%QXv|sm^KK>=oeR<%<;mTHz(AP$cefJn*w?__0_1C^Aab3Zp43roKKWidBo(Lm(|pixxE9&V>o*M`yM8MtKyeZYz?a}F+52jB*i+BbF;L9v5WVo>~HSE7qd)+Oqd#>KW~kVlLsDCn27Gq zTwRuU^o@!4bbUX%GJLN0c#Gb*s_bDf4A*cGH6 z6`)mdv5Xj;27*pPAL8@jX0@~FmQ&Tqy11a1nj~<&)PK{2v@4*5ba5)>vFzP9*MI*Q zv+%b8e#iBj3 zdG?@;U(^EKyRnsePWsP*Q0N2Bw2U1lT4H=$)QnSiH+*Vh^rp@LzK5Bu$&j!k^kjKu z#6VAbT}#)_5jeZP*E#Pb>!sIG+H}BYz5F z*8n=*Kp)S4at7mETm@>=zUDUr0i!n`28Moh5$bZFqpcEwh22n_COg4yJ`9}prE`GD z2%af&87+Da1-1s5z$)V%T!o&7{2p8>IszqdIJ=niyENHGFQ|Z>1bK?Z6kW`wQ^NZUY~T z>}Q+c#?`CbbWKV9W>i$Qt<*6&#`G3+tS)n zQ&sQ=W+ysU>GSgqnDJUr-CE3sZ-ai_$=Cy@SdeEiOExe!<2$?%5fc)B#`%ehtp`!} z|ADvHK$NTbmtQfhaGUaRo+8^@8GuDfDrnqG6N2__%0}6>-=ykUaY0opmFy<|iQ_Rk zG>Kh7-5#Si2;>^pOArSG*MKpJZM(0If5ZjvvxYTP9ix9-4@WbHHVc9WE{!dmblP$4<*1g%zl{dDp56C1E+%F$}Ao@m-tXuJjGOKjVWfx512_ z#gONmLaYgp`;sP%1`v^mTjlXT1fouivSmocPpeYrmpublH4%cpAMjX0n+WYkiYauP zaW8mqUyOPmYku)pgggfItd5zg@7h!!cU_0evm{4M9z&b?K2QF!Er4j@V|T5*lmE~^ zG;DI_2nQLAC?di*AT6ok%{jBG1Y2n==t~@_)<(v*)acOia-CfRrPxxwKr=J%FWqP% z8#;$SNDD`JyIJ$RThKV08$As1B%UXk=LZGW1{wrSkK-m{4*mP0pz zy}5poE-=Bx2M9~l9v@SF*GC=8EN3bTF+(RV zyZ<}+@aBJtn%a2{DX6*`fgXQXW=;M)1G=avRiN!^<}W$f9TwCBa9U4CqZH0N^4=Xe zL7oa58ZP0sLUuCcP7T!Uku$%7FwGM>-FF-RJIbxEPQXEzlVkxp_ROQ`5BxmIH8q+W zK`(n2%j7?}B|gRG3D6$nXfFTxFeE^hdvY52ajwvC-4S6)AaFw^bmOjcar|~g+2t24 zGc#_K3PTP=z;ue;bB1EJA-N>C-(?!`$zTJgSe5RDW^NO+9?itUBTQzoO8Edqj6PDfJwDH)xbw>U;uc!m#M`hg<-p> zGVbE?Ni;8Q`Q^Li*`@!lWHjZWvc&Z^>#59Hdmo5P)_;-?(tobvL(yeVr)!OoVnDS4d zn)c@DQC%HgX3nCYL|4)hDb6DXE(@Uuv(Cnu?!94M%BO$C0>B;;N{_1aCo{jf2ZhG< z^-qj)5O3GqrN5G2%=eDY4;aYQhb&WW#dG=?lnz|_#aOgM{LUf)Tvw3lO1oVg%>FzQ z^ZE?*IVpyW0w9V;q7ELuojIG;GR7_II7h;;NJnfC8MQ{e#= z7;~u&;pW9%1?u}#i(Mpq*$qjIn4S!HRrcE5PZpU5VzE(pR6d?53AXosvBOUbf*+KK zv*kaeFT-727Vs`8AzR#13gKlbB2rLKwub?QA4KvI z^uy-kfl>HFslM+yiH5Lj`4%R|uFENgttV>uYdfh;%UlMQ3a zEmxRdH|>I3m4d}c$f~!9@&o?Urrj|PV5f~T+R%nixGt}K*-~D2@9hN$d`kJ5>sTOp zj$VN~Y#cfAy7%57%3Y7;3HzIl z>H@{AK;-j-Cyp42tsxR9+D4b9DDE1|(L^Z7eqpog`j}47!Smpm5_yCSW3avDkLr%` zksS^o)Z1w<){1Ck8`?ZlP#jmrkP;R$8T!a#X9wl1q1>NJ3n60>8k|b_wW5MQw$_QV zxo|vCx`p0UYWoKPe=7wPa6a8Z$+Qr^Y)v`y{>$e`FK*%aV@J{Q(|GOe$5;n-{jpo- zqH%BZ){`&A!<`;Fx2P)QtA&L#Ayy;VZESgq7=%z!08`iJbdlPv=Cha2?v)6;k-cvb z`UzKH+7Fe<9G&aJyaprsS62O2cn0dfH2^u%H8g!1eu4nj&OaLo1(SiIBHk_)+jqSQ zY~^u>s~%NnvdF=Cos|uLA9S%vfzLSW>aSgcf#L%n*^?~wm^oX}qfK-zjc_hm;XpS& zPVI+IRO=KktiM9lq1--)NmaTP5dVmo7(;MWNR__5EU@_Uo$AsTfYb7Pq1TivHPOvZ z0hq2B5MTpEbv6&Z%fv`me?M2_CYRYl&v0(ky^r^=#b#5ztLoerh@yzP1tCk&kW9Oc z>!KT`FUDkN$KF5G$Ki9Ij+-jQR^dXy&$GP+G8rwdwhr_D-3oqz_O>`#xI+r_e>>## zh|r?H!R>W!yTwl%VnOFguU@4h%jsb5dXtT4tWM&vwy}=6nS3TH@X-JVaOe(2;+FdN z2A5OWq1UcbP$t5-^?nq%w41nl)Jp;xI|lue!}?o5&j14wS%by&;6{{g9zE&xlVo}c zeB1`-=TCAtR~XYoc?S@AI7iKkx=x9lZq-XN+`na#hOKiWeHB%;cPN$m@T>!7T)y3i zGvvXuNO(PtOX^GO*B}B~n65#l9_q(72 z#~cYS0rleiflmq15;x*-T=(8d!6sQck%hMLC1xK7HP7@9O|r0e-9%W~L`)od(#Om7 zR9qiF#V~LhHZJe14MQ`+B>~%1X@I$^_#nDemJ(wwK=1+%rmpVCzt=wtO^O#_ww7{x zQkSA*Z5%t@d*cr_PX)a_K+KcaC7Xla*~O+LjF&B)zKKKk*zyzRX!#|<(jay9lK+#! zR=VrM`O(Q<`-vVd>XOSPmw#jhg_bd_opHgfCUe4tAgQpD=qxg-ea?Gm_EsdJs}jQk~`_4eHA zHQ$Q8AnRWge|XE-5b(?PRuM7uGcK=VD)&Wq5etbxUnr~Qj(jo)%3@OLKLXXNoS~~a zlg`2N%4%FLoz2W3~%39iVI1!z4@y<1{%@;7Zh{Ah(`W#FEX!7EEyM?x- zGrB#4H4d{^CrAH*-yGpeIks}XivBvK;qPI$&a)DOjmyDoDdCkmLSthq&w=-sO8Esk z=#)hTYG~JghUGM_=OLbg-xRYQd$#je5r(h(!BcE_X9E{=n}6go5pZk@sl>Or1PnHC z#Kua{GKi%b-RPb{9FPxk3ICSB+2m+!g-jNlbgX$kR#s(=nmr_l$0$Y%_A?m#=aL5> zo11t3Ow8>UP^C}mNYv!>|eub7IE>GZaAeA#10lvud5PmrV2Z$Y8`!_h;SkN%Mg zt_G+E&>;azoraL^{q*gcE39K)>aovcH)Vcz?%GrUSrU%wlC^9dTpEv;9unkoeaE0U zdQwN<5%+4KY-)te9G^5Db7*(f5B`##H5yjkMj5VzojFZ>@B@r4GLy~?%EDF4t}yQo z`Q}(N!N7{E_$@1KimA@WpJBNO`zJ>M8zXQ^=`6UT%aghI(!?v4-d>aH?tFl$d>XWA zbnE;D%jE-QqH>>)<=Wn!epU=HPC&1oo4eS=aQ11JgcSi!=T0H^D12PNnXm8^o^G)^ zepfFju2BIV^Adk&hj0YTw*hBUo-7t=vglUD&@FSUCh_}i-53oT;3SRrN)%u6s^mD~ z*`v!`!M?E>7z1bF5X6s<>FRZ^y982Y+3LeIMeM^wcA9vy+{}McCEmD(jit(V`j`kl zgJuX`e7mX)Jhk1-vS8td*kO)KgtE{rX8}(0jKzQ?c_6*nBM=Gkgd}%H{02)EXKWws znC>|R8gZ2!?SOTMs6Oq!)jW(Ara_a~5$s0MuQU;QP3P{gyL%9PHijw1!*J^;r8V#; z+=uvCcn_`PCiIm>&g?#s@JFd`8~Fe5NfzJ}*64I9kU6HKUYHJ*_BK?GPq6*1NNRIH zpS7pGmdVa66V%uNQUWNzF{K`tPNbc8WLDPpF+f9a=IEk8bb*UqugrJwr*;^zHymzt zR$Fav+xmoizLml>6z-F|ddRNieK{i|2%i+drD&nYA8!i3Yg|u+1zM>0IxO1W;JygO zuyhRF1iqA!y=b0Im)DZs)QI;@#XSDZnfqm;#JYeG@PIij`57STszzq>^cgPn8$|EN z1)p+dpcO?Kh70KUeX)>Bl>>%8`00?%W;7idYRNy|cGZSHC)jc?tpGmSAo2TWoQFtT znxackZIOpw`i}}bkaEs4UKT>;#*Sw7k8FPr5oiM+5a5E+8E7wlVoDWdp%?4;%mvjP zpbcQqJWl9T{ZRJ&#Q1<#0n7MLB7uz~)zShCcYGcz2z^<=Pf8FsrKQh(M?_cEhvhWg z(?DxdCU=l?Boqjzs$ughBXwv$UfeTo@$0=rwBW?p+hj>3JbYi^1K1PFuT^M*-Y|4nn|fyv zXY>BT$D{0Bo32voM&QFqet8(o2O?_Kh#W?;gUVM0(|iq zPY~@Eo#WA-shNxSJ%H4G9yH_m;mABUsOomo9Uw9i3c6o%lrfMNBMuG&pR>MGW;nI4 z2F4WrbSzxf3o@vQM&-8S$$C0Zs$kJ4yjLm|zl+uvE8-scMosoHNA<~!R%`sxd;@H!im&YQGO0>q^ImbL-hXgWZIXK2C z-Iw&%4me3YFdaVqfc2qI8uL=&Qt|mI&f*1kayeHJh0&95$YkJzzrbGl5vH=f2Rw#m zsSMin#_w4fVK+Kz*`JdZI4(}%yuY`*H{OS|%YRc{PozZk(^lGwAm(ky{0fN5n4aE> zdzl)1;jx$ES0(xsnb+?=c>)cIBQQI`DkP~Vt+xq`eSv&=bPMnAt*it5d*Z9EzSs`~ zYS3NcouW%p%GpWme){?wPKvjEp!WgVTan7^pz51jCgwc0hyc*K3ZEuw_r!8NloDF< zhu8S-JBk`Tmr$h8_TgG_(&X<^GcaOf`Zd2e802@&g%zRDqLia?l~R3WFqwmLemz`6 zNZpp_M2_L1-~{9Xq&HP!yd6gna45TLQ)9lELM5Bv@#Lp22)uvXZuZ&NeZ#Ac>?SZg zWn1l=d=RLX?5FgmX56Lt##M^^|F&L`nVJg%aXlP`cd)#t=Gld;(dWoDqla~@7Bf?M zY^YaMB$E6j;6lW02_9TbpNOrAa98h_sO3yd_S?cb5bi>l8V~fA!i3h3ln}$o5&`TrfaD;{_V8ubU*p7JSsORs-BgQab z!vfH07=gWssL|7IYpZI$G02N`$Xrl$9cl7mE5cEZ>(VVR!eqPrbm(#2;o;}utUpwk zPq@5o;uhS}d;)xuFE$npS9KP2mFyne8Vnf@-0Wl+#*8t)sJH~zGc_8*aizdI614c;syBaE4Nmy z-dzjDi#6?53SSl;zQ|UtTmRTwB|#o>LuLY zS_z#9~+xt=v&iCb{Iy~XTIz?tH4Q$1M)NZqP6wQU22|AsbwPk3& zcLrtf@u+`P0E+Qx;wm zqA(i=!Z7WcY}3t5yV}9VlrhTqm>B^h7EXvDt9%Aj1U2H#xQmYUM|Cv945?To!&U8o$ zkzgRbFh=8>JCIDlM;AF@_!d(h)1LXc-AE|JIr2rU*^(r8+o7RQ=E!c>AJ3D-$i_ z#KWi?u+^)N6I}L%_t#=8@%P~B?eUzK+BNz+XTMzvYc<##OzhgN?raOIpN=SY&HYJ) zfSho|78V_B%H$1v3Y-YOH@lx;sF!0nW+P;;hUvKi2$t5LT_1KCw{-=Y0n;~KMbtll zk~ht7eE!O+%G-Um&K+L8bf)avWXtC7ErJ_n5V2I4>Q zsrf?@JcA^|*m3?CR$J4{nJ^L?8v z6>d`>B}9MuRhQN)J7C_i0rYr#KgjJ$LSbY0mhD)Pl89lL$nnlLVP13AsZs=>B``e( z93X!0TE9B-bLYbP=!hfyVVY@!_*5_tn)8 zcelwLUy>z1RpLx@J{}=HAX|M%bu)3XO(@Z{N|2J(HuR5SBuSifi@AKIoH@7HR5j~? zgR;oanez<|#t|`?BK?Zvl05xNDb>8xmH_S(i9u@Z*?u?&!tf8Cs5H;>aZq9wrLd=Uw&nagVhUdh>C* z@8$lR-(!Tp6vmnRbZ!#)q3_tWjw}vtq3jbBvN|4^17bTcC)!ZXo=9(gd@{&2=?VDY zM|Kf_mKB*khIq7y+z~O}hU zdOVr&|39j}Ix4E~dmC^h1VKRQ?nb&xq$HJYknZjV5s_|&97MWXdJyT(p+mY$y55U^ z^!t0)x__`1%w_I5dq4Zxdq4YJj>V|6@?V}CnL94)rx&-rJzyVBRJgyn695F<2!P=4 zMGkB-yK#=>Bwb-l$kfV-y&6#dKK)JWxsV;y!GF6zpG zrk~k-^hgwc6it_8?R4&qBcdNAFRnRX)l}oF{IO;k$bR5W9H8<(wcD?SzY8Jp)dA+J z6!5t$4s9D(cpUp!ZR3EMrcZPBE38!S1H0lAn4{S+l)7kWzEY1Hk8PPVOfaWT$|^2G zHa9Ok>8}no%1q?d)jRTboy{wKQ~e8m`s6rGi#$(Rz+3<0@nV1 zKb+})%QT!GbDEbaSF5ti&jPd6K!=u%@)F~v;;xSL!XG~^?|tmtTb_&}?B?d8;*x`& zx>sAMOVCoO5b+%O!Z^Jr5Yl==Y*6_1kTdj}sL328p*|0Mr}@bZfqTix1u?G|}AqiFK+_d^S*B0R3T9vLB2IPb}!K0T`R z^fj`H)_qpfHc5T15-v4X{9gnLH$uS4_olvJ>Mv$AG!2sus`Z7sEVCwkt{yQqnzh!N zlcpv0X0bglsF1N3;h(Lh9Y|1gP{CtixWg@YU1WqOQ3ce^U3+NkH*cgJki%n<1+;@) zqkBX+>BI@r7Q%&`@A)*f7l)ar`}gNUK+^vIUHw<=R2zMbX>pt=%!scVL3A z;sqcva46BDBADj3uG?f3jSL=jMf#59F2AHG3@Lr;YHDn7R-e@b7atZ9nQ;lsiOHKO zqho5wSW8P&5-cB!KNix${uq}&+4-6}Uylr=^Ur{d0 z=2Utgp=OR1r@>!tFZ$_P%{?ElXN?`g1?hiSo3Bh0^2#-YIR?k>#^XJOJyS}`nCjg{ zW*FOWW2j7LUZ~%0={L8zR2Y5oo8P!~rJmwn-;l>$R>Zt<@>V0oB;BJ#Xvu>2VLkD>Tw-Q@&S+X3JCKCluaB|s zaByIhjh#z!P}0HW;mzn*CpX{hb;OFb$pp_USbhejcQWHU*`*q`K!_#}**hATvly>U z^RwZg|Jo#^~a z2mQdf>aOG5v3a(qOB$*Q`Zz8k#Lqr=!reI7gYjNqdwfyBP8K-gw^Fi0x9P%vu8ECR z`tox2IC#h}mwXN=9ttNoWSu68vShMhy{6Q<=X`!c*0}wpm*7hOS0ABW zD`V#Us+8w!H+E@DdXF@BW|DFh$N8`25AD@z!~P7&Po?RW;>`0s!1r;3vu;D`8@vR@ zBW^-wAu>=W(90e}Xyk@1qri9w$T>1QENXC*S6G=0wy9iGW{+)>@Vz~m6m65&=F8j)O)=pZJF)y!k)5@bi@VO|&L?$Aji1A!qY*JFdVY`emB_6g9e%oHW;jY+M(IssdV|1r}bc=^MKU08TBx@3rlu5`z}ec{`A=&EV2^r_TD z59y(w65r47U_O+cq!aA}{n)kupCc=!kIDmb$>-JHKkSz>gww_s$bFvv6o;0H1i>Sw zrTqvSiUWN8&*iU*kYqQUteLbozDKkx77Wf`a4PNfA=DXbK$`Aeue~V8VI7j;VHK2M^@>i z=cVO?PZUlcG^K*T&nNdirRU_m?b$Q7oeKg~=MF$J_pG}=HwA@ zd4}oDCrM!lP6sE}*W*Wg#UPY&5Y0KDIh8#Om+Xa+I4M<4$Mb`@W3>r1abM215k=EWKH%ryt2`4r+MA-e8DOcx zfw+;*W=X&k6m|`4e440%v@A$LG197_ExeeDTVX0y@t;WmQw6az+;yWV&u~Vpy?08J zAg0F$YOXFt?(kAw*X5Oo(T9iSi7x0{HHg)6f#Q~3E=?OvzYRbIMdLo#0hy0Cu!6YS zGm6+dqg5abs^zGhP4LK6V6Kvu)I&P`Mj~Oi)$@X0>b5!05=$xGpD6l;c=?l$L|!_t zYXc*$|Fb3FhN5GjSOPgdPTsFz;+Whh@H>g}66Z}yv7rJ=%&#Ym{1_gD+)X-y@3j(<1WztvK*Qc^Fi;-Z!D&cJfQiCru^Bo$mw@53v^=%YT#zXV*ug z+HUlx79uD|RL^IpnI>w<_FqW6m4l>)!FR2b*EKwW)=r2tASegELm(i(w2# z(=U8n)m87Ns~qvpssCCA@IB3Ap`+Uk4&7~Slf_~-7IuC^)&nH%x4_-6f}r0=Kfiv( zfK;Si#qHYFb3eFdH9+$H;dg&79XSI%*ievMbrlKMy!-^fS_TsqYU z8lOA{t68Wsa&rlrS~8Lk2T1-OwqPI<;7xAFE>H|IdQ~ctG&@pa;1}dB2^hONHeHgc zr@GY+x>kM};i$hqIP~rG)Ld-N#D}8mq0WDHohAp@W00zCflnN&Mt!X)yqY2O%$aEX zxK`;|U{&U1GF+$Wr))feV$KO;ZH*kFzMfgJlu0oLAaf}ZvYAWBuP1~ zpSK*nBb9g`5iHR)Nj^iL_*H}HjrG?{G&W~00+iGMExkt|^MCsa{BVY?p%Oxn=CHiH zUS%6|s?U`4RfT1Ckt`zyK!gKigHkU^6F2gs`(8r3bKU!C!~dLklLz6taIT@Tkum6M ze(@DWD+*os4!;w7jS1F|tBA~mJC-ulgxQ}JpZKi3t;gy1b89)&6xw}cfQ^DqfHqg% zPWPPbR+O~}k-|!|mPOkY(5%LP)^j)=4<-;2G;3IY(=W?_LP|6kEapl5+C_u)Y9e6& zGm-#b+VYxot}tV^fiD)5h)5xv&zv$fIR+3`e1S)z#p2}>u(9lD@!9GPy>>+Ui#q|r zw2&Bhaq*y(Lp4lY&s<`0xF_{Nj|j$zpMaYofpD{V2!YGc``?BJmwt2WkVy>ddy%jm z5+{OVlLk+U23l=9*;s96gaIRie5DvB5xsKO)=$3rd-Gp>d9b9v7jKYgfeJU6HM*gi zq9npI^756KW5dXUg+e6BN>1KpqhS^5Tb9=9H)mFjDgQjbTLOeXFB`|EKbsPavT;q%c*H`g94XHsev z+L&6t!b+vr)Kw($P4W9@*zTX;*2z+5XW0qD)L`MvXHU_s;)5j;uS!i%#t5)A^kkD` zr5#M3rl#QWSe+V3{bty{`O1|)(_$0;{@p_T{GSJxNw=4hMfGH`1JdIeP0dEuw#E2I`bCk|(ZxciR`|{Mx zV~FCe>8gaoMkIcl@NFBT1HlqSAx?gO?F1ed6UzBszt4}H-dgltlM9W@dwY}hAn}1N z=qXH~zZ`t>RDHwSw-rN~ugcW#dzN>~1N;bz1K2k-_r}cy<_OxAQ&s6(=Rsr1s)pAS=INmzQ>IUoF4=TK^vz#o=399FK z_!gXG_ILgC=^W_YKSxtyVrTf1DNanx$RC*r+14G+#*gulbVUEO^ z zO1^9Hs~5OZ^~EwD7+@k_*Pa>(ldfK!C+Kq3A6{ht*$($0IH9|Hik>@zX0(!knIAZ6b= za$5ZKmS&yr-^$q9oVu)|b@rh-mH!%D9webHuPr@MSGFD0LWw5884qi=-w!Xg%QdF# z*sUbMy#sRc-|?RnOO#uLXz1du&Qik+J>*sFz#1LQ53R|#u+aG>m$2u7UN4aoKEcCt z|3U?9<9ba&pg1$3c{|C5BPbobg#yMSPS~l#F%id=n5hUyk8ZiV-B`1`+tMDjEt#lH zB+`&6?pwV`jzO%PDZP-Ed(sj6To&*s$Nza0kUAS7KX(Za_6*POGw+Dz!k*l{x8`1~ z3$v_CQeUgjY$7|*Z!vvL3@`yj?EGETHOH*8kwPKYqd?ISbi&^}<&Pn5V~W~Cr8$`j z_QL5!9uSj83yYkVE(cW&X|I1S{3|>D^cypD;r_+np{5F?m6_#>1&UJ;b?5b~;K3jPKskBwlVt zprk--U5>AB&kZ8!K+UH-T@58eFZGL0S($2YnKt!O6g(jO}=N#oYFt^WWY; zy+CS{lLK08li}K2`7w{3t<*2cPH%m*Ltb?FQ+lo1d#IX?#~um7`nHwhWxH1 z!0$qDVGF+c?9*uz%JcK2L7_8{o&6vjUp7^+oHWe}JF~$`>HjE>Jw<>}!Zs``_iZG< z!^R^4Q45AIZEqdQR=6pnxy113MUsXnB&lE6zR*Uj3s-w6l zXk20U{CaJCH=OLEN#jdgYiF>?e-;3h1(A_Q@jb3pV+MS?+}3KRM@@10a;0!>`iz>O zUcJ=M*7P0h`o#1^5Ga5>7)`wB&LZ~@DFkM$?sZhjjF$v0=du^y#@fKXN8dd(HRq#W zJDz;gjS@tXy1T!`^`kw|ZzlJ<*ox|r%J7uSaKC_`o>8cyz6W&tPgdhTa;_A;NCjt% zLmrJI_trY{M*YUi|DssFINT|2-58&kP^X%;fzsNQi$m$+ZjSn5ynfS7gW$;&rQ`A( zK7D>MFR+1-HwIi47C1YYpz*@TH^O{=mXhnZNwZJ5?M|-*BmbH^lo%Af`LU!p$IRjs z<#|m%-kah!Gy7pOnEs@}o-gW@aJPRe5c_}O223`^-FAFxt^vbYVlRiezQ57qOOInY z>Bq+Ci+w}FTvnP{y)zHwIs(V@r1v_$H+@NK(|IK7wD(o(efRC#n^La#ga=K2Zu&*7 z1`ae>=wNHsqOSLTAU=rjux;?8TD6NKy#JmP@Sc1rxD`gGPlf?qg-_AZdp?;4jj!I7 zkQ_|jViTqw!|rcqv53&SlSIzx!739)v$Bk~d+4+B2({S$L`=mpEcK38qeZI@qlXTMVx|Um^`i`quAs^lLw$`$HZ*j@;maSUBv*B9?!jw4ex}++@Fx){IUO}b<-R6R@ZzVEkNu2NPA;2kC zrm`>6#ubGXf2SbBLI@XnZ4bG%JxhR&Fsp}ZBtck&hV5#!>5&xICns51!Bbh{uFr`h zS0k)?v9kOucR$U9>2Yi15*53?<-D6_7a;+nrM+Ol(a5(Sr&R^%1}7_JP_uu}?f|v!*6~BhernW2%7_h|f|t|>vqRb5Pcvo=eR^4Re^>U7 z;h;D3K7_UJ*XoaX@$U>(7tt&RETyFu^CLgzyGm{xjSSyPM)#Ec7Z0Gyt*1J8C z-ush@GOyOEU2>Ff$=Onq5jLE<1yuPl39|Kty9`avN?(L?Vh0~zG2g?JBoz}z z7QbKMC@cCxa@}||Jd5MC*C$W3_ILQSM()g+pp9&-%|tFo&?K4yxVRR8_8n|{)Z zh>V+FToJ|%g&~sq@w`kQNi`Vv>e+GO_7jad>rygt0_IJD=WJ8E;J`M2$6*37271{) zMHYMEfN@u?L^Uh|qT0bL+rd^{YeQ0Of6M1MKgb*`CJ?!Th#hZdx8(GVk8x$GJ@opt zn9FEZhwgv%1&V4+NprqWyGbpuDMGFnL2-|Tru43%gNH)S5Somx7E;oQ;yMzz#Bxdh zhx^)~vokGCla{2m@=|=Gd3P71l?B=hm@Wd?dZD;alza<9Zh#Lz>pB+_w6S*ZfXquO z2tI48Zrk#kb*_fXs^ca5=NOvX{FeCX0zSP=pQx0zpGGn7K3}t1ylQE>n1!vaD2kAi z`W~~z#W-L1qnasN5qdkWbvDJTdM*~WO^+!=5OkTaa&`|Nc>)t)Vts7 z>fx?7;J7D65VO-@@{0(n0x{>2$|s`9LWfW2be zY}o*R{!16x+;96z2RM`#rNuH%_M1k7j>C}kMAqF+$@;q>oeKdVd3QqrlJ|M9ov94+ zK9aE9_x7=xDL&l0%0 z6&;7SGaDagO9eG3ePBz0Jw?B1z3z0dfJED#f!`=P7}rw4r*+_zhFFRBl7V*LliuxC?Ka^TaS%Z59J?`sr%$zY17FoL)s zkrSHyMzXjSDOvWYd>xT>K59fp($}323{6lDa*JOWvRrPJPVT=eWh|zksG!jT1<2&& zOQM9#`}K*2F1;29<(9KOKaK#n`#!(B&vNqNq@kNd#GgKOha8Bo+O_zpYQ<1Ukd`BR z^S&>4_5F|c9^$}*KI|P(zO>Rmvx#0h!682T2uc$t%L^0MxCmf!WyB#M!C~sn;tWaJ z3)XL?U~N9j_1+u_J3f4byN^E>2%iSlwlKx;xa5MioPLB09Er&~UvE5p1oIkW&`bCi z6VVYG37B-8M(aF)BX{#V!;q>WatulxMgU^=cu zuJiJ#DXI)a5oOohA|n00OqYcbGbXY*AFZ@CjgU^1uZ4^J-PO)w%tiMn?=x&t)sXCa z5;dj4?-~jCFV77e=1BpD%Ve>4uBOtb1Di(YcE>>N@bt|r)Sfa#<`!}DoD8S7WwAe6 zK+hFK`Y&_h4TkG{x<{~1;p-<_S?}GLcf5KOy|Y-d+gl!4fKE8xy>z3mXiGrd**1WtS1;n} z*|O0eXB{MueiuTjqEnu8T9pda1eOF5lxFD?-{cBY5=}6J(T|^f+EVQS3Lc`~00}d3 z@K9~}HLch^tz0Gbvcrh-}&*ed*2lWZ~S+6A1FCtZF(Fn#orp|rcgkHaW4q1{q16*Axq#_PA1``n&NhBk^nu zi7pJYvbNrdJ#DwUJ@mLdi}Ixb{yD_^DqFG7{?MFCzu75j=hXl`@v|nU%aj~z=5bnL z0-{)rVlI4*>c_W6ZAG5V#JR;-v(S|$#1KA+F z9{fhun(BSpIH};+ja_utkM#-9U-lQm_eJWfcTRb0V?)BqN!SQh72lE>S5uI>gove< z*7qblHbzxtcD#co`2%cJ58Aw$zc}$U?iNUqu*1v1@ca#N-(~#m1QUrF7;%`aVVsb$; zU!0fM$aSK+iQBnJdB;$^r%-Y#SOp1M&cswA&7z-Qhf4Y zIj?crgtkTElVY}l+aFW(Ns9%kqR91kQ_?EjTs_Yi!vCxX|HFFth7eDRbmt{Hw|}=N z`nc;h{!(XgE;jRS+vG9?sHUT(-E9D`1!ZU!vAwLuWlaNSyy!%{w(3T$=c(e6paxG- zA-``%7H~(V#E=w`4Z`jiC~@|4PHCosPr`&=W1||96VH+dWxog_#aL-dag<`*u-|4& zDxHrfZM~@Uz5&y&&4KGb4iJTPMZDhpK5E(SbMfH>1;T{a(pWx{CSjnYx(sMi15G!m zO>Bg=iaRT~{>4#d%yQ);J9}g&F#KdfnG)(r1 zh7DAPcHE{6Z^ggrs!{Hl=Fw=`PW$_6iriiR??zr*vunL0uCFiuihpyjK;pX;$kMIl zCzABMsf)2(wk9`mH#CL*Y$=3mkX-tV)X_ zT*;^Dyx;u3?MyWeXhHJ8)lpM#XjjP8y&*EqmccinuxDd|0fs!mMJQKS)*Iv z8~v%^Cr;(4I*exyqjQ~PcVF#mF+-RyGn?zwvvQm9q?hh1j@O2QH>m)l2?7;+y%s_a z(A_41H-1fCSTT=w5hZw;>=3#5t(l6{XWPN!Ok6S9I$O;X3k`8qJW9wI=fh&7V{?IW zd~c(v2wJTc*SPBE4m#3t1`(3q0cxH~>FK*Z89GkumM+NVZ)pG^zg9?m)5uPg-#bt@ z`n@o2ku{ViNJP6{k~Q*j;e@sK!1um1GLSr?x6bj%3<8&e@Wph=fI|Krl=Fguvi zB$DQq_AF5K?q^r0lh&*2IT~qK#5Y~GSBhnTMb)GI=Algr5EqObSWmehF}VRdITxk* z)MA5<2mv=6H^Y5b8}&JDkGt(( z321RicOxdWM^5tBC=V7)oE`FFnTG%q5plr#Q0}ZQ?fmYC-qGz=J28;RB1BTbA)z3& zRY0SH?~E<`V)l%)r6G$b&;sz(RQ!m5<4MV(|I9npQopi_LTgFKZQ5snP`<1DVGHZO1zQ z`NuFN0v@XhN6)NVy@Bu-%Ud0eQYY5F6z>olDo1DQj~|-cITxI#KzmyRt*P7T?2P0&JCP?42(vu z?wX5QKo=!xS_=iABHgl=9k{S+RA|H=++1u_CeFp*A2Isr@T}zMiDWeeFVTRwN9JS< z984xSy?vbeyzRj3X#2aLX|*HH@z%3=aBylkB{fCd0yB1L1tk7!x81&i6lQ$zUq15^ zEI>gNW`5!9dy6iwyP=2#+08)R#%izLxwz_FoOM;Fb?8DX45q8WzSEDRVI zaS}HFeux z5d{j0X%?!&`UW`3j}8_QM(|#QlKTW^eZ=VLlU{l z>UwQ662qWlAO}K4T2gp9yN9i2pP{_ zr$ZGg=mb)pqUI!_lw2qF;XYQt2Fl0&Rmh^Tn8(LQ8m~WceyVyQ4z~rd)$2Bp;sY}2 zL&n?w-oDJ6Ewx1?MLg3*j((50?er^Mg}cBeRmHQ8<~dgw_jXdj45^pJDD-nC6uZg+dp!j>zBe&P)f7eAg~q0vD&0}S?ZGf(wlcTBe)*e|I=+{-Xsd#?oRtYRF4>YV1%wvNlW0iG zCXv5GR`8bnX@)xHr6F}7%SfNscfpG7}AbK9u|X2og`E!~ig+X$o=oO1$fN*kFhpI;xcozDEyBeoWK z{GQ5sCtffaM1bwDBC^`lp0(U5aJq@lYs;d?OL@Xr6lfhGsm7EbMU(nf51P4e*QS1m|;XT!t~WnOGH$MoSF#x^+VZfkOIz(2*IJe#z^C3ARaVBmYSsE z;&kBPUZYf}m6}u?>0VQ{xYE+z{MEXTR96e|*wC~h)7&(y?9W!bM5oxA<{a4Bj+%=_ z+Os;JT*n6t|Mp^`6af(`ZxB^4xv(nozbI~aMFi3-(eQqm zt)$r=k%nM{Ap4gScHZhOmq8l>4xSltAf|`*85Ya_91Hw7>&H0bcP1$5rqPQcLIT;E z3c?;+6Fdam7;m-mx3Ht4IOjqXqeWXWry8({Q+w9vLMQBaS{1Ltn0$F1bl6hAR8TjqT2`m)kXbM_l1v#0Ga-d+U*vG_@ffbExc9YKo1yK@DaPk$Ws71=d=9S=Up&;B%+pcNp?HAX^{WNjOF zcV2@@l4vfuA}qKSKuGo~3&I({qmCpJdszg5t^xhK052x z`E?Q44fxoz`|l9!!@rJ;1v2@9%p;17MG%jbN*{PKZy0n?p@(VH zAZ;QN9L(2!xPh5cs@Qjv9=l+iUOi5y4yU{gV-4A^=CiYFv^!fwyz{@^jSwul z1GV%cSH8=j0E3C1GOLklhJX}5cuVPs8`S6>jGdJ0??0bZ#E42ETU(Od*qXqKaYbuV z9+qTyVOyP5d(f&e5{}9di!hZPCkouar?3mh&dHi4V1uZ{+I(d?PeY^PiBdT*mU6UM zYB=2WxXzXS*q!#7fsG^W+~e9TWzf>9_*22wL?`@VzwC1?f2;oLH5P(`#-Fr7O%iKsLTvl>XhGyc6fSUXk3FJDIo^#O(Nv; zij)C*Sb*Xw>DTq{jD|*F_Tt7E>yl)aE4qOrpe>+0x&qY1C9?#)7nodA`#&A#<4(k| z?CIf+c_#WxPQ$GBHWaaV0YR9m(g`z8M2ZX*P3#2;ROAly7>u|LbF#cwq~wVcCbu^ZiciZOYA>{%x~P zgLh3*|Ml67{U<1LOM76-{8k>_auwc$|S$O1lKh06u z5;H21|GANW*sR3oI3_HREQj!%8#46?(q%$^q7OYOnNBb z4HG{a(26VxYb8I{5L$<6@uW*4xMFv|cv46c!;bDtCrhQ;konsm)D`YnZJFc=BjdmPA*sRF3z3KCg|e%Rm%8k21;Irar|ns$3-@Hm(IL8K!p`40`hDFOI~iXKsTtRp%n0P<-l28{5-Sq z1?r&#zu(=9O)Z>Zj2Cp z8}A{>2*`geYDU9ofLGPsXx7^=0$sTjx9WZDm*AW2O*c;(k#6scP^pM*s8>=Ms&R_d z$K5AP$!%IITY*sts z>z(#5q<}%SwWAFmn`j*fZ_$zUN=xzts$lbWFDmodS|mhWDchZd=gZY(DB&AXtSvMn zRrqasZQAVO=j3L6P%@$bK4E@iVZOyV2L>t7_{((%PX77iz}O4Ac@`;;o>N9 z43%Z5uk<-L=8G8p&Yg2A^T3fvpJ~WjcPAG*8?+yAVjW_^%N2&@d~rA|I?M>|my*=y zPU1M$(c5OceuZF5Q(in8p>Y^G3;AYDl)X(BoISo(TM~x0ct6cr^HoLRZZ|BGW3j`v z20sS_NWDt7TFhYd9vLi{8oG-w4?<*AGuzxI*C&yjQdqd!u#?0}_r;Lf7dB(Hd|4-7qnq zU#a4;>KFKu4Qhs&qk$a95* zEuUqzXAmYObOZGuwzGaiAtR#~w)qWcPW@(MztZ43*Z9qc(VVVR*tz&@18tON@vjxO zk2%`vKf_G|%CRi=C)9?;ssLrMMF3vVQ$8>t*R%U8-r>Tm(xZ{J{UipEN?YH|SX`yd zRVbnult1b*@*bhU9^E?Y$Bs&7x=Te3bIY!XrmsmK?ESU+fq9_I|Iw`L#~?0k|6?j4 za{q=^rH#YuG~%W)Y~H4PT4d4C-25`9*I^iFo5M2t2Nx_s&-IfgZ-v@o(#Zl8utm>y zmX@MLEf>*K@d&x5d6S_66EC@M15A&yj)P7F&qxVFyCWlKH5W>KY6Zo20KKbSI&oCBw1Gay z5^9QJuz4f=K&4}Yykd9}ho8MD=7enoos9EiH>!-m`ft)5&W$!U13~itd3p-Y&WbC1==I>)u`Wm zD#IZ~KS3ZmB1n`pZzinOM_zAC?oQEXRa}c>E`w}5$ftw%1^Qhj|CPl)x&HxeepK-3 z-2__OBVs8%V#UY`A80s0>v`}3oeH_Tkl8!J4Ba$U3KOnu78Y3 z*<0PrUqzLecV$Ws5s33o2NtO=j%LV(34Z)o&VpH5nke(RjoWT$!!Ii1F4DI>Kk8U? zCzd{JBe_&)5UX@Z0W$Z;RGdi8;^|TVXHS5mA>2NB4r(=E^PmhSb z;bRj+jPBDeGFax!D3o;rwo|&uDEtNi7)va6&1N!E-m?zZjEPxMsrMb%x!-Q}R$R}e zU0WFvG#t;`Z7EV-F^cWOr79yF5CYBswTY`2$pSCunilX!(Zd#JZu=~Jberz-Xgak7 z0)n#@uu4J=81C-v<>;?Q`od1%eo)=|=(DR0s;!X-!;0 z-ab;5w}j(xxXLM89W{w0~% zJU3TkxM?~Zf=}-i334HUZr;-z|G~k5-dw~^*3x2^D@Q?*zYgXf0XjP`+}5{enzfK- zpMos~Gz)Choy^$^s=uc!SkZ39Um=0rX#t!r)Pl+Y7d}6`#OAB9H>Yjd%^~L6lgEh5 zQ;(B3EJ6XrHnZW1SjjVgN_!sYRuY+bzQ;lh4@7lMHT-r_(Pe+}M`!zdx z-5+1kspRV6Q`2s^F8gaDTOY_k+!x13DY{711b9S# zM*Pu=1ogg&+DD_!ruu+H3j8M3VDuBqhiSw7m2Y+GZ-Y~ciSw%Vh`Nt`_Pa!Ir61pQ ziQI`}M_bPJV5B3r)zc}yi}Mrt#d+^c2j65`F&v&!9?jn2)Bm<`Jj-oo!Iaz_$C9e7 z>GG^u@$i580x4|KP*yqvpm$Jo$6sJIreP+IlI6=&EJ6V{%k$OaiWbBd@-c*~!d9ua z`YWIj+NJQ1O%IJ2XawIJV(*#bYsXjS{GU)g|GvPQh_ZX#EzI!kNehQlWpx9My4_rC zAUX}1w&$qmm+8C5DrcCpx#A`Use4H0J9Y^5I%0^NV;Cc09v<4-?^zE@gXepVp6#d9!kch=nu!s?OFCgA$QAm=(2&yp`z=q0d6L@QFcrSEY< zd^td9*S8^Z`qpFVMpK|+=7Olf{Uekno;hpO#7IuT%T!jsO2b%EekvN#@S}_ND3&n* z&AlmnA^s`Rh(&JgOG{D_LvAd(*yCq8|E3SV1;M2qB3BZHN?O~L5Am1}%GUd`?L;8! zNxVFhQPXbBBbb)?deQU(fiRUmyAU%pL&jTKD)iqH}4SE7_F)2s<#ZrhOvrtdz zOFjv$@VXvCQRr8PSkaQ7-2%>}nOTz)6@{w}TL8v9N2^;qy5V@Ajw*#I#}sqV!`N7Vci;je^UW+n5DyXpP) z;GT*}1%o3x(DOkB`Q(o8gjy^>(c^DuLPw~<(#P|gw!qHL*5bG7iXW;xX|ff>U4OMi zh@MU~9$|pmzX4&A14u6NTvM|#mOtbBr!W3OT6If?A*M3cLcP zf5UwAHbzKe++#!ad(v+Q#kF1fe0=h(;mHm3SeLS?N(l!}u4u z>WzM3U`x=clLg{}V}YJ(zyh|0V(wR+=>!fAXmw$;;UA>U>Exq$cfs#HC5{CLn6F@SPnORI`M^r}C@eo?_2M~N0L?zbuA8H%wV z%~?pF(XOuVnIuqRq~wbUl%nDY&wu8Z1~MFN6zjY()1`2_PD39*v`s5df%X)2+vZH` z^S21HMLX+Nq-Az^X?oEjlDdN3M-TlBPesW3I1x!{;zW72gz`%Ex=>5H!$+O_h|AgI z(+yvWjhguJu_%$zBPQRMMB~LDApAUGZfTva_G^6nicnBopUP;(2<0(9p`U3;&nSq69GO63$L6iRSCfSZMf(9T!GTbqC$!wtODf7 zP!P09Q)hDhv;AtTi!?UlU6k@XBP5B&7&9N*nXyqr1r`V2;yhuX|?y=f#hKBh$GD4sng=rXN^-#-kc>6Dxh zdlu)R`dzl5{m;4t#esg8Fh92L~v;fWUxAI1k3p{53P3z?tzMW zPRmMZ6{q49<3vwDr9-STuT?3^+X_$CFsJ2gEIb5Ek9ibhy=G*9Dyx9Su}&A3hK@E* z<#|mCb=UrGIc=uTl}uGPYL6NkRrO2}GxLw1Y=a>wuQ&mXdw|wcW6%#YbhJX}M0yY44at~g)XQbL0>F~j+!7%INr|eU+Mh41^XkI>OfuoRa zCE~Kv2Cn@^-vW)q_XSMR-g}9La<{{`8~q!DFPMjZPxk-T<^Z}$ODt%Mrbu~J%}PoQ zI+CB>2Gy-?XoUQVN{LJkX_pZd)(GENbTKVY)NI61j6XF(-}WMZa!*t}j<-OLV{oWw zqO_x#0MKl*qm|Ft(Ljj)%A+8vTWz08E`B)@hLLQVjn46<`k}A-B@Qh@Wnpm)K9XN_ zVrdC>PaJdlDREdDqKm+|v$NjF$c@#qp%UP)pW}R6ndJg~NTHi)mEv}Irq8*#k??DR z(*vJZ@2Zrcg=sGO+({^(Z4b3vcqp@c)wXFn=T9bRiAUbUIjb#;Q!tYsMqOuq$#FZH z+QT0+IkN6t9Coxn{tY-naG?5UkCdv6f`}W7e(V2tAOi%LM!yMu=j%$}O0xT;bH$!& z--6!Xw=QBJUNf@-%Nqq=vhd~EHDg!CoqltTfdXg2Ho>7o8$?TpizMlVss+ozeoEH` zDT;F{oKV6%%|aJUeD6I^?+73_M7LtIFn8510j_tVrE3M>KV9Qt|{{uh5#lGXnPb!^A4_|?71z(@N@Q$Kh zoNi-+(cpZp|Hsr@##PaUd!X#irll2-7HKKzkP_*X?(Xhx1f;uLdehyFba#W&-CcKl z&pG$rAKx}#X4cHC^*sN2h7#1Bh$y&Cw22tagqr-%F<6%S5W5kXFvZem#RJ@(6Gh8_eM zK;}g<($=45cTk7%#bMw03H-(#6OvW=jbi#=r16 zJ%gWocx%!rp&P*Wwi+%;O94Xx_^yDcrkOG`*q`ji>r()^Sv`9vO4&q0{f|W zBz$FFBL8j=qGS24kn$l~gh?Z0IrN+=U~gVD$$Ci{7f@PSod8od;i>V8Cd4|eAW*jG z+mRozRwOe^ZA>%51u?$3h=#0x31rIu2!YH289VnX5YWgDVATSrNQzXm??$_ z|9Ab69V{^SjqBe(2&}g36o<6Z|Es_LC*EQJVZSMP6wHNV*nq2=U1J-3a>;TA1Pf>&l+9hW8s}N3?+w#!{U3clnVD? zI_NNE=EexLK&(d?iDC}>xdPPJu$TW1skbC;oRd#r0bhtlcV9p$RX)Z~@sjn2EiwdCFOiPSH*AI1vGtVF ziWiedZfBPdoKh!*h*DNwYtb`Id6`_0f6Furi+j!@uWVm>4^o zZ~tKV2}&b(!#fG$BVG9m+?qh!_9!YA9l*kIgB6jTAAH7_yWs(FZb1}g41vyfZY!pz zVCOJ9&v8TOBZv`W;6Q+QG0LyAZC%GUQ}=ak6Eeg$6nA^%X}WGWR52%YXt zn!Q7`_F=Ui$W9rzx?)v*Pc8dHT?lo1$dND}P3YM}@JUGh7_f_s8BWct`t+77-t4aNZE<)g4nG}?x`jl2 zbL?W$_gGKm{gC$%&}_x*TcKviyUN&6^U>|u8VaX-Q~SBqRRe;d>w0n(uG!|Kfx#;K z{|Pi1{JgF^(H%Rd{qG0v_I?6ofplzy1CnFCk;XDLZZp-U!Fd7W4_&hv@$v8F9cIQW zReXNELG7=s6I(M!-zh?tJh=pwmLJaUO#6`>H%baOt*r0C>rZnI7ZE=9kt;L5nblp8 zR!j@~YQeZ>zcQL`UWb znrShi_J5+0=o3D*nS(1V{5m7K?@^i~h>~4$jv(M^xd%VvKwpM=3t#lU`qlAG+}yu* zfaO!6^UF%K!Br?jjp4{BD9T7Z&UBvkb)oMvh}IKF@$nM^?u@-$dq&7Xo)^ zJmX_Y8O6344MqH_^+x8zo2@iWMN!;wq&ppb>V-3?Yv}m_T2-5G^I>5R~uXZHQy&NX!egB+S{YSFI;(r#%_q+YXP- z-!%=UMr7&cOdRPjniL|TDF#O?2$#Dw^zZ#Y_j{<=EV!Bfze*-u$DDmz>CG34?#wt}4n)gBwpe9c0gKqmXzFecB5Xq7Bg^PaY;SjQiqW;#KN3?h zk7@QSBN6OD5CfE%p-^H4+N4@*&xsq0M^-YDu)wx~R5gHhS%isGUn)*AFlFjAA+F9d z9ix()++b}rwi`EBcDQr!s}ad-%EE)q1D0l%lz^`Mw!2T%fVp7#lTGVv*zjfQ|Mu>> zDKP%q3bbXBBbU~Ab9rZ8ZBE>2g|JXRs8%B zjbA=nnOSK-Y!6$6#}ak0=E`o#P?|&MpJ;+^V^y-shQbe3lp04H^Mk%2@pX=5CsJTE za$mpJ_Rilfqy!hXESTalz+SLN@29EFV&aHt=WgG96V%BSI^YQ@Rp!UG?D*D$t=;TC z9i9c$sS#w#Jw!ZDkiIj%RXH5L9Z&E8+3t%KUv_xoR37%((+EcpvZWZmG#8+2aNw}0}EG%P6P?fc1T9V zMFw=8zT_2vgxg4MCq7Mw9bTq-eFgFp*u&y%O{Y&Nm~f)fQl%SIlux|;ePVojkaR&@ zxdeva(A7InS+Mk@v2y=u1WTbaINJS%yT%93J@JKj__4wgs`J}`lQLTaQfV7NKLx5| zh~vTz87MFwbL2sJ=H9f1B=A)J%TvVt@65}f z%lDwJ4E5WfToMiOcOwt8QcYklJ-yK?H53Txe1}b6-+y^B8%&`1QwfXvm>MXC_IOYb zypr2^q2M`%U^I3Fq*)p$J*}}%rw>20AhW@F361(--lEWUkfc131!lYu6?Bx5-BfXkOl14nNvFKz?bk=;d3)hh#RekwH@K2xd6+a@*nJzy^g3mqB>K>)wj3GM4sJgALu#G zyy}1-o-|!|T6~_H|8k`3>%K0Bd$=q0=h{#Lw#CKB`oL8BY7-CLBFL#O9gH0lx!Jo5 z=;8u=G>D!HP4GKzM4afsNI-<0)L%=f*m1{D0A$Y~8@^l<(si<0^_=)ELU#LQj2{4U zRwQ92D{Wn84qE5jv1U5oeVHG}>|?`u+P4O4q=XNy5KYppxwDfh9j{qSy3ShwHU~d% zN;$dxzE=2cfvF|y9v8x`TYFl+wb5H{9N`KJN*~2pB)Pbj`5-B8Lv%s-gdIyT5PuR7 z9H!oKy+$f!(y)|LR-y=H^Zt&2Tk)#Bn9e{=wy_K7$=qil+W&VDr2@J<+BXIc;gc~nP=Lb@Ey|zm0|I@?J&k1OnIE9*xIj1<;y2-$3HJ-Ekt65zZTuln;4F`+psj9fVoz` z4D^?RetQ)RJLsOy#7Q2vNPJ($1x{!Sg2Q`cLhb?eft+jwEnsFi(LfFulJu3 z&r4=#j;Z~`I4yCcN7k5Iv?rGAc^cNwXh&bP>ijKAOqUxc>!<~0#(#ie zn9Ho5mxQHd>JmpB5ma)k@detCVFKX#E7kPK3C4!*^W zS#{)JoXZ7R$67af*?o)VOr@G29=u|m(_fx|S7L)0?9p5ll+9vj+KFVi%r3PxuDmncJf~ z_058tyu(k;L8xqf&llJr!goZQ4+#NEJ?=6xI$>N;H8nu&nE<2hAz6ZyA|USvf5FO3 z-ueD@Or&~}0nnpxi)6E`e<8A9kUbIst7iB5D}1wXn~KC@k|EOy%(a~O&hFW}2ni=F zl`xJVi5H$K9Lwx7amE-K-c;{KJ?H%HO(1x!8piShXK87@TYoVPqmX#`1sA>3|+`*x1!93IL zEvYhLOpp$bHL7Q0!5|pCM4ETyP`bXzLhKDMMdwM<_FcazVMuUSbRPzYfRzD)K1;6u z`Ob8t$cdGkuQY#hW3x*$Tm-yIIgw(MX?^QI@4yQ3#B(-~+cibBBt&wJ?u?W$(j$n7 zaIJJXlJCG~Vq3ouPUuBp8$lC^Fqf)VnSWord|H8f)8PV06M-2liw9e$0y`tVZ;}Vy zmC8GKo_~z&LygIPZ+>JhsC!P=4RFDs`+gX|qX`E1B@D_LDtO13{p?Yh^+8Bs+#PA6 zfsg0}WXNyBYawJj$$-&&Z*afZE_bT3y54*5{fqej(ky_Fa&YT}-B_qYs3qnpMghTN zG&KJtDe)^I>|7gq3H9ufVZSUVhEOy88j@+{X1X=E=&1Pm(8I?Ql$laf@(E-RWR0=B ze0b)E@db%OXfuvsgc|_uHA`=-rT8>dj2&LXfd(k_yw$?_Ze0`->z|x?vs6q4De8Eoq}YHU)%t_4yYfOo-)Wf&8q!C@2dhFB>wO7@ z;}yk_zo##ExECBcuk!^ryD}$~awYdcQZFJwNR;NjHYTmt>NYbvfXrEjTyC^>RZ%Hr z%y5Ye!$I-T!WzHnFjG9e7^*n^E|OM>Qw)$$)CTjCEJh9V!s)Na@0 zk-SU2at%d31_X4dxxoefSRB zQ0AJT3V04orU1jy%rBZ1gfS6_>Q||Gr^?*)DBL*`=J4-+ahk1BM$fz3aIHoTZ$%xF z6I9$9C7k?=dbeaip~{T^R&!>5Ukt)7cRwh!$6c0723uh7y3sLZp z4OY4XoCwyjt=sj%t$xEe;4}r`1rLvRn^*pC@3Xl#dGzXpdmM2Ud7?uijtJx5aIvwh zhMd$oj!niuzx=sS|CPV#Cg*~v8}t4^cOaXm6`qYZM3@Q{Jn`S!Fs%`jM5K*mv(^Nx zxWF#z|92o^gSq3`VrM697d%az)XPb*j_x7>n!AMUP-V_t#2qHTC-N3lKUoiQ+i0&* z7oj5sBpJ#CyLP)d$pnJ=dVrc`#qa&KsA1icd}OF$;^H|>%wdDIq+)>8$A=W>*qmNm zzRPTG!H<5{SbHN9svI7j7F?MygcPc_p>pNx=eMD#VfpGh&!9u@s<*TweS(-1vG!DHmUV!$vP>5!1y z30)KA*&wItNGrFpOOUn^D!#f*>2OhEJ5q3l!og5smXoT$>LLx-WnicBdvJ8`ZwpgX z#M3~Jb@TyGs8K4WiUTB3d49x$pY&z{d(ubwAmc=3CfE6>jfUEJhb#D2O;YQ`aa zQq2`?(kNla3K6SPm1iqCDfyym_374y*iZObs;7BF+#F^J9Nc7)w0f=2% z?cb|A-^z9z zq#q*epRHXJ>gW3o$g8QNm$}D`si?vONb3H}l3AsKxp(zM7nFIaRG2QG%WfxThm<=n zNo>cisB3;kfq0WVY!M1|o@7SuU&2WQ^6ZwcEMvtdh*?>gjm{;j>2}`M3D=8j*F2mNV_k8L5my}#cv(oW=hINhHxLW z2i69i+uQzv)lDafQec*eW_%SR+tiZdJj$As4Bmv<$> zc0O2VUx0P}^#dnef1p*wUyKSB&8AdPFP){-;kMVkIxeZi!}_W!&*08|v!WTH_IyJj zN_!^!Dpqrf%cDb;o&}$6vg9yg&OOq=*3z8EYHqEJ$m(@XF00dJV9Jk8}%@*Qvvc*V?|y>HlrG-?mv_D9+4l= zmcK-GJbBztqI8^SeRMU#vLyw@Pe6~93^nINm~i$*{`L<&G}}D^NwXMd!$Gw8*Fy&p z9Y@x~aZz9LBq}{FUS>5iC0IEl6@&w*?YVbRN(|jgWV{K($b+=0c?nOJfqbhGmjMrz zauV-QAD(al?l?Aw!aFfV|A%|~*bnErvQ&n`O1l6d!vT=k>(WbJuJqq*yWj8BN|E=k zhI&XEbC9d>XogWH55pneW@^Em> znN1Wh`x2r0vPb>$gp-GthrVSyw6x4+?cPO`=g9LU!i8}-c8)msBao>pLlrHRKjN>= z5_h8f=>MA>;+mj@+&0bLQCn*5So9^LzV_$r;7vrINH{m{tR=U2Q=5a38Iqbqmw=p3 zO}cKaXbrR!w69|+!$c>SOKhtDasQ_jD;1L-fmk*!go#m8)_O>pyJzE6dj&>&x(ZUy z{QlW{DbAJ7xgqF!bXk(figC_d(LvM`|H=Z4!d*%z*TM2=oXPZqODH(_jV*g7zlS=G zg@8JiY;k-^)M> z+T2pvM&eXD5dPGIiDn|E#`4mp3xugsx?eu+tCbBOB-NtznXOUyZq!OK6xw@qH4D-q zctpav%oH)a4MiYvK0f_v-LEgGS2)nTT;3M;JbTtFG9>2k_j!rt(oF0749=*beMWQd z>ZLYd+1rNcksbuC@fJyTgh%vA(@T2tO{I&&E@>63_1`>&5Gnl#mR5G`U?L zwm?D(3>r1QZPMT(A;de6!BksbU`6>^mLAX#uv4|Ny2AO1dbSgWjz|Bh;v%e~xi9PU zpMRQGCyiTXct>qK0gX+F2pmN{yGOR_4<|;*OE$3!)$DKcikd=VFR?f|cEdUpT7x=^ zn+qz!r{qgLhQB>w!@oZ0@|13J#q&VhMd&j-cUEl}sYc#oKcxCe&;kW$`Kl(<>3&Gn zPWlt3G}c!+ab+dJV(;CR$AjOAYOjV@WN|rT9Y2$nmn2)|8J6hnoFCtd_-v&RLH)I`!O7!bt zMGP|fChID*Ya@%&+lH45K^eO*XYEk&v zH5ri#P6VyK+%5FnN}7Mlb%?8LAAoRpUOWA`*Z$=BXG;`gV8?lmTDsP6Em6p(qBQw3 zuV(fFr#?NNY9G3Jc7R0Ohp*pP@*USPW-5haILZYNk z=FNg=tC$i9(X|q0^-K|v6#-pYiV0Ga$nZ+{086nDvH^rN+&7WqrsQ-$Ya{h`SK?P{ z+n>PpDs)k#jTV%}Cnbc}b9WCwlZ+eYK|uTd1WrS>RED!{0{13|ACbO&yzGP({H%j8TSxWJ%l zxt@K;XiI5ctga$N+C8Hi%LWk?7VJ&v(>p9q(zfay=an#A3zlVdVxw>KTN{-P<`30H zL)V9H8@YRRR^%lPG?4Jma;5Vwn|Fun$sJbGw6$gbmjxgpr(t(tz%YJ4cdmI~`V1-c zVP0AfD$lSJvY6Lf{*I)s?gEGvXi=v+3XKa|90ZfJ^VuY_3dE^uI*=P;3 zC_usy$uPca6^ho;wrHPD7uQv*S;RP-8qCHuxoPEBheO)={`oG%&4`M=DTNtrMVGW_n;e#Qm4D%y8J_|*`^RO| zI|Rp1mf)rF%Uzkl$9e1Pz^EQ-dZ@%VRxGGSFtcM_lfGYq~clI`}TmKy1hyd zfhRK;37mXI3T47_Ui)fC#*I9E04Clr_(1*0zm`H?)2CZg*S_&k=sB zfCiib2Q|&|-$bi8jQsU|Mywh+-pqsdv*A1rjdl(?yQU6LCwMbLKC9R^&Q}Wl}l8|$9w}7gp9uN~QRx+>6GZX*T9~{zKX$XVJ z1>=AiBmA|={xD_8Yb?~Nz{JY_6FqCr%7?iqBs&k+jG^K_?B)qi_=LN?c0r}-6UlxU+V^c zM_5qDiqB2yil(YO(!*6(f$N0}`2r~mzTiivnsAvtCtz^LXLFb~MRx%f%&S;N#bF<7 zD7U)syt3ICd9-3aoM(SjR76p3$NiKqcJ61IJwTeiKSRY& z$ud1cdpb9~ICC`%S`$x+24pk}6)bs+6VY!Du628gJr@{uF^lE^5?>R0*p1hGh0M@A zJ9Dbh*y*FIp1J8I@-r><1SA^Ym9q4dL^^jMJ#B$sPB$j9OUaWP zH-30@vtN;bAkBIsZ!RU|JYH9YZyFUKa#Ln1yM5H>ijOav66D-YTn=li)!gFOkC=yJ zZ25ySQH$sgnE++K|2_QK0*RWFHHQ?Zi{qb* ztC4U>0Eou)2ba#Sbw|mXj4qTE&d^f!NKO;=%LN+-wUm_0vg@h@PW=^dfi8;v@ zH;+MT>x|5_#0Y8BP(4Z?Mw|YsdF^RtcQFP|77m9a@8GOn50*j;Gsg=OApV(0{NKt` zY4kXFM8f1fNKy=3vwS*hOhcjh#3_%=qcoVgd9&?6z5xLb<4>xFjg-%4eonv&R3sov zUA0yzUuVYrgUu}$kTPOr$P9JF243-v-fl1)c@sQaWv-5;RW5f(lkG`-V&P8-`5LFL ze|0fg^u+wWO2;`--01o4>z`pj*;ri8ihdfB)0GoxG_Fs0cQTtsX{#qvla&sL-1HPe z!`OaixCJXKe#O5<)Zp#0aMdAt8o`H(?(P}1joolBk$vaVb-FmLuWXb*HTySoU_uY9 z=EU0d2LktD=-LI3UG|h4%+vf5@R{C!GnxxEm~QR|kC6tnys>0)Q8m{u$C|x*ANO?T z%jt|0dKPXz;UBRRaTgyCsd&RLjjCtBa|1>Z~90@}JPWZ9DC%Ouy|$wdt&j z70FY#^Y17c;UB0Eye;I4gP9|;^%={^teFuApB;7?QqlNFxd{;bbIrjLqw}V8W}?-T zdRXQ=4-T4a6aPDGDCK5C)UBqJE9jF|-Hb5b%e)+rdQLeyS+ZL5%`-P?wv}IgM#u0q zAP^)hI-C#RB8ESwERu#@Ua`D9VCp0r39K=Zx)D|jP)pz2uw^l*Z{r3340?TV@u9Bh za8Tnbq-cEcxmkM@kKG))ygHdu51Rv-hdV?iOQIpVlWC*cLXX5*XAJ zFPMw$ll2w*)9F=wd!N~A_#!>j;ng$UjyjWN#TRQwGVYy}w}&fE_XYBuoP?dDosq+vRWSlee=1* zL_FfXx~f=V75Fl;+xRRInpNcK?w2kQf6MxcNp{7uAB+u<14o?%uNmE;nOZ*SY};3T zO+&rYYu0I&?TOz=ja+og$8CigMy=jP*URd6YL0Lgona?VZEM=qD&-wjo(%Vu4Vi7R z7{$wt>z-{Vtc}lb7XxC3l}~`u{q^gFg^^VcSry7L!R~tD?hc*&{l<%~qoEyf+@Z#` zsW043O^3(gm*>Ovv_~024xPI=QA@QD`-sZ?{O=>g9@o~IA$j72*($6Rgh~azS+_Fe z8E1C#JKpIHGqsMfUr-&LYL~0&(o{6%hQ?=maKvWsMzvo$$GN)K5D+*bpD%L_^s<&R z#({h342Gwk{QX62hv`V?Y4_jVk>-P7n7-gjM?~Pz>FB61>gqXC5YprZN+VnGeoYZh zJXIM!Dgi;H?0vH!e(?noudl>Lg(O_AoijA9&ksDa(u?Rqro|6Ns6FtHLKH@p_ETVc zgx+oeBoKH|=5Ha$cRF6d)XbVWL!BQEvs0wfUGC-++&5(V+U0q(Y);fhQXe~a2hsZ( z7)D`Jr=_JOPvY+_Eo`P(ot#cG(@Sj#-Guk+PxBVytKa(L-`wUl+5a;S&}G z64z6~S>ib6OoCi4wx$!zxJ)>GA_cckf8}F?6KhU^@BpXF)7Q(}k(96``xAv@{*f`p z1!2QnQ}|~@HiXY>ahjWW(8m2mVS3#qDIeJe)i>NnyzMm#8M-+9^Rvg;jHs>mYY00n z{GRIBCB~O!g_%rV&9cKZbSJk%b^}kifzdhOxIQfx!mz(rIT_egw3lRQad&3Anu~%E zUx&%-xNVyN`2-n*(5!>%8)~Cdc69`uIU?FKBPgLzTn;O&TCYqxd^-G4iC-YWcIm*3 z!(Y}7R$EEfOn&V30xFMpckrzeOnk_H+#rsyR=nN#iMnTZS5j|Bf1jlHqmfklhN`T) z^E-Q?)Cf69d;@WIE0lHrhdltzg{cTRB-$p`aoZ%i{A*#8=71|oCR8k+^z%2qTEyZHcNL9a3@cKNmv)Y^ zBNCHIBwUUT$%fb1BU&oXDr`eMB zN`5bS$Ukp>ZySNsgKGCyEDoQ`OUrpS)Z`gW-(3@UTs-A}zC?e0Lg6WVO6z!1a4>zf zT@)3gzvO*jgf@p5u*dl<6RzOeP)2B1Ih1|s)G2e&U9>5UCV==ZEG+0YSQZwDFgU!U zm`#5>2eTM#_#u&nr9!S^;0^xXu~Nqkey8C7J(Ah$c(12UUY8W3**^t14cLXCLeNKL z9$s|h&_BypdAlMLUd+D)C_BNnVK8=>s<`w;0*4Fs$OhvoW6aQY7vr&r@E4Ys%nw+t$6_lhg>H@1=z)q zX@M$yU*)U*@u$8uNx9J);SVMD#v9b-w<-IC!X!g>V<;at*@1baIbIxTu)Lnn4|UMd z%;s$T3d8)(PN-7Oh*b2WN0vESd1+`lf0s=Au)1ya-WN|(pE7M{j+10_7?VFy-ZvH~ znva(iUV|$x^BI1VS9)(y-9^)be092UIt>-{BmLePnr1T|O$WR4twq2PWg9pF8`K;0 zt98e2s*2n4{!Dmx4{JkO8KnxLb%K(|c{5UL{N#=pF3)eFS7+NZR{GyuUr~qYWNTVUO4Qn7@uSMM z$A3%y)3pKP84F9j4=c1+Z_pRM{=007_}!~>seE9gUSJ_8ZJ*X1TVuNMM+j#taT5y{ zs$vKH+Ve)O+v$v%!_%2ynx7;*mRhvl@ZLLqlKFN1vVU3}GwNqCZe+#Ea?y7|f{e8YRqSQm}g~0_zVk1qJXHtu?xmP+R^QB24)d z0b>QB3}3p*R{mHhka;p>y0LGP#nIul&31Vwbel5nb)ZLpSn*A^nHv#$%(jUMnYF!> zVx&uk9qR2`#5UPKK8bg*%>9K{&3RZjTk7QYN+p^vD;Rfr>X9GVMDp zaL+2BwU?0Klz?Vl1xmfBJ5smD|JeawPMa(&@W~5xbs`x&sGs%igRhMq)q%V*>(ttu z`Li6$FBitb=qgDwe+j17FRtTAMsI9V6imjG90rW6P$&_r?<5Deg7ae@4&vU?ZNqHN zMOvQaybv!ZDPJS3LxHUlW_@ovfvs%(5!MBbNkURGOf7+*Z;*l)7JY?C?)$&fL3mbY zH3D0iyk4ww{J$%`!dKN=X6o=hM>qEJnvOn3G-_uf`-u4Et6iN6I!9NhW%u6#$-1^& zbAF99Js$)VaP4s$uU3Hy+fQCSI~ZHPrNLQB17vM0GfiI>CUXlV+pp1e!^)q)v(z$J zrW1n13gfocp95Jd-Vrt0^~u6Aw!PHfGxEDH9jTl>TIf8LI;|){lOj0MePt6mFm%RMU`= z5>`NXPQ}AH=k<8Mc3cLzzfEomynw&AL2Cq)poJ;rwUzzW#3RDhP+*nHe1|tzPDv91 z&Na@h*QY>BdAbu7fAe9OK-Z=zfCuu28!Hnb7e;g ziJR9~+>R>El!8Egwr`!f2k)#B1()*Z(>C5x;I&h3{gwG@ff1vIM6?+b&@i2(z++*c zFkXT&JxgBW5XM21Tw>?xUGXlg0-kH!F8eXYO9-}}c=y7b*$=Z)Ne(@|l-#`M{OK!+ zDCzizsOJ^XY#ll&r2wC0g1&y2+RIC^?1<0PBZ(TqfR6X3IM*AS+1t_|eV~eSOF0J? zOFv``oCKaIBBaNeoGkP+Y)i)S0*^f?>h_P;@yprh{__a0#$Wfe$d63%g1uYhGxgg| zg_*W1c@-_gH^)am0d9S{&hEL`8nyCfS&^OJsgg!~v3M#T1$^+Ck=qWx$O*L*%Td@F z(Xu?VvY^*{tsP(5zZWdX3Mc(|E#>}|89CS)s@vN;UGaq~d{2Z~v+(4e?C>Xo1mt(RcMNv|1c5&Qt{>t@<1 zel;lTX{FFo4pK-SmPt*mX0n4LWf0=1CfKif@?3l>5`DPP<<-Epx;}4^HGds(fP5i4 zS}=soq`M|M%ig}sU>M103~`1T=-Lq-z}t9FPM!J9FU3ILc#fOy8B7q%DKl^4gTD|B z`zXr)ns9G2xdqEt`lGVwrE9w5ZQ4-ly~?`h)Qa0xr4xqoFGlUOnFsO>(a}*RIE1Zq zYT+bd1lOYtKPFkC!6tdO)%nrTbR3z#`5+A+I0#56s6d^uwQrafb6v z5}6ITh17@fULf?~;RcpseN#xwpe@-(a@ZQHBLX?(noV0!J+C8u?`+tiF-d8+d7DMj ze9KU_yQ`>N%3ZuJnh-4q_2z1qULNQ*e4Erc&xOlau^{!*^}?6-28pMuSs!LB@iM=? z`q(f|d3Q%TN}!HO0P@bJt&Wbg0~Q27o4MSfin?V7dg~$Kl0GdU0ZZpl)L+{W3Dbdc zf!oYzO9&fos}>i?y6rt)0AfD$x&BKsy6QsACW?OCK&@g9qZLUVN0N)sV zhS|JNJz&tT?PV%w{b6MA($e3(Q_I48bHHZv36Xq=Uw~m{SH@ftR!*_wX^Cg2&P7HU z()cxTg{9-Z&FyfD=5m&b&w%f-IM}Pk{F__;fG!m_((?}*&Toa=-%KqnnTTi|uwA;| z2clxY`7)gtM0vcz=*DtQ6_9$%O!B@7Bv;H~piEGZta|lx!|$ZIyln+P|4+S@3o5Ea zy12c)iZ2wL51kh9XBDW7&7&Ana|P3L309vYRwt+x>W@d)XJx9+<;Jo@=5P{V6d3fp zCbI4R&X@gw`Ps+E?_)Req7lOPZISX;srX@gNzT7GHiT}$a-}BVnI@=t>;~shB`~l{ z+>=!RYa~9oIQYRM^a811&Ol$rA?4}3sfcX?8+WV1FHE!egTAhuPhMfoGMh=1rsturh^&VQjc00$8+b&>aKx{0ej=W z{V`3rSf%C6j3lDWu_~6NA}?}!SROHM)*-3JY{n-^9=K+ny1&W^X=fVK z$-?-}66U;hR&Yonfup_C83QJtA!w=@VS-x(q{FG2u3&w9sO$uD7KMFv;KM6FD^`Sd=qf)#mp%fE) zKpk4oR$1ruyRz<6+#2@N*MWNKp&p!;m&9@KghunsFivy%t9b}zi$;3`>|@YKu>&q` zUvHu}YWLGo|0|xc$M+1BXgrFgMrCbhS4o91xFWg`;Thm*_@Y}XOL@fiUz96Ntg)3? zBDKQ;G_f5%b@0DUj!?aPl6^p|kl3YWFXiUMu0cI#**63>gDcqVgLKn63mc5mElx-U$o)wr)$tPQwy_(b(|ifbEGSyb$z&OK$&d{CSz$G_m3n)_mgJr*W2T1kvCu7 z55AlD;^&It5(!vp`aj0)!9o<(7XK4$kSK4$6fAiR=Eg#F}QaeBfiY{$L@`7 zqjb=xK_w$pp8t-eyr-0T7o!>uHugmm3aRDAX%==?)?KY?i>ds&{MiR(jv+;LyV)fa za2NZ-B~jijTj_WNSeOJicX!_bR|xK7@N9?t*k=Fh-tLnXv}~;$)z*o^&`UuoW$9td z;XJRTM%uTE+u^%~Al3OK%*)0g*035l)Fe&*Jt|NgY)|-xwI&J;vn}^IrNL?FAhel{ z(stc;B0N|d`p=o}l-dY45!p<4vW#~W;dj;SR3V&vms7t6QOqG#{2u#d`qVSl8|a^9 z(wy4G!(L-=Ic0oqf^x+krhK28&IRd$3MZq{Jhqr4j$GOVIFxTF1lO)K$gl$B(J$>W zHs&o2*OSmlnA8sS&MSv2^!v+NiB^+Q@5kEg4L(~SuEB(vsx&CqTKeKJ0rIPXQw8 zTbDwd$!MOu8&(0`m=S^M%1|$wx!5^@cUY>1|F8)aSk9YFuRrLC_^;=|ryU`wJ1G`Q zl&?W=;wldj#z*LvOq$E+IkfmwWJiDTP>H?y#beg(AV&Eb-N~`&|1^{G@iC=L{l~+h zZWsTj z3uJUr(i-YO+;@J_$fuM8UwNKVlL8BBdEZ8IGk5%XnMXzV4NzMFSlXpYK@w{9c<2$D zK$+g~Ly!O*nDge^@s!&sm-&F6n!}&nh0Spj{mCj>`P!+a3x4q?;(*70gFLd1#`Nh1 zq41Y^0om?v{+|ZAM_=*$4d&5$C zelbuk72a$@W`zY?e2~jlgMDuDf^lrr}mAAL`L2W0u~bUjs)slh8i^)*;|o|;xAW_UQ3LcflXf6oHuQu!Qwkr}Yr^0Yi|Oaqk?bfX9&7>B z4O%d~9qx|H8ZL_DurjXQsSW)#`tFBAq@mHRL|z;W3FUYL{eiN4@~6KE@!bp+zb_=j z8%4^aX7fP@9&sV!(Il_R=55PnI3VGmfindyb%)$y3RKmvD9_^$?9=cvs24;ap(K^L zsdoD3T+#XU78qCoCaJ%WJFx1wky7)@VQdHgDPTl>8-BG7AEm%Nf3wcgp#+Q^Zw*Bw9 zkTVZNf!a!wo_-NpLP(QabN2tVP;f2e;5o^=q1tBLP`6CN-F@<**F=XZz$(-@?A%tg zn{gW!c^`*mwOB>d*bMMG)@PiL{QUfK6!@K|;mXNH z4Zm;1ntg_2A^g|^U#D*1*Nz-B3;L%x{)963QOmFMTJ*42T#=H$-c%^0q^xBs?xSP{ ztLegk;>f{uz!=3zhSVNa&d1rLWyW@VQi6pzd#2B$g?ixBIh8RLR2fn$V_xJq^*S_|Oa~OIVJOF^VWi_kuV)cd#P2NLwNEB&+2qNLQTwpc@2Gq)BXNg=6$GR7gaw z$B_L;^&U}7rQ^okW6Y_`iZEjk%d_O@sc!v{qr3ZO2tK`+(??+yr>93Qu{s;3cw!sGCc4K|~&-rAj3 z1r@s(`TIY`J|52$@bo?$I8S2q`i+6qANMB(u5XlsYy)2>>!z5-9Xvu_8;#PdEp092p67&EFnvGV|QQ)PIW?4kw9`ihHjfM-&r6=dwa2`gw`n6j@Iy z!lg6PRu6x_1n6a@J%+Llj{h6^W;~~bC257exWSx@dgB%#K`;a9B=63!lVz<#WvJD( zNNkURjtPVFzv;!9pPP;cCYsKb`i$b06Rf)E@xyYSLMh9aL95WQ5?Q+~Rijd|nh5LK z0H((Ppm}s{K~DtDMa365C4tt@s0yNIj*gzwE3Oi|I)-9 zjwmM)TRgJb_jLh(tl(H`@mUi{T6oZ_ozPw)y`Qj!aZB7(9GyRwgM(A{A(G85u^;SJ zj7DAn&%2P3dV4?;bZ7vpa=-vy5Cl41l4P@$P##7M;Jkr2tp1V>p=cDSq`^3^JXunV zAuh_8#NSl+a}UTtoYkB2%at}`LF<)p7S^;&MB_ik?y%eG!=sDi8=76O2XEkquP6|B4PKd}29<-K#h@VVDWd#` z*ki4)Od)}EX-k0BkFA92gw%D7A~DvG4@DH48p3`@z65kw3YA7LOwK*lqR@ZP)}!CBdpi{lVDRg;W+$l;ZsQZ_C*$Rw*3laPmIDEWVFVuiC{Sto1f7 z&P@4lJdgzSSI9%H4iFPTW@Y@Ul9-s7i@JCZ3{9bAv^UnZ@dXH3D(tv_^7D{(?^I%} zf`dm<$+0M6WLjrh?pP4N;t{91EXaS1ORT;H|Ma^UC;kBFU$2hSgP4x*Pv-J?x>>eg z*fE*HDL=&*BX?9gh2UosjLy*VdpZetBo~aKjzi}kQn7A$QC`~`*urvOOg1>TL zUWyr63n}m060_ozOnb!OEkva~oMts*al|9p0gG%+%wl`hKpECTDIv_+^c(?@sDrLdSbl(8t3Ydb}Z>B1~YJ`P-`tC zxh=G^nUiA(5#9+B-yI}MKPEW%PU(WTyvb=<$Gm*meP zuY`xQ@Uh4I%GTvjAI$%z8}9x$+XwRZ=3?`3+$Q)iLZF9FUUs8UJ{trY>Xt&Goq0>gQ^Dig{kC~R zV;=*&?%Tg%!S+vWWOm&}Nsd`Ym@ghj1o z`sOBNZN0s%bw*9sQ;^?|)Y`y%n2tO7o-ZP&?o-MWtpYa3<|nHoofxd8VV zHHZ7n4_n468zoi`|E6^(v%U_~IV$dzmZo>zdp)%i6ypP>Biq>p8Ale=btv4HrIf^2 zu?uFR7&nju)`UFcyb?0e3jgE~+N&k!Z}T5)gD0n*jaD0WBpfiolTT0jqN!l87O6%!4ST_6lFzhckknoeOR!t9 zKS9J7DPt*9FmzP)HSXswjpts{d^!q`xq4@gb?YX#f0Vty&lOcpSo>JuWKGmr= zirYm=cIfkqdsGG8bvJJF*f8>@bBEW6H0-MJf4FgJ3zFpS+t^tAP1>dYqn3_!R2hnU zR5rHates919~m$h1|A5PPcI7E{#T!sm&27H`edRMfV;zXD>3Ecc0Z_y%5SxQBm9Pte<< zfWES0VXwBv!RoeQHQT75qDY-Kcge-qD(%XPq1Mj&W;@mxnlgD0Ygb*k+2G3OtB3C= z7FD1;!n34k3d?t|%iiy`bU+|J7TkhozUAAQ)7{p5jJxgmuE^R?%i2~b(Jy|AR0KLq zf|~lxEt*O{nxFHiC|U`lu7AaredGB(IM2<_$ze`Z#7~8Kbu+dQKYWQ& zRC*WZ7wM20TxM| zL5`J_`zyQtJY5yKHS0Kblea9tAcro9|{B3tK-Od|sy> z`LCdL%kOHhFv6ewG03-1rfd+Dsh``Vq1bKcSlUe^bl_p|Z1f}0aMMW*tXQRMOl0W@ z%)^S;Dx$2N^0wbu%)$UWrZEVV-|rOPZ(BkJ4k4~$H{zUT+S=)pEu%uOufB zmIni`mX?)g?|=YJi9hpj$dALsVCQ1IVqR}o7jLS=5BCP!?+sle;YCPGYub(IXZB;+ za%x$Pjp`35pE}epAmqgK-A`Rb?Jd$5#h(p(9rmuDH?K}Ab7_~E2|Q$~-t0TNSvsbD z#nG3!FL*wGH3nid>1{aWn4{chO}B2IB51dfB7u|Cb<8W%AQ$(d6<}z3Mh0^fq|c-v z%UFl~T0U6(MGRx;$SPp3qwfo(=$&&CUM zs$O^imFfl&xA`SJye=)xeWKDykt=7B4h;IEjd2MfHW7Gv3kMl)lGmVX@Q~!+Pvgn# z@+D^8*l1zO+}?UL7If}C`y1I4-rBbXXHS&2R4Cbx*}$A6?cBAbN(cmVeY*!VQvq?M zRo%B>zw~?CrqHi<0VECG8c7lv4XWc5v5K+l4$GF^3q`GB6a3BDHYek0SdRzVFfCH5 z@i@m5DScgE9nc)%D@PH*rL0;s`jjQOh6%Y-SdW|!TTaa#AQ7qN1saZiT}59puDQan zyrW*`Pz~JP+AxQPtDAk;>A1OXmRU9Ub(n9w(h%}b^pR(9mIcCBOTY_UyqE4UPOgrA=ME5uT6dR{OQ6&l!`bTF0{l5>5{D%y*&USlWW1I;v{{i{G z*(HX_qk|$oE{;+kE!UbTNc@dZTB%WRtX(*iVdnqX^wsCdTVl?yKCl9 zW|(CZweY>!@%s8&_^|QZF2A%)R+lQ$MZ=T<$ouC$_NQ4mwZUAHrGwfF!e0eh&+*Dr zI!~QaC{8n}rhoOz&g6cidS^Ly^zPM!RAdRr+X~Dso9BU=ih|?yMo-(TCn@MJPg18S zy@=n9I%)<_F5P;h9n){w@-n28lBGuCV2*GhiYJLpUyU^K9_kUza97OHhAmrVNCB>OXkEtG zps?_X3b*unongrvHQ%FrJn)j25(&ujITG1yuEtEz6zbRky1o6&PSCPBS`E%lQ`g~2 z%3Z6#{c#Kz_%ypolWw>A4|3HaSW%5kFN3mR*{P#HXaX=b9PYHS^K5NI0M;yjN5&=E zx4xp44|1vBM$RtOlkoNPn=C^F^tlAHPRJhch&z5RmvupejmZC}5SXVdfaC6!vxGDX z&Xb-m?mPx4kSKnFC^hpgM<_dHmN_0hLm6X`G`k7a!x6_xc>B;~lWd9IXF>Pds4fR5 zd=YwH);W#rhqwfbQ#=&2m5!VitHQPp7X4kz%yl(c0krKHQqIDGuO-b$ROzLq1F%x< zv0ia>*HqQySY;t7_R@Nb0g*U?qC;eKyNG#{){-fYl04xO)nIu5)VDubjn#}$dCjwO%uababATGhZfhDH!a?fF%B4iu$Sr{YFm=u zm3rcJy?*n^klwrSnNv5`UuT}{GC5;)!Q4te*b#4d?&)n#B|{OuV`bO>8hwZ>q`NtI zMEovjUR)}GvG!}CJ;pr!HspB-kC4g%G1Oj~@IVEg<^1qOg7Ta|jM@h5)0;@HQUqr{`E@$3CcE+L`b5?`R_i}`1xveTfQMj^g8zl6DXAx!>xM}zi{%wQc7KXyG%seb`Uz>Bl>2GLAkhEyYf_4Z zJr$DRS-q}IH9;;v7C+XPvXHS=B%ynbH=p@xMCwMbiGKh67TuaRB}ParKjn2=nzgU^U)`~LN^BmkpOTsV9pX5!0)LVmQKr8!nIJNoy9#q~?FA4(V2rQ$_K3&D6x>?>x30jckZH4kk5cDkN_L@N2N9rz z`(cATcCrG;>56>UnuK(Qi)SbzZ{o%oy87FTS|laAe-?9DL_EFU&Zu$<0~qgL89+z^ zqXsGs8B6uXzt@!ynky6 zZr|MQ=VoVk`v6D`(@i!B&0}ccyJvZgtADv1y6GoZ7b0!%gCT33u zh-`Xp0jRXF-{D;tZ51;IOZ$@gh4VtXzlpb`dWp5nmK`24XBKUu*swaug^VGO@5=9~ zyF&D)?yQ~Q{Prt}PcB%>5$cT2wL-Z%(Xku-Qj^Bs+&|j3?pxkm~28%TkjXia~rD``}t- z7$zlB$uXD~gsXc7*xh!IK3>mdVf?UiYCI^%PqYqz;o0k#N+Teqj96s_;!IudN;c@yB7(!8qx8C=ZQ#E%@N-)SB+P3@wd4Dq#sHho|qCegm~c}tAP3>`lHfWQXv z^=YII)11e&?M?);R*T6~zj6Uj*m!lZW))#Ul<<8q%BKoj_l6w zSJl6(7x;y;Xu8oDrd+-#9LLc(Mqa0uQsqS4k36_@uAuE!9Pi)Qg}5=JxB4mJBLH&oP!lFfp_%6rTfEVTd5P22f@7#@SaH>K!4jb5BR>UZ`5TyL1GRL}v$LSIU~6zG7!97ZKY5iIQdz4(Za^!eMpy z(-q`G>sDuThp{YVH5{p_%_)J!d01#M4+Kkp_9$Ij;BxNS2KS`Yp!xtWyB!nwB|!C8Zd;x_~IlhDslfP-*(#w z+n-iPD45av*Vx8|F!zw=8Z#Kzi?qXfi2sRy{0LBCL%h_JL2~V^rRoARR(z7_Q<(vO zP=FaJ-ZoUFML&#k=O7ytc1kW{3HTH<(z~5HIzK`98Q~qOoQ$QN|JAm zu{}vfOW|}bb(TG?Q*SDVZq<09sZX2hbjquZt|<3djD7GaA9!NPQ*M+^On}?q{p;Zc2>VD1mY?s4}@krV#>Z@Z+}>S{TreZInP#) zC5vVHuqSnvDWtn(3%g)d=gGt7;?zY`Veu8k#V)PR!oCKNM&e}__Wn|lwP<}m76AP> zVm*(wkNYpu*DDBK6D#!boaG3=DlyF)ALY~9Nwe~$;65NFhwAF;eLQOaPxL&JS4{o9 zTIDDeGx2!>Mxe;ME>|I$bTl(0E-|icCy`XNPX=_1wJf4E_D8=TNKv)Mz+b4FG#};9 z)ad5W5iQrwl+PT@ny63&|9awKXG8bN852u1i><(8VUf5m&od83_0DLdk#I5Db@+tv z>w~Fiex7mtHRp8OXY`PGWJh)qeza1!+93Shv>Y@wDGH$lSx+pLN}IzI9)S^zjI!~- zh^qj3s}HL~>f^XEC$Eg#HlZ!eU5$3A*rG1?XxHIlK8sfU67r``XC}VxengFa$F5;X~nu^#eG z;ckGiY_bLF@P@_K^qU*fFT~_DN^TTy!guT^!_inJ?X={(Y@5|y(){%cmUDPU4r@3h zBq4qE7MIOoRE80TRB~7wqIh3dvxU5-YydKNDLS-yx>NLpQ6U4q!Iz<2n*y8BDkcOJ zp-K18%Ug?*M;(vg6lHvJ0C9SfWcp``a5YQ$h%pc1H5SpM7~#)+N9(^aX2{&e@bs^0{XE(e z5@@OCP~myHS|o0yeft8}NSt{6zEDxEq~U#OqzH&Od66TvbKAnD>)|9PX|{5mZb^-@b1sr;#hqVj0j+NaLx(W4}zfrk+mHi{;$p@Zss}nu0E^Hu&Ww{+>#cA7*T-e3FPUwT` zLH-N9dc6prNP07VC~b27ZwfYK8=NiPbS+=1Ts|H4{DPF<)bhp9Qe{Dxh&)gdu=mA=aEtt3eCk*2;DyN!QMdr5Nx`ox)5nN4R0b} z1s47h7wj}OzpVsNA?;0QeBfQ(jVQmV5}A<0f1UVL@5|L=ACPG}2S~lwdd}Qe_zS%i zDSl%Paq`l2?h?rMI`=~PcG(HG?hX8S!+O>SPn+Pydg$vLHZ|Tzz2G#TrvFWQqISlp z)Ku-m*HP>$ZNSyb=t`}t#xrHF|6hVmCBb^?wHqJnTO&k`|W9eu&X# zox+P+Zmb1Hm1nCflzUks?PecB;URJuYZdd966@KhDWRL3bS86W z_jh!cV@&rvrO4Crue%L@=STTCZO(RUNt2L&#S3!h2>fT2jXN$%_|zL1uoV2qu-2wu zxYk~5Wx49o$}5%~IaHcN?Nk2W-w@$*=pHmsHw#xq(F)<^%WBY8?<7%|dI}Z0!ON{z zk0}Gu+ugGm?>4kw8GdP{X=eKuC&*V#0ZsM;uvHXvMgtu0rffW>UH1Z<1U|k@5UTLg zfeY)05RMZWw7-oCJI=PY_UPJZ-}^~&&%#`gz`K<{?vlO+zI;Tf(C1-4%53WA`%T0$ z5&v!2!(fM#sl2eeU}|ULFDDTQXAAgUBA*QEJchBB-AH#jYx)$Im|ry z2GA{l`QA`q{1?6uVem(GiBw1rRTe5KR(h8KJXN^0h5Vk!WKVlD{^tF%yOhf7o*qe+ zsZLD;@1Bu)4a_GE3*7jJWATkL=NpRYd3}qDHT-^q7*htCYS;}XWDVTtud_&@#+g4DiMpFu2MsEalNDnzC21V22 z6BWH#bH1o6C-RR!2LB?~exSK0<5UOFAhq%b_Z`4fey>>gRS@$fFRLAC0f|eC;!THR?*?Sm#k6})#AfAx zp7~%y_3ZGW;I%#fvjzgfN)g9VXM}^ENF{pv{txf!S$kX?7YL~wWP zq19bi^!Vvm2dK{Y&4975aED5MqkleQ;v*^~#JCu{nT52l^w;>!Y1fpnn6FC^=p*3W z-%V~x36LoWndyP%p7+*4C>jxF7m+Q*_5A)DqndzUbo8s70BzI#hY{4?y=GUnO_G(B zHI3tVCby!EO1dA^Ij2wM+pbk^50_gkd|~1Y+Np=T|3U)5b+4xo@DF7%2O}Sd;kDQH zvom;+?hw7PGZ%(riE4LO3PIBT(3h~K34(Gct#a{tc4$ltZ%)=P^rP1(;Q07>`5Y>8 zLs7o|mQp)yLe)&XUtIJ-p_;gr3;ic58*){eJiD3vjEaCM@fv@p1+0UA&Ufji2i{$- z$`jI`1zfy(kkCsbEJj$fC}veTzz@QjU9jD#wYu`w1$c;?pI5}(kr79{f32ycWOX_V zkdpUV3LytQlN0B?_!RBg6_d`B2b|DAV#$GJPHs}wO`3wodIA1~k$WNAlSHzXv$EHR z!RUQ5oK3$BB%xeUuaBw{!;Z&k5E-;~b{gil@@>&LMWqv?jjF~%M**DGPOk){F8yb9 z03qGM65xW?zc|1*{+h#(aGeQ2R^QPe;7SrDhrl{YnPz4G(#7Q~V3r#;;IV6rpm1u0 z_3cfjU8G7AVFRNu9>|jV5>HCPMSmorl%a;el6KtX^z{6~W>WZV&k*2Ym23G`Da+&M z_%p$dnd^y6S2j?VcTv0wcSDq2IMZX4TrWyoQMcCDQddZ|l+E>%L_#`_XT5v6+ue5en>I;0wGnBxR=+qU=oL`1*ZuedWJOj#s?!#mv-hCS-)q5>* zEC|I?I+E!mKpgy$KpKX-tnJJ1-P>0u*(Ilc7%~u#39jW*#DAMJBB^1-x*Xn6h=_5T z#*+&c&P%n)X>jxKWaEt^2fpno?|;L(AeJ8x*?IbqhfJ8R=DsbGw<;mVJ6gA&E}$(X z`cE1l&cUSj8tJTZ>qiIhP6*)FM-T@(Ps+Y|xmkglxp^4IiWwj(fYD?sS4$x>!7GNx z-0S((>joAH|KeF zLSITCvw0t5kO`gVZ#$`i2ycJaA{ZwKxQP z7sbN&K$pmKOz`$c@zAY`J*S%9(3onc9Sr6DA*o^#=(>uW?S_Qhq4|cn$N?8^N0g7n z8STnQz~qPKdmLfSL1Zo90q#oNKSQ!bGR=VN>N8uA^{@~2D{!puy#R~<%-erV=xDt} zRO%;H_`pZ1Jd5&yqvz_6^<-PYQ|RLt-KV!#v7l|?`&~-MVTD%HTtUArkOgxKzH%!t z<63Cz;|664kIHlje)sN|@7-`Dw{=06TxKfAi)5}tq2qD_&QA@&{0*1LPmR`Dr5BI@ zrRV_gZQI71*>&=LMhx=r4JgARMPW{0NHZWIb$mm!yoUefb-0r<3hJb(z?h&TGz7sb z1Q$o%<;a_-v|=~icf*|Sh4@&^Dhv8>hX&$aIzBoV9eoNeEGYs%@_c6UMm)(k%7Ynb z5_!pWtTT*uUq-S|-@$LT>%4CuhWoH~C>wIA7XAeZvESkJc{Z#2D7dOR?7t}MS)Z0jgPkZ!xzOO%TRJZPTjABL z&c0!!rq&Uap2*9xmx5r2uv5tH%=<$kq48G^b z_|8AkE^kh_p-P|fTiwPDo{WDdSECJC?Z!2yGnamY(l1SuU8wED@eae`ZkpJS=AWp_ z*x&vF6T}=|%ihj~K*+DkZ`;a~Gx^VXx?1lT4eJxc9v-RzypGwzZrd z?+q6Z$aXom%$_BNhangFi$ZORx-T-0vY2+9oNo~{9W|J?76LCDx;b+tDW?q~Rn7&r(Dndu*Mh5dsKll=f9#u$?QZhL1FQ$$dJjJt6xS@$lrb= zq~c{3%bp9aEh!oCc}j|rViEveGRh#mP=H8hAeNVvdrWwem5 ziS1YXZ4LUF_DRU$6PyRw8RFl^=Cac;N{v^mV{4~@1r)#=jf$_Tk0M?oTUb~v?QnjC z69TDHM`EtZD^=f_PB%qe^vOvZ`j29YW3k2(L*4~s zqz-*}0w~QYtqL>HMp!hy4=?F7IBa)9c*mGUAi=9#%PS8JeESzX^*O)ls$++GF9kd~ z^d9*3L&yQTrqFFVqN;wo)V4kIG6p^4R!Kz$JI~%8n#se?ylBxb1xKl%(*pa9v?Goo z5w?Vlw2+_fa=Gu9Khnx&UL(*A4W`GnSXd+YvS^lSde+M}L)ym?02;I*=C+nRYUyzV zY4;yLZ0b`*>7RJZi@YBFMF3%kqm`xpDNuWUzyCnLTvo#}#ar$t4Vk64)CyGbX9z=D zX;pKF*1$(W0&xd}Q?6`+@8=eJ1N()qUnal}&leo``TSPUPSZB*lpMuf^-l#bqjWhp zEY_>qwnofg4DpXnvd(!6!frm9tV0QGO}_oy8M!sqsX@1uYA#j2JOMm9q!LbF4(k1` zT!>Qz&EkQ;4!?0HC&pDo#NEx+eFJ1hx4;^u(wiqfvSM)+EHImE1o~w5HY+~;vFRP% zJEUFibNm|tl)u-JXJ_O&a6~s0-^8F>f*;=RlH-pED1jA>6Us-wIW&7;6Ax_1-918y_+AvC$V(BT*emI%MPJHRrBt4>&f~I*jI?{vtbFk(x(ZxxT-d{|R=fbt&P-|_ zvc{8uu~yU@Wu28Xa*sE`n>flD&F5pDPWOh6c70VzxiOq;-O4eYY$n9}KFBWlS#g`i z!K0c;9iz#fxItLVW9fh+C4oVMfsQ71G{mjzRC}@BZR6FgRt@ON(&{gzZT0x~1t|7e z<$oG+QJdc(+7N3RU*%5@hfv(F}nI8>!!fy&d_48-O?aXUv1kS}te5RZfJ$X_c zxdZ3gnCAJA5U=KRk)plZxps!h-(#BbpauqGzrz67KAjt2C-2)+hNykOfNJyV^15X} zC2NWp`dJ6imxTUw^Y+gDO%Us0Q5-$Z$1tS(bU1xx;rC`PvYiovJ*mZ(b7tTY5~Jpm zpWoC;+%|i)cjGvLs2d*zY;bk z4!ApaBgjDP3J z-l)bsy968(Wiaz|aph3Jo+Bt?E4sqgR1eOu*jOJ5yr|+q-+C4=CN7}Y~{G-+38 zeaHHfc+!F8l|{HpIwe3B{n6abV8J!49``;r9Q6K|A+e z51SrBQxWR-_y}f+|I_2_0sD;DNV%FN=vgTIA*<189&p&9q1dRIxQNzdt^_C4DGf?z zy@m7h$>ZO1vSqcV-1>A>CX!?`Crl5H;dzk>b(bdDL#Zier6L>C(PjylSP;7r9dr%i zok3$2O^0Vt(W8{~?(XiPdCTE;M2Uvw3G$42Ks?HMkiHR3*k->zeINK;_{tYHD&o62I_ zf{#DjVKnJ^&)$(Di8eSBdu|mz7aapImg_{m0!KkjwrA06x-L+GjU}!=(=9HZeAq}5 zj@j0>^`;XNZLUXofK0y4004v(Y(p>O>@>ni)roof)_h=#dTZaKCR^g&i%ylM4Q%HL z2MohkqXk;7g&2}M-iZ*>q-gww5lAm~!D=ome4BFFqb%qhF}F_U_UeGa`DKEpB1mMV z7@uJ;=c8tJyqmx3)DMe;THC?McV2&qg?w##EvFw=osDAylI3(>vkaEQ?LDPM$IB;h zr>EI;jF0|_n;KKc(mv1-X?_}(Z6@ivzw8bQ95%q6I52)&U7i$y@3WYeGQA7_!|+n! zv`J8^wb{JA*bBaJPZf#B9-%a>QNlIT&?vE~f;Ra$iIBw6aO!h_4dJ zBvhbwx~S)7_Z}<{CNIq+VC86W=QQ~u9E=<4Lsnmk|tGtR#s0nBGxvgTbXwoCp-dAgv5hjhSTPHiH z!TvNGsmfkv?}@TjYIL!`s0al&F#dwrj4IVjm{}o@Oz-jQ?3ofokC!hLwZe zFs!$ob02p-8<|T%3|M*W=ak^L)&Sp{ZSqeQ%_mC!<#u15 zN4A1%J7sc_qnBXH@zO0%j&44L>^vY_p6jtU2-2N^NrSWsLwyv?9>~*KJCixHIvBvt zpmz4D_y6K0Q8VG(ou^WA)z}JTm9;ZoXfx!I1xioLL`o%iHEa9GPA$kCSM>jeFmicB z8u4jx{&?m=3w^y1_S69^qFq0AQvT_fSZ;Ymw6IKx_;kb&Na@c@=t9x|C_uwpE}x-Y zT{nmG?)$t)>iGAP0aJ0zcV_&fY z*-iz4hhV?gTRgoCMP1w5?qyF$=u&dj718AmmG#$T7P)7q||c@k=_St|XZO!hbVm)3Q%2 z=!A>-J|p&8{y4qdV?ojWW^}!HDCT%9o#Iutv9&im>BxR=(GhrisdYg zn7SN9aef_zVlVL@xD8YybhQ4QeAm4liF~CVvUU1hl-uTfq~|;D@|Y8r30+f~8P2Jw z`Qs~W^80GJ3Q(ze{O%&O`RM zFP=H#v;IZan4b=iPbFf??G!w5(AuWnIp&gwlxRm6(hg?2vMO)_PxxZoD)>( z#1PfVC(O7-bf>a zjMTH0sWas(a+vEc#@YdiM;g$n1U( zSVox9Ze)k37`Svx6>#DPG2@J_R79J-$Mdr))?vR@(+o_1Di$W47T6I2ati|Y;jYDA zmkM&6{U5J2xfYdH2`u_mwijS$-gWb|-ikS#A@=+`_&XL~Q1hLRbtmUU`5f^EMW;_u zGtROGMutypVN}XI55UCC!+hfA0PQ4gOOD%`pI_J{U0bVJ(^w6`^9Yu03juE?{kZxo zW;j~`2Gh?hz^Bt8o7A&uJv@>=C)#VL!tJyo%$6j~&u6npD(JqsoM59;+-RsP>L)_6uu51&b4`X8M|0 zl5`AMWO$ZuNeIt~jgKo}?8#f*kBn$kBmY#EBl_XK3o#y68GqLh zr!3%Nrm?&))$sc@gYqTjHXM4a-SgRxlRw9vBY^s{M&hD3t))e$DUgth*Gs{l``v2} zr>J>n4IO{;s@vS-hG+0|5j~doO-f)BVvOA%@_lZV1CsiizvzO#S^r9~iL8l`scqCS z!G)3{R!@JPxlkL#&83P(XlrTcztYiKZaC*ijo&pSl?F#Q@Kl%R)1M@NMm)9pk`jkN z8WHI~Ut{tI#?v*a4aJy;=34Z$Fuc>2tABrpq-N$O*)yR!?~DiEoG9Y@GWv1*@Z+fc z&5I1d{_W=bHggM-6~d@6H?r9hD?!!K8!Gvd4=$Mn*eMnD+BEQjYtV(*X8~ujd(+p` zR+{&f){jecg&#`%lX+3T1_VuxKE8mTDYLioS+9@6?oaLFse*EJk}`nK>mpY9I$wiw3wbmCbDd-^-IsZsy>ySqvTfj_WIABlqebTa;zqQt$>QOhJ(WaA$Kc1tcehlf)Qbv_iy+3y&unPCtqs>WnNLKnd zbe|rKPlp%Yo`5;UkDl7X?|mh($oV|wjW47IsNl9%0Q+oiU(WS)*dX!|%wCDhAB&KZ z9JK+zYS7TT@U9hFQV3}iz8feP@UorsKUSvbtPy1a zVgj*%N;+9D7HAE(C?=}cN6Lp}^8XJ+LA$<7l($3K&pjwd^787c-OAp0gbiZ5{N|-s zZE*jd6UThDv~Os@g4ZsMPgJ!H>q{AKr=m4LxiY}&WR>R<0(=%U}225>A85Mt7F5=;Y)qHiH{ z8(MTHwXFf1<}4bTni`c^TspcCqk;g=Wl=I0V^rXXUc9WD<#Gdl3A>$e7LK zPp_BjcWmqj8l8n>ROs*^eF?Jf{K93M(cPu~b8qC6ZXr&&XR0}q-!f;2POYJ#-ZD?* z(?`DFJSX2&Ofw@V1-2oR|8Rh^MctiT{~b~PI-0|L*FMR=0m5n)fQv0tpqVE_7!W`d z!o7u*!k`gFzuYgrG|2!%{kf@<+uK^qrsg&WBrL4c ze0xxCVqU;Pgc2sOQ-m$E%1-tCP5I8vdznn~MU`JEMdlTl#-0MOm`~V;Evj`4^DOJ6 zM$3GcuP5>PtxbA+;_*k!xr-y3TSsGQFW{fyx}}&+^ZToc(JpHkY-eofsDzAsw!S>` zM|#`MH{t>KU7duh0f3^i~)I5-a0Ihcm5XLueGRw z@ZXfRs6{vCT^d-rtRSMC3%#mw?If56JUXfAxp>xd4mKLMcPSbm+kRePk1o}_wT^>Q zPNriyN4L&32R&L5nfxqZa0IhR3ITIa;%LG-95g#lKTyJl_U|j|q)mu)d&vd8D-*>#5IMpI^8w6h0dX4XrIr zwtNHkwiBm;C^S&KCp(JX#Q2;T5r6v+KQb@;;q@5zvhrQ?kc=r~E109H^1Y8w5748N z+SUN~z}2hQER}jdpKVsfS)-!0$r_o(7!@{?TLvmwok24$JK9@qCV1RxOD(I${DMG9 z!=j^RT*XR@jB#|DqyYGcs7s-@D@1YU>4dt%t8#MT=&DBH{>I^9@9z?V{OE>-8PxW1 zyB{dGa9POVq5l3rYz5x?hd_{%>eRrv9BqJp*fNC-D*}LX^U+W_&diIhUqR2#02#=c zeRO|UP6K2m;2ravt(EIZd|D=ppJ(a;We@`d%q9)G=K+O5*9blB)tkp3ejw0E z5AGYZEKTQCKEu`Z4x~D_voyfm29#Q0xQCXHhd*X6DZ{Gy5op>>9u#{BZyzAL6R_156#~xw&|ogHQ<8$KCkQK^ql$rKmS4#;9EBhB%g=Y z)NZMoL|1I>@0uF88?4hn%r$^ffpJN;axq3lDwDUkn&rJ+WCF`bk69-0g6h!DYde1I zP)S#BK+aTjN>*M{|3Mk8CP64(2I1;yg!RdeT&gMo%f z;D`khbaqavQv*{obH-y5${Zn!Q5r{d-=6WH1FChg9e;O&8LzHPEn6|MsgD7n z7+bA@v;igy)5a~XOnW#3~Elqw*rTjl5?SOaBSP)RWh?AZHrj|s-o(NHfcEdA#p z$b+^W)&49F*$%6|XDQMsw5<(-MznY=rEj^8JtISxT5N27WN@HQh~f=PndoQ#r0oN$ zHCK=`=vd-$D@#{4zBzI!Wh0!td#=9SwJ48V|EY_4<($v&w*unq@+Dv}g9}Nxl6?0tW8mIfd{2E{_ z*&}eR&E%JZIq8Cf|E%f-14jG2>IVlE*ffqE+OHu0d9AO*r>G9vAGd$9G+^yt0mAWZ zl~T?tS)-1&x2#7-MV$_Pb#G61-p2a%JK!AW8_TPvff*M)JpNE<-Tpy9NwgagJ^{NXO9IE=n{r5Fl7;BX8tI|1e4x4 z;6(a0CxC#H;x(`(uv-eFUnV1lNF9g9k^3liJn*P6eF5 z-J+?hE3kO+t0^ytGgd@h%AIQp^2s2ej#0Hd-xyag8yq_vEVDUN-<`TfmgzwHNrI4( zO;!}h_d@7*Z#>XcZpk@{v0RNp83Gnld$g4*_4Ue()La-hhm@u3Us*9n4(<;$MDCHK zm7@qWK)16Qo17L1tWh;GZ>Q@=J=9%a-w=pXj=`{n8kx)~ykndsPXjnuu`S4uiW`ux z659OvZ(g+T|3~gSVfB)$4?JsZQ0P`tuwQV@I{T1ZRSSbg5IqIi zmv_^QGRyC~>sUeHgvYXWc}1Cz@-aFoP6M+lIRWbuY$&Yd!(Pv`P9&kDzOL{(n&$}H zu~eIpU=`=;>S)WW6puMzAt8R6yn~70$03!iJbw6qdF`DK%sBz-;v~%50QCTBL?PTi z_&ER`;d>2RIOmeeAatpuL=cczr&#px|4yjvl9S>!@XiMxoB#IAbLQZ_Ve@BS{gT#Q z(cDKtdPpBw{}tnp57JJyeJSB}UMxDBn^(lpxM|i@3d>3LX~0EdeO^;gFUWco*Ch6- zioCl)_qZ;!Z4zq|=$C1G+R?w#@UE@;65JxPnW(+Xq#XwiM~>y-|)JM&rvjo{@x;&deJ)UCn2sg;jJ#}4x7|nDywrs7t@8Yfj*gn<-0;$&d!LR;j^c=nT z+S@jh?OtWeswCtv|sFG%K9 zT>8&F0Hr-^@|D&2)z}THr&Y0%yn0jT5W6m-7l&iwNvXP}f*r?Oxd#rnjxa1vYt zfb{XfVCNq@bime+U)P#GOu=`_yxG~_QSh{${X<~26j_knf*&InK-oJta{yaZ;2XwH zi4FkKNffTfBlq1c9ql#${OoV7?Gu-AToD@)Nf?x6h(fTJpvET(>NJkWmtK4}j-!g8PhN0p!)Ee3O3GX+PSwoXRM2c+ zu3g;sP6JK@71BToIoM>Nz6vTBzTP4!=&{{h?bdK!7L{9E1SBJMt_V&*nOll9)~9Ng zW5_2#E{={{$90g!D8Yj(Wb#)j)#-|y$N*qi=$0z2$~I$h5vLmV>GwMei=fih+M=wF ztM>Yr`n&{f+wC8%26zv8PL6ynd32Qd_6hnML18kp;()9V+IP4AEe$j^HQ00_?&hicf>;x-q*vwq`%CBC%<}1;QbtItTskw#V zgSl~pt<^5O%MTC_s`n<9RvgiN7lmwXZQJ!Rx=pIwef*f|R}#Q0Z@g#z-M^kU!^-ZD zgN%+(DG8&+CSpXdn_*E-S5ML{gy2zXuqhjn7aO(MLXGpw(0gSj|Qe@MVD_g8FJHI zlI5@|Csx=cD9B<4KsL}fSf0VURwB`Wy+2e2CT#lGNIDzQNwL>nd;6r#HhSVu9xd34 zwWk{D#iCKXlRL8!GM$1ZJ279ecB3SdlUo|tzN+#Q0)ygQPNK9~r+ABkX_G~FpLg@m ze*J>^$+OQ}T@-VoPd#|A`Rng|)7*RFcrl0DV}ASc>&n3Du$23Njf)ZL;qFt=;`gt< zX@NKghTWV~j2>(8m=sfr%I871Tp*Lb7WET!%v;-6%vl8)1_hvM7jUAGV4w2*E(+)q zMW$_`_N(Fd6_q*wur(~5H^}a=^yKncpigQ;hassf`Rh7=;q&1aVk-p<5Acir6PPX} zmhQc=$xyZ~O7Z+%QMM7)x!c9m`MJj)GC%*#@6DUJ>$f|SZ&If?h*6kNThN)c7Fh#m zMviC}NzK}-K?r(|sxJK__upGo&@){Jyzj_HCF?5*GCDD7k?%bLXytGS;P9^2Oik)y zg+vzwdIe%%q=#B#j{r{f+rGP7Y&1W8cAnOd zL2l_K$fwI%jx?-!!Zr{5l3|E2NbGwWg3g*Br~ z!RbQ6a|;MEiu;H4{?hVts$5cnK|tu}iBCUbe)in&%&Tv`Z=U?zr+tBN3hBPo-w8;1 z_e-hg%62^K-yZo|KTzUN6pWm60;2$c0Wex<{z&g104VI1xp`OmC0#FmKYwz5#Fp9q zv#);HG>a7>PU89F$EV9rmH*yHrwyebj>;bXgNvhsm5Z70Vazs*0qXaaS&A3h4$y3qDB>c4)UIc4v#DrYIk!j zOVwGSewO?9E25%rZU$gg`U6MC36zpJAmSqSeitp`mp}WdqOM(vo#2{vbjD>&t{>rO zU`F=^@Ai&78CA?fygS&!l+Dd6=DMC07;Bs)R|B>Fe!$85;U{O!tTJx?`a9pyg)BUe zFb5O3xFYZz_Ekd(SxOdEra;skM-PgXqi_~o`klsnl>#Zcpj5hElKz>@G@Qz#a$FZ9 zlv4SZUp=pkpB3}J{@Z_TKC2ABphAhRqlXTP4dD0Y|Ni?QnE&~&zGFV~*u$2l+3&et zfA0gcqVwE!?2sLg-{Z3FM=ffAFt5G+zWIxE7?}Eu2{oT{VHX*%Ijbv3ppO)cIs%^n#|-Y7<~;Sb`48)TXRql(g?* z^*7&nxOSg<=mB%;+yz@ws4OVNmm;E9SvCu^!;)$MML5rzfr> zkW|~f7Ps+C>fMeLwJT38N#6UgpM+h4Vqs~?IvEFi&~3&|Z>~SLG|=6pat;E{#R=GS zt9Q|VTGPDx%Ioi#{X>K1@PU248$@sQ%KU;pu@u2M$+HPipj3ONlzvpxv8?y8b^HXO z;yCX1)>hLY8*i+Ldkg!8Y@gr1`j&a*fxB%6elcV0l#?1@;eY=Z->@uyf2;bazj^78 zwl=h%^ASAx&!d&^2 z+?qXo?viDbBLTqAdW3!2+FJC?uG&oT+-G*bYk)?6m-;V$bJ}i?0S; zB-ZD(nVi0bqMDVl@6q@BJiKOh+me%yiKt7oH%^=eoCb>5z?|MQwq~mg&#;6*4huVi zk3q8xpml2$TAEsJ)gT|8IcFDSI6_Mm47KZ8C~s;Z5_e&-0c{x0&L9gW_C;5VmsiYH zE##|B$Z{Y+0%Jj`($&#n7si+M%)~{`UKkOYxlP&Hx$Cay>nq1?)p*VAeELc9Z{x~R zW1-iX$8N(Ji-VAoOkoQ3I0+~lgOVF0L}&VtecuuTE$QBGVh7pk5&V(y?fGT5|T~mN9_7jL6MbJ z(ZN^Y`qPgNO4$dn+PuL1BsP%MN-`0?pykyyb1g*BJ~jllgde(o#*+%V?@`cwsi@&$ zZ?b+M5b8%LJ3|sx*og`NB`&Y5OSdTyfuFv_c^`nP_=sM7?X6e<`$`6nu@?qpR|(lrtgd zl-8-dH}?+?m;teNSf+@X;OxF})T!&!ZVk8~|89Nc{A+D}-DXFC=wDIY5SY4s=>-zQRB&Zh=6@v9m{K zPi9nHiHOj1DBgCmb^~Yx~8W=UmtSd%vj8Hq>N%d)9T=kN9zPmLy#74hmP>!t{yp~(^Lsg+6>-w(uYUK6b(WkErFamaSc;Cb&JfHjMF@Q0 z)UtH3x)woXP)-g|i2ab4%02cgk9XJilzAOR*ZOCm*4BrRI9Jkqn5XU6}{%&b{!X4VVi z=V$rZ9?4d)WJ-EUq^v+nA}MBpAOR48$mpDNBZKq(&P9QZ#tjut-Fxfa+H#rb?y6I# zc2%9);oINd<0QGx_v0P`VE*#U&zioTF7u7=|HORjM?Vw&_9q1t*(SL6z#%g?zi1x3 z_pV6oELM(v&+a=6?TZc{OOyO7^^*QOw+04B#sx&&;44Yfq4*c;DeG}6*){N{+GTX8 zt%2W`ab8k7!z5Q({%3X#x&19Z|H#CI5}7U5*7tglPpO5X%k~G@5Ipvr@0haVY0;H4JsS8c7@ zrG!Y5`4<6av8j06+jqL_t&(zhIP=!#Ajv9ey{tr&RoF zfFc`gN9@gVmFA{;>o*p%gI&)5y@Q7>JK1Ji#lOzBg@%ET7CNa^mIi`F0xIj;16_?e zW6LmFX9MKZ>U&tNfviGGBM3|iFr|wuq^aD0hrW|%Eexf4?ordfBuxW2F7`blyXH_D z>gD7J%^WnKJ1{(EgPmcFR2)8x=P6M$my3LG__!S2CX3#6F{iHfc2#{{C=1kczUvuP z?K&S4lD|6f4))+?YGb8RU9-Z`Wu%fHAzs0#+y_4Re9!sb`6ylMLzZrv1&8WQ?vs6uDOkP}7tCKIG-R&bVYSrDmF0+#O->p&`> zXyB7iw_6+hlQYEsul6=Tz3T$CgCL+z5}tSZ;rs7##@@yE<<{+6&HwyoUsTWEJIxEf zdCmNzs`^h43}>~Is5W|E0ZEQ~@ZP(fDeJZ!Ta^@AYfhXVxZaA<+2H9w6{&$?B}o0! z^?OPFxrl3eMJiGkH7r2egnA`!RPrT=Kf*f3w=GRZ1b;HH+o85^7XTd)BET=`qUl@h z(x1nMz?Q(K@FUEF(vh&$(^E>U2k{@8o0{x1gwO9%F3bMHm!WV3z-!WlcMZTQ@xouo zM=oU?!?Q~>tsUaiEZ5BDnyro*PfX5Ob_-ZPgGz7LH4v_tI8yPi0b(2Ti;HH*){XuT zD)OElliRqyhs6+LlJ9=-p#>~%+mz8um#X;z=uge2N&aHogJPZ|`Cpm;IDt{dGp5J56W$NMVpVUype;~rK+lY($}4#>nD-xQs^Vjy$BNQb$R}DEH@fx zQSg*|dGTU-fB)QE?Tw?*w|s74!R)!?7Cpzo)>rgKe_ZH1OD*LDN-MQ4D&_p0ZEZHG zGw8GL=Y@+G&GSG1rJ?=AXCAyyj8E>Xp!$2e%~zj$+I;rm`_0?$e`x;SZ+zDr)VG-R zW2FqMg7En1L9^@D9k$wN<=>+G8-L`Mty>HXHYC1MNi|SN15?v;O46v3U!bg1qG{mW z4-VSaf1CQc^8!m(pTz@Nj0TKdtRANtf3`_}QQ~*DQ^}tJOSCjKSa4z31p8_|0sto= zOPfi*g56JSJuF>6?6EL``Sl-&W57mv>GF-=8o%?+wI57Mv_5Fk^>B_#5PF&K*t*HQ z^x9jY?~e;TU>gI>{y{F>-=dlEgl_1fBqNYAV&MficmlXL-zmq#9KeL4-BEQ6M)AL6^w? zAajUo-w`=rBw9$!yAt}l@jO6-Rcaw%e?h&^J7mn;))^rtMB5PTp37G%BZ@OA6EfPXeFb_qJV0w?P1q;ZE~ zwE6$f-~Wqw@_#7gZq2^6_bF{Q9YP3KiOLIH_i|AFs0%t=d{t4p~kKW1J< zZeC@s%dtx}7L@(FZ{Ml5Ba7zuZ@pWpY1rhpeO|Tx#wlJUKdpRR3KiuSRA@ng1?LDV z9kStwzfzc#ia!mo-VKjUnl9Oy*uUi}-ugFFq$>Jjr_YM1;=I{)%eL#!A}w=xzGjOw zhjP6al{l|7$?x;7j!#TmXSct*Lm!+a85L;)cD3q=>yhYB6a1?+lm47)Ah-tj=;H_n zK}GvH3&}@<%0mjC1rZ!XTgeczC)V|z^<_}$j9Soy^~xypTSJRh5j56{b>% zu}QivsVa^DP%83e^$hbY1Kvn-WghD~>-L$U!b}k)DPDZ}ReRlE`I9FLeiNcSCu?kr zn5RGcnEC3TeBMmY&6)rD&)+k@d*dBj4N6u2haVlccIxdrw?{iwmZR@d@&Z2qKd72O zNm*2p{;Gfm=$DJlL%TA+pl|}`0Ko9MQu#y!wBR^={FMDpZR+j$0&5PJJc5!SjryKb{A3IG1`>m_3)%*5-~DqiP?bu-uffi;zFY1u6Rg!hYhP-{y| zu2bjzJ#1gZkzlPz#h(T)D$YDKHm2f*4pXzv9Yd$Tv-{r*c1_k}Y>Z;ak>e*Vkao{q zw_iUUES(m}I5Ttocj-SWLj!4&-{*bAz~@Eeg~X@;p_|H$8o;=t8U^%Ul7k{QURCjb zvPQ20>aHv1ihm{P8KY=G-Mph5p7U6o5T3D_*Sw8)&COAa`?Lu;7ZPqf9TqwXyUp^UhI<|Bg@bJOKLeADk#O z&la&BP(2U*ZK>t34Eq|PnR_E7`3Vl4lapC4EjOa-l<(`5f?kT+8u~gOR6?N|1)59( zfQ0}c*8Ww1Y*LP(OOVfbJyU>T8XUfR)p#FEea!>}*DIJpMN>I{16}Hg);w>kDv>Ik zCm(yrcLHeT9C+?)1i*q(1N$MYoY!{-zbg|o4?c=bUdhjOh~Ei5i84BZ#4mwDY#zQ5 zNfm&L&9X&i)lPznvRuNm_uj$7=FRsGm`6W#kGb>KTXOB{^!I?6pLp~^^S}K0bM`&> z|GxiY^RGXC(N>+lBPxHW`or|GEi_5m)36;ncFMlLh4xSXUcDNir9;O4Y4oiG)^*#y zMr%kapJ?FB(6D^aCCd~-Rdo;nm`=s^QVK*1qBgrPiAHRLrpEf~b_%+*`g8Wz(LmK zVzi)Ptvo$2Z2ipJc5J<_$PNL4X*WWAIF-uMK$_$aa6bux)T@dWw4!UISAc1h?XFbu z&uWoqQa~YWps8#xu80$BZ*8`Lr2(Hk6pJ{u^i)OE0Da>%)ICY|nZb8RY<88Mk!(>ntG2w-O5M*rp^XRdCD8<%_mmP|vtIZGFALsQSH2 z_bMsk=vn5SqZI$0RKHDvcp#=-NxK5b%L25NLuKUgGlRC0pX#Epl1cKHcRPw8*G8gx zW^U26%E{v1zo}$hH40S{05m(ae&rL^e}b~hmLAg$0FCnkm<7@!6sUY3 z2*8&Edr2`gL7&a{4<8flaKG8HWmD)=c3C>ktEy%WPsd4;mUombi~!K!UF4lirLr{8 z*VCD))B)6%LjVB0@X~9RIqbR5e@3=`aVbvkx$}1OAO7@dRhqV%@BQ?b<{L`(4~>iq zY_P*Qn;9zLTefriW^>@kaSL9|p_cU5sRnG79f`%&cFkC zNz}0kw@J9b6G`9!v?Xg0b0M22oZX#Pp5cq4>nADcSD>pVUL3~uA(2Pois! z#J>Ty<@s>kVsS*Q5zL(!VHok^QuB)4xn$7`Gnb?s0{|Pw7ur>=(>*O$fk}mTY+fMT zxCJ0NKI_&^eWr26cYwGc@vzwl{%72CPsb0Zf$$ldaOW%OQSa^BZ}uvXI~gFk#f0h4 z0y>cjSP$vtir%QC3UPgMsz4NIA~6{<&g6PVi5w3j5rYO}IhE?w04PKZfLd)Xg@g+I z<5}0B6Z9h(scWH|s*oVZn5c27++Jzu_|ZeYecD9DMb&mWRBq>*CrHgX8Iv1(gVPE# zCa8a^DL}1Pk}NJS*)_K=U}Z@@9c-u(wYk2BQK~cZjqmqwyyw^}R60bVM8^@q!K{=7 z9$=$!B|UE6sY*dPJbMouDMK!T@-pYNFDMFT(wC+H%WfJKWt%xA93j< z9XGfJpg?VDZWKdM1~d|Ugv=4gPM$F@z4m6_s^hR@yFTvp*&%a6eZ!x8^nt+bZ&uEw zQPp)M*XPBA7FMY8;rm>x%IGjcFpOa8QW-EUg_(bPOl}Qq5_1#n2UzQK$!l-z)i>~v zdE~yk1p*CcALbg{&+h@+QMLbKrfvV*@0NXWL;w)%n|3il zhZSP4%ruA(jVt##rq<(ks|CkOflUAq z%B=iPmn@Ad&yoJSG#YU4IhE!*e4TY_YT8U{qbF1l0sX>*PY@KxkZr6;{WF5hv()>- za4@54faa$9Se)`wUtfCuR0GA-0LW%Wfs?#>KyZz4}+&&qS+Oab26AiDmr0Po=X9h=YkpaM?paKXRMnJxGsRYviZ4yYn-&o0CXIr5Rjk)Ou4WK$Y2NI1TF|@xdhlrplR#_J&EPKlsu>RVDeKdHPB7C(nM~Y~H-l zmvX$FaE=M+Y|w|!OLB}jGK#!&RKkBfx(@=jI7F$0(|~vo?15nerQyz+9#w7{i1YsQ zj^dC(ujWUnNKj)^i-7jCf3|(P704zaPPJI6Wp;kvI(jZq*FEa}l&MVhr_*BVfcs7F zah4bQRD}@_bh22+EeV2N7w?$`vB=mCLi<bf@30hohvgVnlfV|lR;Y>}oxprTfRM@=S3stZRoQj* z<13NAE7f$r%K2Ie)Dpy+6VMk1lQ1$iF=dXNIAePQ9#$~1oCPKjzNf5-H4&-JyE{7U zJ2j_X#bJdW@`&d@1Re*sLqo)AcLttQ2@k-Zofg2~R@$iYFUbOZGB*6!_|1tFKpp?` zgP)pF{q4~Q?lJV>EvYErNRQBGKld5)(0zLp0IiE{I$a8EcC@vrRnPv|&XEqBd<_6> zv~>OIAMRIR^17ro#Fee&_q^r29~I+N6Yjp_HtT-}oh$wsNjTQA^g9(ydneBfSti<* zU|vAqHQHw8=7J5DWADY~WyL_2O;3Qzdgg=eI6V`*Rng@umn}mLZFGYO@oAnPmn?hZ zQ?vG4)ez(ND#$gu-uKR1Z!yhk#rxKK`z;Ho>+$oAXF*naPHANur@$%!wYy6eU7j;2 z(rE>NuMk#R#K2+OfvR95mBJc0J2+yQ9~v4yX-`u4sM74wsaPRkCE`(W3HJY~vqQGf zr?2~F;4HrTIf0tbYfGhaG>~l1wFLLUqvM$*S(3_48o+1*6?836ebe95Uli5=KoX4J zV)SS!JezcnQfYuHOP&LgYp$iUz1{ZcB#7xp28TzjaS+AGrb>slCqa?ke&!c+pV4cT z{85bV)#oGuMkO4TspTjbc~l;P2o77(jYHn3jpw0KWL_-{sOoNX>v$MdzAS^JrO0HK`I)~#fRAHlYB z85raIJWrh+t$t3Q?o-AZz)opus57%#Ps6-J1jA0KQt0gPh^}&tIjf4lau$;6RA1Q2 zu!uHVt7ts=2F)*qry;(eDiH1gHe(p!dzGsz}flSo|ozWF`2Quo9E9MiVv_MM?Gv2H^qDNO6RRQzec0=&f5($rXIt46(} z@aJhOF)S7{7h>%P*+i`ZP6rX<5xgfHA4I4;g^8@Y(?4B5<9Dewkxi;O>_H9i1h9e2 zLV+NN2c!C=m<`HSHHuDXf8GYNG}yg{|BL_G{K7o|U)+|M19()0QcVHGj1L1kMzFlxLCJ z>iMQW`_n+UXn?=5D`u~&&@ZdPfh1MtLnEUo2kl+OTy%&FxuPHybkHPA!Umc`oVMZ) zfnElOvKT#DDo%M7bU{?skbr{9$F+2{x5%itEaad+u+oBP_6T&=Zg4*r*Vk5PJd9-e zTCtYb1fY2|mrmC*cF~$5l31=!LI~CaETNXyRzV&icW|SbU6d_nd zvKP84mx8V^+^<{?SJmojF0c+!t+!=kpQ|A&VjT2>v%|Jo)S=w}1oqk3cll z`y5NvB=oZ7ECi)tNP)^UY7&^fk$i)q1bIdx+Kav2on^zcRL=K3iE|@3-mFTnx%nmg zetOqHE5a9k^LzUof909a1)?MO#z^vi@d38<_lY@VL2J*6BwtE8_n0(*zsR}_m@5eL zTWd?B1&*XWlruL5aOi-*;jrCp-x3~h+|t}&>m_L^aKQQgqdL7h7G1paT{WNAUg#~ z=j2~n3FP?3lkx8HJbAyeZ*_&9)aRDDT_xEItdv~>WWc)w9 z->FoN2GS(Ik9!G~5icOrZXR9nq&_^U+^7M9R2VwYevqV2C7cGVfvkXO+8Z;R{wn1t zD8!+7!Vz;V1avR!eVv(G@HI5Oa`i|?dav7C+f+4r!BS57I%!Nh5!hrs4@+5}Yg(*% zAlP9?xT1u5xe4pZ==hZVj**xMBXt6KfK~@AP$_azl?Q+_T+_m0FDxmETFUkm81@9i z0JCiD?T&QI0I{GWoP3wn+>s^~;b8ic5tLzLjoq#4q9L_5sHA7Rka|3N&2IINEhJD} zHmjOr+A<)xS>M1yeXRfSZAm5B8sPiK^8vLnLC>H9;N~*Ba#iIdwV9zZs*j5;vfqVV# z2M5gSqN@MgqYs$dc5I91#AQ4x;GA1`Y?Yr?V_to0ZyC=tom#0h;Q5O|LBQ;v(kp** z*UrlqIjTxR+S;|Ygr}{}FkP-GYun(Ht1gd!k?^8gz9S2mw^gkh0q~AG08Q|`pVQXS z))MH#TLs*J(?0H&i9S(UOj*2k=Sn@4tp`AyCIx}1!;GF)RF7u;b zyj-;SG(E&AQ0Y^paon;&Nq!2Mu%%te)J$gny%9kQJzeboGGy=nLwh>Eqeg<3liz zY8_XfCZ10yH>u=FCD|I_*=W?7N02uRlg-e`m^pUpwArm{p-pG z+6qqqP*H@05aZ1VG!WLWBstF-A9kJR8UWn5dbgwKqa2RqegJ}k>1FMj;9%B~4-Uq_ikdTjY>z-#N& zCHA`@V%K3Uru;7zV;P&EP~IfwjBH5;zd8a)k1tY9cDi5P+iAv=lC$)S-=b4oU;A5Lu`he%WwQE+ZZc_#Ds3RE%xF_Z>VeJGU-pERg5R#)AR^t=woJ z9VIReB;Rw5OEWpf#TbD?n-?4WIiN#lP*vqfeh|d(9@;R%M=jUlGqBu&=u#J3l7!a z{L^>M-+k+Ob9yj5h4GSTxk0#sZiNmxtTfcu%79)It3eueZeflCy~?}LsD6bmg`E0Y z>6ta$&pYPw+}>E&C{T?O$c`#p4;8D}KBc;zsMoOa_>qYT8{GGLC~n!aZ%O$YELA4=cU))G)In~Q5FBEOou=w)hbjKLt#3q$}hmosiYcMsR56f zrmLec+tJG7h~Z!-x%35Bj-RxD)8=VlXvA#IQAz0cdhoIFKLH!3;!gu7)P@YZtgYo{ zpbEXhi3bdfjQSfa$6nY&_`1Z2@D+myu^3`9LFuZm*!SzIiYtOK6Sj+KfVtA^-Gz** zcmf>)g0E!Z8M;-BKeR1*>%IL&xl|jk)4l3!_W?wAFe3_}h;wjHT?kAt6p=)YHv(iK zRt2ynmHZkwdh)bg6S_N!^UUO*P2s<`Y25%+!Z$K5*>~`W1$njQD5&81f=Pw=PAZk7 zf#iFxCAb%Datis_m2;Iz3Dlx0xS7Cx%StiNB%IUh(OJ%PrAg3ZQs;=n1riY|wK$|k zfnfS8llb-1i*0df$ue|os0p9sXU(L-k4n%|3jH^)T)iTPtkLN-GL_Dawi?eRI0p*x z14oaWgGY{=U;X}d+m|cM#2H8Kw|cWST1Pz07#!22<9Rk&?<+wT^lVeL$91_Bbkx+) zXn`~Y)k`6;Kw!Aj$1;^0H87zH0fH6uT1DvHwz;oYtQn_$t%)~|mb{->i|6H35}YH! zS4aqaknplWZv`z}N>sXYmvc$S3asiqG5d5L%XV~)xJzu%+&$`Gn zxL(0~?53m(?Rw@Hmdt{-ld7ow{?C3T>dqc>|DHQ7m2=XsFYNqbX@JFtXaDfAQ(=cm zkE>7(Q0>cF-_;om2ty)~_H;1Yq+)D8U|7xix;t(BG^|dVR4u)hO8#*-Nw~1{NYZf> z2dr1YNH#DpHXFWb)E2Z}(FhE7LD^iP)czQ{{($catce%Imd7&@M7UlAzu$#GDR2G# z#(EdSmGu}ss;CE4aa=7RiXk3de{Yd0`RjRp0THPn#)Z(*i|EQ8`qen6xY`dfKUdQ_2!q*X*J)u(|AqCLaV zq(@X-(ybm*RGs^&x*Hw^sVpRT1iF6uw2^QO+IP+M^?P*VP>;EkOr`Sx2|@P0I_H(n z86@*%8E38Lr9ZrFj*7ZJjFErx%%G~(>+Kfhx^c)styfBr1!pefgaxbADWpT3egY^K zL{K6cSZtRiIfX&{iDtVtuBTbS6sjLe1zN~9udu)4`K_zlVA(eq0|p{`4jz;3L^L)7F4IX1`$}u*n%d2iRI(N;vc(^(eVkZ!oty{gbd~75K7UcNM-k$$`0CTw;#E<`s`O(|Db@)PrQ2}}&bagj1BhKe zW2)poD+V2y-uh(g1H#HCHoh-b7rufn5S@lZkDM~oRPAbwLT|1a zI?lxqbewiQmVFilYydWZf8&ql zp1W={_wLyh8Wd8LNf+rPfQkSxpEx^YnU{)m+VnID(7>#0B$(NtaSkS0P_=&O|EgYC zN8jG}p(+X6>=s2KRBZz!98BY{zSIavh4?{_5?3o@&YmL`P-RIH7@-rakIiE-5-)($y#_AAVR=U7=*D9Y8mRao8IEC@& z1svhqOe?BlNK+FaqP1$>GamVLE9al>-3xL80KE0;sSdcw6#yZ~`IE9Qv1+~z6m4Md zCP8>MU{YJQMR4Ij-ubTcy#q&VA;)I5%E=}8u{>WhFc--OOr%xuNupB8A3e!m z=z5d%ciTfz_h)6`mi4x`r^_;Cyt(&-BwbFZxe~aBN)Shw^~I0S50%Xoz=xi-sZ@>z zNF+ilNq=ja&(H^!qwtNNRaH$8kW9You%JuPDnhLo zRA}#n{n6GEG!ga5nY>C4+L)9pu;P>1#{VzYCcd9_wVBG%_|F*c8His1ruw4-e(Ut_ zFFp0RKyJ|hpYhJPc%$>|-+R~XVh)+87>26iX|Hn6L~@#B7XJ-uLtFn{i7EVNj<05c zE$z(Eu<2DYmul1~(nO`WzID^;b)8CK4UCRYSyrT6bm7=9v_D|KD1wiRuZz#i{Rtwp zjwE3Z>zrS~e<21zyx9UR{T`_O_!ja;65Dqn=nfsC%W9BbRizU{gB2)Fff-_9@tmPW zK(QnPB3ftTZc)!kLD8~+3NGf4kKU}ptcj`Q7gWbwj#+2Iy%^;HkZ(CAN$26q?x&eLmow%U3{xM>Dip7NBLjq-VO6 zK@VHWf8}F~oOG#N);-!Q@XorrKAfCc^-v`^+bERyV-MV8b@-hhzZj@mlIuJ(Fd_%M z&N|^yuDjB{e^9f7jLt=46;@XSI&c{0p|^Jg`UpA~0*q`eqGXIJ-uvtF`HniLu*Z*x zg~Ea`R#jf`p0U08-hr?~C+0ZpwJ9a}p^C%7^CNtd*hqdw6@qjCqUwYm4;rM4dhYc2 zcr-wwnfHn{$FJ-^^pOE*b^q==Y%4VHxc(mX5ZCj;Gg6L1P#db)2K)R*ccrN=gFZG4 zO9ndP9je_>7g|-=FR-xjZhIASp$Dsg1F7^Q;MJm}63=zg1r{!^-PV(jJ)qV!o05J> zPUnCB?pNrN4FkxC2^E_mdmkYqTvU$x3RY;>y&6I2m6vu?#QMIEBLthy7;_7 zll+{YRzX2p6=8qTIxgeC1p7YYa|AtKF1;`hbjLNI%W9Br^_S}joL{~aqJq`?ww0hgz-@E!xW&Jbjm|teyAQ!?j zc>MGkYs2*AX?X#NgxD^iM%J`cDnkSARq}CVkYDert*mfCZ}dC zsOc-)F#Zo1LFGt&L6B>B)T`ylO`(d6~=1T=FaVJ&+4QBiBMw9LLNW z%q6Jg;ThzgS?N}CN&FL6BI-q^+dFO}%5N3E;C4>&n}a>!4f84zN=vh6#b4}w!5 z$o*FGLm9|+QS_ftg`gnkZ)<@`!0lQbb#CeL;WU8lPh}OpfFB`H`0fWETHxQK58ivj zA)EVq%>2Tl-ER0p8U27=#JA9eK=pb~3HEd6f>-jpIDb0EjT)fRkLS~_jgIc|jb|wC zPb4Wn{lLBExhFqup8WK~>6*Jr{o6KgRGi1C*YKD#`+e0Z(jOD10sLdC>pI#)fbF$r zP%(_Hbd6O!VOQ^Ny88|sR{UUt*|ymYu&Af2ttgem?le%kk;avXy(GX|UvF2|NnpQp zw6)sA*ld(|qii93U3}i4pUZ9m9Ek}9VQ0Xl=6qqk&#+hqJzrMk>6hq?7#TWrWeesy zSGoM2WFfW@jCgSh>n}EOU&Ma)LFNb>Cmm8=AU37U_HqQ_OmrjKcq{Vu~|Hh`*=EtO~*AffuP=pr!^QmH-- z*qk7-V>j{RtO{(GQ*urT??TTQNI)vPnv@`pB5PzgL-|kNp;A?fdB|F-bzKV%9+hw; zj%jb;Lgtj@nAO%VrwSoZ`m5_(a^tK=UUi`T)6I!`~x zFD;)}4`exE9j$(L53~=#j{5pqjlXP}0slZ&y|96zirb-8=QS}@BWp4swS4j1>doW`Ki@Y4_8Xa3^L&kEQTB57Gp7Zbx=&`+xX zXT#%@HYpavDbgVmrU9>#6kj>0WNaG!f9ngn@dty>pI5PTy#{Hi!BmF0HPC1Ct2=SpxP%s z(%2B3X@~yqLC=?KUbcs|l47o&?Of+6r{DJ|9tCh@!Y7zF?A%#@7Zg9C=&e{{nIRM# z2-0#a*W8Hxa1GrNSx_0@7FozX;h8TLGi0voET`X(TKi3v{CWD%m(zT`x$xY1%$wft zT>SUQiBoC;chTIv>$beZ0V1IYZ%S=&hz+Gu85#)oC9aHX4ty0P!l3h|*APM=n17$Y zc*!~HCh+9S=m2^ck_f04G-Da5KAlR1X@CkgDkcfK#w8tc>@Miz4;^7X@sNF;9qtq3 z`OX{1-!5p)W^0yX7>8hpvxTAD%{Cusc-i`t)c2%U4%Tx|eb!8=AKK4;{afoqcz?_F z-{YsxS~@v;jz<-M2DKY%1lBlb5wlUpE9LP73a+Z682UcfVv*NMu=DzVYr3(!*NsX|`ksY#CG}zxK}Fl7KUAoN0VNkx*O|!kZqc>y+U0BP2p#mm$y) zRWdy|FQm_BR7IB2@z^ZuwE3ayC&&t8#@V3}^Vq5?elCfm|DwKqFwm5v(CUJ9!WDBF z-yqf``bh=_kJ}_al|Jg(8CGm1hXPHrb8~5>kd%mRP>KF5_FHNX-29f1grpjio~~3W zrjlv^8;Px_Gq~@|I|is7CV*0b+RNn*4qukT!$DCIzEz_D z_QV5S%q{A*mE-Ze%j1(`V32P~ySF4shX8QIjj=ae3HmsFcE~<2ckJAecUWRT#Da(k z(WWSs%FsaYFL7mDcjT*}5|kH+im+5N8h~nP!-n-X0hH<_)j+C&@ERCZB^&`?s;1+T zn)Nj{xtT$R<~C?Ai*Ashb5y#1j8rK5K;OesQ3}ZzQRhroh@g$1`O-6sO4k8ZF1Qdc z$=|=C$`Ob5p?mjOWajVR+-nYfbfREV;Pk+-S!XLLf>)$uyN)gaK0tdW{rhK`9b{P3 z581WYb{h(IpeMaPMHeb701LHg4Ph;DAp}O)XakUB84=B32CR8A*k^Npp8{g7=FqWI ziNDT#_sB9Y)z%o=Z}_E*A#lw)PH-%$U_Z~ues=kMMx9f7d^8P^)Tq@nH9A(9<%MAG zp<^d)GBsP30sV{tln|umSt&l4!T;;jrH{wkOYHs6oILZBtL~z-3|(*_@2-zI-`^5{&$pXE7xOrm-*hd z%6G&UO_H=WH`|JPmkD;_bOU0XBfd$C+WfM{#)9hPzXNnlr7|>-Ciz3&#}14l85nai zpi;?b0Oy#6HO}heYSDGGdhhUjEy@T@rTR2*@fux@3!3d*+RM=Mcs+P>8;v}JJc~S& zQG{y&F5ISlF5b%y2CvfJhMVER-T?a#hap?81@-oj}Fx zLP!X+4XXV^l4W>g(!QsySt?&^YSx=OZ{2Pk^kb*aB>6J)&V6irQbE^RyXD<+#UDNU zZd|rrRNV)oiG>6q|*vhKhxEKlsq>*t$7u5RbJJ_Mi;?V)-Tc zClm;AS;>#>07x)Xsgv6t%cfVpVZ1|e+KbuFTgks0ot&~ssiXx9EBT(pd2OD3JgF8( zDP3utnoiExwx_ZZ{jI7_4l1n2*f!WWrLg~l&Nr_8M`ULL=cyON zL3Y!{%a?*4dc*!R*O-sG+T9ll+;E~Y`hkQq?WN|ImUPeKX{U8XYX;xqCRG>1vQsRv z%n*uIj>-OuB0R^_v-7r{gfq4;`RnL@W-i%97E$z?@7OuD8zE-8ZS%%_yOrhNx>b0_ z`U&vJmC$lxWNgChzWo+oEmSDNtI;+(Ss0}6YL4kR@oFIWmoeV?l-=kyDI_C*HA=5b z2CJn)nw>4XIhSHWAUJVk*6M$$MArbp%_|j9DH~lAxkt!2dGnUuH1}S~{EJ7&6b2FP z_likIlenLqTeN|QT*pzZ&9fMluAihQv~g6al`80AjVM)a_jJbNkm0D3Bm`|6RnXJK z)U-{2)MRmTwr=b-pHYw1*WTG@-rjd0=jm`ZKy#8bp*qrq&?C(T1>}S!S~5T7uW}WXeNqL`#W~TD5#%AsolWk!>vjvE zIePqbwm(#qe|3(*b^ zjgFh6C(hUaL^fbB!7Ti)aeV{*Qn(gV2e5(ET?0M2IIjnS#NG<}y%-)FzsCF&{`-!; zserLb&JujDT||Q2R8#dd0C$pD zR2c^su^#DLh@Df-GC4J4`**vfPX?r)u^-(fL@MzfS-Nml6L0R%PhCyVaH`SgBNVN9 z@k&@0h%E(`I)OuJrFc>PnIG|(1Zhze#=phhy85vII9H?jvWsFL;92Z&R6yb9+RI^| z4D-UWf98dNlfI;wB1&4)%NeC?m^ zRQ?(W{v|H|YjQfnOcLPYO;i^oOF$A+PN+& zN0TZZ{dYbJz1w$ev)6h2^jS-xmegy_H}BDjX`7@ZaNt*L07Unns-37}K`8q6U`vMy z(nURgdVD+@XcJoxK`j!``Q){C-m}bK+cs^?x;{}^OIx(j@o5VT%r{mk{&w|>t58g4 z_R!A4kTb8Ih*tx)!{8X+&APhq6qTI6sUgE8;J2Et6nan3nQ9;o4e6U{-c68dg*IxG7zmE0P@Kgahk1h7D#eDkBkvu(Um_rxYNwU!2zZWUex>q(tq z0%P3{(H%gxWvgX%8R6FSQu8w#dGQJaROUoA`Xex6kNak@R=)2n5ayU6_gjnq} zk3DdYIePMp`QeMd$$B5@uLLVVZ%=3BIe7!nyOf}S3&sL(%u@Upz2KnR;BE{P;2q99 z577b%VWvi4ZE{*j_!dASZcIJeL_l9WJ{671F3k0~I)Hk)6mRTck+4eM>-+V3&B_Ja(9cMpaU8>q{*WBU6%k@X}Ieo4i>$J`TPnCpkqKB+~08L|C% zFH%W0kZPc~8X&mOxBv9uaB)wZfU~{+?mn~kz!CGWKlyj_FF*XLIjaD2C5uhch^f<7 z3WrEC@eORO$KMrbE02F&$vWiPu%D^&?(NF{oey*UF#G_rql!9}+!`Qh?-*HM^kri#w1)L%r-1ZL~m1 z6wtt`$oyS^tSK%!etN*b(n3oUKSF$-qL1P9`SPQ#MC{`x z@|%cDbvwy+1&HlOzyWQk+nJSSG#nxTmFw0fzuLDn8cfd`L<47rhRttZf7?0~hd(-L z&Z{D~k^}>h7b;r?x2x-w3GRC5cHiwzrB7mLRs)i((6v8J6(SEAN{QtyTWH4y5AyKfA;HF%(0WFZ&K^%_=HXJ zZ`tS%&3V%RS$@$1@?Q@5f&ddq762?%>yS)}Lok@x21SitY=Qca_u6F=mFFAE_L&II zZ?W~T`h1xrntFuF`N<~_+A2I}iK*#3WWd8l7$gK>6v)a=~4*)q{&J8Pzrztu9-5cnuJp>JV5LALr@GhhFge>2BUoe48kB96g!o>7v9HI)E-aLKS1j<7R-Xg1=Xh73 zd=4Xk$4N~2ZIyzJgsnu&Ny{D(@T96aQCd(hW`uW>*Z~_w4-~Oflk|3XSpX9trZ|Kc zA+f^tYs_GAoFmtmHS5>fXM;kCuyTF=diPuvSpc9R=9e;<++JhsSFKN*H z4GL)T;Q|Z(ubRp+j0&4{2UL(nm<#J1F^=ffmZc4P5VV=r`_v+;{c~^ckG}TY zahq9Ca`E)wP{mbZv8IoVDM@KTD8XztaZ{*|<<-bKAqc{H3{|MBAc*VdJRL8%1^|Fz zTjZka$ENtjZ~tJEr@L<3p4YCRYK1lI?9gcNiNt)w7120DA{yN;%1qRwYvi2>3W6Ng zyG)?FB|XHmUEN~?D9BX7M)u92{%9@nq9~eoK{HyOcj9?^czxv!4 z%+sHJ#Js)lL-RM^_?~(1z@bF9Z(4nw-p|%PQ6)c>n4!S-@|gi{WdK_dGKt=V68K+$ z>Le}jUt5)|b19u-<6B%lZ{t5M$E}pHcz(wPRBQEFN!+EFBl--Rw8baH77A+*(c|4q zn)|s2USSH18DWID3B^HSK@FS}ApQA^7i>%^%p8(@4DIgFUz8RP`L6$x+G@hOM`2K? z^*7((1^z}{6d#%AF|6=>&I)|XSZhNt!DKrxz7X~(zI!S+YJfJZ10!P=hzO%d(ga(f zNkw-{T7!A^qRNb(y;EYO+P-gG>|6w3*1K*`DMLyP+qJQ zg7*ZV&WUys>d9&sXr;ZRVYTbR#r0#DVsm3^R;-q=)oqeh(*kV5a*h2dX31kDlEx8a?YE*M2!8h6N~(ZPG+~%P05Tb%%NQzCGrxeFx3{ zLm%B>BvN%tB72j=o`^VJ=rdonX1@8B!{1n6=S4LKfG=#IpwM-AJ?u-7V*LGWWnzH=4(Z&PBR!jWul6xC(G z3KRkjYqTCQMie30i(YZa!v+SUoIgD_9u43daX)jtGdwqMzkfhX8tvwmty}U=f~s1! z(*r|!_bbI8vCY@%TDe{qV(A)H1;@8KC^HqnS1CPGFqC`7yU2ax-ApCbK(%Rrm=5e5 zCkN6b|H^xhtxrovg21#_`O+7jFn{*U=WJVpfB4Ri%qwsGG3R=}a-1an`>HAf=LHBQ z@E!()h@A{TARmZ6N%{yg*9GYh9d;{$>slJ?8*Bkqlxkbob1J9tunBnfI%)Tpg9&iu ze4AxwVRralC>AYIP~1`y&6ux=LHKkT&an6KKxnXuPI?2h#h;4QIM7s9iS z-lvtoP31-nWSCEZD+#! zR$p6Z^8r!T#ccCQe^(j}^wRtzO7hUpiyTIk!deEwo2B;e{Xlln73b745O zxVi~=B~=gUyyMjXbY8E%wbv#WpMLxivrQ;9k~MGa-S6bX81L-;jqdg2p%+!Wvc^`) z)rg{%gkUxS<;)t_roP@$g#>t$Kn3%n@paL2DCm4xqh{4NuE`qR@qGHI{A$oPIvyQq zdrFB7f{u;a3JLe{na@2+oZtNXH@~}H&CmNnQl++LgAmct@1Zvj3{vKWA;>wBC>Eq3 zKuvRVIJ!SS*X;GMA5m3cv4wM5r!Zs?B-?zw9_|Cf$)&IdJcC_V=zXZwJGFi5CUZp8 z;^(ynR+7+jjbL1u%3oT%u$B|_k0JnQaXuL^fPE?ZtV;h| z*gdG!_11Q(6Q6kW0rNjS`=ptXPxaqK-T%wqy>9)h($Ch-46qSlLF`eU&%`2`hZI|uPGVjvv?gjbuOzcwZLdB%9w zT#If0pzm=ZAHnr`8Ly^B&xXEtm&K+TR)*vs0ItdS2ab~b4&#equd7^-gZXq&mHaSF zBry*~p%M0#7ypl{=XX9hU}Ic`;#{lG)7lEN;yN=hT>KWJtLKvb99;ud6kul;zs31N^rN4~$YL_YK`n5=914C51Zqn;_r-8G>Bj$}ieqc8B_nLcl-ENZ< zy+VnfRPUL=3i&9Je56;{dPnHbeA@Sr(76KgSLFmj%^Vh;z4F`y{t4PI2>r}87Fz%o zfUL!7@QZ8=I!CD4*i~U1{mACNKJ)0O?lo_|chJ1E?|?l7I(>q0RNsUN zROLG_^m_9OvT#3>C^+~RE?pD=q9!(^IH+rS*q4Hlg>!Ol=HN1XVP25rY=-<6vXkmfpF+7bk;30zMHV9A)C%~XB_ z4d!Iir}w>z3gF2$cKW+i1Ldv(-sLv6tN;)^8USch?$;G{epJn_7btLLZ*P(p4~jkP zYtKDp*TMhvkKZ;weDT+!Sgs6P>9W}L=zR>(Iw;#wRDD;<@l-i4DG9&1zsLC{;TK;Z zDI6gY=X_FSG$2VY+W62fK3}^(Z_2#e_;q1@J3vvexrPNB62}9u;TN>wwFj`yh<6bI z{@WxLGkd)e0NhLGGV%Xzu~Jo=Sf&yIpvI?WEok2L^$*EEBvx0it=Ixv9kI_`Y-E1s zGp=u@#|-3We?R-Uj4%2*Uy}CmKuv&^{4kf4+qGeP(n@wt@!F40oQ^!bH}@Ve9V$eo z7&(Y!m?yIP??J{%_bm?%R8^9nmk47$2=xnSF1ED_h_05N&NlTgN|B`KgO@!^7oCKY3=zu*7xs7084>8a0JlswPe&TX4*V)gC!_g93PY(E~crDj`m zTwh?Vj9CoYq0#J>xwajN(8?|X$llypPfH;rrKoDR6xI3`#3dNw;C7>ezd%w#RYsps zKLXY6gZJEJ`jou<=JP+bm9~Hl2()bK@AET6pnYI0xF~90`qCv)cpj!_<}DiCH2~te zKJ*~)EL^!#{AZq?*iiMmr0d6avi4Jn-r?(d4fzktOmgmmz2}AOK#zN(x%2iN=J^+XW4`^PpIP6dRDRyd`9=9RRh$5*S_S$D3v$ONW8-^K zjk37BWEPf}O#Z+4jTCgqq*moJ$~#XG3B_T%5uQejc|Dr_He z7%Nye{hTYC4)L?_l1Z-mkzvKGJmw=m`>)=IWFXfX_Ieo)9|mv(z}9LL?sJC<9eKv$ zh*1;&A~wlZYyq?Z&~GXS8=0T^P*n~{p`qUG{9BqcAnIH(TR(HF#C-sjQ^?gRP!qA6 zqzUiSuG@Cl_4@C={R4AMesWkF3Kk4}xc=@A(=yv@27z7P-)w^8 zMiAK*9hn7MX`LuVk8%vaSyi?-H#Wv<1h&|q&^j{`AW)x8_h*KvHT(xgs0M!ZA$d5Nt7i8u?S>=aP}kJ3IuhD{_XX5-t)72V)h{bK|n4{ zOBkyEVF4HurEwJD8^HG42{`48kuJ`L#-!gyqFW=W%KVTCj+4fJu?*K8b6`*KPIpqO!VW(;XnY2E{Je ziY=_$FhxYEph1jQDr+A3)IDbB_O0gUFTZMDfA>Ax6S2S7JzZkE z=>0BOdcS1$E2m||ypsHYT&N<5L+HuLHZ9uEVhgmXB+1xZ2~I25L$A^#$`YU@Rd4uZ zCh1gyzrVZ7{Bi%`NEILDrmK^w>9RxSUjQ#fp5MCuB)M=%qcxKx&|`s zaMqZGg&W(rK-GWb*hw1@q}7sN!OsGaa!TLBauI@meB;-z3)c@e*L8U7>u`YJ4(l1! zZ%qO(Mimly1bFayQ%N;YjT!*>M&pI*?x9e#Yn`7!dCKoAfj2(RbNr7j6!lzHVPn|umBuD z1%N7v(}XDwA~uG;GPYOs#l5*-Z7oe|X|fQDy$BE;s{T^Z^?xjOtSELcHax6cu0aa) zZReYkD`s2hy{Xi{A6Ml&0P`?0jdIZmv6YAKzuP?d*aPMr2VQU%$bkC+Fsa z8)x2lBvEl7396B}4JP!nm=rQL$NsRk0M zfy{cdzN|n+rB>van5b+LE)Oj=>tT;5zrXVA=ghNDJZcUJ(Dk?9_?~%d@BT#Ei_n1s zXa#tcN^lKa5vXrMZ5|p`_(L+)wRn~MRP(2Dvj)x%4O=EazeNHyN-*IQ%*!qhf}XRi_SgMtXsEN-4pq8K!M>(dN|OL8*rUZ3==ymkrl!NQhYgQSnnqFey9O!P z=S|OUQrQ&ax|e5$gEODTHJ2#k5_9_U)1Nc<+;xX};icEi@85Vgko^Pz{lL)^mQ`g_ zU-05C%S5Z^i|03$8PTYdFzF})y?8b0 z8K)TmXj07r$mUH23O+h<#@wZzHGQhATrGFsb*n{P9yxw0bRV15W0vaYjFTf^QB2jv zE3XR!x)G}#n}mv9Z{B)uf0$F2-7zFv0dK*E6DGk< zV2pPrNZ;^mO1OOaYNSe>Y-6UsD-R8HDKT$*TCQm8!SizT#3{4q_ML&MBVp4Z_cOQHvd6NW%Oxn3^#I__+$t7^Z0D`?-+ioP13sANhgJIZrq`Fz0Pi%c zgxK%+VyPJ0_KW1*N(Q}LdI2c(jM`%U<(Hl|Gpfq{TT%DF{D-$~AxN%c%JKJuBjdIT zGm7uz+5a#hRGVD7a#>W~*LuRMeUEFQWfWDdfa(3vySqMrxsP|^^pN%Y{ZE5JR1-`4FO8$A-D6}>T9naCXop=yn)G+ZV%M2;jP0L(W zgjiiUK5rhK{>ih0PNzo|KP(NcIoiWGJz18)ShtA}!B!Uv@S?a9?-BOAOD098S}))L z1z<`1rl8Y{D%f+)PRzLE2WX#0yff+2pzqpOzxb5tR8H-CKmDZ&OiudFYtX8d>i?}9 z`{ic`kNaZ(;sav;hI#gruKP-xf12bka7XQeq(!Nr{^o^)BpffJ-yT8*PLdc{B@FdXF?Gd;7{ zTjq+ebA9^p$IP9#?l8M<-Ddl!haD%^arB{E62OhVbV(H6u_;wS(&~dGQXG=$$+p3a z7rDps^`6m9?vwnPp6qMYH*72pmBl*dq5w-7(31Ziicp?`#^1aDkaOoW){D(Cxc$DU zO2IFi!RldXBgFc)9I;K+;OMB;D?zeUDoX=RjdijcM0dHOrNe9QyeAt?6oQ*K+Nw~$ zSL{{Ro-8B!Y`~97|5{2!zt`bsze4s{wlM%x5af#M8y)$lu*aO!Jo(n{WHxY;YN z5nl%&IhB0Hw%b-rqip_3pm$XED*&KopY0g&fgbtPz2<9w`USf-{rz`-Xg)Z6v}~@) z&qPLTrEs)Ley{xs746j~_WhZfHJzgBCz+_fUcqDtE>i1j8 z4~PfQ#JO`9l-voP_MTr@Ftq|l0gwtK7DUw4AYc$hV}-$0#O65$a6lNrhha^-s8$-- zPeFy}dvbctv@3q#Dn1r;4yE3o{0G~>@Tk+N(32c{mex6?65`E60(u=ic{*^ilh}x5 ziU`4!SLk|a9nJGLKIxV{q0o6<{GC`nfWLS49dt3=^)VQi9`bEUP*)1`POn&zo_p%E zmPPU#|MnB#@7PBtP8-;5cJJDGT|Ytp(IRjSYcTD0Qc1Q3!hIc+ea(SpPr}d!g|pDR zQmxq}!lXOY%jYwFzFGwGD2A|xVUT7EXwmPE=MPGA7h;_PfeHp4I|Y8`2K^V5 zbo4!nQ$RNZ0E_Y*S|>q*b&?)0!=j`{d+VorK zdTDP2-8r!_KSIkoe4Pvk__CivuJ40=$8)}Jz5n&$`oY)!p~G$5MA%T#mljUC!=}`r zN@c83_kV0Y zI1~(^HaM&#zuqSj@~(xr8{gq+*{4a9Qv<{1TR(ikeD}vMI-UCQ(}NZi($U^-*CD5a zmuN8V@our+J^t|h=KT+knty)&XKGx>R6D%#z$@D#_MB z_%Cv@uiN+R37&vt2ARBC_^=bqqOWhUgZQiGQp%qvrl;i)UorG(PQ{%DaEz#i3WKV3 zH;dAaM1UdFlje%~!sCx*eGk1E?z!_eLj~)own`8ZX~hr{ft(8@1q#mj)eS}{ z^m?;Gd-_4%`yIDf`+z!bYRVDoKGe3Q|_P z;P(U2BYBZZb!h+~66+Ve=&M}-yIfWrtfM`+F9Kj_=k_hu#y)!TOrlf?djYNNqV$Bu zc4VXKx7x%eL1$n}QzgH?m9B*(8dcQewx@cqq_pd~l-qbD>Zx>x?wohfkMNu}HPl*W z+MqPe05CbeRlY%3vF8^9)EuBieR~TDu|W@%SZLpDf9Jc-@#raRCI6Hv`6U^sib&#*cfRt%T-yOi;ZFFpN)dFrzdE5U!v{HK5Xj`_p8`+S47;*7r#6GS(I zj&&aPUIjUBPHZ20=hj~2)^dM*zLRT|VuS$WP<|NE)}DuOcG zL`pH{LZux2?S@BhHq>#paE4q4I&H|*2U6Shwa?5hSda*nsBsDjFyg{tzX?3MgGwT- z7wui-z7H z8$b`DX)5{oUSzY!vW@d`rt~e8UT+@HT4#y2pKrdV^f##nf@z>vtp};(_drU)j;^e| z=Tyyw?bI1EzSF7dIw-t{Mkf-pu8^RfRPwm4wk}dnWdOdg&SM-{Ov8N61c z5T(c^>RbSKJ^t{g%%7__I(EQ+{)g|FU;X}dF^3dxPdTOn68k*IP-H<|r%vl0@94^Q zXi=}X7{{YKVpHd}X21H@UT3Fdy~7vlRhy+s6N<*Z`rK10*y=Yw{KZS=oevJ?w4-32 zqR4U29XliWgB{l@`2l#Ql57oB=aQ844Aj}?YD-^GfzBPPz4x-P{Vzsih-mVKPxZJ0}X{u z!mJ$2&%aw#jRq1K7KD|85lNLceHdLiF&JX7qOfKbTX0067K^(Fbh8rQYvr(#s1GAE z3aTwEE?X3Hlq!-i=MHiV&jh+;zDca)rvj4aF^)vdN<4U#V#%;7D>6Op+#{Qz zpkuvAS|vBtX!IB($-g#WWzojoZc+TVn=?bBc1B$p-r0dH;a@p6%SeHPqn!djD8IM<>c7S#t5> z1sg<(12AZrfMh?6sgl%>EQ_54KcO)IJ4vo_sa#*DEohKG*LX?&J*|8Qroz^cO7&@A zQ(uowG>s;(@2%^*MWELv`8&hKckp4T$OK%7uaE@Ev`bqnThTw`Q?}~@+`tAmM$I#$gyOKtVl{LDXdsUDxYZJ?2wq;WJ3g@J10rspjL`h$OrZ3VRAJ_N%M6P z?z0Z%o0~az{)DlmbC+KYC?KqO&(EaTCc^mUtcf!!@MCR@gQ+6ybrthC%VwoDlBDgl z)^`^ISYT*ONphI7u6`PctFE>>*-DR=L>_4@C^{DUMv-&A5WsU%wiX_7zB{j-ax3`m077zC@u zCiz9z?;)L6|GgT1hW>Uz)zbvd}1|8)j>A5-E zUp4L?s~+8M*h*LW7+$_|S^bN;zS=kvaQ^8zGXxthEn3xNXfc?@*n0e9Z80g&Jt zvn0CdV#&6%r#d)Iyf6iTVo02Kw}9Qj3jjs!71H+$E_A6J(6$=-)S+L zVB3bJ>ql?skYLffgkP4u52WVt4PlEp*Usv#19N zB#%${OMJ(z(vfj0b<}fHbhxgCRw8^`XJoJVks0}@QQv3pJhkd^T&rLvRjk!4!(tiX z9zb`TO7&@=y{*Oi3b22LNdQ%n#xn7(tNP}LXKLKs*K3&(##L#S1ep-Piua$^M5%<+ zz~Jy`tW0U#A7T?!brY*d)ydw$FAviaby0C|sQw%E4C3EctMH8RtesZBbr-Uz;)?|_ zL%?WN4gd#1_`W#>=5YVrcbUKZvu7=6=HGt)EAyTI@xo0-Q7f-K0}>NRA*Sy~OLK#@ z`DtyD6uBb*jTmf)d_xM@j-MLv4P?%DjjaEpvMT_Sl^PV1Z=O;AcH54v=1-n{%$(HP z{*CYdM1YAK{X@Qi!vdGEHg4a#Dbne>Er{f*U5g_Plpb7i4WuEqJhy-kKMTkPQKLW& zwOXuEqeVPJS6rP&Ci-Dx;ZDV^2KZ=!kW5-K#g!I*{py>RI_y*T-07>@4`XbnK5n3x zS0}7p&8ZT{$t0!eA%7!k4|a}Am&9Zu1KyPjgJaG{rBty68l!pzunAxk3T>(ra&t_BMRm2iBDZ(d_fBi~=Gf`9%^J zHjFSbt{^)0a~vR)F!M=|ajOArhL5fPbk4jhdO>`I#~!>VbTHKGkv&l+$!C~|*4Q9( zty{%uG+W89Z!7kSAK{(GR&&8fQ>%5MM%!vz(9b+7X&(TZ8T#RxO1iA{T6}7Nq(!$X z*XNfO&7$aKt68Yzr>d*7BU~jvy4$v8qkPFr^36vRp-SvL>)(jwE{gCSWTP0S+GIdU zevku$ zDX99*C)kt3S!x8JFO>|+52Hwct(v8+l_dXKvD}R~QNsj9E_w!gL$;H1GKp1#U=Q53 zW1IP_FF$K`sF3XW7k*>@>AOENCq)y#T8R0Pd}rOWEGDaWaPV`t7;9~%^{SYygP~&^ z!kaGz0uS7Mr)7EjQ0)(+3T^-RzV?Wjko#1r!nHkg-yZYG{d>&IufJpd?PtHVezkYK zw2(V`^0Zy^+afTWFkU9fuXvf5FH=dj25!z=C;NJw&K|-bwp<`RUOEE!&_#jht40CM zxU_uUba(mh{Zx&53Ukd=OY#6hi(73%kntzK`klE&1{KDYAEYc!#>T#GYjEy8a5#}F z!w`a2x>mrVVo944wX9{ss!m%e7~1qlgA&b^@x)qc*HZ-G;5PL{RxE(GzDB zcsOfF(%-UWbLc7IXAywrdkXNil*0N-Qa#KX;%DAX*V-eEPu&Q*%yumE08EJO>qf+` zq5m;%b&GfXuJl^n+Dzir6Et3!U$Sf@Nu0l{IkV>C%XBC>A6ELhJ2RF1Q#bmUVaJR6 zIC|UKq^0PZqX^$DZ19n>@W9L{=P%{4P*ykUTgLY+4&nY#IS%0NQVHzwIIhXXnAj(S zBNMhwL{K{W`no#n55sa7M6mS;S4Oo%@CgvEA)jRE`lHdgdqoORh_k*06)I_Ag{;w5 z%|c5EfJ}oUZU9`c?eRSE{VgXU2ED1j*L>|yzhEAJ=sxq?H{Ug1|Mv6d!;enf@cck= z&U(}XC@xp>DB^`Ri(Q&`BuU^gVkQ&vX^0;^aL-+~(CfgFW1(0yR;mx+H!N?P75lGr zVSRr3@khlfxzqgM#h1*>e|YP9?(u@yi;tZiG+X-nA~Rs{-DW-K+iKVLN>^+JUtj9M z=DoR@o}(=88PntDo#~s-mbCt~H(yD(-z+P^zbvGCUzsPQ!d+c@R4Fuo!Tr-;{?;a# z@7;ZSAPSx?Asp$;^VZ(|rI>o;b1vy!T2dcdS~V0)_@J|f!G5sPS;mM2z(bJ2bH1X^ z38#n+U{o;$IwzdEs67__&bdz2a3sLORPysJ;Xd@0Jzf7Afl=ygJCU%-KEL{mW4}p( z6R2pWdNEF+q5>PDSnL%ZvruPD>^jis5zLuL5Q?1pOLYi6!ke2il?OrZIgBSH`VSmA zUJ~c}BPC2(qhEggT|<&Qj8Fjsom)p+_^Le4o2lfNKIH_4kxURzc6I83gi&AFA47Ey z_m}|KnZaT8wY(=XV@iK-w|(9SJd{#w5K=*x@aecF=%AQ~2c zmBdw$p61zf1311$8|!qn$$*l6RI}E{RLPG%z4$A|2&rT=fd7Y$L@W&dHi*!w2c|=8 zk01gl24;m8wXcmL(C2ghLh*Eb7;KacHQ|e4@L{R6w(SA5z6F)x(Fyx($Hi0;bUvlp z9~NKMxq;y^CsUYEUvUz`0E5d>=#T!DXFqSg{DsFYi23W^dfx0kaOiq`XGZTIt?Z)u z*!ix#!>i;^P|tLd#l(Hv@|>q#?FE_l|`c1HhxR&bBKRw`}D? zBbjZEasF;`S^aZV85V`gEzbERd2Euam(-g(iB+;u^iDscpS|>|=*oLU>vNYi=KYRG z-#`$c`_%__S&l%`gh0qe1(56O!aFZvu9-?XuX5izo?r~LZ8huH3gnav=H%_|@4r+{ z*J>;FdPxufph81kw8{*BS7}UxoL#Toi{E3|51msP&wn>6$XL#TlT5O2IwTUl)YnE^ zgkrRB@G9S-&@YB$lKfYS252e9HMuwk;4z+6p4l)JCj_5v-Lb_SK7P_x!@HPMkTH&( zIcpQ8CkF=2?^Q(*Ml6U-P8*J56rpD^_A$>|7~wu-*5i1nT*J&MJ;tpDY6KkQy9^81%fQ6avhE+L{{Td3J5^SwJdmFZz?_6Iyv)xNu%;eYD{E%5$`f zqS^*+9aUzn96KrhrYZ)TcfD-iRH{=0n^h47>kYubY88M!OIp8MR5=)h&5d5)#51O6 z7qnglqXI95?#BhDWF2mCxUQDsI!bj;Y;EE=4Z7!XCPBUUYtW>0sq|8e^OA6lMplt7aj|pQ+?` zljMh4frM~xcX=lHx!3Kw=FdI$ka_O&pE1K?Y59-;{oCfZZ@eR0VcEtu0V#Bac>nx9 zM;-FNy~M8HV@kar-vsV0v1Z=ePwlzW0z(GX)+LJ2#t?mP-q;sqm_!}BU4Vu^d**YN z+3vePdC`KhPN;PRMP>a0$M8Nz6n(DYrMIf=6o+CBLn7T@h5q(4JLK9{#pgRq!qZ^z}Mz7-hv2Km|5<-c+0< ze}8wUy+$0tAi{dOu&}H|WOGqlHUycYx35Lg%UB1H0HaIN1TY2thvTl~r#b;$Ei4u% zIpxt00(go;=>5#~ET2ynh>8ka#Je9sb;EUUkt0cByA-`;>2V558|vGC``NEUIbQ(Z zmbAvub1jNw0{)^3SE4_RU@t>=4ge;V%G3bCKYXa|s^WA(z1i2(X_MYz*&~vqW%`lQ zwrN^bwE2W@ul~;hX7cS9VQ_Vh$0AvQ0`$yBOS1HJmrE-7e_4_>W8 zYn1aBF4?NFC_13u_qI*_W_o_!%xEijp_LalPOB0UQKg{)CcL#aARl$Cq#RGu9p*CD z&$!y!poGQR1{b(f<1N5uG`HMkKAwe?mujvf8kf=-46~5?A;VNXOc^2 z-SR<~ZF|S+wSRNzCP}|{letOK6$hQGj2COT!&LI~(Zui|2$&>+R{iyN_n9^!6(70(-Vi;F)|eezHdz|CH{bgp zNtbTtYVWt$8|vh|7E^k% zP+mRmXAq&6`mCxI`$IVBLC)i9U$5unP{Mjn&P&gbQ`FZ|jhW-)?np^%uDv@~t`ui2 z#&IG+M4~7T0f55uJu*5O$MCU@Ns>G&eJB<1p!nZ5QM$G7d_d+)Z@ z1HXCg&A{JxDh;R(?(2#6Y&5EM0X9Zh8d|$9>a}y#1}2-%(%+R<15IMH;ak7!w(Zu2 zh&r3ieLZ4UTQc+7N-37PM!}^xD|M)x<^3uI@QuS>%*5%j&x-TCD&@V*b#Cl1Y~DI+ zALQCA{e2~CfND7c(ixWWiEb>g%m1e# zVm(QT@lT1CzecSqi6KAP>wgqOquLc~b0M%85&uss`CVwUbYoQ4TM>hvaRhkTKu=biQ29G2^j=NP zI#aLSMyZsC2B6q`@4#X6%isOMfOt)cV6^N%eAFB}Jz$=B{E^Uf(_2+-Nw2NFAE-}~ zfDTC#=pm_!?vl|`OzCQGw+ZPR>6bsYhWq+;RPMT)O|z(INfH9+D3(mmiLPH4Q7rI8 zt146pz{7+PL05BOXl8pep zENdxkhT;(H9e@jA6TETE$ua`x?OC>-QA;C$t1?y@U$*jks_l)To4Kt_o7`Gar z5*7-@Teojjb(jPCKmly1@S=rUDFa*c3asJ#MCnmoNYkt{*bg&Wv+_wC_9B$KfN#?3 z$b6bl|5oW5pxTUg9P0KkK>I@a$s{yp%=-rq+mplKixa zh$0jXE{ZvaD&i!`UANzA{^~1#Vs6>G*~T?tHt31@8emC*WrvnsNf3ZvfNJWrA-wCi z?bvEQP;1>K#l6GGtiZSca(8U59;*J8dZK9Uxu+f%V4_>!z(un~OpmVE5wZD*@zI_x zmE>w5P4eftcOa~n&MjL+Z>IY6=AG#;jZqYomigw9lJ}b{Lb(&HwLOQc3`JG0cZ;Ck zZ(o1g{KtRzmihbd{LuWn68*)dMh~kSJ}AF`<6YbT@4EEhL1M37#>TGYdER9qtx)zdqKmW}u=KuJce=>(YI_Y|NC*x5ta895=7YXlv(do|1 zHl_NooP{;_ikPrmF|}kuV`@=X!f#fw^*uO}Vb1Gpb5m8AYyMul)B&9@sX~nE|Ni5|MPQm^>y?k4Gy%l+)#siv&wc*0 z=Fgvf(lj;1>)Z0|1K8*LoCFyh9=ib)Lk!`Tt(y$r3P53D1Y3fwzc)B*NtpAM<1t${ z_M5->($lsu@{#-RcD5eD3V{7r&28Jt@qP!LQ<~(@qcJ{O6H~Lc?@y)&>CHQPm0`uI z(z|&ErTdLd3aWzBCi#WhO~tJSNDMWoBAv=!jQ2tU<;^c%e$D*-w|{8&$IyT4{e!j= zq0q7A@b{x9Pn);)9W?jv-eqZg!p;dtuuasa7h1NdOR4N5lYgzWGy`K?1)NiFUvX3bhBe zUGiN|(*8zZi|;l}MX6M$2C(5-ZziTy$sa?4mPFPEhdweV&kUF!{^B=fspRu&6Le0M zf>A+%TwIST!-z>xDUYvNuELs2?5|&mD;Lrv+FPoe|9|%0^tq1gO!G^E1OX5r0AgRb zf%{r}l}b`oX_s2+o}TXM>G4EQgd^+-fABxx@BRxMw!@CFV|wiFneJA%M^a0zT`JY4 zTBu?zEbc4zomhw+AV~W8oy#mq2@-1Bvz|EU^&U-V@d6(yX-bwYT-z+ky zgw8gx^+m~cgDy$Z@+$Qf5c$4$VpLad_BvM-{|eh;66Ou;6Q0)~RQ6R>YEeBeZX;)oajaG0>0-i0}oercOlE>n6z`-d=@% zo%jXkD>vbO>}+ju{^XmlIp2Ksi;)k1m)a%*G~rbaLQ8A6y*1;-_wN?l2w;ey?GFLu z@!GGL%w`qtyAwwa3SjaL=a85YY)OaiW%qfsbVzc4+MpE-3jkb{ywkCR6aDG+>D z&Qo8>&pmzIRCnB6TQ}_n(*GXy&s1t~kI`?|l@%rUsdgqnVN+5q%HyqU7*>LhHHv=X zaR?Q+1_YI+PcPTSVFo20HPx4wv3NPg?P)hspogB9bH#ZK{q%4$YGR@bZ!}5qdrFEw z0?|Y@->I27Qw3^+Vl~$oTZQsnzqjry7?kT=as7QlRD#gMhoy+eks=WF_6HxwwzmvE z>~Tt;V5D9_yw}u^?flj2YH@ST(>T@z{FsrQ0HtO{$Rs~K@Pe#w?srX477%0g_TYdE z_gu-prKtge57-X;YePu7vF?nhipwu}&c^lKbGVhBupdPwfDs9t6SU7KE}gcGN?Z3j zJeRn;0L3<|00=YoV!a)J%4XGLInUc9`5SkdLXRle@bLBem0ZHl$CdziT57`j!pnv& z39UM@34DcOuHS`pOl2TzA7DruLJ|XCV=`owtBtY5aSq~9^f-qJ!v;7c-FGW=8*MgPL6k){_KS;lJDgN9gmKOZMux%SDaG}rz zCd{%F-Am+)Y_umUG=KJF)7~H;)qSD!+kSs&Y|K2fH3HSRLYxy8h~NJHOrZSPyb}Dh z6Y5aAh|Mh67nLf!;9pyUT?z=w_`4+x%Ct4#jd~t?cf>~3kcW8{qsO#|VMTrjNkQm& zOwZv6sVgI&NEu%Me0KmyVLp0Kl!%$pzd)_A3tIO2xrtkm|mu( zEElfc5Ss72bL`N5^_2Tp1cCNI}WKZal$EEiPpQh6SSpie+we4k#0z!&}WYQ^4xqgxJA zMJ2ypetY;8dv2hLZS)ZIIuFynfh4OyaZw76Hfuct`zo9#LT5Vy)^Oe$frX(#zxHVu$8pnBe|w9Ix)9DjSDz`Wq!*1!wd#tc!GA?mU;ptRe(3!2?f0E$P8?QmmT#FpH{6fSLPMT9 zcF=k6qt70z&@Qz(LO1k}KmMf|){9S_4AnE)of{j}303nGP=MaGTx4A|umpFD5nKcm zu?YRs!^AX!P7tZKQFW;0$N&I907*naRH*<5p#KLU7>7)%znnpu%aM*#02N?}daVND zxN%h*<1i5rD+eLSVtV+MS(OUG71l_o`r?qSs*HG`|Dp~=f$W}B%MJR*M=6z++ZB*@ zJm1?XHlp?8EPmIQy;wUy5wJ+R-bUpM*&yj(uiG3<}GXJ(wb+8V>A|I^10 zYkxmyfPU`ZbNzd7{k}6L9n{xRSr&eM0Z2l{o8KxJ#B|rBk>}=&Guv!c& zVN8q4*S_$qVMe>Cf|sC)(m54rz!ujTqTSz)9g5HBe!ZO48b$(=j$sAcV~1Y( zSWQ?vV2jvpp_PtdcGI{A)fe245L2SvU!5hk4?ou~)nj?yu^Gd(@N+BEDbh+LzXsY> z0T1iv;!^3;^%K*cygg$o!_$FZ_;eH`0=9Yk{j>QeobfI|L1O#ZPDw#ahH@$85yEqaUc68imH%mPbDu zaqe-<30w)z3mYVrLz!~n%5~>||9}6%`NO#j&Z{pzZ@v+X_&8 zdf{^B$4ln-%QtR0AAEY=IezefbNuiDYvX$A7)ky~W5DASB$8hP1g%YF*u&~fXweMP zIb_vm{;Nj+uAz666G~f^IHjez(V3S4zTmN%KgEnPYf$~RffZ5B`|}=t5d=Y_8Z+mG z4v;_w$&@&R00@DQWd%f{6e_w&phkVKw=^|)sxY#_Uy?GgLlvnP)_^fCYV{3Y9$34A^Pd?XBE7IR|? zflsCn!oOnh9nO%R~IpKAh&xu^B6L2WBDZHYts_ZdL*+h;z0nCtL% zX8b^yPiYfUE=5t^W>y+UvA(evOpARqDCn#-X0o|Qkvns}0rm36o9{TU|L$#Pcx2Sm z>;nFxgD9EEE|JKThBlEeMI&)d1D#(AmtW`i=ykE6_3j z?%gxa|N0;QP8EEk&Ue51W#>=7^YtevYe)#}m*ddc-s1e|S8qOc4);Wb<@EV5_sYtx&dt8TlCB&`AM>hc!&a2gY0&o`w?L%`D%7py#{eV=YC91M z>XLiY3&pez9T>VHty8Vd;pyG{jHgQdrtirRO@DUteV*ecRk53*J_}!Io9|lo-rM5r zWc}CoNtJN92-OpYZZD?i3}m_iaR?L_rSR-7=^1X0I01zdj+3~k9(uL}TkEgC|0CyT zufLs_t})2D{={dF9PBZHrUA8su?Nk05?$#28Og%(eWGBiyRk#ZHYC%waH#WrCN3i0J!;uXj{wVm{Ljnc|}2g zY?{h#33#F;zK|jfqK5TGY)??l+LjAS@?-1oYG`m?ed#$9;Qrv_b9tY4+hfRgK5UL@ zQDftal0*$F6oNGGsyFeB1<{I^qEHzLD7&HF4j7aFT6h` zQOOS@1{H_pBqXVB_79pERkwPYW7|BZivG_oT@?_<|9XRMLaYNIdAoXD2Nl;p!0+%P z#K|e4!N0O0?rCzxhopF@4 zAOd3*fD!haaw&2NGEZBd!RE*I#|H=^w7vL0{^tA6e^Sha*h`cG04rDg1$LkP0y_39 zFXsnLc~@-;M#R)hu>kEmunEzSZ6<+giUgo-eQqWl=9?>lwjI*%f|51_7BQq@miU+7ys6FkADvkR z?EmaLUv<9n#TRl?5Vz=lKP)<&OV@5V=j1$;n$S~dU})4B+D8uVw>muf^z7fW+qrzB z&m_*Ro;;$HQRN7VyU9YoF~37>Zmc&B;?hG@Gro52D!TP}<8r_d>x0O9TdinbfP{9b;p@{@cW3?T-HcH)`@Ih4eXE&52bVOz>rlXuuosSdQ$=Fdf6m( zPM7Glj~_bVT)cM6=`R4*vpSGGCu5(UQqs^=N?T~}BPZXi-BjTE?Z2C6m{#Ay=cOG> zt73(T1k->4r3%Ce^HdySD$li6ms=%2ej2eUVpJ5jU{4?F*=wp=fAPED`&OL+kt;FZ zsKgAtuY(GEo{7(B5A-51B|cDm5-^kvp+ySu^pUab*S{|?gZ^hF5p4Q-YutW871&0J zeen3V-O<);inS=%3L=2eXH_-aDBxRMahV|FES1l2@A53bLgBa2fPBJ_fAu@(S8u)V zymjUy2T*I2QYilNoEQm-|9o=(vgcs{0-Dr%$36p)E)lyLpl})H1&T~lY|?hmpPVfo z4Q#3;|I_OC{mj#+ROM!`>VdC`fGwMwar}9=>v-v%J;;89=`;%Y4VZZ{>kJS4WFzP0hDB081VJft|f3onA zv@7V@)8$<2?N^{QnBiIYDt3=BsAwfks4|XQOeMcip>9waL^#Q2+Xgq(vY@8{uf!o# z@e;7wCB$c(LXaL^y37cLu00P5dSS@~{cP*J(6z(7UsFTnUxh<2#&VEUW%Rnh6w%Tx zr|Y6xK^Gh{l0;Do_Z!IWCOOJc3RNdKihz=A3Be+&00?rgDhO^<#sokia0FeyEunH@ zMr|LWt{{S%9IE}U_Uv?+r;Z+W4(#rBe)ry4=T?92WW?ejm|18B6i%8{fw&Cw zjL!f8&%gs!|ImLkkwgQ2G(du%M1Sw#h@X#HJC;BcYaW5fsHMKHUe{XHGnRXeWev71 zroW}^Gm;RSRbZ&0Iy7m6uvzbb)VyISe`qnCt+TK;s|9 zbM1Hh1a=McPhE|%o&0`2+wtfUX3RL+9A&$Hp}%3vVAI%S8G_L*#f(y#>;;8%;`jZO zy=te@+RL?5J>)A)GQ!L8rE&~0DPmK^sEAb|0C{`%b~(R%^F32R>1V9?2>7-Xh=eH; za4Y#IJ+?%0<12ukZAf)hwRv6u+hh~|&iZJhB^~imUUPE`*)Em!Z`fnlfcQ~~eeg&{ zonoU{klhqC)=DM+gzh;C6t)#1{`qJotaItuRnj?6uMyR5_^nN zhSZ9Pm?nV6Z1U8JBj%aDeEnA5LWT6tj!#Y-a07)K#FG=Ts{!I=u(VPb0stqG^3=el zO7i>til%30%_2y}Uq0C>v z{@cI&R|ffCRaF>ZacKWur%MTm4?ek2a%g7~{D5N)_w2E*K#cc2a++X27#z0WQ_=c> zi+n<8vIy+DG-*TXI|f?R7DBsN(}RK$@I+${12d{|5xl#1|DIksUs#wo;A9l30nE61>X-l|&s0e`(VqT7g zO&O+#Vjd?m$hGz>IEMqoD@rjzQ&mQ>hi4NKpWpfFmrQ`~?K5W$pefsN^sfVK!pnx> zu&09J{;B{BUB40t+1d}-7*}$RjjMrF(oISJf#IS|5<7wbbC@8y*MYaFy$}Iys!Ov; zDhW4l{I9KI{YoU!K%xP=8X&$=ClG7@uw78mwOf5Axzyg;@_B$7#yxvlLN4j&M>W?n70KJYc2yJ8K*he}mK2M_Jj z>zf4Pn9k)zU?8Njo_L21socKRtckh6ji!C6{0Dr{V#tJIv3KQ>j*Hl1 zlrkvhHP+}{dqS%ihXFXNfyPe8NQRYXF-MTaCh-m57Uz?2kS0xlqFQgW2myYQ%w7`Bn?)x!=V>-^1?0zW-**1bzx^lY z2fuh-Rr!aVzx?BGnSN(p(Q71WUwP?Sv#G!R!AF4%Fq`vAKNksjs`=?H9EDJwP6hY- z>UCJ^f}u7iL?v{FApheQsuxIz^FlBNG0fskP}vdKF1rb-f!kg8A;$)?bgnxHFtjXWA%Y;S3%9fqy1khq4^-x9Mi|SQ@{%JG(aG(ZnLxI= zTLeLzj<{nY`85C#hr+Cx88MaWS@j~^x9^`l=e+mPdFOd0_+NhRX;Z}28}WWTsoWUP zSfg5TWE0k$m!Cal+GY%mdMn6-jesPy&6rNMv2EtR!O}{84<^h+%2NZ$N?PE1BQVUX zkU)=1(20cQW)V~AtDqZw;}fy30XFen3gGpNn)K5Pmz}SF`3ugUedp^Av`F%xd|mLx zXHFTKypI$>fP$|SgyaBJLnM>;3e6ma;6SzM_3nqCdNzcj7>Ad0rq+)+)Bc0+K5hZQ zjj@Om!*>InBnWdu-)eC%7^bu%h@;Rf^1aNb+qR1Z^|0|$6tHJ85|jN{_x59tkwK(C4Y^j zN`89va=)`s?HR0LEXKQcCZ*Tb>_crfyfCAs$&Sf**MH+(m~Xl;#XT32VYrO7!388+6WJzV3NS zRPtlv(*h(c(@?X{;l`s14Zy9ew`CIn7QhG01=`^L!v3qb1lX?I!TPqvv<)pZSKfDv zMq1U5F_A|aV2^}qAFu@M4t|9=Fsyj>h8a2j_^03T7W;T&Fn}s%#hl=^k{_1DpZxkw z6|Zi0{^T34nF2Usj<*JKxA_Y@Pu99Iv0P*CK6b?Hy|A=-0dUjuh=QncD~bxM z&p1C%f}eY3RL^uG<*9*WCAIXuHEDA*M$z2nseHho++mUa)zVk7VNSy)-m|ycIeK8f z^RK@3n)63r`%+-%33TKOPn~eC-t2R3_6?MTj;M(09UL}E?VWk56vGUb07bVdF8hZ@ z4N5S~xuX6oc_M-Usep<<5QmMik*LMMc6&UHjf>t{K^a?>pY9mr?D*d_S&zIY$XvuP~4BuHx$Pos^UZv4fxdnU_F@AupQ$T z*1Wq~$E&vIZ{cQRTvV8GJr{JDdCmt)L}TAj5nN$;z2D6A|E9;V)~C$mOQ!yl-#3Q_ zOpj}=H#aQ#%` zQwjaY*8s=((qH}J3u5qDci#W#oM-m8c>x0Pbs8JOGjB8+kYCE5EHqEp3SnQf6(`y* zpK6Pe?f`DGi2>=$E{YSRpFR#+Dd6wH5Sqv%4VWaqSR|m_4^qr9Ix%VNdi-=%u{}C* z@>5m~5Oag2m)3?};tv2X-aGrLbNR+C=R05dqG1tY?IB4XC0#%EG<5F(K=OeF@#_HB zUA%VFFra3gblM-W$94*!07xN`Xf!}U2+!l7M|Op1MvR2RWy6>jO$>%zRh6AyOogwyxsJeSdPx=d|HFUy3+HR9;txwp z^U90QnWW5Tm#+mvTH4k=&UBvwwe;57ra)yBf>E(|cc)2ueR%Fd6qndA=d9RLVG^IT zP=E@S05Vkl`3$pCpF9EbMLEADS>u)l8MXc@`Ny1Ui%Gb+=4*N)YON}lo`|%W!3l+K zJubnK0$AX-!ieI!_cSM}3{pt=XaOzseZ;Y((#w|gt1_3wFE9F4+7j9*MbSf%L}P1H zlc`GM{831%C%3mGKj4VxPaiid2%lcK^2Bf^r>2}r1<(kh*$@Ku+|zzrHms_WpMW|D z@kA00_|*W(*KJ$3IrLLM?={x54qNAs9$Vj2+sy=ZQ8O^dmm5YW^`* zy1)O?r+!au_4ujjSyO#iC9twhq2ibIdmu+!A)D9ob1sSw`c)lBP`g2`QfvyWN2VR0 zK${y`pzEJk%R#`VaY}>sNdr1o`UB+u7aK?tFIjx@Q{sA@$X! z3O;3o;{v~VR-T9@^OFSq+aLbS1mE2Ga?cIB_H31%_DaxMlMo~6 z=|@s)#DlECO+V*Jj@_IZAQ47Du^5%0tR++`wuxppYGJ=Hl{)HioQ;l?XPSPv?7yF{ z$Cd`w+T39LSwIAdBpTSv8fewCjvszYG_p2?e)OwK@*h34|B2J_96x>Hh+*^k=U={Q zfYwhO6w@E5tR(j3CI01SNX;bq1)#Zdv)9kZB0m2wnEQ;6)v8imr-v2dK^Y;ymIG3;A^5iqp zSDOTXodp0Q6~5T%*gN@zB8r8j6|3!wTziIiLZ(nOv8We709%sk0Z+S!aZjgQT29$x zwCGLbkp|c!TjhIB!qj%ppHIkH28YK?i=?DY(j%Sb`-wGzXC1abfG_!k_kaBB-wCKV z=DhNSXFq2T0ccPA9AYoTAfph#f`EZK3bb&6k!=6&ZpD6PoNHnn&L^-D7*-(J5=s6k zpQggShyTfoHV=uErv{QFf1dAcLA@8Z${E2SDI`$3v9V3f%oaLyL%dIq4v=J2k0nwb z8el_z>ewMeJ4nB;Vha??w0xkqF&lhTLO){+`K!1345eFC*9m)GoYXr)qEq$jg*_zf z^)f$?(O9cKXLsbN)88&tNfu4T8q`e87uJ9%CC(nYne%`_7su!B!^+jn_tSreE{h~x zxjK%m>K_cqjpS^p%e3}r#!5fLI0bsa1qHxJ5=9}fZ@?rJ7f5Kk5WZZK=Qyn31da=7 zURzUTC~L8~E?vJF_O#~n*kqi)t=Vp$Rcyni#ztp+Ds_G z6>2YMK)zbbs;liiFW6StH1k>f3YQsimkKC~C9vh;^S11YH-1p8`Fg!B@xlh}mBb)m zs_`p>qZ8)dC8?N5VGRIurlr@6^j#AuyO#09|PDEb7Vj67sw(C zp>U1y!FHWbc;;b%A*S-?`yb{z)U;|d@zSJaMqWLIeG}G4T2hY57EGi(HIO9vEq!mP zq!m4CH+Ii0S6xuf5#-(_s>npjLj!DzhkN!ZiCin%o;Qj<=xL)3N@S8Gd-i#H@abGG+>2% zW94}RTw;*TN;2k?A@w{0naVy=jzS=m0BEW2e*~hIR3QMFVcd)fB;TnP^1|#8#p(PE8YN|N^W!Vt=teT*Vstjz3L6WO?>$s?O zS5{YTp4yy?wGq$^NqOik~A^R%edc>?Hn1A6|l@ zajf^b#<|h{PO|U5S{3Y2;wX_s13c?Z0%!tg3v1tE|KVSeh-(orHk**-dgY~O)RTJE z`TZY0Hh{Qn$3^|Ok%=i&ZETy(jTT1)j$vW}cvvbz!2t;>5)IgPHiZgTY$Xze8&v>q zPRNeOuCu{B=|@ihw(&n_%Yd$*HbblxV*(?fTeXCGBYF>;{wz3ck<+2#T8>Tmw(pT#z~?!5Zq zvx)&c&VoVAk^QcB$PQCc<`}SdD5`-GsgSe_^h6uY_doh9Z>zAY0!gvv#9gRUX#5-P zePX0L1oX7Us=3khQx5ZqJ`!9GTCgxiP?;D-psF^5Ox&I_mA-WzgU_2mC*?55?#?#n zz`kzhjrY&y?0FT}Jgx~fPsLCiY!J_!I_9h>*?aNojku=g`#KQ$VGI>ZOeMdPLkn_t z=sO1Rf`nl?%Yym?!Whw3fGR4AIpXx-jFQxaqA#EHik8oS0?L#EuBkyiC)xi>Jcvbt ze|crifG_fWA5pxAGZv)~Fdf%(Yv`;i>;@^bjhjF%l^g{3!U_Ra)*tqQF2n3n>0G*I zr;?w3VurcFP`ia4>7&mxUV683?kJGX=oNqvmCFDsy~;uL`hEKNQRjm8x{p4)Xg-FM zmSSdk+`;#H&TPlg5`y)OXNsh~7vVUPldN@#BpL{!0ru71S|eC{Xd4!%U>i{_=#^O` z%-C38Xx}rdgaCVMrnvhpeK~oyZORTYe(>BrA5kS)6Hg`FLdCnHvsy= z2rM>}YKOMByE8AN>VvOi=uz^H;=h6z-Zq$2)!Jj!7ce1~g$aV50(FDy5`1wh{?y z7>d|qYznl>W<3=<=tdRnU_hz6lr-{VuiDT@osx?rf0tHDU%yvw+%mDKr%xWuJ9U!C z#}6HF=0wN+-q}y{9?P4|tc5y%QD7u0|7{8D1JB>u>XU5_ zq439lZ&tj)mKc9o{>b3SXu(qg1R$YGGBS}TH9%4j07S3YYWxbI4wz%&F23+P&Qdxa zfG@y^0ksUwChR$Hy!Qv^?1d}N3s0R;CI6u(_BJZ)S+9r1CK`pfOp0EZFW7(gcC{Ou zht@f{gti035QvQ=5}gLnajkL!YwLIO0!gA9L2~A>8n7D~2s@e*`#fkg`u2M1-3uCu zG!B(*pba;=3<)*L0k zfF^A@f~rd!<4=Kv=A@I_qEu#?B#^F6gmAuBq6gYBGpqE^(YsRBQ+XLq(k9{bx~ zwe;Adq}LCB@p@js9eue0Sm5K(mNTqi-)vWXjiK^#ro}tG67S67zyc3;dBITullSdD!_jiL3erHWWw-}>`I_Tn+n+B2F zpiLxd0r7Zn{~l*_Y|^3EU?~boQr3>0av<1L;})K0Dh^yqGC0j2CGzZ!rfFA-`2>2z zCL!#qlDD5Vy1_ zFtKPV5grd**cyika?Jn|s>|xk*}9H<9=>x|pYGp+{yRF6sx&uLpQ>Q#?r1e&3~aD4 z0+O9dek2OiH!#stdL`VJt$ zmasQaZA7xwYw();WmSRBIM{n^%`-Vq>1kl^ZY7BRKFqi9v2!sMAQWVcwoXmYYd10|{f*o6QLkKYcLlm;|2qYC*0 zV%sSsPpO)pwdSMGE}1>E(D63k$3LGH0Glc* zs$cfUS_g7TzpBu&7i=cI0J{)7;gw6!IX;$6x_+)jbusr7t=7_oYXQQud4HwsInnj! znWsC3MKN(K>(yeBxxhX5M)=pQET;(2j|?h-$o<x^Z8c34-*|ZYyZ#V}S02|qmM>QB)(Dh^3(L*s2w5N+iKpBiDq6&u2V?PN!NAfJLod zh&b>?UwA4FaQdPG=cMbbS>HLjv|;ZUCtvz;`^D|w(`70|di#gcFX{W|ROLsiJ%i1_ zkUJ21W@7_gQJ+oQLh>BPihjUB#{R8Wzu+v&$$tN%b55W7?QsrQzc$87Vv%!l&*fX+ z(y!(A@Wg3MB+)=n4Ron*GJ&2rsjS`h83fXUJdb|o$}>WLT`HWiF`6(Qs`mg}*?yi< z=TrY%*-nWh8rbX_aMzpLV%7;NB*any>U+-a&h~<*f%)l_0Qv8!KdlQ?EchWy2LQTc z4QQ|k3QQl`UfFk4@{f)`nVd*JF0P+1U%zD%#GP%;hVjq#b?HZJl9mqvr?$^u3gwTR2(#p{V{#cP(i!xYtsK-YCjWuA_he)%8QVEd-}vt!yE+jCOXb0eWFLFqSq!{ z8GZ-t5AdOKNveoK{k`4#;3@%ds6<~s#V5AnB%x#y{tuCe@U-|!8ZFH0pPt( zFZ_8wLY1&7icxiTBHlHy9mWm4W(5S!lp)dY|LpbOJNv{2^1?HxGT;9EcVdpj9$`VT z4U$?|T2i~zC8y2D`hb6RVBcnu;GQcUpJfgaPJovOw{JH^7uZ=}$={96x+OX#NkKt2g^ol@dw_+7j4i z8uWeLR-fTq!N<&Nxjj6a6j)0n(LhiQkaT0+Vaa@Qn*;<8zx&8-x}&!mzk znS%Y~g1T%c`3bZc(4J^iH=E2WFp0eecp{NR18Be``Q=XlDhj&ZKyBSSIO?>j-!;jQ zLc+68YrRPYSI_3D;w^NZAn#L=&6m$b#Hfv)_}`eKe981 z;V03KiwbyK(yGee#l;l^6wY-{l5J~u#f)c}zP?7_#``c^DnXbo0NH7IV3V<+vT~b= zyLgF%E-WsZ7-FZGND_Hc1AtO@i2Y@7#9x1R64Jy30SG1HT?1~sXHfb}7xtO|`qSSC z1U%z>_2uXD#=G!kc8mEBAAZ7e`?Gn)1LyV3u$Fig#`E%XPa81J`ODXy_!u@A0B2%E ziA1jf{2=1zw(pJTN1j{(4aA@MC?matYN8U;D-{uxBp4xHjD<2%!xKZLXEzD7a;27j zVnUnn4+Jfa9^CI-xOU4K5Y<=Q0yQ3vZ+B;WXgrc7pp&nu>arA{(RDdBP`mF`V%>Jo z*6a7{(m%O@tf`Q1TzakI&NU`J zPmqehpxe6TaL;}d5C!2M=KRnrE-k4CybmOKu5o7jJ5F|^64oT)rYu)!(G!uns}j&$DQm9Y5#r>2b<4JFc}LQ=os=U;?cTndi#J)-r^~y#5zOLxn0%eKm6tIA_YWee#>L3uup-!qXD|q z2^idRlRb9(uorp z`h2aAHl;&AC<;I_;|Zzj-8XBK&FgKIFT=B$$sCY*jaB(N90RB%kta1k%nP7C;8Q>J z-fogVDlq<&Bi$H(0hq>%ig{PKkI$aF==@a7AE%EWa*iE3P|!ypHpTM>xX-2l{94hA zxVs-YcBse1v92ms52zAb?sT<);C=J9GKXKktag zCjC{$wQ8XxiIBeF=Dk3fzBx6m$}MdOBz6)hUk!lpqbJ@4QM2c(G`2lMu65-`uQ5(K z74V2lsEp_l4bVja8vx^!n&7|yEW)`eMp7DmxX4g`W(bV^=LEzxbNCf<#ob2#{B=@U3 z0&CmEXmrCeWj_f4DrAD*s|1#h9N6a^>e*|YeEWSQx0rR6wJZ1P_P91b4+P>(An z`OUWmy45H9&fJ1|`TlT#4;W@30eN=CMAy#{-5?(Ad@_Z<0q6qQl}Mt2L<7-kV0UM@ z^(IA6Kk0OevCHc-+uGdZ99IACYhvO#fBAZ}6HGfh)pAsCMag)zN0oRmtw9r9Orhlo zwsVVs=Qfk1>2m_8iS0xCq>U={jSpk5hSdu`1AxD{K!1M6OU&u=jXnW)x}B=Z?Y=(# z^vPoaK-W2+Uc9Wt_v48;l6v&hCdp6nge^(Mv*kbK=~bVKcQ0F|&u9GGR9puhCX;b9 zs_OyzC#DThKA)u86R5|yMLp-cUOB$ksKOSK{jgSem7$Ro$Umy}*XuaS?KtAIfT~8c zf4KrM=0E@7rv|k4%1h6AiXXI#nF2qaSfouMw!^ED4vXCROW-hycMBGA{H9+z@eUf4lO1=`1qW&9OFGpwK~)Rq0@*Dlp>q>zv9LQXfsvKK)Iz?ew&eAMLd0|QPetCid#hxFi zV1g};T6^LaDl&1T2=+!HQ=(0;(i+Dr4uLWe8;MsKp#q-gBeU9mZ+Djxc=HCR6ZHIU z&&bSk2R&c5&jEHMD0L%C|F~?&d;MF#0(WjDuh-9H9tX?7i%*>}iGGp=HU$6^00j2) zdwG~j(yz(0$1{knoJgX9FdCpDmeyf&dc`3mX_i;k40{FlejEb(2EeUs{Se0$bg9}d zP?x}~Q3%&&zsko#6vcIub0!)nMGbU{{mulf)2inDjB}QR_MJN>;7t<5ivR|A=G1X9 zb~w&Ye*H$^RsmkFm43|V#FSbwswz{7#i#@dazp1^BrMr0&I^k~107df-JUf*BL2494Ur_Zu%`DYs*=52_XKAx=WP zli1Q5?|tZ8xO&5RRSEvX`}ca960yg1eHL(^O<7U_2(Na575&Qdr%#xBh&YHV*b6h7 zYj;Ph0g||XPyUTg1LOdBHYc@zu_h-{UK;S0y)Lgg6>YRFjg2PBU#fIbwiI)FM)cF# zQ0Og}Ncn02bouFHhn$Z-yW)#lDeR0f{&2<)s@GVG#xCp`y?vfUFTJ$bus#tzU#SVn zvI;;&8fUS@RE~)r&WI{Oy#k#4ju#3#5a@SRRUMUJ^kNX={1!u}+;0F3*bnmcEeDxT zAib;1`@GdF;pftGMfIJAJulA~V{9x!l939oR8TXTYMv?GN86R~i7M%nseA5ko?-qA zBhzvoND^m)1`qDz3X3tjT9_GBc zKFcbH-&3)b1b;k9{_2NGd=mPZLINJJK~w<7R<#dW6liawGt*`2HTgZ!KsF7)I>)m> zQZ0xOc*i!Q($SCK3mZ>|Y{&OM{>*`0G7g!ZoprX#|ASpADxs$?Ol!P~DKI9HJoM8i z-S2p}sb4y!$@lI>1Mu}bo+yqV8Ji4kuZ9^Hu^rQ%E2j26e#c78iTFxKYjdDI`{n0O zE0(bAoV#$vfbZPvPfXDcaNX%r;>)J^iDy@Au^Y3(w!_v+udK)Zx_4jpm;6rK@tlgS zDdqrTM!95N&o0GD#16cIv~rzmBmbSYsQAOMqj-^-JM*6dzxiAu{xndh_$Gk7$*Eb> zdiSTVzhf9_zWM6Q0uy;F_#pO(KQS?BFUe1AXiiLPYpZu1DBu0cv4i`~zI>x^@bmS~ z1qIS>IFWcXKn{Qy7XT7K^@)_12JG(4G0q)C$+^!iI+w3pGgqCTpLaeubJlt1_a8XD zef^;kCn@@JfBg>WrOiT_*+V2owZw8M7H*88rXnOtVwvD3xdEg zq_~`)D1-oTt&*-(f5#yJb&ZY0PPgVbD#`JCm zICk|`-=jZT_=(CWDol8V5vuL?tA8D!KC7ZUC-*dznqF^d0}2r!AgaWq)%hhSD`^dM zwYM26*`P@`zmqedLD>@xbh$W$5bnJvL>(w zfQ?cU>|SC{(CHI{3L;G^BBQ_ddpghD{3Aw)DbaJ25C=qVQwqhiv4>zMq0-Z~J(NoJ z3KXM#*cE`_u!ce9Z9B=|*3@WX+MGX^U~duohAn}ZP9oVg(5(9uP?J~CJM1>xOG*3R z?D{P9SK@{|5Bdf3+mq8*=b{l$vHe~pjp5jp3!vBmB)`JhZR`^@kA;jm}H*}%AfuG z*UovNBdLP?&2QgwK0N!0b4yf4{Cqo0<%6GL43AD2!=<6#dtj7>mcB_;)TR0``?+Fy z9E-6Ey)HqJ)~2MAzdXLR+@svn)#3Cg0RZ|h4w`N5pWeY?6FlU)aSMpTRwW}pxp<}I zXx*q1!8kUk|F{+N@OoUky1LQ?j++|oQm~~@3F`6(T0>S=gZJA`KVP<=S)&NpLTO7y zYcT{`H=IJM9I^>Pt~+{BsZ;JkP<%~j&$|kom7=&wnw@oO-w;!hCS4E21)z1myM8y4 zKrIeX<+e)oY4(QRP9Iy4pIc0&=W^bjjb6?RjU^5tJvGyX$`k^2w*%6IvDZ-fqJLzU zdAH-z$Kd5>f=(pT>1PKuoekkR8WXC1r0K6xd26(SWOgRwenN0>7g~59^L!xpeK8VH66|D(b~&P8qPznNKcg zEr<>n&76p*O)2q0+ov!rLWT6h`VA9FTshzBMrD=hD89#U3M|8}%N|^Tqm|vUU91cfKc<gPdo|&J%{+6k%WS-o^*gx%-+oy6p;&i#=*$+%&JPKXIT40zM=N=^-0amT) z`N8h9T}aPd!ENOOi<(M)`8wwL)0~i5-OZ*^{ z_0h(7lXK+O04+F7JbgYl(@E}6C+`$f17){U@qoYk?ir_6i`apleMWcV3W#DnA3t_P zXvPkwUW>|r9ICj59;FScW&|CKle7_qYH#S6N$4a}{u-dd`I(bPWX#t%Kl#nuO4vl} zAYnjE3Ec;Ma50=gs+0EZ?lQg6-g^JTlKKJ{7M4shx>yNmR~g(3R9FyLic&C6SJbbG zwL1y{JPhzVA zS3_b|B7yE#=-)Uirxl$<9hB*8Yc)Uw0@gO9e`v%w##BX=vT#rPscPi5=lCE_e<|xB z8C(1sAW77$fcKb!+(AGyfT^P;e8_jk>ZO0CcM=#D+%4 zisCb1fsd-%lqBh?V~3NQJcI@Y1;jP`aS_3rExPuSK-|1aO+NVeoMFZ|Ao}Sru?QId zdUTFIh|yzkWZX=1oJxNAM^xh46l@P{om}y3H|b4Mmw1s)F|k4U3HMYz?~3cc6JsaV zZc`fT>&^N+w~)K%G;1Ps)i7Kp0Koj(Z`XYa@MJbq35*9Is4gVJ*TTBlN7r(Fq&GowrsVco6LE8{k@@~ z!Rc&ociNS>it-t6Q4o1XNx0F8S*Iz2;J}T6;Z%ZO4q=q@$g9_iZ5l(h!*@Hk_#~KN zRL=seF>Nhv#T#SZ33z|AcE?Ubk3Kc4zNJ~_TIftX%M{pCNEd}HEUaqpT9$*p%~@Ji z@SwumzHVzQ%L24$hR%wT{PT;e&gxx1t5*I|T1h)kS^RV-r&Xgw{&e_Z~ZU zer@`Hx2?6^1lsyXCY;j&?)}UwUbxz;?^BgDdZvoDc4ZDK%kTFeRG4bwp|L61gpbo( zZ<8IqF8kaSDrC7=xM#R`N_k%Hn3I1?R|Bi-_nk%sfJW3NW9H7XfK&d~2fs6$TUb#` zt0kvWw&SvVNY#D$Jw>47Yia~45TN0z*7zem`#(R&Y!1q<%&U#h0bvkdK`PBKwE3Y^W0YTVXwJ#Gl%k?x5jZT^PSc9$?w1ZeUY=LsNl zjlN;i0YI%P?bDh)1-x(9JCK%*eTEfRv4)ydn zAAIzg^TS`h?)-;8|5H)O&pWkh>y_z0g&q5fe5N{mpSj}Mw4NXyOwxVzuJ<^(*KyU! zZ=2JzGBK6%85ZO-FD$JZ<|1P4TejNs-*@b&HNd9v$=RH-3u5}PmTi-ty0m)F*%pua z=T%)LbsT%sT>(>vMkfW-Z_7P4*;`{1sY-tA++wXAxd#~UJFHl__T^k7*-=|(>=FuW zUV83n=YGKWaa&7^q3WNIUlAmJmpMLSLbPJ&?&xq*oL;@zYxdEey?cyL9OOK4eFofH z-HzOO=(xraZB!(JS*)n1C-j8ZuisL#@V21~>Xze!Q$C?8s-3DPTvDa^$rHzNUY&*1 zAP@7{fb()FM$T_;weq&idE|6Xa_vsP~HhsBWcSAX_h2m0*D$D!|!wK$5k5+@Wycro<} z;+y2>+FfYo+?(~G!gbF=<4@%QRC|%@I^%n`{cB@Gtx0sQtgZ-YueoMS*UR8A{p8VukA05&F@4D+VaUD0J;S{d*M0ifppsuUj0RXA=oNVV zRsGxxOi^EmI_x`#|8tnuk<%h+x8Ko(#IrjjVeLn{D)Ia>^XUX{%0 z4|6_6mGOIvN+Qsrz_2wW{%sk5llF(cUA`IZb8e~7AUItbreYVNR{N(&aQ$-v1`m&;= z!}Y!aXSe$M(&j3PH8fohE!wN5RoS?#7-rmaO&>=TKM#+M8GDsjO4NGHHGWw0ShrKv z*sw*F22pMQ(1dC6vA?^^X;eS=uyRUvHq_>CzJJ#F(u>cV>RafmZR;JMsaCc+J~NP@ z%zHLfy{{mKv4`y2L==83>RExkx3*U7lG=}{4zAGmOl7dm`EouJLR0`EbDr+^m12fd zkMW(g&UTFfTQHIA8ra*}>Abi8u|VCkIrYgW?+Ppna||HX)}~$APqvx<$}Qz1mW^s) zF!}d?`fbCAx?e1{L7xj^khIeyCK)t-m;P1ZFHLFfpiNDszTY6@+Nrh=2gLe&{_+(S zAYO7tmGk31p;dH{^QDh3`PsV$@Usn*o$@!i)<^GW3w9*2!oU9hj|7yhb^fRS@xKq$ z_A1zwgMF%0_J^`d@*n8g@BGDIq$-Bqc=H{TRCw`)XPuj(L;Lk_-*mdWI-NV}Db}-p zPtI?qYWr44q41pV^TyoV9fM5WC2F4PDsO7izAq}^9nO^-b57MZ(JWTSi%iZtAxm%G zzrSuag7(&3p7mns(#dHe)WB26k0{aD>|7RjW?^AY=({}V%Yfsc-_;#GSI}lVi0_j#7=* z*C^J?$;n9}+-tNZ*A`8mJ9ktId043mz{Mx#2>si_!n_HN>}seh+8Awp&pWEO8-1^= zEIZZP<%G+yFC|%C+UfujtlzoAd=~&};*|UMD@DCrqrQTx($Cf>?uRgAnVFe3_Yu!+ zTx$o{A&^#4@xa+3TZjAHuguQgGeI2az_N|It))?c9R&hrXPm?PyM3L0b-6;77jydD zKIwhCz&_D^-vsYrBbXC7w5n3;TF84J04wL~Y-`G@7hj5q@ZhTL+YCqx+qcjfE|b{Y zE8H`D*V(@B2#1=SdV^`;ftb=-o9e~Jq#lwF)(hT$VW*F+NHt!a_VhUK-R50NoamWh zjZc5Z9>|`i?MzKoWhCRl$NTis6(#)E%{9X6uN*$Vvb?02!kU1;@$IAKpqFHfj*|3S zR7KX{{NdB{O7d)Z>{-mGAKEir?;Q{rr^?yW)fVbmJ#t{5339&v&b!WQUw9_pWdpxA zFP{Y8q@%4l)L07-fc3jUHu||hg*!VQ*wg<%Pm#O+sCmkqd} zUP=C%9LLxiPr09GcV~+MZY{`W&ZST0IIURGvYuB!E5%wj%3as!?O8G70>}-*rs&R} zHWb8gh)LRr-`&4=*Q|R@jdkXEb+6<1-=hcj8o%q^KYXadylLn8r%ssS+qmO*e#gC$ zj<2q&Ga%K~l{G8ztsuIYn|t7#mp@u1K<4Vol2hB7v<&y6$I^g0dKu}%W9o$B*td7L zc~Px@>f|w}y}i}>_P4%fDEp!Ff8qIOon0YOVf1)Z%1xWP_e2E?Y`v(9K(EoiVk1hb z=hd?5ld=)#w^`=|lKtyneo?`Qe&_0qUh^#3*2S8F9esnNGU%F3GSK$<{hY76qg9kY z>(0p7grAQ^dhCLF$WwW4`jJI?rP4TglUPrdWZ;9;jzXq|5C)CTt5{N_%4%LCVt2_D z_Lyi~T!J(8A)9p&$(73>AAG*{(k;8bR6=QB@t6 z06K^5h$>p&@bia*g#MaJ{w~>Y*#vuv>YOXull+S5AvQZ_?#ZaXZ`eXw^&GS2aQ}x9 zX!k}`$ZXiK%&%C6BhbF@_wR$JQ@Hg&)UGVJu1rn zsO7e(C#fI_0%i&`wrVQ*wa#+=bRiLk4x)ON&@Bu^tVf$gX#1!CZ3ug8vwUZ9&66Z% zlPdX*&7ytGtHA7dNA_8}Sc&ij!U*8|9>u?2fA1m9&{K;_fN zk6KgBmtT0+oa=X52d2bGNk47d0 z$m-gf0Oj`m2N=_s&-Bcl#|M%28~wmx%Hx$tehsvVEs=YEDvv%n1A8fz<82}HHTg%g zIo=H$G-k3&cFY6|Hk<@ zfBq-VtFOFl0;K83#_=k&UH+XHKS^kdMw^ z(qd{qu*U*@L-crEst_n<;+1c$MaWSqM4{|26&V`8EeEDrhEa8?3@Ora>=Jr91sYn3 z~Y(adybQ1xPtGI_6m&1H&VxqV3?mz4}Zzif8uQr;i^p_Vv$Se>;+Ab6WNs zYh6nWi5Hlyh?(^bj(FCmtpm@gDva+Fy5l(Ptepa^uBsIXuZ<)CJQvg!3*WDpu?s*3 zyVNcKpBh`U6ol54Hx={ZS;_@u4l+(!&QZNPFg&VOS<}v5wVbdm#$T1~F)Ma5V%nL4 zSzy}v7<(w(x|Vrve$R{#(J}y^Zrj#M>(485MQFp4OXx2SK$%w}`8Cj_L@YX`ja#;R z$Nlu8)(u{r5irNP*GT^LrU86%lIdewPwpx<8CEEuv+=wD+&X@6zljyoLX5(XIQHwX z^WSKn=iZ=w3hjo9A+%QF9wnCd+jrjg+ts2NLoSJHmUWUpQ-f4l)GL5!V*$Vr$9D0H z7y`LOV#!v8Fo~d239{UmRR@X&z%rftd7j9rJr>*tc=vTI=G&Vn&AtBdHAQS}F{ z+^~|cnL;1}$B`gYSbFcwa~I=x?pp#`)3YhM&ti-r-!-8YEavk?5`Jgh4)vg$w+@0J zm}Dw11R{t_Qk65+>Iu0V7_B;jUKk;HM=-0a(|*gGFyqShS)M0=OnyLt6>`*K&XZ;vdh)jopn+OZ=aXFM zSFk&3nN)ix*fn^?DF7LG&gK^%X9%GZwW^XfB+-n=0M@k**{JG?Q(s}OMITWD{pdT9 zL<5-`;8|q7>GhG69vB)mY#2um?aw@2px;j%IV7NJgF_WB085(!Sd*j+fz&tvqu9gr zSS>~+|Cx_3DEZoGnAB|QFxR20+{LA#6*Hz*{7wb0wumCQ_BjVcA%0kGoXQ*Q!E5Vgs3WYx;Qc|JiS- z5>BMJ8XyKqERt4%Hb7EC0{-Je@Cqx>iK3qv4n;E93yH+90iI`=7#Do7SlJxqpnQDV zsKJPJ?&4K3%!DhZu{pLNEL#ehtv;0&1}VWJ?>Ff}`Cs0jm1 zC;3Q74d6iDS4A__kWt9E9BdNofWL|*DI{x^61z_Z<3P)f3F*upQ|#x#hJM_nFOC}l zfKrn|?di6hjzDwMdIPmGz`daK_PMS*p?lo~X3NY0T9SiL;1lOEn?M=O^MO6)g%Upf z9JI(_PvaGbbhWjZO8&4(ek%E~wYu7i8#MKzGXinER@AGlkPvuXQ4dDGk2pXVg~pS- zU;Y}X6P>JS=W(+y^2zXAQpHPat2o7Fz8D;NypliaF|?}skN_-e^k$J|^#@Yd6e12^_~E+qTNz)&1ZmE#tUm9M^!I37@9B zqqz1eb|@_-reGE`K2{vF&wUMIwzS=%&CbcA2Xh}7@SQzf9nP^s2c-8l!>AVUGdF)PDym=KvZVe6_R?m|Hxvq%mGKw5kG3FKZHIpQN4h=8Ku)vVm?z7_9rH2N^ z6jpxti{Co`^Z)XHIe+)VpXWuxg}si~&qFahqbg1MG{r%G>vdeoadtGo{l2%m!=T&$ z<+pF!IceHiE?&K1sw&wmqA=3aE>-YHs-LKam|IA-pD6f&7!CjcQK|LD@0t_C&?316 zlwM16+M+TVNVegm#vPPTR~GkqWhIrXTa@5RIjipbv~;0rw^;P??l^4y3s8ZcrSxGd zMY?`iXbjy`1XTcDbjY4bP;+K3)mzHTXS^OWqkDaedQ!$&WxzcRQw53JAeA!&!3Zc* zDGJSIHla$Tjn*m(C|$kPm+g?8e;ZT#lQ^G$=s4+TOE4%5D070!C+g1f7^4!d%euR6xwTAGK5(?O9O*!@>8`ntAQN9{ry{XgddapNTA(9vqvhj(T zz)EC8c?Rs2eL?^Tnsu05f2`bxyFZtcCecAc* z?|#EdwQRQW5j(+$Vhzg_xDv1rV7o>4wuLyMyB~Az1u+>_04k0EP{8@IrI=5yR(c~^ zqgX4F^#W}bmhR40v!>@;*8u_GGj|FgpGa{vz;oP8CBFbYD;8SH3<=!m=EK6yjCJV0 z`PYBseD7~AO z2zrC|gFZKmkO2DCTkjj1x7Xi0pF(4xJ%s5| zs5S+*g>?$Yv%a=Yr?&5a`WY{+O>j;@^{30IDEt9+KtUCSlbn97IDUq<8wZ%m>{6`c zAD6xj(~ca5Vn~WsvC-_~gCpTVfbJaVnMiPgph%qeIyL}R_LG1*0#wcb1ep?Q9Qw6< zNndEfK$+9q(vJ{0Ha%YzNRC4YSQE&fq$jB67F0|hL)lvFPo*y01JLh#0cr`;A=Z|A z50j+DU6V^)ugUnlX@KAl`|F*#h0s1%p)1a+La-Qg{fw(wRc5U5Q1l~!YRvkp_HHQl)&LYYoC+Chy1MKlT)FK3yMK6TlJg>L)I(%Rc50kY~?%&ht zz=YPXRsc2v^6tP0;4Z0}wY52(qzmf{z+nFEABnG;|IEjq8U5|s-L0wH`(9i{$@a_qU-Mv$hDZ_GMVwrl|m&yG2F{HZaI5*cgioe?{nfa(FWk< z=bnnNWlxRqp+-gFohc+|NeudJX`6YTT)*Qlr`qSFR$6QxDD|raDvm<7Z>vtT zvI(sfi60Q(Po%I0U~DD9pNkp6*t5!$u%{9SOQiT3*rE6x-$iDu((lN4Vu$s-?$Gnf zGiyVhJ$cMIb?mTt25p=#Ifp+DG|JaxPvaH05Ko59vsNrxM-J{&taGRH@x^PO`@?aM zr<|^rB>8iETWml$9)7DfFhEFjjH-$(jC^|WiZdp|>ZPYoIs3Z1od5iTpE)2`!UT`Q zjEB#n6DZf|qXsorA{$QwP@5kP%als>ZKoEm#^QlHU#J(NwT-bM6v1)e2eyoZL$Nnc9296_QZ_n9(hJd6z38U z#5kT8uingcNY=m8GRai(#}t&=3KCvDUI7>6>g$+wplN^Pb0F!lJ3-Z#x-d>&Uix+u zu!Giro01b2^BtwGqGWs#YXC+j_WC}{pw!G^kExms-;U~(xCI~-OaidcAZr3f@(gXQ zh%PwFb4@FRSusqpRwt5ZV6$rg+p*EuxO3`Hnw{1;T}Ob80sTQYYf~zfkissSnOn5l zo0I_Vva0xY$xn|ntor z_zqEl3IJ6vEb95ph|}=B!&K+me^CH7{N7vTaD2PgbrSr!D*0(wIVoS4whT!=p)|fx z*g=R%-t70rRFrGf*haMZs@YLx_Ssy=C4Vm;4bW1Pb}6}@=O|~A0uoZlw9~Mm0`?#m zX}eujlyfN7xsyz74$aeNiXOvMg831W2n2#~?t%&i#joCc&lpgzz5KlMy|2Gwz{dag z-~Z4g2Z99DgN`Nra{vrLr8m~8l3xaOA{$c!Ck`Jp6^ZYh`NUI&o);~U;OtdT6*dYg zuS!AaWtOVsUnu%;qUuXu^oive06`YklqgX5ca%J&|2b=Mu>~490ArcXW1pUf6y31D zJ*L?6$#s3^h|>?<9l@+plRI;BCRt42qF9omZLO6PWiOx!P>Cc=onftrab`*~@ro)Fq_3bWuEM3TPu?F z$=qnk|2?6B230h%fAtOxKk;7rABfYC#DHn3XHQr9TULHP(z8!JvFEZT;DTPjR9-4T zPZefdLh>7?rW?Kev0Rc0p8#yEtfqXaMJ0^!iK6n0YHF&Swe`DM z0otsy*acLQZc+P%L<(zwbsaD&`(L)4#k$x(0OB|7r2TJVZxz;P^hW@a;uV%@5+(rF z1ppk0Yzz%hXaiUcz*pR$@BZNtvqoThBQHIB%CxS#aOK9P@lAu?MagcJLvvKexs?n# z4$1mw=uQYML+2J$ZVMpr!6)ZMnbzza*tc6K@cqu8edjCA;OLn1H~;j{rqV0uxx;)e z6{MXtm55Opc45YseAbQzK(y}P)2;h$+_`kkU-e|J5nL2iKN|y1b*bDt(97XCEG`$P z$4t7e2;Q>ZY$lX&n`pR6d}k)V)AbVcXV=KAkFg>p*I1hx8;oO0AINm+8yYs;*L#VI?Bn@fB-q7^%F@`>aaOH{Ll>Rg;`(ZP}IYFzfFD^aJ@##Y&JQ5c|Q5 zfEQ5Q=K~E=P0BL_ARt>sNv?^F%F$#!jSJ|56V%<-s!IMzPnGT2#?5|2^+RV%JaoC) z#^C2~sY;)1&K)HJ3yE?b2K>NhijgQSG`{5h^3=etMv_EZodE>`qLe$TXa`IXRV7)z zu{S7K#u`h43W4PfU=$VpQ3}uL{re)Y_mAOt0nPEUB`ZqGtg4zHwyC(JRq+Pa)ZW3-ST0Gi0rtva zfh6Jt{DKW584f@zThc3*$y>e%`_Y|+B?n)g_(icL)h@Nt0pwq7`GaD(0GNrVm8-pr zK6$y|!gQSiVBkB?=ewmJ@Aqfy8_rL19rJ_MBs(4^A^o1q%kiw~4`l14K?tNK$gcxR=U4&-%un;BA?trrcm57rM)W!nk)X=kvYE2_&W)Ho-tzoVFcC-)`foeg z%mhh_?R1Z06v?S+tZyh4^+Z07>Z)o}od#`}ZK2oq7BR8JmEg-a{tA7s+DMf6dwib# zgCnZGu2qsV+CY82vDtplMcVsG#@Zf3uJaMRuhjm`9$hTip-O;8IX(jlSlFVynqN#M z*>TX~6l?%m_fX{!#OUXFfSMk|J>Pf2e$=BLv;;?oM!kVYu)SvQEXWoUU0h7e4cz<8 zg|%l@)RIAjL?ZpgJKD=5$M_iB3t@Spg+&i2+^9 zPHv6y{>NqlAUUY^WnS9;gqUrWmzv; zN0frlFCB)!Y563@6l~!SKDl6?BiJj;Rj@DW^uqSZv;l_4#!Y-Bsu&A?0w6}>b0mae zeuFvAZ%p2+?%DGx_{qjXh);*TRV(`OM7(K$^{zo{`|WA9Z_J2&aR&T<;-0%2^ZWna zu9h5SLjx|eD6z>f(x=bUzOX6g7LsAcoP0LF23U{D{o#AA-+w%oo&S^J-zDyc&F%VS z#F3skc}(%ZaR&faBDpn?B>8jeCh$AZ&u|ltjQPL$2o8Uw#OghP3|@Hp*Qrl^d%h=-q?-iHQGtLI5wZH;sB+6^m}0@r%` z6o{pdV5^T&M0=<~8mcP8AWgFvq0wGFIeU~E7@bHVN$cz4t4PT;9}>Gbi0Q#NdT)=Z zvV5l!vrt}d6afXyE3A_cg+4~5?_%#?*aa;BtUy!0&5b6heeG7Cm1$1TD#@<(3tIbf z5&~hgaUqaXDDm5~cekPE8W{0b?S<_HXbM{_E+P0t1t`HV0@gufUR6YVlwjN$u_Outl@D>i-KhUsTQ<8#3fLu zQe`zQ1{DAXi6k0GG~fuxmty8Bw8xW3zR^GAbP3o_65F=GZUnHC0H}@RICcvOF8;-a zk4peYcC@yb1RL!Qq7>H9tEvj)-$Sb8k5jM@scv0fSu^Y?ab7#BYo#jr7o9G#B9?+w z!`!8oaroB7lwl|3fUq%i`WS*%wREs;VjDQ z(MHs?2UuHuVmd?eT}WD&LQ9G#23MkgKwr&D@=uDHI9r>&NA25oM%IF%L{(F=!y*<* zOp^5z`z5Ft<_kT)6fq^TF*J}`|LRRE@@$};C}%mKcp7$MOInQc*^@`iSO&z9n+ZUR zaxCR`?)ak>WuT!I(UskQa6fP|yB7Lc6ZG`&u#Ag_k*eoI`}a8QEss|{vtWMrt6wq; z=U;#SN6z^x*EP=gcod_v?rR%Ck}%^>(OnCryopxi3RT*Jx=*dQcGt1l`8gAekFwIu>-SBi zKKlWvWmo7K$lBf5u(A46L%q&Eb>(I}{WV>E7Wx-Z1(owva$=$iN)#G{`+iqPn+f{b zYJ~#DJN8OLL;O^N82-GLS61BVdH$Du;g0(C)r$?FP?CS|?o=iJutyrX2?Z8-=5{J@ z6Q|I=piZw_14Dr>JGZdl%!>}KtGzAI=Zo>79W|-%@AgdoBtOBrD1&sxn0s=~;%k5e z8kMKqUo!$cMIlrylBC(LfOj#(t>kwD?MsYD_7h2?s090r{>G`KRJ`|bd%HSJwI?*c zQ3^n7sE9k%Hn5y4tfSPCpK}99CDC6hpw#UuB&ig07k))CUWkp~Wd@Np3h`L*t>$>eNM#iHB=>y&*S(GhdG%f%s@9&UrmxwnFPS93)q3{1@bQjaMs%_HrG)+=fot5O~OJHR7NKrf?|pglL|WK? zsv!LKB__>9zc6|=fARUJp4j*ZGSMsjkG}dvXI4GM{$bx^?0k3!v=->&Hh1m zswMHS>0zwrtG)!PD%zTx481fJMK%N`66}@EBCw>ep7Zi+>o9M|&9ecZjCL0JWRDDb zoQz>r=mXd&B#Z|e3|lNt;oj#Npof@W!Lf#_UtOXcs`Sc5U@1ZF7wFTrqV*3OmRBN) z25f2ox?rjcCNcuiwyx3laKFZ0DTYuD$a4dbfpwyPcqG-Mu#DD;V(889HBC=m>9-K* z%`nCnY9R2J^#9C$pq)v4^30r{CaVK56 z)Y9OtK%b-I-r6w*)g?A3zV@+0J+aRbs&VM#SCrU#AW-f`5SMu>y~3`T^MZi_b_-&a z#guo>d}09Q2h}^c+y$NDdu0MBscx@T>mphhMIppTD3Blqxqffm6oarYN3}LlrN1m- z7T|?UaedXe{mIO6l+*9V|5n2IwwR!@RsQ0mvzNx*X0V)eSd6i{?Gr~r?8U>QDJDhE zmB_}>0P8V;oIA1+<79RNRIsErDvkB^kAYy`md@|{rO_*$DMER z1nBv0Cq#ke3c#6dN^mx*AK6BdS_KGsrbxEXORrRfv9Nd6Xne(%c2SdZU-qh}D+!x0 z!m~rto1WCgNK9b(^UBu8u+Y3lRUH8Yo5_Rg)d{rF z8z9KY9s~mc!KTVe`$>U9-$&5*YihQeb|i(q7XtWv*RVk)l4!uD2KI^R1OF)q*vQK` zuqVK#xPTTfpRsui34VY-i%TmGRs8GsQu_wM_o#%q0_Zltf)F6zsKyt?dD+K!H8+)S zALS_YlAJSL16vGWeNA>(zV+tZrK<+SyDK6f&URHc?e1(>LUY6j!fEGFhvMU84 zrc-CAD5u{|5s&U2eD<|0)bH$_#crQiPWp^%tZibDCZ2S? zcOcLuVa&v5AQo5(@z+3;NIvw02CxaS4cXrSgvKSbStZ8VEn7ETV3|99u*bQj!e~HP ziDcJ6lH|{>mr#FA(Hm9xMkTX2o;`nADC7MGF_UW$7_wjc(o4?E&z*MO{NSwftGC~c zg|1>yz2A03h}_II-{kMb(E!2t(?<_Gm#+63gn!Uc(4Twl?wWJ-a8J-vMDjU|&_*E* zX^~;05+ag8Zs3y=o+fymqDKpt>`Ye^fhvM1EhQ4DrDrhCFul`V!NJ)|yNI+NL+*R! z{%=!bqjAJ<^bMBGRA%nXD?y{6fso_H76KPkV8IwfGCGVb%4VRieX*!V-4s&_80l{& zrboMU;guQaz}`L1j4IusTrDI32w2~-#p0BmLOOFz0`Gp&&Nwpk!XwxeMde>qdcg6k zkggefKEd&)Jqxcy5)IhYKwC?TvEcy8L?NuR;-JvJRy>y(Th#=i)f$C_7`79E=ct5d zx?yLnf;y_iDl*emRCCNcFKi>Ls^-IHOC-@iE)4*H2b4$QLoWIB!d0hUOhQqyiDBOW zsHOUzYS*yBGd^=?K`aH;u_yWI>5dB+e8lQ8TCR;BxV{dk z-JZW|b&x(zh93F88U$R38x)djm{bOnI2;|%Z;R&oyp6wKzJAMqZcZHDUsiT}uDRzb zTBC3W`y)CVWQ%;bEymX2c_GU*ep!#f+2#y`y(brDvSTThDm!P$&=M;`fp0FLP84`))x}caSFOH zlxEs9Z~F|qlQbBcoK_2yROL*Z;|_DJ3I%aUfEEJE-MxG7VNlS8hJ?9l^4UZKk2KJ2 z=$_Y2@4iQWwfEDAg1l6$k!UZL!0@r4>bWJ^OA`X70$L+T8kO)&Vo>YQK>08 z?eo6#l5CxXU!THRdsHz!4S{F zZX#(6Xf+CRh4?>k5va9Pzt0i<_oO} zthM+@+qR0aPIg!nLQ)#0u|?Tf>Cfn#d|Tp4I}IaaG}n$|KBaKZni~1{dJ*r+6nee` z@MrB$yy}#tju6w^T2;D$;qjQdE!~rHJPT~Y|$YGL51yizQEqi&{%lNqn;><6h(V|Xnz4x4b_Wthg4vz*r|HI&Lc0vAGv1YPnXjZb1lhV=vV?98tA;l{5C3gImtr3qE zr&Y;PBuu@v_drf1rSsL7LR_bOB1#!(ml_acVKp^mSAWqOMKVOFOOlDay?4K%{t+l+ z9Mq`Co$DdmVRNcr<{~+uhNs_n_YV z?04^G)vzViBqS(gjWLq zS+U;|vDW~4v~#&^;UI$;0Dss%QE7l&<@$~5vagnHW@V6NKoYgHyR%)C+{>1hJJCM5 zd`{r?q{^h>Un|5oj!tXj-?KGs?U9vWzEYBIfAeD6PW%m(W-?KPQ~(?a(3b$5pHxnV&F zux3J;T7?v7s4bm%8KGJkM(DGpF7z=~4EjnIWkJzoX7ilX)Kn>x|Ax7I>54_B5QCN^_#*&f0~XC0Pq2*h^^Tfdn}GRY-89ToHz~msR72J_Le5=3{Iz< zN2b?izUCT1)q_o^Jp}ooNo2!ZBGW!d`++QeK&KJVWd3R}rlX4>_Q6)cnS@UDzZ0i{ zgc@jRXtbc+3u5buXXD~@BxnT1XprwH>sJEDr_K&0Y-_q-05XL3pAsEGWy>aO3rtP3 z0x!*LEgXa_EH0Zt+1ov;?OKQ+KiSZ=S_d+o5(FuPT#&>3N5A}?IVe!b>q^`>dg@Hp z%^>lIz%MLf^ersHoW=8-4&#^KZ6;ZRQUO~x_oU5G=64(XJ|Aml*3P3U*Xl_m(_KtI z0Zx>hnq&dyjPu2UNtRO3BJ0n9+(qCw>I^NV(@FH}jqAp0BixnY`On}`X(0X8DL z9{^msOnyPP)tx`00qm7l`6lr#kwu-U%-BqR%B4EG{!Dcj_Wc1};a7`gbqT87`0)YK z;$Li2Qgf`Z9`CnF(f676_Z^J=+WjpJxFCN(k3o|LNep=gymaKSjP9|iDH(^ombNUN zW7r2H=(#5!H7`B$xY_gZVe_M3{MMqe(>*@j{isQV;ap!=m|#M>?R5M3s{yj~wr}aR zL6bN4?92G%)Kg*GxwSXrWAkwUnNVzJ28Z%-Id_e_G(b(+$fD}5k8}KkEE=*>>H}U7 zX3|UyBT5pG5jY|0cowb%B}!31!-EC}shujjH(&i<62&nA->}qwYtaG-AjBjnRV}8n z74*BJtxz=5EC>PWToV<0Cc?Xb@t(;dAAPmyoxxG-mA!~}O&rxYAX^HQc>#Eb#m8x7 z^kgDrkX%yf1v2vr0Zz!ItHv7x9u?SgQmHOi0wmj@A}C z*1*;jgq%2g-oDpRyq4qpjX(0;4-XjV2R6Xlq z5dsmcNir>$>PP!o^NvnVnMZe*230>gfguSnHY^3$@JOV55%y4UugJ>wbdJK3?s9%y1j$eK$n1U zyzBTXGZm6INLXl*AH+#%X#ggb4PrwD%$5&5VE6W|HW2>XclIU?uEbuE42LOA_+(b~_jJd8 z9g4p_I;nv;bW-_jAVDh}D`7&!2i^Y&ePNZcHP3vUoM-jCfdC})H!MPSJM#hA1=9k( zU<7xFwWU<$_98(}pfHH7VSFnyF%0d&anh+e(lfIQmU=ncHmzPJv-7eosgmtEayc$y z^jDhk42}Nz#YO96tf|^y!Gr0v0jGJJ94$j)ElL*T$4Puc?^<(1VUXn1d?m9z(@cJ{ zRV?VC$k~~TXBDzWGBsBjy8%uFA^Q#kg3n=4Haug( zgFCn9^U{G|3;zXyZcnEK-tpg5ZP*ytwOLbOZ>_FgvSlF%!Uvi4?SVJ}`q5WUSWiZh zMmBjLxu0oj09%R;mN(uHPrP_SeQjaN40`QwKjUivKnp&~sTc-^_D;r`Gm~q(gN5?P5PaJuu=Z`{O7nJeFCU#ue%#X!Z%b z*HmR+L!7J&=+Dxr6G7l-@4>@Dt2C-!SXWlvrY7YdeeDa@2>Qn#ylzfsNEMsvq6BL( zkUdEqyEB8s8PyqPqK5&3fCosdFiNDRbVb~nQTc4|)P))ASY&}(E@HpnD{s9u{hL?q`L9Tb2%1U=GGo7Lho$N>#S?iH=^#e*%|b5L?3-r$mSX zsL+#=C6Emzny1rCD*r&{K_P0-(*~T(Z0I)=Ay7pirB>AR=?amd;+JNWC!FU`x7uDK)Z8VpXVajO`biM%XzXr7WOZ$PmmypjI58395*m* z!n9NfVeVjkZ|VA@0$%xQ%TIsvmM!b~xo4kLmSuza%FEAL+THz!j@}j2Ot(Fg0&ug2 zhmN}t%nym$2~dT4?m;m^B)TrO<4>L)&^zB`*|-wzSB#&to*~c;fB|8Bl@HN<#!CJj1LpuZWc{x&%uk2}MaGvcH^UrUBN4%!2@eV5;+ESj;ny8Q9ZC_l$HiycBT|LY?=8G#$L7##h?-iWA?+lwps@BV*=MYxmyEe z739AxXE<9lqU>&UME)WOLx3KHNVc+x8q5k@0T@UnpBz1IALOSWebCYcrP@Ey_hg3t z{uf`e*`j~vMi|dzi%>&lSpGP zhH6-(rn=g;3Hl-GB0_})J!Pp1b0u;{MHF{PuJvBW3d?~>Feqez(qAkwPh>V z7b~hKEkMfv06+jqL_t*1Oda?_h)290?G#C;9U#QXoWM>>S8xiDQG&BzR&^2+3CuCr zcsQmCAachUMOL_gaKx54iMuFOGzAbTWn^pGB<|Bp5nx5z-i!)2FT=} zUs8}?TeO^0fS>HsK3gj^co!RBVW><`DZrSE;Nyva-V5WL$64E?;PbFRoT*CxWZF-X zZLq~!m27~GRS5Cc_g-0qe751!6Q|9;{P>sV{%!qc_s(s0E`#QFYj3xCbMOAF)*wTA zUc7RL`Rq%oYf-uk)-SS$^C__G96vK)wkT1HU`QzmHM&VmxHuOH?(jbGJXcq52(+6X zQ`{g-n_4z-&EP~#n(NkBASLS+Pv*4N1a#C`?}GfEPBQt~C`L`dqO5N{A*m@6aF)v9+U14lhd=-K}(G!C*`Mshwk5D1JFlLo{39O z7*8WS+k>MMrd#!W64g_QcU--@{chikGL%@b6e`rv_G^qDPLc@#hOT+nmLu@5j2Jv$syLGMISTpS=P;P6-$BkENH z;@zSEYCZ&}Jp$y1kDt!Uj`Z3%J3nv1_|)Gngz(-qHq>c-G;QBQKilxZfkW~^tT#QK z?S7uND9*(*`mOK&zy>g1eCBaGhtO~U9rzs24h)<3K026jUkIWXvb;m02E!D^T470R zqkIZk?qn$Ub+=naihN!>=xc4#x{LMZ;$j5Q1kjq&l64s{v1QXn7)lu1Mqqx^J%uVd zC;>!Szo}M0hC56mtjqbEpQY@t2}-A&*Ov2;ZV3!}GBPn?1OJ6$AoSYde#X}T&pPiO zzDhFs(-r(cOG-S+#IKX?x!rz+*8pq5nvInf@CEoOXtA060*-XGwjv~jF(~iY=Ym1>+0QvYOjI#h($VMup0OSJ1FgB$i zq8xAab-}Z$5$tAz8L8Ces_ruY5!UX7zSpI$>r&_+HD_6GEG^w#%7Tn|)^uA{vML`! z;sx{Sq~_R6!IshOqt0sets2VFpXcj$5&_y<8Wjw>;EUfXy50Le`BVUg^#<^9A;p$I z(L9pNeafW$fKA@~;3IS3$O-ezV-K0#JGS0F9Ebg`ZCfnk-FJWVi;UYeB6gQBmX~5) z#8(L;L_TFm+2*YA$@oUMr6iODvG%=eD+He~ne~7vf)ww40l01b&jBWxQQ~1k<$62z z+_(|RnM}?Pjhdm63G?g|5Bp{qLR;A*C-a!lk396b z3WGc~bqLzzqLWnAkVJJZV!!u$RjXo&-k}}*bi$S73#Tf6yr7}7Ni$t=?p^RLVQxFGX zsY|!AW@eQ|5}WCi?|uVNV=nF#y=s_*xwK7o!ssohaHee|Fg!0Km_U1>oJuUz*J*>Q zG7|yZm?>R9=LZzPyL9~2`M8(EIRo$k-86_MuuuY(Z=|MOdf*2#V{j-@QUYgIu}KZ( zsczmdjKA^3a|+@}bK*1*ga%mX<21Y=)-lF=zhrKX@kC581sqT4i_M%<{~}B!*EQw` z>tN@amFp`6KCQ@G4Mwh6D7mS?1-7GCexy+CaMvvk4G<8g3|++Dh>SM`RjD5>OpqAt z@{=`=z0lv&W$gzTlCl*TuK=A=5~L79KLFTFOif#Wq+e~oHa&56(5^KLUBV5jTL2mP zVhw~<;-|lP$6oWRue^9qhB<+pM|bbE&+L2qGRx*a5A&4P9%S=}Nf^IblMrlT9gr_U z*Hs%g+AUw#%-3}QPGH}ACAnCyV9N&$+U5Lgo_~lnV_+h4igh$&IG|2sRi*G&F0Bes zS-7=iVExV0iD749Y#+K+@`-gZJ{d}-5>s#zP7UCX##fmtQ8l{t%VINuL8V1Z08UC% z0|3n#C#MC74pIp21G)ypnvPBS*n>N5sq(?mv9#h-LAE_i=MHej3{y8VYs<_nr%oUCg-zs$9myP zJ^M5hos{Fg-JPv=5q|LK$+UtxhmN1J^vcv^3bQoH2!(R`^!cHJG!yF8n_56^ERb`h zAf1lBr~p-d3ADOwg?8q|_6MqX7OLZNU_tBG)0v!pDE9bb{CnHxZd*T{`J<46v6w4p zFNpRN0?w_MK8kj6Oh%7}@te0Y zV50lR?*Q3srp(5|1Q24|+^x4`1|g%`h-LX4&F4_jD9I0E?~6mJM@PruT$3#k(IK5QXiq z`TDCb$1PWev;K*Pc3JT3cYpGWv}-V9>qAYyQ2|(h`e75Y=?U=9EiB}%X_OFyg^Tq` zsR`NG*Q`Dx6wMR1rurJ|-wU#@yerW3Ba9EM|AG?E56F5(00v4cBKXi(l*!NYG=+_JF`0~NDbhloVS_$OKD$*_l2z-l91=|oCk#{Ygj7t>iU(BcUc6X``WJo4wj!e`hfYAhAHY&RkTdq(7h>2|XGd=?DPY(=Q zhN1gKOimO`52fV;!Pe)Q zuT?-hA7HbDjZg4JgTNxp+xe&)>@~py$pl$9-n&-5f5LW2tQ}T9o?Uc{1ikKLjAcXC z=x3(o-RyyB{=>M(T9zQC2UsT|V@dv+_7JQm)9gm*c0;GcQW!Ich1hahnyW~BaS~n) z0954J&m@z7ad}y6AF%g^%hwiOEf-(Qs#W&X`ulE`ZV9Rf*rSwgYHmm`i3wJ%?ft#x zW5u0d@o^H-K;f+UGPna!N3O_Vz^NTis2O$o+@RU9wLfELcf9jOf7YoCL9*E}^fRfc z5`7NucaV1xI&tSz@^n&;KdG6wqrcD66COK#Hl+$qh(!WwlMH%H}PtE58tBl%C-MEtH`_yLy#}FFHQCJ_8H?jYg2Hy)TLMT zFIglre-sitv#_{i*>acm(YH^_hS7HBZr zto2rvx`27*xB!f?va+HYxmsj~0nkCY3Y?5HDj$$aa1hcbkjTX-4Rv+a`Ip=wpsbt% z1`vu-uddDMYXA@lcHBidAFe42z%TKb-5LrEz~SAA&JNB;#vs>ZY`>HR2rU==V>d8% z<|7#9wKD>myL780ofnStzI8vH2JY5?1+76ZuM&11ThFxYIBdCH+c)36bHIOX?d`UK zDd_sW09+=QB+HUuTJ5GgAfil;_iJD3z1GT!eDd6YFQ;VuIIB!{9LD!=>-XiHxtwIZ zsN~t^wI&Ht=og5 zAN3*g9R#U^AkN*|+-Plbd<4*Z6HjQG=#&ZONB1b0xEDJ3?9c^ZTOb|VP zspS0dxOJKW*eFIx_51-eWuNFEWij^CwK=>R!08O2)KW%j4Dd?;eiAIAE^Q%Yoq{Tj zc@C-ofTia0$i!scy3Y9hBL%4b^|fE*)gF&zLeJ_xN9bkq)?s&{qSSzWAC+YUr6|u1 zKvveg)Un;%)s@#i;Z(=wnUbxYO;QKIOXlZ7uhqREDdmg&5XSDo(MdZmbjuzqv|#^K zX&Y0ysvzay(c|XFzk1W`R00v;2Ij_O($dsuo_q2ULn+3iDj}K*ls!B;ZYj9K3QA?J zCg_w7<76_ur28Fy2g-C%(x+5yT>KZSDu+hW+69ey7UUH0I*V(-V1$q`6tq+bfWtN^ z;1+@{0vm6A$V-j}t(RQS->S@Rm_K+&$mGY@>ax9a)oQTE1Nei%Y-TP!u&rhCQ$hpp z3bv7x($oO+fDQ6b1Y5$5F=9%w5qzW`n@ci{5}$%SmiE1~??AF$%j&!2mS0(^BGTDN zO_3|gg3sNdQ&YXc))%8zTC89c{^p&%rb8L`?FujkA&)+=)1qxgb9B%wX`w_wff_na z%3A}_jQ4hSn2$a=rpZ5XhT_3vC+xu7(b^O^rI#Ofq^+smI%;s(az;O< zaw?>IPjKq6g|V~<&K;ydMhn>-r7kuzS%KUU)ttX_3mIRCKHE%8&0p=U!OXL{^obgP z7^TNfrDjHny$--T>IbqIOUDR~J>IXs*R5M)l>Z=G=UzJ}-Zdsw-t z;GAXg)Z;$$Ndp9>0hW#W7(}_LxdUex4(CEkh1j&bu|6MwAg+bO4u+I`2=%ys^Ueq6 z@BZb7<{$p;r{gmDo&v%{_HBHKAB5rE0JHK06Iy-|}o+ z6rjT|zkR%xQ09_&Y$HMb6d*3P0(56Be zi-8a?OZD*alNQapUGx$`(V1`2xiP}dKvyPIOt#|DRW`dCI@zkQWR9aI^7j57P5M_f zkbao3U&j=HA!q?YycncQfg_Ai55ZduZ7zj2dW$&5-CPBO18Ox8rD4eAjqOj1fFwwR z7L4mc5HN^spWlD4Qw?Jf+)#*m^ml$=Bh~qs_pWJyLf|Ki!rYHz=%mW_WGVnBa4r+r z3fkF{tZnpxdWq@McTuO_X2#ZL=Q!bdWSotNk}G!H;Qa?c*&^Gk&_bpHK`Tlscr_m5 zwFyd9oM!|KDg&m!jCXxMK5J=KrUrC<>T5iq9E}sFfuJ=2eJq<_<$q{zuH%)TeZ~Rp zWeWPSMoqhIM8E?U=7dj_N&4L%|J-!R0n8Zx;U`Cuf{6oek~_WAKtc^*a}kiHjy<3v zk~szhX6sNbTbPFOEjv|8&!a=m^xCs>*plsC=z8K@ANvj#2M>o@WNyaeNIy2CmuH2RHHUsX_l|%$%h{>@9#gP`Ew%aVLay% zP)K$*eA1Kq@Fl)if=<}h`H=I&qc#{_Xttjq`$}E$NCvmE(vK7`cGV za?uk(;Kfxydxa1B#_I=JHg&4^cSdymWVdG`<8oeB3cOHg00ZL|j!Ux0(h=&mQ8OC= zNHzkhw_7X#pnR!^8!b3xmPC;shI)V4OE5xB=sq#GBoZhL`MxcLOcROP9Q(PKZ4>*s z_xHlmqP26$l6B%V5R?Wydtz+TKRRS6e{pCu)aTCbmTSMMJ6NGDU*dr<6ESr7*$x7w z-~G|g&5FkLKm7753Q|2{PM#ewXMLjX_Oq?-TuuXMz;i;L9~!lQJ`ySbsKB_9Z|Rn= zXI_EvY=Yygzp$$cotaG#yhRRV5(HrX$wlz9!D300%?@Sqhe7c9v5m*rt$wv_aCpr8 z^3C_H{rjbtpHDjjj_YHaJ-mB|1UXnk1LTn}2L-{BSmAmqEQmQ{0`|;ty8WfbAmn}mK6B|2S zVZPYf*JJ(vZ|(Ug-NQ!u+rUGAZ+BoVsFcy(FABHg zXU?Uh-%8Yo1!djMtHxYM5j&t0_04^rod!T<_H}nyhKTbc_oAhsxH>;LVrdK8v~Mv8 zwOp=h(xTQ_zJbuEL_8ARh?xFb4T+6!b%ymC;X@d#MYQ zPA#HNA{b4b(n1J$>sI!+XY-uW~hB2$vNI1`zkowef|?|Y#I1|-%;rv-e?MPRxj z7){3LCY7`bgJAEo&51d|Z=t_$@7-_Csn37#rO#&^gk{Xzwz)^y|2^iXzj@ncY{zRO zWA&2!nyjg+HbzR&#oMotpTQt$GfRE61CsUU+|Y=%Z<|UuJDE}*Wb!Y_pM6vRtyh`0 zd65VflXC5AK99GQ+gc9~jZKQ#Zc%{t;_wUS zQ%7OD7R(ux$bj88ng7CTH%xxB3k#-3i7R3Bzk2O-KXcar&jmg_e2_uO;K-QPx>Xs2 zfwl?IS*66m4?g}hZ@RU>9Y|y2J4n28VoX zKRN^?5gDA#mI*_B-`Mu~{WngdTGi*O&^(mw{F7^&a7bMe6?&7X(8DSKA;?@hdg^RO zO=R&^QMq0=L}9a|89NEAlku8L$c|t?<^{5CvX!2$4qGQKy7o2;(J49gU{DIf`wwMr zTT8Rb3QQ$kY)s2OW5Lu`B&BWQ-7_ePv7HlvX}jX+pFgm-qWHu@iwT-F-%$G?2Ty!Pw2%@YsrGEYDDkYxdDYN%5N?q>7$ zhX-w7A*kzn=LKj>2A`cj%AlN**KhYTdkx@BWE&hCQ$Tsd)~P0_T!?uNUlIU=vxBe- zTur-E_@+k2r|h^=ullEjmPQ2=8RO0k59ibrvW}0R88BU)Fe?<+Mvm=Bv_CNRyKMg+ zQ0Cj~Z@jD01pQ|Bj;&dphCu4$5AHIT1sM3=YrlvK7J-=x;Bk?}u_j^e zV%`cu*RgGfQ4A0(f$&ll%83!=XO4t<5;o6#2w9`7rSh?+7vDDxT ztC0yJ9Yz$`EehcT%vHbH9AB#p8n$Dn&t-FX&d1=CSk~64fI(OV#vhy*$1)7^PfgES z2Xmq8sMZPqh*LJ5k{c2Git-7$E%N)iJFW9>eCpzDN2FbJh{ak*(xHKPjhMe7#tolWjYg^J5q+s6?OkjOq=yDf9_LH>? zp!orn(O{lz6*xT~5&`@5*@U&AuNUo)A~Shreo^$}|Bvy&eld47rlHDICvJaZDO4WWz1^HTKWqrq11^tUSqpa6OQk~z?1>-p;>L5RW`_S0!Jfb*38_TRa>sWhg7-``g_K5 zzzW#U(E{*_1)hT8Ak3{DLQ1y*3y=;E(l@}B3;``^= zrLcDasPa<)x1LZbxqtrA&+Yv8$6tNL;0X4Fd1Z@iCg$#+z41;~>sRkw?x)j05E|H^ zz+Sb+8|;w7C(c+Fh(a-b02J*I4f2TU9;cIe=jmfs<&z>Hrrdy+-uZI6O~w zlw`_Q2v7qeY8ATywpbX%t1&w=o*lDhygpzWn{wqZRrJU)@KvFrmd*AEu# zloxY56jTHC^_#5C$T~X+Vb12Cg#Nlq^<#6Du8uZkS}a&c4Ck^=nT2_?NectoRA|8| zR#Q`D9YjNNyvGv&Uh|79_W2C5bW^-*`~4X#CG%o^;vPo}V>|XiMMYu3DA8?lzn88C z>g!alLo9007w0NGy9Am5YllI|w53!DAn|+&nab#MPW3kZ6dXDK`r5DTdf^Yg^f|L5 z4*0QC{UpPG&;COK8TiM};AcPFxyn}qtX=AAt1STK=*iPIs~^W?Aq1c(flWa6)J#uT zn6D`rM?bjxxKw+VZ)@axH9f3`Z5)W|xJjktu9@N2*;e>Noc zj#R=v8!HR%k4n`}_kHdfXl-h;G0k)s9rLt~0j<(@#8 zO&h{;j0^I|+zB%7ncT@-$W=i9RSDTNuSL`iIlw^`LvMabL5GKhPRxhkXzuOquuhtf z4jsw(k~pCWJT$pUIb+?G`T;$i?KW%j;E@vs$})tY%LN5mVW}uK!((eqd|#^>mn3jiG}vtSHHjpPiq#G^CXUt~a`gOiwwxadxE=<_R`G zvfZ`_L`J3s8v>AW^1_TQ1rP?>S=zMe7M<_VX#9X94)*z_CDR>phbetYB?JO*Dp+9W zG<^kITV1qta4Yfx#jP!_#oa1U+`YIHq_{g2C=S6TNO5$khrsAMnA})ML$jhE?P(>(Rm2-$lo9QeE{V0nTB%2 zw)E_YQ`u{`2U!P+P5LWDFN$$@9ViZX$0|ke>@b$v_*~ymB=8XNu!%f3?w{_DUPEr& zG~cy9HSO9o1p-FJ4qe=EHbWc)#l^y1k>SaeSrY;z7efr?LamXSg%9KQ3>Yug9G*oj zBTvhTD6|V+hw;uINZZjf?foLD8QrX%?`;(rbjOSxj5E;%#R?%;nKJsVvM*g+0Z-%C z@z?r!fAQ$@%HX{C{m|h$`#5+XCN9?}&`3oG6a5qFwx*(m0S|?ebV>U5ikdS&__H^P z;SeH}C4gGnn=kc%M@=U;nOVic^cK7hm!U2KMzq^MuBPvAL&4U;*L5O|e;$30h>#*C z6=+q%XHOD33qp7X5J}2&Nc1vRgh~hg;Fgyn_-jK?^lG+DuTo7`mdwe}>6P|hL6^^@ zKkfL@;iUK6vOvRXnvjvu4LWA0<))hg{3yD+q3?ZvV9~>avd*7)aK!2=RkCjxRNy4R z*0ZwJFUp7H4+{04&@JpvE_+g@v9zqIctoY&0zPZlgAZQ@4k-0j} zVwKA7=F0o(2!aHbNp*PfFC1vu-vUK=v{CDgUU@K{kN;XySzu1jNm>O(!+Sr4E^c->UxEwP*x~I8O=&^eyT~Q4+9fg; znOTahc+)d~*wPWMf-v9U_nEOaasPzg+J%#u)>={_Lry2rdo{EE=zW&mm7e=R?q@hw4PbB7PRcn_fagFF8qCS zezOv7SHo$qaCW{{ktN}ooL5pNC%%AsY!`FfN+nSTo_l-63P&oEl36OcIJ?(w_S`>R z!Ny(H#dOczSCDT53jh6|-$JQf##X*Rf$3aMpPcl=kB4Eoq zrZDY{Zu&JkY!|bon8}*Fi6KHC&yJLpO^r849zL83XlWjl#zHxs9G0I%}ipW2eEw131F$ zg&rncGLh204s&J)dnTJM?udY0e}4{_5CyS+Q_kR2^R&el#5iZh1aoH#0=Q=Z4ZM>g6}h&Z7>Ru zQ4cy9*HPqFP0NsA=wG|Ykum3ns-nAT@Icx*E}Z7x%vq1C-nSw3KKB};uP}(8ht(I+ zn>8$*T+i3++%Wl!%Uvd;dE<=3*=GKd>QZ?zhHhu@T#7Tg$J=IB-cl{=zHM`94TDb` zZEsY@>q~NH0`zJa*rhs%jX@(fdqReu&=lwyrhKUQJHmJ2C%mUeRlD~GyU>QUoA0B9 zp3Ge%bQB9VB1*Kc0R3@1H}8T`eO}qt%_;}KO67-Zd$4Qy*_Cxx zb%ylx>T2QZlwL|$;WoJ-gb{3yh^%^AbkXro$lVSV<@AdfRM<( zSoL~GUVB}ox~c5h?z-!(4*mVG=I{cWPBt|vO>%UmkY}2uz@p>ZOMBLS%C+SWl@o=f zxas5L{B1R^9andKcTY);JAxy62< z{S^+}XIpU_6S$yLt-t_QoSc;Z;r-Z6VBj`MD29$0eesT|Bxz(YPwn08pU7X>(u6{v z#}ii1;4CHc8$ZV(I~P=ovT3>TiH~7Mn)t>sN0}&n+M(nQ?v5g(!$Xmr4#g=QZYeTZ zOZ;z1kuYk(g>?JOKXxmQEP4Qu?wvIQW_7qjLZky<9XGFxxeDXJkC6o}A%Gx}H!{TAfW;*iItXLpfEFzl$4wKUvo|WZ-gY8Fe9Z#iU=lQ`}QRc}- znBnod5>4-KV{Ff3+Uy+_q~E^zlN4_cnN*RPynI6Iz_XUaob?gOb0^k5QS!1W`V8T( za8(Q^%Co7f@SJ$ej6w>pPu+86&&|J^VybCD4c)BsX^0P!Z?lZNae`v-k->;_pA_<# zL3D?QCs$P;zHAFWO+PL;gG=T9JgR9ay~`h%MBuzDs&E=ZLba9AkA~GZE%u5}3XdLZ z@L{guRg@Gd!w^jX10C*m(0$diDps5O#6F7V)$ehRS#7{em;ANz8Hxy8IWTrf( zyUTvP_DMKZJmPnE{JI#(SQ=vtPn8gVW|1X__3mVVK}klUKl*ZtM`o3^fb!c`W#F_X zQ7pT=Ql5O`WY$;O$F$r0ocCm70x5gbI~RqQo6ClND${(w&B`cm{(AV+>*$b-tjz~G zCj&!L!M3pj@0;{!(kk1GY9@&ucad^}Y;tgT3|l=RL*{;UZe9BtwQGpmE00R>Ob@Lm zt_A*XB$eE%Kvt>T_t0Bs3FtX9f}VB>_xC305Nof&fv1Dx+mY9E%9rqafa&21iOQ&r zAWuPD1Gzn)5q5C*_0ZoYBtdzjj}}sR?#gq^dW2d%(^xz?V#M3dWlyw@8WRSpY*#O4 zKq;5{p4&AGdX-wgf~OJd)oPx&0NX(wS!fiOP3}i8_W5Fpb=Br}^qJj0Y0Gy`oMX6jsYLjzrwy=e z-8Pa@SHVJGz^o3vp?LQa3!|f13rhW$wz$d=q|L^?35QtwcQ?F>c+%l$bn6qKlmx^D zNr;vd!MFCHBkJ^^?}A4uPkSaZ`UTlO_oel!K!}eZULv4M4Eo@F9}xJCSC@fHPxv(> zY%{WoN8=qBnyQcDrx>JN0c+!IU<^Lm*H&bSITLdsws7^08MX*+vXM0GIw?-?S45va7;{+Q?i|H6l@RL9xc_qA?zNv%fJmLGl=W&Y z0c$%SeS69YuUV=iT{Zio^`&|2X{vL`BNO(AYL0ep1S{o(6{@S|t=LI3OiX@^z&eH~lSFU5S6JA#0h&+K>gy^{Y>e5*35zl(!?MS(VZaXXaD3|CGV{DtWuddf*RhRM*cZ`q&>aPf~lBn=$}i?(@{D}DbM`Sy&$36HleNK{1x^gGw9?chM6MV?#qdkwG87!uN-Z6fcl=8HNBf~&^R z<@aQgDpCbziWoJDR!}n@yc2B&lOojNLlbgFFw$y@zn_-O*`*_79JA~k-5bu#v{}^K z-k^x>W{6IV9p-e<9U{hPDOiUZ+Auhxm z`=YUtPcJXsW>5~H$1i2iSAc1~55F~JBG}K+X*`U7y*nB^&F=Xy`uG+A+odoF(ppFy z&r--OHW^RFFSJg(`j1v7f(1VOqT6{Bt{fNH5+dHV1T^fA-Q(Ex;U5 z3egW)CzHploj1rtA=Q+CPnXN2MjuBJmlQ#C=0l6Cb{}hmSEAP5P3-!8@OkWSVvr0B zUHni%Ayd7xROZIGvK;~55&LAxRz$?%JLF0YSWqUbK95Aj?z04RR6E~E-YMz3^T3iR zWptCL-~_7^Em`1WBX_S>j%7AwXW7!a9oIs4PlOHhcj~l-qx)ZNs@HFR`!+N!JN$$)xz9zHqt$V z>7PJdGX}nM?{F5I`(p$H&5D>l5;?hh?^iaYT;CI0dMk0>VrdyxvG=m$C z$`C3TPE$gY$^#OLU?07h`Z&IzFwXE$(uLCQ4Kxk7jLUOp2mqvKdR_ zlL1eth-DjRPo#{A1fch>M?e}5!e&1sc>i7<_41}s^G#+8FCfO4qfuSapM4V-_Qe zkO!8tb^%zCYhgyU$2mu+78;^B{ktRBH;WvSe5sV)Yd)9LAF${BSkbsAK)IFIVK=4! z9rMRY72$b`O^a?0W~OE6d{hvA5?dQ~hLAm9>}C?0zW#R^1*)5FumjtMTFLSG(hx0A zD}sqBHIqQ7H#(?ipJA)9#h%>B;sx#4 zEwc92|%Yl-zVU+2q!9^&ES_0+3#^u>LoMLRQr{fYlIq5BQ4 zM5(7Wx=SB>eaj0Ol>5bot6}1bijaaSdDV?x9j~TI4;UXx?$0Y$YOm@QEbL>@_amkd znz=Y?$#i|QPqgnUh8(W|oiI62Ql7ri?#QS`liXA-#~rEgqx*Wm{{FBC0`qc9zF}7s zX2b0BK83pSqml+$pDgo;V}8b8w4iJy2Fd97AC`XB((ttLQ!w)ilZ6lT(0s^t4d6*l z8a|h;fbPG5PJcYf=_M;kcwOL9gv-dpQgtYp26Nbek6u{NO{QEx%3*(7D4!W+Do5$e zvH)?>neR*ozjXkD@hskVo~;G zNTc*pSX(mbir6oWksmdK4|D4?LIyd@&Vu-0;Fd&KM2){Y_%pE?h`Lz;QNbVv#@Jgv zGI$bFe%`rmymOawxdZjN;~d(%ZraU^Fqx^ZPxgJfowMFKhWb3KU53Itz|cEAIcR+g zs6?kJMnDw#PD~WC?qBRHAkoYfe(! z`eT`t)pfhrp_x8o{tAEboqDVZKx^3h6`Ub^kx6tg<*cX(W!S(rI8`RHf+B< z8n7Nq7`6Y1+&TY^M|6x(B+dvY{R#9sX5roGwcWs6?9^2)Jr1|Ug0kL@aC6Co#u_0| zGaTp8U!}fSSV!IY4&b%7i~WB#3p|cuN39%pgwFQeXR{v~8|%|;XTxozLR39;!n*yw zN+I8TrvK`3y`V-DwUc=lYEzr1@WPFhPtUR&@Z__UjRQ4S$D-^v8Tc4Vh-##a5({5u zQ4DfxUZ%E~r_Xi!H0Sj)&^IZ}9P^y|)iCS>!Iz=8Ge$BA=K}29HZ@ckv3UK*^hNO^ zt&cu4lD3tq8A7K$jNHXZeZ4tn1CkVPT1gQvvp89TsTAr5t7YHiJeoO-eeM0~-ImwT zr+l;PK;c^YO43ObWEA(XKjAO`TH&-C!K)@pbZ|DPLlq~u8g$7!M4iL#WMREDDMOnE z?L$KB)cWSvqKSypxfCE4$SoN3Ab;dG`dbt;hcUSPYOlz#{)QG(HgJ_V(Db~J^hR~( zuhw-&DfB@nPYaboFFlPDTI4%kny^_qkHlz0mMJq-($IpNvTBQu)W74ByUg|#@W3hS zj8AwOjVlRcM2uhSO<#(W*E(p zW+vm)JoWspEzl_c&YhGnEdG(pTDiqCesF#!Rll{aPf;E8m@IZ#4vf8qXbO@1f{@J9 zj-YM{u)b>7f2t%G^VC;3i8^yyJgd{t!@*lj7JJKxvL64Oc~V!ijQx@1<`&kqZmke| z#&;BM>EAD9VaYsc&ThDi>!rl9N>s0CM}cdA%k}Oo`ZVx*jN@jNt~VgkV+=4A|Hx~> z2vSG*uS2yBG(JQ{g^X~~8O(Y`l*G$Wtz6!d#ewIb$95TR`y=FpKC&o5mqBBoS-(F@IyIRfN; zX0k%cU+?KC-;%qA?7z)(e;51}t}wAXClbj$evnwC_)R1xgmiodK*=e{n%|yR2=!{~ z?b+wpf=^OhU4A$TX}NkoV(s=6^H`xKmVk=k?L%0r-v&r2UTz@%LRlNUX+7cdKRm{qC<^dhoHXi zEG6tOI9Klp=r&Hsb%t;CUn2`!u`Nc= zOT;1HYp83~ooHVNlg!vTi=(|P;bCJC3DF?OWDNdU$OjqcmLa4?A31kCZf0QDGb~`Q zagugLiMU<;)}?gXitRB7_sZ9Q-3T`_SttY(O0LD%QIe+Q`(WJkAkARsb17E&Q8>)N zZ8LWiPhwufuR@9|^N}0<$2upXlMZxUo(I1mK+OFa>_3I6R*C!UP3* zKJ4Gf=_Aa4luN}<#<-yrvC4FFbA3Reo5x$N)R>GLB<}QCsL`%~nQNb5CHKucN`{gh%x5-p)q2LiHOU znJbp!-U0kij5Gq9=p)3t1N}L@z{Y=vrMnqBzFFZJY}_9lS+bi z*WpnBb1Mg~8ua7O*p}EFwguWkCu*AmYZ(L7IK|xl3FmHpcD2NvxjIuGQFBO@GyOGo zS#V^`=Qh**nfElAt(|-)Z{YuD0o;|^WtIO|=6a4veSvMVpx)3iC6>)KzC2+I%aTYK zfy!pJn`0OwQ@j+5-d*h448ZyHCGGqrB02;O)iZ^&LfiSRTgHUxcnJ7!!+ZLSIq#7G z8`YLCttd#0bZc(GyiAjJP~4yk7;|54FI)XnSR-e|-KFnero;%Y`FH@UD{|u+XUQ)I zPBL+3@!k z%xJll7Vq}>@`foDesm(Z889Jrl~XsOY(OW;P{l}2klPD(BD?(Y_&UazusK0)c;ednmNiENB|Z zp)?#rf0;7na&^1yU3y_v4H$+#onRb-d{!Ml=O$MD8LKzgG#ab9(JFBW+@@-cpTED0 zlTn+sAP@V(alZ+~Mc18{>&qjYbB!-7nLHEvr5c&~7)~TgfQ#Amc@{J?AVaH_%J2sw z_B-i-Ru1kUGzn*AWf`Eb8Msz;CCA3Ym-+@sJ59%F&EOE+ms}_FgHqcCIYCXsPQ|za z{GQJ%N6Wr^rneIKdRI7aON?s#vV7>@6fdig%M_iFbW(JzD542 zQ{ROh82H#vAjk3iLZ6W0;~r*Kvo9(P15;AB|BFCXI+z^V>$xD;YW)`Za>?Ysy`YN%sCn(`1@bxN@F6B za8?30m;iE0ZT)CG)YT&>csOc0GNM?qC@_8Q;sCubhttWFsy~?Da#G(FDd2eXk$SnK zO|)clXrBtc4)W_gT<7Uu%%{(l89bKHN~})L+h<1~DR2=$^IAH1qt_<9A5K}<$Tl8&d zo*a&i+oN4G!|5aAo8T}BreS%hP2%}#;d|U71kTJD!6qDe5zKEgjfy%FO4&{X3;dn9 z{D2J<9a7=-+-8c`Isu6^Gp2glS`(kxg|wQ)Z~1SYRx~};U;{98xGyIeIuO&6uC?ex zHCj<;`fMA8MBOb7nJmm**hMB=$0+HR`wvhx4QI05hl0=ee8I;k9^kzcJTy34j^pRi zdOAO>z&E%2rKSp?_tu4lTvMDi@X=qLxafft({t-QDnC_&+{X7UaprK@y$iqUn+0~6 znSwpP%X(X=?8k4b3HuiFXGCf?`?Qttak$lI1W6(Y*58CDl=qN{_(=EEg*#93(w`APJs+H&!9DQn9QQJt<;{A) zEl&@N4d-;^`5^G{-SHBGXQ-nrkKw`i)-%N-tD@!eO9Oh%=~@B*t(&;klXdCwQz8)e zk?nv-sCKd!uv&n1eV=Yh>FjA6I%=o{$E(xb@P(7N-#Q3A(EtAGeoKOlenvSbKC*e{ zMB`s59+hIP#1(ja1BoT83-+UyZ$J+LQ3UOx`|lccItfTFj1zH?tllD(0nKqFdE>8qBY}N?<&X|0Y5Eb}NLq-`v9k zXR8t+M(#5(!-iC#R1ovmk)*(kkiZ%*M4~tRm*x8ey${>U2R2P%;D)^i74pq?R}xfd z1izFdI3=qgiR4_=0NAh$A>xokus=CM@}A?Eu;nOr^uk&s3gh5S026xS2)zMoB@}XC7%PCw#WmiP?2b^e%NV ztwkFe6QyAw%GG)B81|T9@UdI%LwoGsi18>^%js}Iw*JK3#3rqXP&P6$`0YD*^r4kr zHow@xr%#A(>T4uF484v{E1i3K#>^Al^?N5tCl2DV42uI>8h^h`IemD%c}{X6J{(aO z02rUdK7Hm=zI^havY4|z4RMa>*ULIldk?oHo?+JE&sT}sdUa*|FF+%}$`I;7d2-{QIDa<$BVrx*D! zCV+u~6z-d|)R=8vHU5t~0kkA!ij}(V(uu6g2|7vT>PZ?a{!RR&Pohsfb}V->rDMv( z(RkM|Z#@cErG!6-v_CQ$E-|1LMCqM7b{k?x1?*nj9oPl`1V z;eFtqI?b#x+aa)?jJj--U?_?3G0j*m-AaX{`6`cbmsr-R%*6!NOb{y&2bD?Lu4L6c z!Tb%=Vjwnx;m3T0ThUF%v;1KV)j_5*cF`l_zNfhg%p=mR<#9-8rtWQ4h=6~Eo(?m{ zUUuB-OIJTtaMjMjHTXuGMr4ax&3Ku`JG$tnM?FVw1U^|i;!`?+7{)17z?MIpI2g)W z^vfvI(rd9iI|2uN>eyD?Q9p7OfA4QRaK^=trh~wm+%F~e3ik()5s3DX+;SI3&9B0! zUL|aNhA6$S!*|AB&%o%kb{CJ+?HLsnEknoyNp>EN9|_W{|)5;`8ukLfap4gpNWd^mb&ipkHujSf;D-DD)Xa4)w0<2m=C zb0Nhu#duO_2DI#>Bq3GckWHWe5l! zgQctz5`45$7EL;O(MPvtWRhef(~Uj~Kyuxjo)vGRL*0Xpn(}{sD{E06M;WsY?A7N$ z5pMSBnZQ5m*yqTWSw|++q69%$D$0|$6J|=au`;fgo^ndGG7@eAL_<4A0tM?e>5O@< zsQ@|sf5pCQ-W7IWXgNvcb;ztl^W@)+6@YxOy$hpOFO0QMt|?!hqUmiQj7h1Cj|RT)H z0_hG(&}OqeKIWkt&E$pLvJl!QdQWH2?je(_d)w4u5`FbnQ|`e1=Jumcy@oO5DQs$Y zi4=P=Jno1!1n;%%&V zoyUEgRtbpRST*^#c&Vr#by3k)4(IQ>6>_mBF2{EsF-baT0@8SZVVn z1oUvuw#ubZt|xaGw=}n@efu)g@B(YCyY+M5IS=^@f2lS#v$AvIs7>M|_%;wIG>eE% zB>#P0NC4lqlR{?v)li$yQFWx)uZp=A8YRGLEUqJw)Z1l7o`ZKh^wQru!L0YR>VgBW zC$p*OKN*1ZPnulqK6MyXNABUROu>oJzjVL#X|Wny+(cEqpPlV=OeG~ohdSJHgY7QL z?Ia*FZ>aP_hKMLWp<-&3Su&L2D|);o&UWsZs7pPs4gqY_9A(l{S?Xvr;7O?=w)6uT zh}rFA$~TW0d#Pjjd+g|)@v#y(Oe$_71-8|EbTyvQq=e1j*1V8G1*rkV^95|z6-k$3Q*BWpX2Z4Llu8ui!TJCo6b(oD=>nCJ( zc1QW7`A`QP6NFDA<0DJMPglT}KQH()N3FFvWLIuiiT3gQcJO#Azt>|3|7^W4#8FV` zKl0;%r>{;mPu5lq$ktYBm~Wz&zIJTr8gKlO1u`n)F)lm>>_vwp5n3t#dq_T_Ep_4w z&UU)LP@B*IHo$I8@#m}FTH<&_lLV`vBlm{=M|*q|K>xe{hpjF1D)ahpbo7ZN=TwC( z6k!TFndLFFPb1$tV_i$;e^<4ya44sfQnVHPq#mX;k#Wq7!6FEbh=G@dF~Hz zIA)B9hK0NXoFg^I)LtZO?SO5fV4i>{8TxcEvbZVVn#jtCv0o-knbPs_=v(Eu2lYA* z-*+oBQkfWnejLNA<~W1pUg_Y0qM^&WK@I zYPYHJ&~~L3BF|SMv&=96|D)l67%N);W3GKQV;^nyoHKX%sfW-KrD?9e%R9V^!2v;* zKwA2Dg~FV?blp<%+9`8isHZLm6ToJ#*&m3_FD29O=8m_>r`sHMuO$)qreX3K1Qv_3uMUd}i+JMH>l=H~SNgnJ;_`*_YSrQ`KQ z0r5+V;mg*oJ`gE=t2$Bj^!GUZaomzB3RzQyhkII!?rAHy<@lQmrN->cPfXG)EC^eR zGFe^lP3}NZxS8p8od14!(})ycSvowZ9Y(9vUh3xxry4aK#f;*6wIFM2jQ;nGfD|bT zP7)D&uT4}_-t0p_qhlPAZ_&#u^C#EgNrMF=alV7*^bS>B$m@FkO1;6z}ETck% z>PUj)2jOMuzB3`)FMrCMLb}hb#pb|sa?IdZ8tMuK+%qW082q$WFBb1~a3op6JEZw7 z0ObjihWyYh$kMakh_fh;rd5&jc0T{bMKNAm(dPECl}+gR_rZ2;>c4>wHT`Q$)73l$ z<8N#6e##7{Jr8B8mFdw`Y%~U=SXd|v)K($q@-|C~>6aS%t|4XYLcdAUCqC(p4rvtG zQ;3}Xk(epZzhUC4tm0rrXB%%aG@%N$0W10Pl6vrUR@w{Hi-p^0v zd9a1o>CZCcMM%v$QPg~?y6LGCtU}f-Xg`}aMkbjMeiUu$>q2xKy!Cm)xNKOx_b-Aq zPI#%ny+~3vidI_3{$+7q1(Y=ng!#o8c-+j;?)2aqpUo>U-c#luxl7MT`v=2|71FY7ttf=tk|3cGwV%E3;lkn=8xl&Cm(KsyCYlnsS(OpY zI`kOE_*bjHeKoDkR_e7)RK5c%H?@L4U)w(obnk}rZ9ITfGAuMBR7klMI5kjc{H`B8 zo9k|Lz#@ESQi@4vGw!-Eq6aR1XF(h})kfZ0h|3S2%|TXe@6ao&S~zrDNo;Kfhk@)h z#V8W0nshOw@IO>N?^gog0Q;!*>e%w~gw;!k=;P<9?9RD{e~|a4qa%Mog!7iqdQ;a; z98S0T#Nf*hTw6@P$x;=+hokXO`Y(>I_0$I}Yh->NJrA;)F|h&LSw^|0Uj-0?H0VF%+{Y{c6vekYBX0KsF-k+!eN`?A8Hj?%qTmC>N-H2y=Pw?6^s ziHsucYH~s6o6PWNz#+_-G~gSp zcX2Kbj9l*g2JZw6PUSJ7U{P{oI@jgYl*!%VH0VQ+f$%OycLRo8#DK!?(+B4#f+F4$TsZ(o#$g{t~Dyy=)I?{J{Z9S~dlEO+V3wy*oCa485;}s2F?^YLZfdPj1FF4tf46e``C?NohdClhCyomFgPlI5JYBq9 zXG2#`Rr~WRP7>imF0nszptVZOF0del{&)kTT3cofZ`%sdv2BA);H!`8#PuxY5Ox6! zOOur!)f?`7SU?PT&W@$G1Muqz!_2l-53$*q|UtC5dgQ+4W z16|GzXp1zpt923!DnnwOI$Kn-w<6@we*o?F58%){V^&55ybVFV!zSBZ&#JW<_PYb0t1 zscBlu6DZyxnl{U$g-{!GCq@n1@m`fzAPF7*ykgNX$YuiAUrK5?7YP3e;in7m z27m^a>zn5OEx}27J+64EAyziH3}+KD*7M$nbuv(OSCG&BahL{S$#o|9WhpRpGirlG zVtY#ZPsxylM^zuk50hqQg`-WBVk4YgiU;$ z1p?FBfdityY=AG*37jl4F0lakFwT!DF(V4e$&EEfH-CjWn_@5 zAKbsy2WUkRSffU}Auh+211fZ*=|dZP8v=g;EO0Vcc);v$ZN`M%q|m>HtfLT2l}g9g zAZD5HKBII_@^;FSDgn9q3#Dkqa2Aq{_<#+gEg@o)OpoGC>(!CUPMZE>PE*#`>(`e# zU%Qo+#|hz5bf8I8dxkPOsFpQ|FSp2VyrAX*DcOC7&Jq~GS%{KYDNhvK9*T2l7p;p8 zSC;>Ad!{L)7pzsM%kY_YHx~ZsCqyU;dvcWI1hH1qE$GOw6?zvW?LSLkj*&kL@4S>f z6V#=zDD~86$f+fygVd09Qkx>HVE*&Jc3%>-Wq4Q6FBD|u%$Mr=JzmxGr@05YQsmpZSA{eV}D~a z7w3z&9O$BM=Ae5TIy=vpHGMhs0wkw;ayp$R4A7*;zgF_}_0SGd%Gg-~WA+iQfLIk) zIB#1>-Hpm%)Q#nt64r~}!j{zzB(r|>eTl3{emzOk=N%I|w=g8# zDnH@EfyXqERHitbFA~&}nW||R)M~;tf7keVAkA;t_h~rG^!GkKc{%V5K;ua2;XvxH z^WI6uto`D|igo+# z0vizOzPH=3?^*#M!(S0$5AU&dRu&H@UxJXtC3V8N;ipDS2EKgYGca)LjA7d8>DxJx zM;c)RMmXj33Z?RKb*rg!<@&z`MAcZ=%z5ngbZCm zk^26p)>iq;S+B>Nqu5*^3#24B8+Z$)S^XkJ)H%8((7|zGtvBJ|vG0<_n3wmX%%`S< zS%SP(cB~IJYDfN#oiO*#lI0H@nS1Kg7yRhs$K(SJeg?92lQ`0YWN*Si1$)dS&0U35 zA@FEGLl&Mgxmwyx+vswt=XM0W5_tY}|g?ZvPNt2 z0;s~S5JB(<7oxAd8Y8pq6qlR&9i}fgg0Hqe`)HY`;H@#pX*5>_Vnw9aFJCg{NkhkI zyE)WewcIK}3^!R0Wd~3tgh-^fok$!WsL8yH#_h>g;-f`aJJ+N}$i=S1VnvzDMz5J! zRtfe0W#4fmGD1xcsal ziNTB-$C%>G7k8co#s7n?z{{Puq}tY%PD#=vOq;ieoN1W3&Hct2z})}NFGYk1LjqN% zUk=rOV*i_&)@?B>D{}P73|Ll@4%2Oxjd_)x)oyU-|vm#1;?s+bRPTa|BPa zt|Dv1Gz00r&4zu4FeF&vIQiy3X6rWzl}d}cpSZLDrLcQ%Ko;5)pq-p?jpzPeM;E4h z#HLj691&(+BhOL+72V^4o+_;}g&bFUg)W_!z4ZHldduY_HEKyw>yS&auWD1I&B7N| zXtvn6$3&2nB&VaI0x?^v0DxNoTsqZwr@~mk(hcQ4cUkGu_2Jw?r{=vSZwtlTK<85% zTl??huQLoncJ0?~luR;q>dt=O6Mwjnuna3b!zU`$pz$Dgn95_|#H3(;)w2aSJC7-r z5BR?s{T)9>SbqlN_xD8k*?`Q6s;jH5)FEXM1rpW{D4lEj6}Q+&Xyg&sN9&J@ zir`4Ge?0Lg3sAGqtn;SI6tOOaI;WUSRJrJfx~7Z_;Aj4n@Bh-y>ZrxD)!AQY1N(jO zi<1^UzyS!8H4p{ruO`$4I_Sy|D6#W-EOO4;(WMI}GqJL#G zXHM+1Zdo$*Tr$wZR}Ew%z5s>Jjl{32Mtp6eEB1VGSZT%e+(%D+$@$X&Rq52F*86l^ zbsZ1Ek@Fe^1{h?cu0>OXsf6makzS&=w|Co3k0J3fKFKBS-0gLCB!>pd3oJc(_u($i zQu&zi-wj?0M-c4HVbL4BWL9DN8ix0N3p*tmN>V+fa0xKwPqr%V)OS}7)oyJz=L{WaalY$4Mdf`J%+JR+6xVeCy^6vn|&cWd~4lVGF&r=bL z>CRwE!XjQwYX{;OU^}F}1kKn%;lIXI5ii zI$cb&Ycuz`-SI@f>KmhI%{~$(o7qlwzS=_&w4qw>RuHL!xsF5rsfy;9G5+&Qik_ah z7dnKBn%yKD0aPhqg1*P8oa5JU(TCLgom20wqdiyEpDEy~AChNFuCDbd;SNShHOmVA z@Rz-jX~n$S+q;&RKCJ1Lk?_R-t%Y0%9^F|gt+(iV=6b~`mf`@8Vm z<##>#a0IW1b+Ig8Q_#pjqiBlOu2r=Q7++5XgyVBPC^%!80I2PFqeaW;2-wj0FoP5k z(gxrO&*J!rB?_hZxpirM4ejuG*z4JO9NwqQdjElI1MB$a=q~LNTY&tyfDv(XPQHWD za>RSW#6&qyM;`=SSM4<)$Nz1W(vc4sI_MI0o?J*7nU2&OXB(M9ln{XA9d?sFD7h^8 z7F4zRgA_#9X2ks%-CoZRi4+2HSUa0&)f0pg{(F0&uP{m=XvD7FvfgG9SEEmbv&Jm2 z8e=Hd$D}WD1fG}3abwu8>`|v;EdcmqI?RH8zQNik!Z0Cygbtj}l(CwV?@qds-L|+| zb3;8(EGOg%T*Y_^h4Bxoygw<8jx|SH~hD)$geF7L?5(-PcS&1$+NBJdNDr zD+^vNf0=zWc0E~69*Mskp03uzXtgu=MNMh|Wi;>(9Zj~ZTH%tV4F0g{*>+6wPiJwv z3EHnqgD=a|Mf+M4=1lz+WlM;gt~CcLiXj((5g@+}#P+>uf9aZc3BVIb|BtD+j*Ig9 zx`tsGC553uy1Tnm8l+1~DM1=(92inXx}>{%5Red1iJ=imX&694y7Rrz@9%!zzs2h_ z`<%V@+Of_#Lf3CD#kf%77Uj*m)O2QRfeM&a_ZOcPknWIJvUt?F+vB}5*c|5HaaJmkgL3=K)IvCmJ!k2^ z8gWig_8rYgQ)8aTml-nMApPd~RNj*&ojz46uAvt(>DuJ`NL8~gIhB5pWP+{2OckWWcXN#45c zDRcXgK)z#aBJe~Zd6)7@DNI`DEdeq|$1Xm{LnrA}JE1XIwlB>T_F;PX)irs}<=sp` z&5}wrr{>Y;x3aRYxO9QTxrrLg208;{6BGWevv%IuJx)WX&lfhHTTY(lRW?~0I8tT25Y~?z{`9_tf3edwVe~$x){(E6p9ZlL@tP}}7N z3(nM^L}qYUFr-!-rb*#AVcgn)He*0WliK%@v#2EL50-`Hxds%kzuQe#33FJ_BAa!B zX~o7?A-xG@7501_H9KMlDi|HgCmOXFS@|6f+yP*iY#Sa7fq=;6g7@+pc$-`*J&q@AAI>cxOO-Y+Rj_7~;szK`<9OA=2A1f4POVG8Sm`*tMoTtG~qP#svb4qb$fZ*8%(G23InZv+z$+55^dW0YP$YoqM;vpdn#Wym>uN{8U^_9 zI8LM_&J!v3dwj^h`7m%=t5y~fG}|eh-UAD|303GadA})v9<~0~#)OU*9431VcC27e zsw(-!Ds!1^8|@`$H4?+cDkP|crm(EOT3hAj=x>;;Vn2b@{}g@FV57V;o)%@KTX!Y2 zSYph(D>ulmQS8vk>hLCQF#wliMP>$K87toB_11~+5#OK^juy$FF=&SsX$0INPB<_B z6=Fs=7S?&m*cm|pU4N~hG!ywDMNu%7LTG8;;strJ#!IIpcJ4vF;j;Qv*FrBlLG*_t zav*Ofj|31|TfKra)(k%`qWN8c2q82&o(gRs4{mmcK_rB{AR!kpar}d^FZBsLg2waV zIP)E;>UM|+mbxxn&N3Zv68X+8uqZ<&TB-|!Rh9q(c;089KY1f5hllP%d}!VpTIRgl zq0j5HugpdY!fczVwR#3-`Y2b$c_GCGGprSsALPBX*s|0J^aX$aT{r%>oe?eU>|AW$ zu2qOM`G0?6KGp{2QtaKVjeCdnp3TCzY{(`%!w@7`ygtzn#GI{cX+d!IxBQ zb33Cx;`2nZD1^$Jxby*yg|*t`6Z~s|2=_{j;GEPQ<26?sm~LUXsTM%)T;)z3b&M__M&dI{ z6!gcqj~0IFg0OP28KMR@J)14~MT?$)Bv!`Ck&?Ssis<_$FNCe9m~1hXZqFj z>KQXt3Z*&-(~ZFBjTe)HZNR3}#Uw z$$W90zzm+GKs6jH3;SB=Y9;zS?K-onuZD8`!AO!}lqqmCo0pcBCIbpz6s508-UE4XIJW;(NJVql=s zS$TnkO9c#*NGjYJM{JA2bR9=t%2{OdLn8%(En6l_L40a)e%a-mP2&Wn8DCNv(2_T@ zowZfp7P|}6SLhvF#JS4}BL6wpJ6W$MVpLGK4%RH&F;&g|y6HUWag+@|YjyNyschRO z>36g}Y^B>1h6dbMZ{X6BzE&d5d7ywBLv^@!+NYb!r>@>(`~5NZ!ImF2dyp7uR}o2M zzW_WRFHg${HvX0+a52+2f>x<0Snb+Z(|@?4=)}#{s1q?rD6Hnb3KQrUt8+kTth65l zONSjZ3crum)Y5};+j~dVb5f1Wm>$1Nt~H*3QBah0%YGqAvd#DbVddbKkF(Pj`%G^V z`-<*px`tJuzg^K|5dHzcUYdHRB6AS&ha>E&Hc`(W1a_}YQIL9`a8J2)rF5fN>#u1= zvl2rh(v$+iD|@q$NaEg>%ZK_i?^T+RXCq=4>U26o9(QAaH2{odF)(07V9~^myv79a zfGZtlSMN>Gxy{2h$r+0QkDD^5fF@T1)l5xis^{w1h47$*ooAWXN5}e_R%DD*uSkL< zW#~BoblhO7w|37SZ_jtQTld(F_4) zs)AD9yO?@5nv>z?bw6=o;lMfpXRDWrswe1HL=+S`xi3Wfamt>j#gsT{UUqd%g&$>w zZ^v1Vg~_sGj!)>XWFrfpa~bpw(sQln7fH#<*{BBbl%bpq_k3tf+qqib4<|TrGPHJ; zEGa6A3679e$-d%bnHo7oWDHU1>l+8x1<7rRr@;d3D0t9QJ+1r-*8<4CcO8Gi>Syf= zRNCnxKmQ)5K=5yQ!XQh0c5FK^N^PGnWqvcx!=hPT2{#6|MFY^BO1vn1HxqLcEh*L+ zX<_QLUzYGol09fwpUl(RpqKipC#1-t5>%-LLauFWR5}O+?8SJ}dS!JC>)O2G>@3RG zcX%e>6jus>CkE7EUAExN0Bg@@^0+}HCVZJxdNDg^8jjB%J0|^nH-T3yuOSqVRcMbJvdTzhZ7f!Mr1mkyxT7K_(`ynre7I5>F3|C{3uvpj~WcvA!3ic zdSa?AytqAd+I9Yhe85)#8?sSYR8m&FPFEV3upjW!Y#fS6;`aA`7MyUttFBT3k;RDO z(@xw(zrI^R=CBCy35BCJMWi>S=?#dC(ZD!afS=(@-kU{Q(h;iUG2_PP}nWrn@_G3n)-bJKByH7NRONr^N< zwKHEE<7Tpk-=wxPMWER4^4N$UxXPtNRD@szDmxxMNY?!0na{PR+ka>b?zrCG zTm?e6{?NSj%`p4Mf{uIZlL^3^`U9 z`}Lco;&wLbj2D;UbdA-#f(ROdwhP0Vw%*!G{6!usMy(mOOjrxy4l*_$8(s=QsDK{$ zHS51M(tBku9}`P;&D}&naFCWKI)fL~TD zhI8%Qvf@p@_s=L438k6fp_p5zNSlL_sTq@;FUzzHxYz0O%y*oQDAk4=9|dlOuAvte z%zwjd946Sf_`0Zs=6fFJFx5y{LH#7({!w`EK&2!J&O(rM^@3AK;$ktQO_i7ae7F5!0Bv8?iyDz)n>avzeKEHgp)e((u^w?k^_VP?Zf<>IrUD5H3jWxa?h-&&5%A9-ox{Ymbp z2cKjZuL2)=$GoMYrixTkz&#L4X@p|z3q%W?&(xWI%rTi-z7&f}^C=|$94S7}U33gj zNX+T=ElnEyF5vKYS3c0iWh81U<5D9&uJ(OEYhL`icXC-;R+R|X>8YuP(+lA84aMX) zmg~-V0BmhQo};qE+hi({#6F{D9VsZGu^IEf$eIMPlF$aOqF@j3**$Ey%DhG$tnV!; z$+HV%>=$H>O8w(DniB2xdd*It8RDD9rUi}P&biwJ#yhOd(6XkagaFu%K1QJsYD$(# zT#lBaGJ1rvL8G-?wrD58SG~sPnxM>sXYX;;+Z+ASBUZ&Rs+$K*_$Z_MihK%-QaZ^B zFaLN`^vByMDLOein7-NzTo8Ld_GJP$P(8D9opBpJh;|fZa|zRiKuc?+vc(J6czq2kS1pB&99lvBiSs`= z$kXD2CX&~>uymWW6Ot2aNz!|_iw+Lh43v2Aw-DH~(I=L>3eE=PetE33;E8MvhZmSR zGKo91OU183Y^dK|1ChH^5|&=>fp~X?KO{x3alan4u(228HvAfnf3{*?RhUgBpvPy5 zRjV=wnQ>a1i{XX(LA_p|5!f?|QJT0N+IP2znT+Iv;czFl5H6XgA$DB;SOV3vPEH@a zni0+K6LYAp&ETfJ0;>Ou?=VHeB@L)IS%7FFjc023cw}VAV2x@oF#elN ze_ZCb@}UwQGtK8b1g)#k5kdEg=z;)p^C8W2c@4X9|$BaTWuWV^uel2yVxW&kBQOhj#7`F&pomPw&$#Y21QiY1yEURjB&Y@dFQKcyVm zz#KIV&hbokG)Mh6x75>yF013L|WHM7dhmp(GrbaA-g zeqPvF?gtF@(?zce&Ob9?eq4~YHwPO@=&i2!;k$nkt4lqY>OyR0MlgH>A3fQff+{B$ zTKZs;cEHSLd6lW{@KRFPt7V=HuabLeQADtEd|iziI>?jo(6>=$^)=hO$F1|!BX&0CbXI<#z(?bTol7B{1HK_Ir}@4oXiI7f#A$K^ zaG&PwL#5#C77`QE6pL}WWi$l?7r-<;1kSo10&XVXD&8Jy}v%E@Y~xW*F+f;WenfCSNAFjK(m^C4cL;C z)P$Wf6>YtM*h?TM8|guS9vsb4qINE;iL13Dt5r?R`j_ z<;wh-$Qa@`Sgky!Vy)#0U0-ZuN04Kq*>9mQ2sh#8dZ)>skfmc{B=^`7AeBN$DJutE z>TH35!!=$}G|9;xG=AL4fqr;mu{w!eCFQCq9xQ>9 zbMw?U2{my|ozmuPRg&vw)lW$|2|-q z-5JA0VSqV;`oZ0I@aNdVlV;Yc^b9*;`Rp^#eM@zFyl6VQ&aJ=h99IPO>m2fe|I3F+ z0CTDf0bXrmCJiD!38nj8Mr3j# z%v`DcUjcZ(l_@N>&3V~vWUH^wPW2X-c-@=xU!2SMQYN_oX}zhtAal|Rei?AV8lC<2rW7iRxF5eR|0H)hgyd6GCI0pcxJ zrPak)TF^S|E-hiWkjYF*kgW%FAea z77w=3Wm_n*mZ%MA)n#yYbm%DfloWIGq44Q;f8X5&>EdWf4)lqX)ED*X5A+g7kzG6k z$|`?%f|I#Z>vl8EQBI$DjeH>G7oV^G z;nPE$*aw#KlYDfrX6+E0>Ihq?d?N=1J_tZqmx3%Ci=0R5Hucrb@~izAh5Qj3jBhA9 zWFxykI0V+^KIO*8cWa*S^8EO#9}CQ`&K#sTx?=n`l+*Ymx))@d!k_V7{oq2H#8H4wXP6d>~I%0}}20wsrJQ^|>U zJa;P_$5TNkC7+yUQJV5N&~KAHBVT|&Z9&4jZO5DbnACFmTk{w`T(+N z#(F-ik_F8FzVj@GvQU_;Su3-(HlL7rPA(u0^LqpZR8?IHRT@vs!O)fCM86On_)&!+ zD2uV5tE(wayv$vmPIH5`Dbg||mA}y^{D);)El?%*cOJaL=U3>;Z~uUjYX$c0$0oB; zV$qE?mknP9OlPq*Lq#tr94o$YGg$r&Pk&cvfUs7s!B@eY*z5JF{KF1|zGF}CAaP+W z!(UAS=w7F|^qWqsRIOz0IbDx&^|5hVo7Ld>^@$IFPPnl$SoE+G62HjF)}Ao;ic!V? z>Y)MSGo3t}7Nu|MDSd&U$kV;NKN*jI-;BiS_$;iR-bmHikU=W^55n=uohu-aZ)mt zn^a_l10pUu?$2TV;U;F%TD&mI()NRjf)dvmu6pN>8_8d--u5#k2AL=w=KZhUNX8(; zfY-1-$!84t?b`SnA~E*!BCRE8N_cI1lZMh_vv%xO!u@cSsP1{>tBZBfb;rKq8c@ve-qVulUkQ-tOmOHsv76*?6yT*<#3&|L%z7DX*r6=Fbcos20bB zs@d*G^av$ZjsaYjj#6vRUKW!acW)fL-%(Lm2@UEqvG>2p$^azUJJ1u=>oy=%GN4)S zTlA``bM2Bd=f9-s1cR&?)w(H<|D4AxG?B*I&ihzh9Tx@~G{8D=bl9`-j+CVSnCbp) zS5J0^;Jk^G;X?Z#-orW_!D{O7wNH!RGHVm?V5i+YxM;4ZN`UV*xl~5FPFD~NgJ81# zm-u@z-F9cFyhSeuO0#*$o0o%D4{8G)E z7LZv-3TO;89qD^DgJ~AsTe!2si=Uc?JQ5YpLqbys?13 z!NJkrt`EzaFi|jOs^@ZxX|eAf?EkGbo{xiF>^8DA$HERO@+YwBY88aHEkiuvQcGv1 z4x;C}+j(+L{(_oxM>;&JBjN1-8Ak+ebSf`+Bhj?n!1TT$p@X&J!nv|SS2fFeHL-6` zHsq953mSaP;K@Kl&#&<;q_*Dt$TX>1)Tj!Ygf1dX0Nw724^<(*^4k~tmOfhDzW^jK};-ay4b~EjirIC5^ zoL1k$-v|nP$c7At*6KZKd5ts>y=W>qi9h&USy`wOJ@Tu?f2^g3!%HjkI#)nkS`Jn8 z@`;6aQO^6o`$pe$Iq8G#xwUP+?^X)@<)QZ^Ns9wp`3CYoNnP`91h>g(i&r1M$^~iG?Tj}w*p*$W+ zxc$FAb|oNTLm!iN2mvOg@xt7s-(rjc;yOJuosPT4*nqn{>g;R0 z$p7{#pRqIH#%m%ur529_xuz>;K|TL@{_&bl@yN;wvbWjFI<`DMi@}TR=SB&e6h3HE zIhP3TY#X;D6R_+!n3a*9Z7}mfOstvDRl$%5l%}-=`y!E4`7_dzs-!wQIaxrdhZ*8t zX7Xa`%HuGbb=ntEUY8uN_tFQs8mX(74GKivBj{Ae&Q)py0>zx=NRRbhh7&6)LgA9did z*BLsmAe*+*5}`nC4Mg_`h!uOx)RbWtTb!68G{Qu^E<(gFc0R`5Z%-#pXu#7;C*#YZ z?1?KUO)|7KB7wp(c;BYo_2PJ70^6%objob^`NrF96pBA4?_C4=1GBzxidz0(@6VV6 zi1Q|HJ9~3*hbnJ|S#y{yCV_~D_P3IP)~7>4q$DSDKkVIn!;B5I#+OM-5gq?qeEu9+ zX{QI*j!rI3J73=?(v-y5rtRNJz)e!5GRE8i3{EDY475z>Co5bqC>l|?)@-)htjb$S z5n7QCQ~}Jp)Sh+R-DJPn)r`Gm@`Wj0UlkA&rA(c;U4d@j9_nY;;XYFX+NgWPPABuTPX-zCy?jFYFT_nmCWr2|dqd-`JnQEWLP zaNm9LOs%QYx;zUD~;9DePS5#?7} z({>TmTpivxjQlT;032opYk~-l6Ih<0^8>FB%e%d}d4*kw99;#5YliZ%XVp+x!l2NOpP zA8~W-6}qyl&-`i^hWB`=t;Qsxg?w1^3AjX0aa-b7lM{8()Z@QM?Tp7ufnek#L$~Lf z9Ds*?vq4#tf*>w2R>0`{T5^eYou=cO^CstRji--!B(sp$(uF0N7s*j_JNi4T=7tRW zLriRi2Gr2)|CjAo(tyH`PD6&XrPZlFiC%E|{VMzkxMSgalqt6gMOg^NJM_psEi8sV zc5junW1r12y; zbf{nbeOYyj?%TpRlnU1U`VQBZ=UO~=aUb|3#kGo_JaBaJ+rYk1_>VpTw(te*Y}d*4 zl~mRDCiJpvLDjQ`H$MSeSjJfEx*>e%8#q4pBe~xN(H`E}LK`a1&=C{y6Et-aXpmD$4oQf6R&Hj@ZQH3X$3)Lmljm#x7T4`12@hUg=uDQ^I= z!F#ZAc}G)52;pjBUS`~0KNFVb)0)rk1OG+{2x#+3E)I+&tg{_o^aM?%b7^uyBDcT$ z!1o^_{*@o?7(^rJ7Y*+d7h6~swol4F#BjFGx1iHW-PcSG;M3h{HR_tk^w|QR!x2q- z1p=zB~h6_R;t;UNlFA!NA&kz9!2z~ZLVVJU|BA@uy-I}!CkA=l1a&r?!j|K;Db zRG8C$gv&X(^7861(=+kL9rreioWN-7OTb9;|JDfTCi8N;+1Tm?_|&L@Yk&PvZvhP( z%M?Q2NYl9gvHS~M*aXJ-lp$Nm3+&8mi#dsWhgHHvWF*W4d0CLhJW%Zen!!bvI@m|W ziU>KroTW+F&UU)pBxfOSk@IVA-lj^V?6|jw)*WmdBKh40fqm#1S4WzA&=(+3qd?SU zuU0MNemrKcPVP&shbe+||6Uf9G59D0A$rUk4msZb;vT*KUCp-1X7OLFg)qUk`mxfz zTE?S{{%lg@Ue8euvU%puC2(?s`IAxBLE60EWW`q3!D;?I$j`d6R?Cl7zgF9(zGLDw z(k**yrhNmqFk!G#f=AJo#7^p%Ad&ubG2I?WRm}|L=(4OqgW4zLv=9+1Ms7RqQ z8ip+)v2gGbHz@m=pcmv1A|C23@@&E&Dgq$7)Wo*HXv&*}kX36>IX;I;EfU;+x zZ{tMoX~)T~-wHBz*KA?c~@3DSSQNIf3gVa z=SB6vU!c-2CP-m;u}uzk+K3|oAE%5tJ7mhAlN=l9SNSQ(e@2rl8$|QKKS1v%-Au(l z6m%)(7Q;UufjqoW&_*Uh~x@CG&@&B(TWW*AxBTq8|~l1TK}ZjN8r<>ASdE z?0k3eB!aS#G6JH41B2+lk+RoHIP&SBo0@e^xWd~j7n3VyTV{$#@oLny)T<)OXuU$O z{JD9NZ93Msr^vk1*$J@-?Fx#dRVan?E5#W^jo8P}kUFe>FdYWjs6PCM2^CD>CUey% zP)}x=#ZY>Oz#7cXkDO;2Dlhx^<*nF&X$O}7L78$H1@3KZhqxT)1WngTYB&nY-~SBI z5()@a=EBq^vF3urj~l8CliPk$7g?ZfVOrVq+-cnMrDVve6O;>$>N@blf;uT;YT*NE z=Cx?k*a~g1d)vk}`B;>i3J9XPtf8vCn#s2RSuYn9efX!jPT- z8KYLGU0)|E|32=nczrxBHm*H6eqy9&(th~o=TMS13tVgV5z z(CeRXCpFK)9SS3XV_X_gF$=qSEM>Vys8nQ@PdTz4c2RYoA|)a=iMRwI*nE#@m)iik?I& zv;1EFBP{fgm5W`QolAB1{f-s}V-5-n`*R#h#W;00H`%_!fZ-fHk5zwnzBD`;`G+9# zvRt=n&L2oH649&5!=(%vqr!6_yT(lZoV7QgA7aD)Ot9%1mbBV$}5MOEGmN86*L4{mm0}8gbcJ z8SjtW=Iu&TH6mjHG=cIIA}+Zs*?~#v>87R4O#G_yv80EOEqJ!9H@cLxCjs4vl{rD} ztVVf7b$+9kiperHf$A#y1B%2oC?O`7i4sN_%YMC4j*;$-Wu|CBbq8gn)b+Q+=>PUW zgH}{r{*8p%uyC(}p=ueWu05qU`>*3ok1gn;_Y zkk23fc!vZFUA6W`5;BjHv5VuvH;ZldPx0Y6DzB#9N@_B{6gdBqzxCelZeaM2$SUPJ zc^?@ZR*_=WUB77P&i~eo!u^U8cC3rhC5v*ZQq4g9dyDbxpn!Grm?Ql*=Z?p(+Jf17 zz>wb(n(P#Gnrb8Ip(u9lhu2fANp$F^ovy;4WKGLm0Thd)J2=fUcVU6r2mCpK>7UZY zg%;sd;u?)5Iia8)+M(+s8CIbNjWh-OX*RP1w`mJ)7K%P+TU`v^D(4$cIQ)BUc|6@ zdo^Xd%R}r32z?Y;`pyNkVS&qjA~?4^cbwq zEXMX_f_`cq`gq4VZ&V{h5_;5AQd33+GcN^L_fo=DC3RGiU94@%C|#6<78_m_K6e@1 z7>galX7?>9W-f@~b3jLXfU%f7P$&jht*Pp$fA**mBP)$wFR~_U=MlwTAWP7{^g;HL zpq36JX5jo7?~$IjI!RiDR`&|>=^|l@gH&j}d9*|#u~omxm4EODWsc^!o1?JZhGcm+zb^Yw}b@FYY^C z3m#gx9x65)Ya2Q={FgH4`<~^MtVZx_x$*Ca;bFr$XGEDw(OV;O5Xz)=^)5S zAn^Vi=^fX-Uu{E-YrBWdhx&G#v<%jWyBcOM(UtK}%-bB%B`steTfqiy33IN7-Ave( z0-$|1{ED6@W3T56l_5JmtKJm}=EL7@1j(n8*3a;xada`{r1eelRgG-P>1uZ|YHMCY zKYrPbncqUM{EG4$rW5jhfHj!(&w_FuM=HXTD&~%BHu-3As~4dK_1=G~F(hwN7Cc8S zqp3Nf?QMw5c)@Qkjo28A-0>Ke#B-jP8(%2aCqGus`EX9u-w;S@P+F4;LPJv|Hb9Q> zgmwKfZ-p7^tC^}cw|G18w5#PkuwI;>;JQc^7Slp{GbGl$l|S+(QKx?UI(@Z~p4u9l zo37RZBTQaAnJ9Jio;+v=tLP8q_)f(-XBn|a$hbY^eWLEXDxgVG((+=gV=3fLkEeC`JPR^bT zh+UX$UD{a_zB!%I0S?O+Oj*2vwFF#_iybY?b*J6z?AKIR;|5S?x$rbwjV=rG^C@UG zMN*n@&i}@2jR~X!s>4hC(RF*VJfa~FN?i1{{FvFgdLy@yGIL_>y(Kb0%faxXO@CLU zVl3goBrz(^``2b8W=E`Jh7n&UtkrXz@RYeCWSfunQG0V z+nGevE%D0R)G5mn?Ex;VFNX|cIlkgQ!?-#9?Otp*!ro?~jHFJ)iZwbTOz%g5V0TUA zH@TlOq8f>{KYqKJZAH8p+DJNZ(=S(;mKF3|_dR9VX3j>j2cgG8)h@4GbxK1I=;{j(keFufF$1-5*j;Hu6QjnsPAeySd2PL}ADAo3YrS zGRhm+57n`3KYXTfHT}`+z_v;8f|ps}t4d3Af0EW{8!bW%48qQMcqrp>uZ&EdZ#jQY z@jN^`c{FMSw7|HY3I8CBl~JRPO?@8~_15-!B&S2sU?N$NpI<2wkH*yM3v%&#juXK( zNG{UT8;PI0n-APiqo44Q#YWYB4GoCkV=0}*mugH0-oV}W21(a}6VRC00+$k!{1C}V zAdVCFpGv)blz)MH6Iyh;emUC@9~Sxo~GXz z>5&$B^~{YT^Dec60lhkJPD)$rBc5V=5ZiX!kx%2#*Utzal(6*u2n_Ye@dqir*+#!G z-oR0L3e=%04`ILhUx=OwG4;&?Q@TeJx*%@yxv6bwc@>T0`STdNr-4sCJ!epU6?gh$ zHmT~u$|%@IT0e>}SOt;|38y9AZH8u>?cKpx2jtcXpT@&Dt;fY;{At>`p1p<0rjxqS zxe>vE&S|GOJtQ-oaq1lpLO@!+k<^m8Hw6Ra@43x-ymi(Ww8QGK@icy3KpnSDWSe4F z?Yf2`!cctNnWCaE6opL>k39fpaGCAT1!Gm!RM3hSIGJe<+L=}rzZ}9mT$Q`n&x`!M za}jkcv_MKitZh(q60NP1AK!g%9zSd)U!I+Vt#Dhjs+`+5!H1{yzBg--ujigCyIx@=UK7i)u%MGgxjX&-D636k*oavc@SgISk?1b%Si#^FRVKsC5CEq@bJHGtq(;=Pe z;@?ie00H93qczF4&ju+`p+P1phtFSN$nwjKPO5RMyh&up*r~aZ6~o4~MK|AT*Q}0R zETQG#t@{K8nYz75m{abcJ&W9nkAM?3rdl#t-=Xn|oi|IEntw^4!iyNqglYeMT@7Bl+;z$|H|2ClV{NzcfDTDff52A0Q@1DmbLP zbn_)D=@&o68?Zfeoo`9I9SFMRxK=Xew60!bC~qUtNxeGV&7YYuLD(6U4c>VPVN=~J9!JlGq3H?Qk+NjqdIA!g)*RpTPkz>) z4a5MU{pesNp+}pR7Y2j%s!gP#BZ)=Yn0#pgby#6xga%iYU3F#>uBx&8<^#*0npooY zs>-~PpAZ<}$VgURNtK?0*lt~wmB2C^f57E>(~&bE0l(eyn6?;>sVmzu~T>9yTnWX|n%C}-ER=*XAC1~)93oh_Ip zCszfu-y`1!{c3>NOgic3PJDNu&L*5XJ==+ zR=G0H{%lnLln5FqCJOB-vtch*5r<-KcXn2Ryu$PuXC#^54}3dtt6k%SUo=y@ z1{dkhXs5~b^e-X7@UM{CI(I^c2tJ>FINVC(m+IFU@^L%ohD!$xi@Sp>jAJ=sR7aAB z63s?0d)m>u8eHFq)I2lVMDN{pZIipr=H`$0D$HYM;t25Nc~cUiU?>`Rp?2Sbvl0Vc zG&d*Zq*h#->zY|L->OGe$=nC%YT9O+N-=y z>yvrnT6pjp#y(|kLwII~pF){e+EeLp*>kcSLP5mXWy8X6xJEb!ddIp|u9{++<;2Z#Lp zN%YbIwhT&p2vis{M50{{Ex(!rs3~s@U72^6feL+GB-qIokpSE_Q(?8Ifpv}x|1~&` z;Se=k)NZ|zN!U_l!g$4SO&2NYa=6&8X=)(kFyompylgn(xUCYo^(42zTFXOZF2vzl zEOmW9TYlZrmN-OFSPzl?ITrX+L+c6z1&bkbk{qWh}3~`6iV_#%pOxyF&Nr3sy#c? zg7Q#@Nsn_3!ki-hhZeOctTJ%T{h0^~OQ!FSzJBKGT2-25G69Yaz`*!RA0$N{p29E@ zhcvz@Cp+&aQW`=)W3RUQf^J$A7n$ESI4J>tH;#`%zs(X%YJU((!r!sC8DnS~!LegTC$i-rh7!yB zo9s40=-3yhZ#Ptm^1(G#;U6Alyp0R5H(m8_f}xr{Mcg*Mz!nsV2kxk6BMqm-UEx$n zxtusU>^mCL(Xa_Il;qs3zi%s8nSgNTrf4LNQEq;B+rJ&S0A{h6Oehn8M=Iu>C0l3Zp zE3Z0sp|>G`;rVRJBjD6V#jQa~vaai0uFWCK!!oBQ_W+N(1#8S*`#8^opFl{3nrbqa zSDy5IB-K(#IZFlfn!tG$+2iWH9UBq0t6=Y0Ap}pnH1EETmv1{4F zfWIjpF-Gktdlvp!mfpU(hJcv|UgHPj&ohr31a+)lzMQeGJj_Eau|v!Oh z8*)z3IFvPglF_1kTH9e+#{Zl3xsU6r1|OBljF}_z+s8*^hAa^Iee9_p6ON7y<;t!* zb5$VLYpOpfdY8+cN>eN}i59xTMq8*;SRzynu#6OHn!yUq; z)K<8uIx|&0C9qzdDUdL%nC;Lr*59Y4JLXm8j%!p3e2Bo`keT*7h9skO1Fe9OG*_d+ zV_W;ejpHW48=gB^R94Tl5y__yZN4&KYM3<(oCQ^oYoCTB%7_Xm(0zz1#Dr-uLB7p> zM01ktZD&RseslCSlnjSOSlLv0so`S$3?qIWd3lb8CHLIS^b#_f{Dn4l4kPz*&PTPJ zn5wbFu3I&u852|JmpH?UBHO(}JzzbJgFAs&k(S!dbOMad-hau;b{`w8OnmHEZe*s_ z4s`QrPr%0Q>1*|ITdIwz5Qix7VQfv?MuS!HNlh?n=CgC-+rJt#;~11F(DA$@SRyNvwHF}&U7NoAc-xpg)vY5WqzI|P7Pa*eHdohie@H$$AtK@jK)gwL- z(60)+==4+h;HHKuG%j660- zpYT+K>$$x`b$njfEEqu;xOD!!APL2)}87w>b z<qhbQmx5!{UlrmON<3n_nPPc4b#_vMfYy)fB;mE}DiE=mIG z1%dP-3U-Q&nSV*jNG!0}oSZVgBvdq4ma(yjE;k)p$~2!76y^tBH5JLuFFR@RRoGsV8d$vK zwxq2p6bwWeq{n>>0ySu6(Cy7=eEsp`$zt)|vte3~pj&bWKfl5Pr{0whOx8!jysS#* z%dTvq4QhVyV=0Zzw153j444prH%QsWtTnRTEn*N2y>GgKb~|S6_db7rdgmQv$DH9bqO-0r!ih3k|9^>t|b-tMBs26v#dm@%9R$9ZCQ5Ot_65 zmg$79)+!Mi<&TB%_k9|9Iz9xl{~cR-f?&l; zFVbA7P@_1z$*UdWc0{x@Bv8`MP#jyHHPo2}(d{mBe%Uy&wb1-AI=W;AQSxI8Pw7LB zX}d&h4_sxqa$Fc!WcS(}OYmCugyfur$;PQX7MOZP_Xn?tX0L%GzB1v`Hm(8L;rGZ^ z;um7Ng~~d^78#~9H8tHF8Dg#7`LZoWItrivghk6ANFwirrVh#|U+>z1He0mmtiSCe z^4QK4w^({B!JCOZ%9jBLw){c^P1zhPy^I=G2>4?qwen`j z5A!8u5o@N~=KZG(jB$7}Pf;zns3-%UN0TfhSV!Ng^GxxfrZBHys8U6UoA$hIi1~rY zD+%t5smds=Yfg^n?E&;$PO5kzZ$D>sqe)<+1)8T`(yF5XIy72M47W$zDFvrKpW-jR z4Y(bXa?c7d&2$haG?I=aYz(UMi@2J5i`^>@VHg-zlBAhH`ojmeidSt}e>5d#Ul2iC zM4o6PM%gXSZ}N%qnAju0(0L2$b6J?MKH1MIQ zZ!z)sk+#g+74ePqQ-23?KP~Elom5NpK}n(D0LH1#wcF0+Y($UAb+$lB^4A4`$!mKZ zCC?!j!;C)+jwOAz3pQhdT5B1KvO(ze#LjED^eRp1gQg2gb)#{qDf%2$=^TCiBt9iQs%s zMhl^SxL#U<(mst8gpz<>vC7;b8J${h`|x@&hsF4v?aB!r4~9d@j^`kp&^k&&Xb}dz zhTB#;e$MmZ5XRV^Q{O_t>pLTB!k_jiUZ|7DzcKSRG42{sfo!gdd-{%jY{_8ju7aVnmJndapTF@2>Tn_eq^34H zivf(TdRZCyzB{9&H;oM&mux?(5BQ#S`!>zK`CGCzs36F~LSPLDurzODBe_5Qi0qF&PFX7ixvat7 zWs^cc2xJ(6AWC34VMky`y#2>NB`s0UNsLkLN1h-*0{wh&L~^M>+o8i`e(#xORh!ld zD+kIF#sZ`pAW&aZ1slIcUA_IN@#@EzU|ru{*8;I@QwXd%0hP`6!Fv?)`N%l% z0BNg5M`Pe;?$|~COW(mrcvD5*u6ZYA2SOmj2yoqN?}_8&yWK=STt~t2aZEgaH0MA{ z^s^M#HcL#pKKek_EjM}W7(+H{h)a5$JVBb9f=#XV+z*lO>?w?;vq;;r zWl0V0<$h+I1#av;px6cbdP(1r*o@-k_PG|HIkN1;ul~W|wb*>hw#mvAfxaFxw(o|V zYIsEqtq3*di%V7ZcGj~mu3bP)%1Fe&<6w8U%2G2nZl+MQ0U!ttRM*tiOXNCvobno? zQv$fJ_Z$S(-%HN7e~)w9O943KIV&*xrB{$m6@TVl$m>Jq&_xP5XUX;uvX)k9#>0=0 z@9j5{#djO`y?SNnUu6~fS_q^q0lsH?zWQ7RyZKOhivtYJ9jIvy2bMG;AOwU!Gz2)? z*?sg2^?bC!A#d#2AI<8kE*rv}$Ux>k6%|t`=qF?MzUTrFSKUKo3+Pb((rhOATU#jW z$)_V*NtTL_0LFqGTyEy4F5#zz-I>DFqF(V<#H66MgZ5R4+eQJB)W{9 zyCFplBVPo~*nfb0h^Y3VuC?vJgAqB(BG{}fA_UeAfgoy#d#+w26BBFBMqsJB;BS6K zj_?1FY>=F>rAf??)AVgkds0EQRW!ts;-Y+8lo3AOp zw*08e8~Wi-P=+M7%zI_j{ql42!@+NR_Iapiw2>QMZBoa4@h@Gz(6QOq6Y1I9B;L^9is*hPUY!8N2pT5CJr^zxy@i{q=t$ zV~H^afUJ9pjxS(4xBr=g!~#3Qqa+D%w8YLkhW2bNcuaOUoUzKa zEjf!nCI>%v2DN)l)Y&v&e3^X61YpO2_v)2|^flgNjdcA0ob-XA0n#@ki$99G>}&wh zO3KI&#|W!1Sh3~iWK$kqr!HJkC6tN^!)Lu)>G-^G+G{Wp=`i@M*f96iAoCdtLe1go z=5zG*R4DhNAb8ZfV^fg|`v5l>|xGW$9(v|?OFZUOpA^;~3 z4wEOyeEtOrpp=jMlaKJ*o=DqnOUHzO5J)ou?1V)bE9dWjOB&A)*x{OjOyHzZK}~k8 zyqtoQkU595SK1m9z&tt7*GJAbeo5{V-;j>-Pqvp{B^`mJJRS_;T(~~?1D;*+j|UeY zMG!UWU3eaH2Dt5omocW@N;Z^-FdukG1;Wn$`#&Se=$m!zC4% z^B1=O3<(ocvs64U_pUaI=W$OaDd_fc@5 zZ}HdiUI++*6$k`JN67W{S1?u}dY+}9Ie-&#@b(}6SULWu|K)$FEYslZ%nDms`74&= zWqCo@J8vnYo9!n*hnzXG?h%|9S3@Q^2MX9xS|B(xMKk~Uk4pBM>m{ar8DT(;wehv36hv-ZL$+Hms?CHj zZ}3AtboQ;^l3v?D#v?Gw;e2T@0NPLwmE}^MzkXwG&KuR1?=H+4OzuxVCcUN(`>2mB z=Ya!6knO~B<{p$HkR}AIN`Rem-s_j=#ve5`O4-LebO_2qm&kMK1cLUK2JKt*Sf$@W zV2;4rj6c?fu66nv7#^mXU;UDTPz*Dle3G>4nzccaMqA}_J?uR7qsF(iH6nmRHS|ooX5714&d@@ci>0`w9WlMb~f z)i5SDpLw2aPd`UK1mw@YG1oWp`x3S1#B2w?AfDrL;GP#rJTw2}?`Zme{yq5+;BFwu z+5Xd?ll_1D8&&dx)y>jHjezZn=0k^+Z_9h^YYKG6d3|w^NNWo@-~2t)80Hcbj4j(# zxeed7>xp5V;ag$3e#0=Rtt;_?G;4uwojdvG<+CBhC3&@L@$*`nZ3a5pFraV~-<<9= z$OhVf_Q&cb?LKpg+^7|(Y8E1DNtzHy1_2ENtx`z2aQTl**AdaU-Wm_2SRxiONQkgm zZ$~f3YjFYr81%izj=?cfMYac#-5X^y@jME8T&U~SP6WfeG$D{s0*H|3O!Rb*l#lwD zVZ7klm9SJGixkq6*>|039BFQQ?FXb+*Qtnmc0}=Y8dU%%8%IV)$O*|i*JmG-hCl-Q zPybjsH>2+B3M=S4cA>97R5i}o*x~;C)A?Z2=$v{h>>}Y`1*d0}%%+A3zp-IcblYbq zGG~FqnCXY4xfNSpZVnaXaXs_7B1)!_gefuRBpd(|7#v&^=TQi<>p^UrWf+ygYLyeC z6d7RH^0hQ8$^KOqUsqpq2@!Vm>)YYvkE8Lmr4RR%0CR7{& zSds2ema`xMzTaWo!ewF7abC#%a=!nLs!PiyXlTB4r}^0T@J!rH9?07Iu0(cgZR!LM(NtlVGP z@{h1%T1}l&FAEH&SAqlI+toN5LA+LOY*+O)Hqz{`Unl<{ep^pHO?Ei=7b`^Yc+fm@ zgnS*Xv5yBAgTgiE`Sv)?e)t}ld-juwfK?4n&U+h=LG6(5XGK0%8Sp@LDmcRd`KiBF zL+FR^+EsEw;b8Wk|AA)Td6QVh+lDfI_MiQnOq9^l8#dm1IO+8_R6y*k(tyATPEt_! z!B~2TW9Ivk^AGP}TYj>?@RCZB)k;e-<}ai`PnRlR5X+|5(>mS)n$(>h(S=MO7({@koJ7@C3!ELgQGurbCT{ksY>x6EehDVbz3C409l@qLTE|a%BXv>TfbH*MVeYP0axd3u3F69tnG{0m%+tWpIL_$;lZx zaE>OLTcEpJ8S#xs<>lejGPdqiPB?#aQzVD7K9+%V(%;ifK^WJJ4IAS+X1=!g8e>`g z$fZ}k@A zOR=3jt1P|fYh-fi8#k%ID^Y@2#75qN0PHmg9&*0%I-Z9gAm|N_x=5beNn*cb;C%DfsNLJ1w$JgqotD3Zz)A#=*ap=l&(~j)^OygIQdqxM zrLb5@iyi-IF-}#sxps1A){0+X?peDFt{m%l@IF2Wkmu6KPN*ZZDwqk!c(@=? z9b-o>9l#~*obSFhS2}ruOD9J#Prq-ciF?CA9d-u;NjUw11mN_Gr;$o$1RxjZ5sH4=DYMB4&bq%oDBk&6)2c@k{fPKZ4 z6{X6-KQxm5b7W zu_uW*Gv!M}J^l9Al=IV{k%=3mh6kWDA+VMNN{S1qyts%iUAu{7-oSmqL;QXc15FMp z(&w`U*0L|j-nKI4y!AWc>>jSwX+T{=nh>x8fgo=&2Uh?9KmbWZK~#+SZe;ng5#88| z;ISC4&k!VZIrG?e@)(SnlVo5*Ax#J*f&fSIyAVn5Z)t*)s2dU3Rk+VqC1TU5-3E*Z zp6|}W@Hm0V_s>+I=5igtx_NTz6~Ye4fE;h zP==$dJyAOM=DVjOTbzS4*bo#L9wGz3ORKefWbzx39A_n_EZoBr-e5fEtQMBg_rh4m zMi$O^A=j)@e6j7=O**946V-e;r+mNAqfllr_U<9yx8ER}zlH3$PQqSOiRz>Uw!?Ex zf8$LA?9?L5m(?EPYX+_{|Ctly?`$XAZd`9L7Om7mZjTB)>Gj7(gEw*4r*p+%6xf#0F${8`*jQD+s~;6VAP8m<`wDhGZJFgX4V*^$vj zwl%nhqUw|^oGt|TKIFk~!+rV$X~=#e7@D(QdW{Ue-{H5D>ZY!-bEj1ns(A!uFzz+@ zqkmd$9mWh#lJBDrAi@0~l=ol%4>C$hQ{8RZzYvI?06Sq(mc{kyN8~+zj6%2`wLS#c z-ndEz#&LG8uZ{9m{QK4%xK@*{Z_5r%IGRxN+kg8ORP`WF4W}kMX0@Uc@^yDAnQ{|z zfvm@$AOp%-Ix z8apnG2XQW<9uMBzDO`8{-d=Kj`~jYi$o&4%s|b#-xOa*B{$^=;g8j^f%yZv2Uqb=s zQ=*9pav)RQz#K8@%rUn0-Mp?C5NvVwKmIe!v&t!i{;9s+*ZVYo1GYR4@gTvK=a62Adm+hx^Go&^JWTNzM#r&L>=gR-rHhb^ zv$kZUnCow!Pn-x5A-IA1y+_j|>scven8e zRCcKgk@6uJm8{-4Q`LX{I_e6RA^YWl6%R+_08T{T^VrS{BjD^m{QX?*aXfn#H+~-Z zo3P9;{sj*HPOSF=nJ>J&whsPy8Hz5rMGF1Z-$}nyy#7zJF-=YTO z-;o#Dpqw?qH6S_Q#E*a-t`iAMc0zW`a<;*qE*R8sESy~xVbc(pslgBy?COeL4l=@h zS!FTKeDT4kX4KXv#Br?xVruGD-~f-4S8VwGuyPA$py_bD*bhIpvSAlN?Y385qo50! zOCLcl7*6|0S`eijd{-`$#`i1St72(N&7-bP1ut+Qi!>pyKp+U2ZT?Mgz!6L0a#o%D z{Xb9;0b%KoRmnAu{Wq>Fg&&rjUfkFjU?4$+0|fo;E#yYYA5}7G@q+SR2&6RuIQS6+ z=6dICRhor^R~&!yzmpw-vc~pZ6oB)8_KjcC%-{Y$aL#>3K~@9e`30UsYuSmt4);v^uPTRP5 zNv_|#PJuqyCOOD9t=n8cj{A#G5#0X=%qMfn@fUwh2FgFI=-^*CCLRxd|Hml7G(R33 z8Cm!y;(voqWPYQJ9xErC&m-%7!^Tw|^&BY8bBU~sW8gL|rmvWLdQ z2*h0!fMlX95eJ?Jy1FfJZ7cvu-M_U+u<>KAJun>ob(QR18~+_G**i^_{dv7UI(qV) zl3h4-U^m$$(>q;GP1t?}+S`;gsQJ)ANFy(ul-;``M-%Gh<*GiV57}aTQ#9F_m6aZm zpZE4v{>~f%(=)Slt+APEs>*2puI+aY%n1YB`(J!RmF1^nB{1b;&xuqB|%&o$b{BREYn{Sf`Qv4odfU5FE*|~G|ET4K($+&7I zC8TBNkpChwzu3XLBSPcdDjf@qKzjYeaV42)9DI0{rSJb*o1Rv68Ew0uEP&tXdT)L> zzrDzQF&;gllvCp7!DHdc2?RA=AqNa?T;6BdhB1l1t%YX)^&eHOO#AaMBQvgO*+uXB z5Wge77X6pMBMl0L#nu8>p67- z*?u?4_S*N=Z#NulF4w`&H&*sg!G?7R{K4}=nh;2L0vzxf?CBvF6s^2hFR5{es^^Vl z6AtXt@toj$zm9u1A^6OD@q%&+h1fBOQZ-6}17kXMohHhj!{c$71hHYvlf=+EFu1y}x zCeij3@YC$S|BLdwnMW`NE{Gz_Hr;_TEV1_#p8mwMLTITP9mDZ}8*G-@)vLIh*6|8^r~J^gItgUT58E2{{BcnNT{ulv1sDF}mu za-d>j6i;Kx{t&k5Z^UGG|3jo>GAT_6#7uyru-OswHR{RlK@bY$kTr~7Ldb+;0~Z@B zOwKg5;IkwIXDHXK4`J?Y$FqXqxR0n|T*m-KRX73aNH>;L2eEiFl1^embK>buon`2=3$IcDk`Hq zu)B+EkUtC|13Z8XEBl^(a1cjnjJmt2l>Fs^r1Z(};Go3s>e(kAfzr|{hLgp;WP>%f z5rhMmuT?LuRd$^Eni`?JQAP%`FIdrmuU{o&I)ANl`7nM~a5iZczGsf{Ha6yazC8{p zTNuMx?JSyxpdmj3-i%FK$=m>kFLH7_^Tl`@pLc98fUFvxm)c+i zUDEJd^1~qNM1WEdQl~bkk3^C|#oJ*cAgcwiLWlRvw-|dv(Qeab-20ZdYj7}%`+bak z;Mn0hukrl;bV!E=p{^3b*x$zYfYpqTOAcH<$TR|~mhh(^t87BslTWMXF_*sJEHVzD z3BUla;~CD+i*UcuwPeCsr>?h0Q&hL?H zd46Qu@e|;2i5~%&TqE0upjZQAJsX#-T6BL_4)pp4rT(Mgcg{8)ALg+93LrzBo&W6Y z=P`>KOUL@VhYgqS3&EkGx#0SvU*TDG66$0zmU-x}yT2+7gqqLzz@@%*hD4K7F>L}1&)6Ywon`E{6CP#g~cr~B7lM}fccJW{7WXlO3WzE4vClGHw6?mqWg<}wHPxZ|0p z+4Dxif~Q(b75*=Ue#O3azXt$n8GQic3#^c&bh6~}GOxX2GRu~f?t_aiN%VZ3zg=Da zRtG+7Xvn=`0D}Z)on2bvx;;5;5CF9VdJ!8#09aTux;;LZtjUEvpCcu9PBU-W?R@_5 zVk?cpFV9_M#+-h-x#seVZ0_KPcN^=!x{v-EYmAAFMjF{)f4kpIJ7xS))K#0MG#Fb+>g;paK>YDuA*GzWVQd1-uH{`ybHl9XmIoH$tOvBH>$Wu=-!-ftq z>oqx`2R4N>XT5rDj)|?=E*rO;2$b+Po=9M=IMELlL*0Cs1cTBib-AJ*PC|moJMRL!zT_!-bUgozEeN;Hx*5cL?SDZEeVL4 zOg{E`=h-A{e$%JMpES9qd(n1yQ7qIwU#UqdMG5Oll1TE;~H z{F;wVZuv4xS8BxIKteIao+1rMP~_bweF20y6ndjMFWUrIN>VX3nUXqRnCRPYj8ErE zrREL?cyjFVmNvI|<-rcVN)K|Bd|t6;j&2!IjQ7}aU&XcRS<^EoUCkjk-*!ZIl=N(~ ztPz$XbtV8MiX9cL>+FBQ)iX>{onYD1*TYScTGC?Di`Dz)5YfBeFy2suV5J9`JjN=mkFTdUpvF(>9Fj0=1DW`zy2S# zP}JA=n(xDRUGMv<{#V=m8SNhS!FweT`~LmNRo!H}_zi1yw}u~QxW_#@ z`aKG@#r@yH&aw(0{Zo?%pz5~1Zc?yuq$oTZigguR5tj}M{@e%eTkU#Fu}{&;1BC0H zb1vQQFbSxLrE!#D4!)JgUTP*zHlD`%A!8*Er9TcG$lCW(_}wmDxj{v(CLq8K3Rw4Q7JqfMJk+e3dKPnI|SjF}c}am0Nt8DGF$ z0*{1mzG2!Q{*xJU`<*6r70e6$6o(yhFR6G76G8_C{^*N6#=A4jC zzTAs8{p-d*^)xH})Z;x$3?$HT$1{I6ML+?;HP-@2h4G+T)A=c~i=89SGt%+rCroQc zhiQ@gR4v!l(|E|~NqauKa6t*IeC1WO&0kOF>8IC~%kWv#)ODX*{+0=l64Dyi-I^)6 zxGbMh9OH8@xpe4v_qonXaf#c-`cT;SC2Q)W&X*hLZ#EDlkFCF3<{pvM;8X0Orwl*I ze1VTECcqPqZ(Qb|MUOiI6>bhZhoRU0~O4G7(vdQIL@qEyWVVO z%{hNssbi;5DB7ZbD-=qlHK&4uHt0tcNak04V4ZsZ+L<=MamS6b4!(6lE&)YJ(z|KZ z&wzBRBb+*+?^Dp&HP@`_=d$ZJMQxm2ywL6^K2-g2nKYNxMJxcpP{3q~1T%3BcEBD- z&Kn)H6h)-Tn5A1q^!2Ukd?ZPAbQoQobi3s@u7_?Ov@o3qohCGVaOd47_r#-gt-oxB ziJhnlw14OE>iu^SddoE0yh`keD!!Uddj-4>TEG5nw*las3*g~3o7B~_9Nd0aV+Hq& zRRzP;UH6&%8?!Bypsu+U8pi&x!q@vs1-?sQhNvZpb8zrJ+NpW2G2B2m?{##3RGBm@ zQn<>kT7_=x95n576;F*Ax`qIN5#^vw29$AxlTJ9^2JDVoBzIjw7rzo>>(-gJM<290 zprR^wr=Cqwi$o$F{Cyg6i8pGriFlT!y%}7@Gfp>D)dmnydZQ}*DFI%wkpvZOuR5ai zJ6M|DiVt#hd?5n3D(!7&RH^eguB+0s<{{ycN2{~|?WQ{xU!l`j6SI;I;ytSaKK+`` zcPjgvSXUaPnLqx{WL|&G_#@Hb95?=u1fN}h0RdP*%!G+mt-ps4N}#La!Z%HhE;(P6 zpLI&|A3lzw{@-GQpxaN$+m?$iHXeImBbuo23I0k<6kS;|1$rYnEhe7ai_hCaKbo1Q zMG1LZY$sonu&Gm7Ypxi9%4Nz>f?ObWk3vx%fPz>6hW^#xHd!kY{;kSByb`)JQWPj8 zi|$$l3vc{*03`T0Jr%agKKKCpp!K6xT870U6!xi~{e^MIoKfCIC|^o(PDNY)#?$d< zuA=z3hWX@pbLfzzfbIwsRv{qL6!~rNJC%cXjt4LUtAG!sx?<(3O=AT>ByhKX%SZ~N zSDWzt6{fIZoz*heGd}DWzq0p0cR;tQ&)emTc?z@n%^O)8CREOz)a5t>{yzcKc+#rI zuQ3$c%VuV);ljmQBq`puNB(5xuL6LRVs7iv_t)M7(~v_GRu12&Rx7-FW9578BCV%j*dranm`)#hkqtNdGjOni6tF^P^#pi516WskH z_Qn`|3+CJfG?e<$&rI&s7fr{Lk6M6Y%hfaO@4!IfxJnR2#A92x*gQay9DB>Hx7W6| z#51b+kHTISTz|94K6Jm$Uj~yVe=lkB{mGb%Jh7?PYLC2w52&KA_0JLO0w9U#%dhM{ z+Slj5KUbapqd>!?@(Tja`-;nK-L_6A@8&e8j~HR=Icnr=kAjy9bZW`Tluna%SmoeVV(PFFbt=ij{Kp@d)LGC9f-q}Za|Ys` zfslmViVxSAY0&bEnqPJR8llHum~AdO|7>%{=#g=svLB9zQw0TF28IlG42}d;(lY71 z6*hiqbr(~-=JM~RRW}0#y7IC(-5s-EBB{5=csLez4eMtIr@E_Ypl^{3N)vF1QaPAG zQEDV;|4R3ho-T)=rufa*mfABQ@b~0AH!YqS0F)_Uk$eGl&t{=}Ow(Bv;nztbK|#Mi z0gB=>I8V)5z1R2#cwvL66`wQy z1s9gOE^1iMRHzaSp>hQt#b!%KO46?pO~0WFEvfd+kV1{su&?{QTI9KT6YvOK_vnYZ zBz3(hg8Q~;C_=r}XyRg#)JO~5=#ciKCj;${W+o(l5Cy?a@^R-2SMUg)H& z)>N3!ESP73mg@GX7!1fWSUj-bM0DS_P{^wIH>`HxjrKk6MFEi1krq%i5Vd}6hL53c zvxsNUq0t~XYZ`~VQqY233Ule-pStGrnGPes;%h+_wI^;=22Dv#CtT6Z%5sF5zX+jIdQY-qm z=&)3;MNg<&qfN6I{y|M;J$E_2irtNK zA3Wt`6RyD@gvnTJlU@BmeZd0X0Ltgs7Xtt|o(jV;nxqpv@@$3%OR>nAFwb5}K2zX| z80K88Jd!yG<+tB)r;W2?9DYL`%F0ksPl(80N_IkE@$W%b>;d7nZKnNC_Y*_38efco zW$b?r^#1}$ID9`Kerzb8lIJ}K0J{DD-&jeOfLOV|b9}qWId0wYnYrSk^USHIG_FKp z4Q&Grb}>#bc(0ht+n`#`cIuA1 ze>fvSpo=b9DB4j9Zz@dlkp1v}^S{cQ0TLHtL&!pJ=ba55@~)jGJ!iH}0{ay7nwwU6 z(|E+Ea2WGw=w;tp3bpi^6h}!mRJ)%Xaq!X$DzZe-hFK7S3NAboO{-xBl*psAGWz^; zlcNX~f>Mr1;HX=`JNH84PMiR>t6c37A)q|~y8n6$0pptG*&|>|%AxgpzLX$*FoIkB?j*R3%T{*|a;6!s(K00=6xaGsUbP(fTwKhf@8wx|`~nQA{$ zP8!)rhXm*M-l^-r${5^yi*c4LHfiF;!rm`T3rsA&iUA4bWdxkg01mZ>@8x1@h44X9%jH)2gp2z_*R7_{~~-<{5;#KUCmD|zoZ+$Pc?_?^*fRs z0BEKnS02AIyJUeG`jfwiC)W7>#LPGHv3c%2D^&@AVKAFzV}m8RK%8ELebNjsgUza% z2?OXaNviBCFOnNuY2yWd=1lu{C5IC;o;v13{O&aONb#$Erj>sScvjc7ocJdviusQ| zFu`ThjcC*3cwP^m3vWCU9G6^nq(7(+l3&TKT)}g)(1PAPK+8#(_In@Dk&z!W|4r+k zwzA)QfQV|Da|>{u^Rmg|xBJttu~hv@jK7hN>Cr$JZHKvF@gic-a}I*CS(C^INfyE8 z<(XSGFqLHh7Co4a#EdD>oR!}w>x;>MsIU(J$ICBR7MO77c2oRfFOo6wgSINUlGBl8 zUh>Wd=0IC}C10!iX9soWIggXL_se}|_=#u`kuYsqW6wbTV<>q^WQ*#l7IUCeR+mG5 z?OL=^_t$c$=69Pj(CrMIc*1cO-L;dV)HmN9u%FqoW0x5MDq4Sz1}l6=k{e2LQ9rp* zkajgoe~$B-{{-4o9EPShjmI-dW*%mgXwOT-9uW8lIfSg~OP$wzk zL6K}oxW{b{I)Fl#hg-H->a)0yKWL5zBT50}c?e>;SDj@8cUyW0;E5~3a3Oo1R-&x~ZO;xaJ;==qD#a4mb!SHkY| zJtqD5BPHn+C}sVd=b)&FHk`S4cZ*t(n{urn?AMssbue*GseaD z6AL6F{o~=M*u z#$K&1;Plf^+5*3t0s`1zTnmVZ;lCT{OEq_OQv|&xoPI#tVn#AImzm_8t}bq4*B~ zX=OiHVT+@Z!l`ErZ9C9keF@+m;3W2>T4IS_ue;M9go3}TKiElgpMDOI>9NO?PXMqC zKq?W_oCE$4Am8IWSsF7so_@-L6|3Iw@i#K$gam-}CQiZUC8rlh<3J)45Z^(q^cO&; zkY`QJEwTd4P8ssUTI>9)VCh3bOR|P#8FS{0IMp(k6+0TJaIN2W4s4wOmV+hlTdj2wh{a%kTVBf^w2auRIJ_Y`xrMq^({uq4?9BRxE zZCdAqlSpXSh~Uv?H|-pdP6tc~wNTTVGZ6O-ICLS1s{hNczcXJQOx$KEb_Y<(>d(<& z`IYZdbpFz~Q>Gr73>`L%4z)AUh}~_XFX*^!T17J;y?9X*710^@LE}sBS7`A7^k#W&#&NE`y?%+&$<7b3BW$5>yQt~F^i-P z_Jsuzwg2(==DUCV|4NR%65IskslWR##)necYF{6ud{ofmUvsS~Wb>x|(T7X{>fnKD zO6q49&ZDbf3`LqV$e#3kB@Rp}sXOj9gx5^yfPyX!EN+u$@5ujG3`$ zLHFn7RC{h`>^O=bL!qAAX4E-cH&c-VTJT~vjppO|+*iHaSd|Wnt|=u{^Du6el_a1r z6@kj{Ku_4DdL|87f?{Ca-Y$d@2x3buwx`w=>ymj`$vBB zr>5yPj19_4YMLGk#)1UyX zp=+jdfG$OT9nU5_*=U;)x>a*G#Rx>FT|lQc10NV89Xf+irNjG5-% zSHUwR@JvIq+5W)&rtQH$n()iL7BCUqb+1YN^e>F|?w*Ph?zjnbxLpabVwXuj^Av@% z#zxt8J1gvG>DOL0hI_{mb)Pir`Y>lSw6|!lg@CFuufJ-k+WPoP-*1yt(Hd!N;LToJ zvmuW~GExDg0AOVo0BSyOV;;aG0bPL7xWIZ}FSEc_iYR8-F~;LriPiAS-805OrTMk} zoqIdB_Fb|3v_JYVU68v?ihVA1>m9XsknpTHO2Y9q)dq=X|}ugJ~e}m8Kf;Z zLH8W4UvI}=&A3D6(>Wmqrn9kQtqh8o3I>~5Ntmr$xc(si=cCJH+GmevQ1qxOT3G-m zQY%TADY+g{?!&o9Kl8n_J5^mZ+XZoH5Mo`5e$*E*vcawC84`8Jt^8*uHK?gR8 z{{jVvw)=l;JJgDFr*X9v9je<8<6nvqN=zN8>G=NYFs{+`-1_JjOQIq@`_-z~?iz=2 z^nM**SjE)*+BK#FhOV~X{noVO*B6LS{VT6AsUQ8+1b6=kMw0PI%zj62l9Md$`}FiCys@z`z3ewEf|?#5|wbXU7-SHqTPiN;9BtZ)u!|0FSb7zZFv~ zjz%=yv&z+Y)6Ta3Qggm3@*iZ{%@f1NnwMoS6q9fJBM%beB2P2pdXu{4c2ndTkIR%d z@Nc};M97&P_`~nWuPslQ0fhaCy(rIq=>m+jF+f}w)7x=QrWg)oe$fjob3s}xQp~9+ z`rWB#_4dtJ-_!c+U2u_QC@pYg?f1%Qmc8FASGsE_1@33?g<7qTYia>F$9i@3IhgI{ zr@LVk4LsYcuoSusQkwi6q1N;?npM!B}V4 zn0_3jnl(>7Ww_Y`8@yQI8`X}EmeML`w-6No@`7IBPtzKE267*NXtic74RQ=a6=F_{ zVYFih)L&m4@6<98=P{hSd5Cyspa_+5UL9a`G?X-b8Vj>DVeDuy2%*`r`*RZtPy)S| z)*5%9uQxbXxwn^EU95lvM`zOM@LyGqt057kqFr#&CB`|}aeMgZ`s3e9=E`|+NtgvV zNv>rF>HmE|Mb7MkH%nT-B)mN7-*vfPG}r#s*O=lrUzv_)pE4d!k1sm!81*1k*b9*| z$&t7U#*MSuel~c+ksiWFJ1p$&Czj%PDHcdkut`>c)bxpX$il(19bC0w@DcG?G`KaX1;i2{SPm{rJhCs8Op z;uR7` ztN`njfTX@^RA#d4CP{U8AABEVQ3be~3Fe`UMx#0+J+tYu1)y8`6k+vJwN zh3+^~C2_1pHOy@Uje59lou%YehikrAM?KiQQ#tWFCfX|j8Zu3T-ZdwE&=;z-dmTT+pgQ$o@@hSV}o9t~cgCIxaQOx(R zpJi#fx1BP^AS&G{~bC6!~@W~w5AHt=Jzt=Ty1yJB{U2&WC zqb&P&0icd#2jjk;-o`6`(8wP|#tF(Le9?;m5nj>fdfMADt9p+Bc|eB0{nLY9-x)g0 zgv>?y#pjGz3{!OZmvXW_A64^~LC#e95=ogR#>(0A5E|kgZ|a%G`((1oZvf=2d^Y*v zYIB<|k`X$Fy60_U5|oCzqUpVnS6z@G%D ze)rxlEhGGp*7(W-EUm zGi)eqLVb5;#yyrek0($}W^SoK?xFe9(B|UWKlN(yMRs#=yM=&)33Z_hERO-M=5Bnef_cp z80enTufA-%X>w3oN-sEd`Q;RCE;B9~;I?SePmWbBzfTA)S(f6Ahm)v6P|{Y!uW%^3 zgpl7_I#I_R7mtSGK!_zQ*cyw^4VE!;=25L$e7A0Y{WitWP zViVDIUsN|gVAyEORx{+MKQ}J2Aw8}4eN9u5XCVKovi z?^;Z^8<#GfTD$nv9ilnZdMuU+w_M8Y80MgV4?$-Y02mcrf~#jHJ{0dl0bg5;1CUbM z(Mg+tJ=#mvW$ngz-ySh;d{&o2Z3v@Gj<|gY5*pR+8Alv@5WHZ0@b_1kG$39J2D$DA zmk2nqo9b$0@$@&PUhPg1TdxAL7OofXdA8{=-TU$=6iu+qBwdpwQlCvg8>u z00Px5otkwEzQrk~D7Kz5foF;z+=)mCZP{q@bY8d2xS_T|#rbWDmzH_!<>!&4f5R8(t=j#%+!OZnImV#< zJ|NGSXTELj5uNiPcih$Eey?1GWx@Jj1>l6MO8!w#$3NO%2jnRUh6oya2ausdezV$r zKzplV$UJd#C}tS#Dg9)UR?04%Z$X)oG3zvU3^l;xPlO3@jb-_%NuI9rpjG@Guz#FB zVz^neVRN&Szl!;*%UAY5rQM3Ig0vNy*0?huN){!b9g_cDk?ggO*aHO=&U@X2AUsnf zK&p27Y8^=P`xVcC=w8zVM}@ZU=s0@cv_MsCQC$@uDY-ACp!^&+dZgK~dArGwSj#}E z;}V>DWd^9pjjVpS*`dUkA~AI)SbFFrAyvloADvzqmV8-Nl6Vz z!%1k-_UE6r`rSkBx(B*j>Dks^EI;I&W&R`D4aN9NFIX^>e-j;aJ)wu?6sx0w4(0so z1R*#=9-4$gAlG6AfEZaoPwAImvbt&kV<_y|Yobrrq38OP$x!%jMH4A?!G#p*PP6FS zYMsei##z;O`Q0-wJ!`c~ZHH+cl5h$b-H{_r#PgnA`JNft>|XBqr_=8>(R8dK5Lp_|oX6bv4KOBs%qg#6ra+wPNF{GcF9Lch04j(;Z)cY*GCAmfr5Eo|#FXSpaobjtp))(&_mvrj?x`=yq$-$0 zL~j`ayhy14(9_!1=s!@5;}cj)dU~Q{7F}!7AUK^P1Qjp(ZijB0L=2#bfK5eMVDj?I zNIsN|0{?*n?7MR;gN{Rx-tzOmG^R#&h;FZe_13@mauZTuRPsDH3qS*FaV#eP=c5mi z(m7xYJ8>o(UHlPm6FXPsrD1@kPMPwQYQ@>58R zwr{hn&U!ABHfQ*7<6S#5>C^mheEe}w-QaaF?OoxUubFW5$EI)<1*l_>t;1Od{wc&T zmSYbVCAnge;D#HGi#%74h9B!r07$xVy^YISDTMZ$0c4M0Z!n@#KbL$;Ek=yvNM_=W zdrY1^qvP2>o5IfRX2{QfMH~rm%&C-JqzH4&96%G2lc4*n+5%rs*F=FrF%;?j3Sb!+1N}>{pg@Zsfo~zJBqk7ZqBrSG6D^}lbl#izp4VDN(0-pI z{0ISJ($79^<*TF{@847<#d5!2%)l6c8YiMNCR*TkilR}q1!&1fWZrz;79iE+(qsZY z^!FEF(P4d!h-(a8@U@yi@6oOuF!WtY0iLxxX~X~3HKcy@Gn0Mz0lLedFR=>709fc8 zx|s4hU?xyuT`_j&+Sd7gUH-14KiKE>XupSjFu40Z6GAzkX3sA|$=-@Qqlce-$bK)d zI)0_P{?*;>0VwU4`Rntd>h}SMoQ%!O)Obp*WsYJ<;$;=l`>)7fv<)F*k_>LW6Hun| z@&}Ez#Rv2k(ma(XH{AkTUw@d#9I2E4v%Oy!HE}koRq81>>UA{7brjD)ek&biBq+Vf z=+nk%N~(%d&LV;+a5Zh<%s}|@suEIkJY8UD$cW;+(R8m<1f^$;J{`LLJ?21LyVaGc zBtUJ%U!(BtHzvPk4Rk=qT3W@v#^^woEn2-i^wa&WiAP`^vX*4#yjm(I-hGelmX(x1 zX30X+@z}#AqM+3u_fhoK7pCK>$LXF&%knx{1rF(e?0%O0v(Fn>W}-XUe{UcdGuWB_mO_010p#NQ<7ZI zk~%YOk{G(rhBltV{HGgi2dzJ2rYRmwYF0XCbp?phP-{uYD|A4Rcs^w9<#|9y#uVw0 z5X(yDwO391AAfJ!9=zW~ZEYri8Z8BAsO6Wxwv^H-z&^p1(`}+f`oPi+Hvj`Fu_DJ+ z;O_DJiq5%d%m6?KkAir(3%b=h+?$2(U{c`z(m9jt+7P9#sO)p^psxrRQ`(#%AipdC zHY<@svNiRygbMrzZ!BlbXe-qr%InhoAI}}9?t(mpEP;`{iFDh?Xwukr=^_eAY^dwB zY?EK}2~>@WXJTT=@F$;XitOnT0BEfhgTV@PH08Np+;I~V2izR(^Qpbdhn*z+e79xF zJKC?e=Ii4bE|AogTv5mDS1b*5z#iOdf&bu1J#{t?I~IW6Vk@&#^H{1pKtyaP0JR4w zAufp&v5EvY-2z?f6&(8m8>5BWw^!^-i{7!2AwLevA3^j?JgkoA^PB^)i49F zM~R-holaVJEUas+BefK9oKsFV>TWE231EHm#u^3|&sUYp%oC5c{oy}Ne%ofNFYaG= zQ#=Q69;N3Q095N=cA3?$&QeIM%CAu_;e+=8MSla36!xtoK&m}o!GL(j6LSv?Md}8w zYL2^HrHxTp_YEu;-t_5R-bwyHwfCfE-DKm3ECprJ!^dL8L)o{MSSdW)#SfURX{8xZ z-c!v*K84#XY;f%lJx~I{cg7%!C0WDUXx=asekH?EPOh{$zva(>z!naA^AXJ+1cBv$ zF~(0uG?#UVjq;2MCDv8<(8`Gi2wlx7;U{ZLIffX|UGAaOuzA99$J#0SV&B*0ey#4G zYZm$IKIP35Rx|@jG;JdZ$|CPih_}dJqEns-qIMerfD++N8wfK{OrzcW21r^G8!F;R zhct`h9g@IqB-18v%J7pc(s%u)Z6=+`0+2(ClRpE}RD{~F&eA#imtJ0$hN!yJT#9zt zl`Cu)b$?#B0y7n~qJ7Y;-3av`G?$_m_UTR+HGUR{p&bhR{JQ!&4u$r1lX)J>KIYcD z6ofgzqp~{WN$UHTUuowz{W3Ikt5@5CN=0)_$AqYU({##Y=FjEalR!?=QGlb@Y0gQ* zjephEbk*Elq7WZD!DL9n2$UdTL>8S_QIDw$$&jFFG%HX*OOr4%twT$w-+Fct)4?W9 z$PgrYNase=j+_DQ?P<|>vo2axdtAQ{sANwxa3o{$t3P7Tm~Dz3fUO$T#H1lAN%UNd zifpx*H|-qbU2zqO1iJGV&gZ(*9FDGi06EcDUje9CL8t#?rsIi6=x~~A3iL+>=x+yi z-f5%{Erfih73miA>B#V}n}uFA!53YIxs~s6tZe5;djDIAF_-|f{<>4ATFtlKzc5JN z<^FV_pLP5c9n)(QzvvLuyE;f5QSimx$=GG|>QELPcqqRbYWry07L%5=3_vs={lx$P ziV|o^n|H-@OC2qp`*=WTbl+{;X2CAO1$2+_T*PUX`Q?nUmPJDv+vO%$B#WQ|hA61x zN_gqM6hN&Y%6>Gl2ZQN4qwQNLER8ja4VzZm47lfT9 zsF?-x0hx?9!R>e2_@tbXTSf?C;ArpXB`wu*zLdacLI8u&{%?&l0pFn;nXzh5srVyx z!z|;op651yYC70UtY&Oy;=iv06XT=-PDr=h1>~C)i;#E5xRSIHG`CGaZ(l!35AWkE zidi%L+*8B@bnIPC=RGlct>#~HDUCuo3qmP|`C1QG<@fQw@$sd@-A<9p0%~EP^ywLR}= z7F3ASpQbhD4A=w~=x7Cc#!_j=H%{q#xu-)rt*#?Ed`%kwGXRYXG&0hb^(h!wr~TY> zVe|29*J9P(%n4ojNhcm>PCD^;v*N>#&Gygsn4v>j$qlzAbhcy?3hv!n~sr?q;~t0kDELa11dfyZR&K) zE?Ho*?=3ezw8HA5t*FZqS~36HnZ&9^)BY66FWUiFCi+sS*sYsQ`=6gAhe{U|cus1b zi?ADHVLFJQ`t+^<5JB;&o~AvyyUOv?FIg-8CW4rE?mf8#=9GH2?!1bf4}x zfMAOk+5}rYp6hC^K`bJFck`fD=oJAoTK?!C;yHvu_(NJj!v02JHX@Ba;J z#p5Q+YejV{-18%f`*#_aWTSH`T?Kw{ENC$*J}JRpv*8b@J4yP`?s;ffCSdL<#3MX|nYR{O zd4qCJ5dowqko5AzP!nUq(47V)BA}ka-o51vBi_&GSxEAE=w`IG7!3`$OOmTx&F5|o z(|qaR3`pD49W@qT8hMKb2G{lH4<`RqiR|`2-VfdWyU^U9YXQgt*_3UpxBlf<*d(~R z)%&BstO9xlnE{vLj+I5~CVnXUdb;^wQC+yxuVxRKVc8+t{=|II;i4Na3~&G3i%c zF5Abl*JX>&pW*{+a_-+5MhIGRwpWP^9ht-x9wr zC0Dx8i!`4Vi2v$ptg(4=f!X(07!O7@kDPe7zcKkO zvDk*6?XvzePgY%r*?F?#ju~c-WgplHTXUn@pssGO-@Mfv%a5cVs!tQz8WfSJUqwIJ zwEE0|9CA@vyAz*Aa9^pgtln7XHT+YCV?D|IC{vPr?rXNo-I)UYt``H_%TO2rU?>qlvh9%v&4GXU zpZxtN6Dq+#M_bD;e+4c7Ppxky`tiOBTej0!ARib#%+R4Q0X%H!g^K{VdJM9d=NyTV z!oDv_)Lb50r*T95b~Ow}-6`@x1@4HxO|D})0%(z2`w?{R6p`!DMek3YhQIFeY%e9D zwb%GJ-$o}U>o3n>z0BeM_ZWwyTH>JEEF3`If{q^dj5DC?KPH*W008QaA8!+zIplE4 zvFNj%$Yi#j$JF|+cd5A z87M;8>rS3x*<`}auxYTzHLc4Gus2E$qwRtF0j+N~0jylX4L2i&R{7fE0A3XheSY(1 z8wYebOI`l6085paVmv?{wjm>lvvNO40z$rA0Y;*>Km5kz-X#}6PM}94_R9jEC;&_h zuDLeZj%g*w`3)pJHIV%L9_OykZ)gq9ECPs2j={h3YU7i?IUPf3}-fx4jDe)Dgp9RzWxjjNhPzRYtqKXz~i6a?@2 zj0EvlbZtYkmV^;dyU>>^u@r405z({;o&kcT?b`^VH`470ebb23zaOhl!M~s&5wxG2 z@kTLi0;di?!ED{J%Y6OqcXn8{L^HODUwTT%bham$Rc?!LsDTu-Gg~(J4F>; zS5qNfH2hCI#&bT_1Xo{U3sgOgL4kL0+nwB7r;t6B!MX0h{ z?)!1o8r-@T|KiI?5>981*MT^m*Yh*LnyTj`O94TefJG!E{F`ng0Rr%$8l%{!fVcHm ze`iv6&<#9sGRgT*=z+jI&;$L^ynDySbO3!tz-Vxpx&Vd6a||<6BdqN{Mz_X^LO1pD>v9x*3${WSwJnb8oxk$wE((V zXmsT_9Qfz|VgF27b(AiWSEB}PO%1XpATbEv}d_x)QL$7~+4sc6=t4eRU2w19kFxC?sjXOP2#)ksWMV$L$h_hZpA=6M~6 zhx4zv+EU8r-g$>kQ1ba@bb=P>8qKX;ZJE}bn5CKq;vDuiX75yTvnrX@E6+AGDNp(d&LbKgCTn>a8JqjnCqyx=ODQ@Yf5y8DJRjM^PB zub7<+NjBqkck+X;S!U*;t!DeK&zPno{m)*$TDP~FMgB6US}0|kh^0`Kg`QZ*%%&wf z16pJ(5>UxW3WZ6TH*P$g#;wrLf*@+p2Q{f{A4tb7BCnAX^<#SnbXiv%GISMZr6;Z; zTH*8EhrHL{{}?^;GWs(1QK8?PdL~7zx#$By>lSmNU)}bZ zmtQb>3jLy>G%CW?Xg&Z2Q5-2TZVR)A7uA^q$)9BAQ<#IUTfd9ww}bO8prgRA>Fh;1 z5aocUZojKk)KS;O-p@?Ob5EO&N9p)|^+ijYTxf&JznY-0iqVNGPL$%_1Zba_m;N;G zf(tEuU4H$##0uuIT_1_5EI^UmM=MQ4Vk5g~elq?J_kp@>ZIWAoTheyMI4jGM!-q=~ zs5c52Oa-p=t1sI`hA8NST94B_0FqkI0_Z^lX`G4*xJFha?Exv09+DWzzVo)}cmmzL z&koa~6~Kywsw6=~k*PJJX-Ce0r_OMS#-blig6XD6ky{;)4s0d@8`aCCp`4}|ZQoJSalxBb7t|d|0<*^Fr?=lgov?~Y#2ja}r+fLZz*q&CIb3L> zO$F>qy&Up~w_rE&xnBfsNFs@1lSuo&BT2K?q>=h*fAk>>7|L@!0d$-WpiJ#O0%<1k zT-Xa$af#!1poi|91iid~Ncieai)RKr{8}p~Md4Qk+3ey42U#^5Ga(kwRhMxTd~r#F z?}0K~WRWiY@S6>qQx9(wc^Pkj(9OcKgn*h|#-lWN;LPgi>C5fbtMQMoU)~=~7q1*Mi zG5q%#qeq$+=*(rqaDIjU8c;^3Q|7hVwnM|C2vtoNn-9g3iAr7d0&CR6L;;?i`-Uwt z1+#9nsK0X0EA94R)-4u!pLu(^X@B@nB^peEy_yZG$;{k26m>*J2KCf5fWq#iWmXa-_rd!Xb?=Fq zuxZ`T04z7^m=2*lwqUaEKUMXsVu*&u!0>w|ZB}EOJxqG$ozGKu=PZ1)52l)Z#v41XySHn5jbgXvPQ|k# zvMsS0j9Zq~28Ni9hyG~U@iK2MM%gE_iT9MccBbP$++r;_{uaz*Z`$^W*UXCDlI>)ChFK~LTL6M~;(0kgeivH)j`RV4n8ruowSpMLoz z<073RaBM6^BZ|u&Y)CmYO$FwyiwUgo@ha=jiw<2mvU-`i=6&h5BMFg33aW~sz|^gNbW5L-&XZ4n`o^m@##WmD*Df)tjLC|jx=e@iaNMy zjX48W(;96(@7(jcEi}6$PNDN?0#5tV`XIH-oTk%GIhi%4RAlr>e7j114Y5%|eqiOr?zgvCdhk zLkn=PSNVP<4yL1_i8&l#X!Sar_K;bY908ue>VI7*FTvY z@s@MiX(sifpP1C&{5`%stU`2vDG90$F6}|l=etb$*+27qbksJ6!@eyoZ!rbrHY5MV zd0diOgYNk_Y)+E+r%lGe9@Fy6Ut2(5$1C`ul6bL{=7X@KpBO;?e+W~D3&WFh%7}QT zT|PHjs`4e`kS8f;P-2P6Lx->DCIIYLZ*u@iNH5tt=X{gjvJo)$gC^PzxS70MxPF5L zsJikm0YKEl9M(#$U*X1embtDT=N!oAC{oy$WLP)O70H&AWHQ(x-0b)VP6c)m~Wqa(gV5{}|l^6L| zu>_P&u;Hw8Xh*}}E`S=b^i^6UjP~xO$iLJAVyezJCN+V`H25>GFXa!Of7;~Nl0WFH z2E_YmWIx4*>ceQ-?BY-A$7von&KVGxMNB7!I!NqV5EYEM-k5PncM{8%nX7kTuybjU z$i20sY`{&NbEIR9IQc}vRnL69js}+ow1HJ^ig^OR`ewgL(P67Uy_s%uCE(Nv<&uyH zL??CJ38wAQ2W@9wpNo8N2XDUD;|zqmDF|xb34)VOta<00i;NxgTI&+(0+(K`5?ysO z&P9T^>OXxsE_J(((uq{8OrvS$j)B^?2g>y#L6m6Wqc7>YQU?@Hc_Y{`I;+Qx8Ce>N z(->nadtOM)W>>sxiuuq4GiKJ*l_zjUfy9<{xD$OA1j?dIEd9c>wxI2+!-ND_RmMG~ zAU(M5MjW26Oxwc`*zP(@^}WK?zVL%$?{D*Y1A=V_-~rk+FJP556hDg;oy8kjFsY>4!G z@N8DKSF+aBVj&6X+=fpqJ59qSG%C0-|H`YaBtU@#XZu4BK;gf&zU@_*}2fi zTfGJ{$dsgFTqe1zaK|<)Wn@%59fanMf++-jKT$UeWLT-m_sp$?kwR;s5)gGV z?cveh&jA*0!CxYNINZ2Tln|AhtKvlx=AHokG}kRgfnpVFq`sq0u5cZc_;X&dK*^4$ z0F1%ru%C32@XEB-!m92Z)D43Y$ zRZJ&b%`ie(pF%)5KF?5+jTpU+P4Pv5EKG{7e(U=`_Au+!D*TTZjR!5VI!aAaVCKUI zCIJ8G&Hg~Xj;R~fEeT?XC})2oPdno_*se% zb02cDtV#mJkSA!STj^!eu`IC91xH_m?A9XpPq z&kZI|(jpDQZ|GnAk_B8kA^qGfSjM#sq>qGaKD0FYb)mST(?G#-xMK(Ol+a&}h>x~W z)Lq40JJH%zcgktDt0>Xmfd$#pVM@!gO7rLLd*)CiK)5DA^4? zo9@UlmJ%{`>E*_)V8f_#s>=G>w8<09>UA5c9>x-Fq|!mG#RSkOQuN7h z+GGNp-9#if2-1`|?0Du$Q{>*Y8q_BZuVvT(8<2t^0ipXYgc+dy>Bmhg>%V)B|03E`L2pC6yggz zcbWo@nNZHT4(CD zXC#LK&zKsY&&mhDwk98r`y&r)K?d-cPq$aai8%Hf%T_UJv~d9p7_UNi8oxk#(4y82 zZoQ+L%V?%6<~XZX5-9 z{6K-qqAxy2I^}0ojOyt4iuOY>%H$~eh+5hoJ-Yn$i?vE1xdNax$wAdK>FA9;!-T_* zVZZpquDed_p$Sg`t|IOi2a0(y6(nLj&o1|7e)9%PQR-IM5i6;lX!B-@Dw|2fd~FLX z{`L4y^3&uut$#>+r&~GY$mtsB-Fa=2e-pA2= zHJrnnmOTmY>ip>pUbqJ@QofB@3-QNlHo6qzwNhM9N9?<^crlZ*%cY<$r`r z7Mb=Z9);nA7~nFV?K=CH%AG_s4vFC)1wbh&6SGOQV;j#BiEdca95IsC;+OX{Q1(EZ z!-7Szu!B4ng*e?~Vm7t=4t{XO#xa$Y&%FMc)mZkgy{@JPg0j!A9MRM>jk9uv1=eQ^Cv#u*9tuA2EN^PJ6()i?w4FL*{HspT##}vpRY98(S)W-==@7^k8xF$U8Ku$$ym}tNcOw z^*Lt9&wc^p)sgu?txb&_DBQ7pD< z<-x|HvysA>QrWyI2RC87F(15ZvgGD{_V#-3we|mM0%$@gYaZeUcKIFR6qE0hkK29gjV1>4A)2Me>hAsY}_ z!<;pR4QZg5iqa%QqtAEKf&KHEgJEX~Obbw_t6R7TrDRQs+8*YxKw(=n+#wpesk>N{ zwPyErzw~gNBVHyM?Q-u0(p4u(r*8bp0WeCI(7`tS+T=R|)ipJfAcW30b<#=S)&@;D zBmHgyI{_Mz_M+$+i|SSch#<7Q==}ZDq3ONo5(*p8mn)H^_6H}M7Xr2i_`NR~a&!C8RLRqsRO85N7B}z^o*`K-^z#(tSnP_Z(WLg2Z zdPfxbp#|Q$)o49VES|u3678yk&(=ZLl2a*=iCCgYLB|_E-lVR-*#ZW{pwee?zw1Ox zHi-mj?&&8?dhTl_8virTs?x8c+j8d))Bfb66j}kZ!OAC~R@3S?11fA=&@8{J;x#E2V$L(E^I)h*2&gaMuo%+>+7)eTf1B7Im6r>+zw&cmFU1$#8PX93Fj5V_JU>>GuB`%3Hi zaKKj6r<>sH^Xy*g0N$^})ZN?u&>s!E1IHoWD04rq7(VpPOc=obTJGX0MXd9OO)QVH5ZhB^5!RNnE?SZJaP>oAVM`04ZY__-vhP_Fh7N> zSF@JToRz?OUCX!|{=0#10X%A)D-{X*SR=&1$^Iw?%HY=98y}^ANbGY0ViJFf%}n$|V;Y(ln=z{v?Y05&ndlvWyq7XU?kX znfCzPI79LI)~>bi7Va?`acI zg0L4;p+TLd5rw|&+&4@s#h1f7pnCawVsD)`;$-vVJ8n$u_(>eNaG=e0o(r|=bpbe4 zeJ&CKO7`YnS_N1z_p7f=_|7u4H8+{uz3;L)Hfx0d9)JB0@vpD7&X&LpQ>GYa?Z*^MUNFa@pV}xojfJ*0z%eh_ z4ix*`9Jcp4r;oIq40)I+l)P+#DWa--n7}u`>OB*Ewi6o5ivVuXG1p%PrK&gm8e>MD zZo8)|9tIA5aXVCw(11&`I)KW%W#;u1Ye6JyZ6nFin?IOij}2joO5IITsgCj)6<=os zU$ATB$k8VCmw#2kx0)Z-47gJ!oBTU(Szply4Ak3NSI&$nIiLH8z3`&TkDB|doa^rP zGwdDy1s7SZT*(EMTjUqOG2DS(@HrP4!#-746G&#lL=y^xgg-mv_PY-`z4VNuJ-dx} zIo+a3frdGw%e~kF6GXW8vqR2b&OZbYwn>;8yJ=O3SK>GK=O<0}^;eBY^2v5vmotbi zpHa7E#|zKlBYa`}yY96h<}PnH|Fd2*p!Tsa13jKmF9o zllY^@c6~dx{}#h#xOKDfXWfFY-5T3*YdMhea?ZF3rsW_0Yb}SV?Yrt=#Rn?FvJwF( z5j z6RQc-0{|w!X+w#1wOXJb@q%hy5pi-JW~mk!oa<>WHKz$rM5>ZvFTcTQX&{GQEe#@lkZYVO=7K$NC$n;jU zsPw5eG%M# zm+|*~Wg<|SMG)YkLd(5{ZZ1fB0qHy>A3QmmDncK!Zqmh1S7VmWXb*i*Q4BYtsmn)Q zT=^vnK(H<}EjQfU|Nep~2`QX}=sR1A$>Aq=J9R#LShQx1$S3@VRckeDFol z)#lz`X_JBKqU?8rR#lpeRi|$*xg6#nWG6nM1Dn4^6RV`UfGGjKpvQkZD&QXmnoxl* zOSii>a*S~kO2YU1+%^YL(G0jqc?k3pZQWvuB?rxL zq$5H)T=R6-1nLAQiP5W(gisNy@Xa^IzqNnlG~|biQa-nQxoN%e7TAv{q8zkv9d~wZEpt%COvY0F=Qey|VC^x{3`d z_{36-7q}4+fxy9H*eW7>5l1}L)2w*J{`u#*u9Yw|iNz)Ex#W7pVyU>?9YrT}y)^H8 zy5I(T-JLp>-+DcJ&H6-&{Ne6h+`Ai%fAQrem~lN|f^DXNtV`}A*tjUF1^BcU98FPP ze!3%?akm#QSA~6^hr)sVmJLnG@;rH$=$n1)KO-$@-km(zf)_N^MV`}0Qd8Tv!zS<& zG1_7i@T@2ww@+0z%lCF}xa;>qLcP_F0yUX!O))<2{U%m4k zC!csc?o(*Kq?^Cdiu~Sx!v@41m%t!@_vkZb+~|?!lJn2*@#uvw_LMZ1*z_&E>tUaC zh8;sO_*fIs-4<=zLSh$cLlq(74yq12?@|3v0**ADjs@fptZp51Tn{UP$WagyezelG zK+RbzWvaAx)6YL+BGA@yAYBl{?&*L1Ql)6Q;)ZvJw|-VmRo@Ynu_-&h;obxZn_!zzkcixj$@(*>oHgdMbYn$8&7g<9G&GOjFR(tl2;)CQvo{u zf}|c`^l*eU6}q19sAwvQ5~0Bxnktb!Mgd;|Um@JQ$)u)1C)%F@AbYK}F+0?SGU9X) zwX=FGE*8b40P9is;Yv$o8%(0xfw?c+bS|Fz%)&Q8ykCg?MdNDjm3>Y6Jy~nqi4$Sy zU@xaIAWGBlgZDu|ud`j*lIU<*KkdEB=ViGii%e!2iP4KMF%F%wPBTef_LLvZA1o61 zYK==#q^YEUdH9?ZJJP1@Pxk|;8)2+$iTt)fZVp9jNWmONe}DM&6k*}Y z%znidM||nF(=l0h!(TGrLIWtu&q-5_hfm&V88l=IbV&ImHRSgk^~Tz%L~)V0DvwUC zE3hRU*S!|6IwwW31b|-^q@=qW(0$r3zh~!ontzvO0RO@6pQoK>3~N@NebeB>x#U}H zM}H;$MJ01ee3MuA?Q>7Vrt=wvh3iTZ0_@+;aQ1MLnxUkl6!WS>xyEr6Sl8Phyx*7; zhg(LZ5@5cP0ax}}_k@#7g#48gMo_BbWA?GuRd#eR)n7rkPEo@e5=Z7SCp2myx>{l(DoiwWZP0C4oIeD{)7%UKYyo(Nd z`9BT7a-M7D__T(ImB5=awfwI)KiN<-pu9tooNP$mFhxuj?bv2A$jOK)N%y)_;~vmZ z7hA8vWI9Y?^JdGsBmhv4rhMbW4_J0g|4N$Qx~w%ley2g+kkp9o9Tgah#QvW0)U2r; z#9wB}F!uYImaVcuhT9xQa|X(v0T_Ms{3=hMiW3wKa6rsMdF1na*oVY?REv-85DOLn z9U6dHKK`y-=XkiC*yX(s*P3ZlCz!LROe%kV>+s2Dk-v{3zno<`y*?BWa-@$iHhxPv zi({Qv;Woof2I94r1ld>Co=`zR$ahibbB&@cRuNr#J_-seU*yo2Y=dTbE>vxENJfeq!H zicf+)6~AgUEu8MC(FF9+XcL?jzJlUX$nar9s~BtLgHhm^dF4fWje#i8Dxfr$F0cXc zAn=WVH=dr|rjuUwkC5Z{&McTq0_Re6)2GEZcQ^~h1K*iw9h>_E3c1?cYqCPYSrQ3^JH>zZb@A->G>?o#AeBFSS90Qz^8zGbnpwLQvCXBI+iwA zD$j5&2>|}*41@BUeX0GC2T1&SCb(Jiz<%1a<_rh`0$54(>?xRN+!~}TEUOJb5_Q+P zlc&VDe-Gqe37=Si8!rt9~T4YR3~zTWRp)fvO5G8oS5T(3mRn&Xn*t}TT}>cycw3g^I^#VP=ndH zSCU+SLJB-{KC!!ttRwH(_bdKCata;KK4}?5eDWCrne=+UR(yhPzM%LYe=9;t$QCiX z%UyNzwQ^s|AZwV%4^(|+T%1j>H|}1fxVsfEP~2IpxYHuVp|~tgk>V^`+}+)lh2rk6 z#hv21@Nz%Tx!?0U-}b|`Gnq^#$t3?tn!RdkVeHe~4=}!SE)}?Fet@lW!WzYN(hKa6 zA9xqL*gmD+;{kUcUOAVVS7oN!HT6zaoDf?95wziD8XdinKG z8(N>#s$mv^+#V{1&M@dQUT-uyFLwIPxACIuj*CPbzRwE3`8RtyJLxySImL)qm^#Go zA%P5VUl<7iTxJL!tC%f}#;u)-`#RiCf7%@leg%q~okt($qz4lX-FzDaTO3kdSEp-% zCG}S~!ygG+zXQm}ymG6rH`hU29XZ;BYMPAlQ8!aKKiHM-96g5xuPb zz?g+^25mqhHBJBfWs&I0jDR~qcr2cEU6v5{wfzDRAfL03tb>TuhH@hjo*rq@B|aX9CS^&+_FJOoe8E&{ z6#ixyOPSzuFhjg0^)3Sn!~WehgmvClbmt)Ib2=?=JZ^)oj8He}O6pr5(CFr-V-6#W z2J)j?SK$VMZC3XV+f2sKKg*vKy0EBY%lQ;O=|)%0$R)v5aoqh1Hg^1e=S#QGQ5&fB zRysNT`<+V5sN&tX)_qO!osH zM9c?4bX?njX{v=5#1Z0al8i=A5Mc7uwio!fGwG)FP81{*5HO`RZ+PA0FPo=6m$ot$ z8jHe8&=pX`w~kRVd2ZqXk9db}k5_Xl_A)1(?Pt^eWBaG?pvL6a3@_g%>!+S`wgjo~ zE98Z#xwfOD-pn~29&6FEKOa}01&F1R1m8dC;z!n<|Ng|~jDcukx+GLN0LOvC#Hp+W z-@?pMUq@7P{8K@Sd|mn24+)`ahNP0?;UoKU#}t0&)BMN$rIfarp~A)Uj2~5ewQ5I1 z5bhUpk6$#(bVq6b`Pj>ApcvYqK#)j}6jtqS_3QY3bGREGD$;Fd z-I>?~k55}_`T zg(u5aSB|^)4fpm6x6J3Ws<(uQHLQwGMLP+GQG)C9{M3V$c%@wKfW+2?{H0x2&U^TV zIV{fj;|9M|>ppDLaT}42$8wuLz-*pmGt&*r_BUCA@n0e3++tD%p8ia+%BMa))sI$| z;;n34_CGTjgLdh#5?&WyMm6pUIsD-8MtQONzP>cnv!kbO?daQh%tcu|FHSWzJuev- zd@Ehz1!0z{yl7EY@n`!tEfh?Cg`y2#TN?Y$WC+;T2?A~IuF>*pharvl1FjJEt*bs` ze__WRCXIk5W#`@~T|c5QOTs<{t5`o3Su z!slEJ%*&R3fWxmuGIJdcmF$WNSIe||Z{Qq_aHWB40(=%OkLW_+k(){zFu}>-0kI)o zeP055ZDR1^EOc?j7LLJ+{cv*$O{W8Xi>%=Qb>Pm||Jr?WeZ4-Xs9HG`E->y@6qPwv zQgd|=OQawF;Cr;LfO?9Ngrx*QdPhH6bmcYFnRUIfZMpQ=QSr<#3W%3B@V2wM*@$=- zs9Bn`0$-$O*Bo;)6Z_lI%HaDF572T4d2_^r>&Ii{+VX-TIh)8o{36QYn+k+4+O2wu zVy@TS!IZO%g)(ZUELw@%DWcr&?*Dx<7e5L^K$mRBPVW7gicNQAUaG0^V;C|+RuWf^ zB{}j@quX)*`Cv*Eo@%<;o+rdexJ-6jt|CnDz=xhPH}BhwUn*=^)j(bw>4J})dP+}n zm&;-%syOQdWx1RWwWu0pt%=7#_!VDRDxdyU{Mr?q5P`ER_TELGR%(33{1=|LvdDpC ze$Vk-5fiyVbB@3kJ?$2vtm67>v$s;L{RsG-X^(%xCe-1f=~!m|&(az@=Wv??WX|>> zj-C8iU%rG@DKb*=Yg_u8y!8J`as6N{KjB3XFaft6Es1=6D;y>4zWOKo`DG&+in+mb zH-*g4ai^!sYC)jJ#h*{ZV&}=+b$NpzfOf+63+Aw>SiY!VX30^{G?~5e;?HO5)@vvy z#kv0hzm^JH*XtECg|^Rxy8zkQ_6bOaAA4KMVgZN&$ZoRD_c z_%W>r1|7tQFNAi48bx&zy zihL|b!e69vqws?+NHgQi*>%~C@^*;GFZaS-9>v@q0faHylg{LRVWC2ZPZ7brvhg>| zA}`Y0@!3a&@-u3|4x8OplpBo+Z?7_DoX0SnedV}m28oDt`$f$1P``X2t$86{I0EGp zY$y8=jJ(%M!jOl5%^J(aY$IlZ8-3@JhHaK`6Zn>fu+b6p88|VBWP@R7;qBaC!##$< zERoqlWDO<`rr?}XCU4aWbCc^I7n5fEsFruN)CVE(tEikc_Tu(RC87F7hZJTy0(7kkZigG-5McGM}3laBbyxg_*C z*Khqn8otg&X)tB4=7kVKGum!qUQm%x3Zxm&a54%wexTKZ9*R3=NCXchD$dzkSQ)$rwX;wu8NhDIqo(=0A-lCE=}=tpI)vJufSUG z`~#*S8zD*Cj}SjFVy0&fK|)R%mVlp4S%DMps<%+hx<*N>5(%Ly0UCFU8FA>jS*x*OPVTEn^I)Ut=X#*Krrx`j zD!}E@&<@7jo^TyW4|dj*;fzVIkW+Oj%^(e7gr|XS2Gc*pbNJ)T1jKzp+f}r|#IJvQ zAScRVP4b=G=qC9c1S#u|EK2GZqCA;Aydw=BXT^mD@kF(_F>YE@pChDELCJdh<{R#& z+|+iP84CUvxP$XNJx4R;EwB`?5^S*kTFo~(Ydj5+npg#l&#O;@qVtdHxccHnicw&$ z`c36yRix%sSviwek5Mh|=U%aLC@hRMtjXK6Y<%(cGoRllppazc?&;6lCfCghW(EPO z31dr*wC!KAqT_wg7LEcaqKdYXU{P50t@@}yyF%u#j8)@sLF0<<5m@D4_Ap%?HRv4P z>P5r6ga~+JAo7auu>6iE!p)H(8A23_A)rOSHWhvu{<+`Qi|)mAZ{$O6Kt+FhHmA!f z?Qcxy^d8UlX`3N2qHH%q1NmEK{I$6Ig(RIAM`(PEp(MF-jt^i7@+eg>^@XA9ykt)L zA}dp1g8xIoS?l=$$gs@K8S7QlRvkgtIOHDY@aIP%O>`KS`m4R=eJZz^%CIVru{A#h_fb zwm7!Y)NfJ_4fx4V4WXUn+RUyh@_`_}o-bSLpLGhf$~IOpD*uJJoqsIOvLU8gRlG_= zjVbp!*u&5ifuB`E=gn(lmx@2=XgTURh%Dn`ay01`NN#Y=5PYI}03Z{es3~W{{YC$x zEBEAjj8*?C-u}z);$iQ{HYpZlSXx+d;Z`x>9SDM+c?7XliLC8aZ8H+N17Q2>31lbS!lK3F56a7oFiUjvyWc zld6D&VH($~^jXl84Or1=>ErIPmyucDUcY~mfw^B{Y3Bt_wpY;Rwoss>kH=@`lKb!u zK^*7Tpy0kr(%tLYw_r?H2Uj;8Hz>#4k!;A_85C?#eFpvgdi+6n8eXd5qSSjca=C^c z|HFkXF`{v;q_XnJ%y9QPEJ9=K%QW}!ydu-1P;Q01t#V!EZae1-?%2rvR^guxio`9b zmFM9DRiNhuvZFhK3~+8_$xzK)d*|~;rTGCb(5>A+lw>HO=pe0wZQ$=8cM}5=)wY6y zolANb9Y(%z$vGH+d|~cUU|2;CmbbS5(Oe2JxwZ2@(9fFF?f9q0ZZU*(Z6DJ`zGk7ZGd+#1$Q(XI2?-nq(26NUHBA-=c1 zQ}z*lVn(id-=}94I)_KhueDtWqJ%6y%A&o$+zhWGp)!k5Q(bkf?+j}?5LQ@9Yzu2x z`V?uhB=?f}=Po%5v~P3eL;omU@|ER~C%gPTjN&!TjZ^cdNd|tex`$9cX!n!Rney4< zXT(51w*40?=B#V5OWO$iL5M)_8gYfvch2pd;~T=U%pQsZroP;J`bT5tkRo46N!#<2 z7ZHipe%n!l#*ECQ4CVrH$PeMt2Azr&J4i~W^@~X2oY2&W$}h{hbyvVcCAQ+%^&2fp z*Vo7igZH=8L!5qdc@OApZv}#fBdQ->?kZmHR1x`vReQe^6!RNgL4UI_`>}U?GH3Vj zhUB!qGjY$~-CeCp5*e^fJ^o>zRrT=BL0G=hQ>4OUOUG#DxzzY!g(uz6PpBwrHZe$W zWp{RnVwdmbCsWP8ue@X>Bv{$^S zBW5!F?J5XWzb^aR1x)B}}AktR(A=JQIwZotdE{h5+;WjA}%4mU@Z!3AHh8Y>a# zUF*$W=r=83^M+>dH@TbB=-U3P8{E{`}<`F3jid@rKES0Phz@q%R&A$3sv6* z7trmO86hG%&RSJ#PI`-ZjaZJNz_FK(%$JV0*vn?m4bF8sfY*?G_BZqlCCIbJHvXQR zpI-aFsD7=^SgGwnqll_L_A{PY-}?=fl>F6fd$dv+U45Ze>T_J$6W)H=47mf6K7*z>0CcTN&flv0Ir-n?QecC&~Ud6E1T0Wr$`YCmtq9Y=H+I2q_;zXHAAe*rzFmyRdrLQvTzf%y)g2$+gLM@3Z zUSUSRyx74$IlmQm9*7;*zBiQrevy-9+K{UFQ}$5-e5F#osGc7C^>IhRU0d{5J)g*C zinZTnyQEIvS){3LP_ip=Mj9)CA z+hMD>LTfg({d`$ZL6l%NtT$S~U{0WuAmgbs7yQ9{5*`_JC&Kh_BFsi>5xAWF%4_cG zG4OM}UrqR-MwnXFpl4*lU`*^apAMUT(EK@}DPux@AbC{D-V z=Gc$dku~!<8dZT*zUOphuFuiVIiAxSHx{0x-7^xMffs38HLod~c8i&Su+7R;>lAee z`x|1?+zdymJKu+WKiToc*5y5>$=onHTSatK`|uXmdX-mqbg3{#N_i~KI>1f(88k8r zxd~0gnNIF_c|=!P2K)E#k7GA#@sw4jYT-Z2G^0l=5c!IeckOpRYQHX4U$=V6UHYke z+;8cO_Qyc454bh8egBM5w~wAvbL?hcajrQaao`v5$H&v=%HAb(z?UYe+7BPju1gWr4!*cqeVe*JFy;#v^*i`sFZxve{Cf=tbi- zxBDbq*h&FV*|7msZX@~kpSA9@IPY^KVd&k)T<4}`_xM(fXLDl<3y4*C_GKhiY>jiZ zreh@-v$40jgXJT=I;(NgqwZC2baL1j^LE=j;Feij7@=%re1=h30&QxwmBs0UIDg1K zhmHN=-Oy;5E&igQ;x2yC-O%^p!vYE_yy(x-->~_YTA_LE6pG#~Q$6)jw3WD4B2ry| zPMID}ZkN;LFZuR2CFF4glx=fnBPUovDrgG}_?&`#43uf*0+t?S=52T= zzwW%&Y$v+B_J7@#9rUz zL@#2@Q6KlFa(?<&g1j3@;k;V3J39(!>^>&;LJg2-dsmh~k!tIA<94+iZcHT15auHr z$o2q_%*JIr&0$IuS?<5~9x{<0x}ROUxQI#(%o854>ZYfbtMRK_rjk<^l^;#u)fBiV zs3T_0sQgTRlt7KcNO;(?_E9DBWhm!_nDIvSxqq5;A?!(6ePJoIX;cmSO3LkQ75Cs5 z4I+D*Zy*__PLVv(<5oAmHS^Dw=R(qkgys!mIM>l?15Unb_*4V1x`bvAJ2UFBcK#OM zxQ|g~p|Kw|0tlfnapg=+q+4d~(dTA*@*^gKkg9kD(;uMGk~W1}^7WHOeCN}r$r=e* zJ-wkSZ*e`3`g`B+FHpimU~SH&5b+#^@_2iJ%$R;kvzr%y)8L(Az#yzbFbu*nd8Fje7RC8BH2_Abe1lJ@c?Sz`0%+;5cLl0Wc$&_1#IU$fi)x2hZdLsbb0mt`w<@6-nct6NW9&W+2x?iss1vpb;mikka0(J%A`(GI(=CGTns&~$D+%y_6 z+WTl|+wKVH_*vB#sWkPal>M;#SrH4I$alg0`?yufakWnjTT9)u8{l|ocdVH$8x6`m z_$C8d2qSU+6-~NJmTj$t?K=d@ZWo^Id#f0{U>%kZC@d$`xvT=RRVz!ahN}~T#F`3cEBFo5%z^`iCu20tbW~tbS zkT2-M=5-F`ejl!kN=#u^#&r^id_n4n+L$!{a`2RPtg0`Yo~LG}?@~O&UM&7zuvG-e z{zrmB#4{m^Q>~$dW|PPHdcTlx>Y->N$-K~TnXlZN+fD2xF!L?ZQzvAx3Y4|Wm9;>_ zFPuMR$gjLB-Ln(}HRSJHrm&|xA1I$HDhgc{?8&Jwoml(+niGB27+N?#F&8th`Anz_ zkG*fzAjU=X}ttD)A4dn~56$SSF0xtRkH z8IJCk_gQxVyorbzGD^G1wre1WbN&Um|*nzODw zXx_CC1Sg)g-aS1u*`c`=1lJeJ2cqwf4^ZM9Fe2cA!rgDuPLg8{bOx;~GG3O(TNBSu zmc9*@X&ZM-jiL6Y7=P8{OmC9XOK#I?{lnGn*gTtcdw2g>|KQaQZJvjAx0Z?oq||ZO z35v;-f+a79%e4l412f8j4RV=g)GMI#3{~QrW=NtZ?m}nZG0^*=rKRrKg{5}k?-n{W z(;^bq*%I`3PWM+w?YLf#?^~1~-=7ihe}(qu9NLgNm7dCQ0G1b?+?*oJM+QfxrggIl zu51CE?+%(Mbr#Cq7>kie$1)@~N4ltd~5Cbl!DWSiH z>ekG+RO`wtP9C5!6G{wyri`ZaFu9V3&KC5bBdD0?XRRFHfM!*ruj4lS*AK zf8oFX0ety@y=I~4G6FL7XJmq5q z&EQHkDkF6STgOM z)}EQq1p%Q;s{niWYU*b+r8JL=)s10{@63BGb-8t{#*Lx-FJCYxp4XAs+K0*<+WNMJ zo5Bx^R^s0+5&0c9Ya@6m>#!e8G4v>HB^$6q)8ubj+yt=s#F1bgWHk3s5L1Y4K) zUKSN?-l6__h3mc}B*s8N=|;?sD-OZ(VuRJWkJk?;u2*TAR7O-j> z97Y~75Bt@LMy`^a8m-kluE*cvolQs28yS0qZ;JudzB3);)R=J4XmKVtTEx`EAb0g@ zh$-rXx(1GtYEwpvuNSm24*DxD0a6?_!9kswpuyelm0(dY`q?)>DADw@+NL}Ns{O%P zR=@rq_h}sWdk7|W@{RMSRx>&FDwsa^^!dZ88iHtIM&ugHn5HM2Ec`(EalDDau?ju2 zxrI`Ol6i-vMoR0B3{D>as~2oHnmnC|QMzn&80`>$I@Q zWEK^_cNG1#*)q>QzHX|R*J_#OezV*idRwvg7xS%0i{xaZ9(x+8*GsVs#>_0SnU;R% zpYEJXx-7ukf>|b)YA}9As+Rs!-*WxNV3LVd9B5Pl5s~R&*5U<>)uFIP*Q#4sxRZG3 zlPZ3$6bKTDJfRR3aG|tSQADK{^-^qZPxF(rwPyB^eH_ob`4IS(rM9##W@$}W@>fC2 z3Ep}Kn;bYOWv0zeHA^c+ z4Q%)tbmX2u%f)XK6>f3Qbo$f%ul=s%=41Ab?PsB0im_J`e}84v3hkZPvv>sB1aru#s?&g33%XTeRGcZ-`MW(DJ0?9Ip{= z{r1OesIQ#{kTeBiYAajM#@$ODc{?7`=3Z|XhL;y=#r$>dO@*=*kYDk721jU1u%ZmtA-+1XSD2n`4{Xx#y-p5Fah#v0emDE7r-@)m`d70WCO zF@ja9fQ+t~&Oi@?o@&b4(4_N&3)YIGv!@Tsg!?StXYr^v@t}C=Xm#~YbHpkLmPYjx zfBUXAB3P`Gd=$V>O?6YQ{+Xira{2qB>ECLp_dKoEcdh?8e|G4O5lXzI!c z18p5W(3xrz_sXj>7o@M@wzK0wpWVhtF-Nkc&KxAKQeD|7Cy3FJ@~DfGpF02bkPl6% z?J^B#R`0I2n!Q#W!`HD;S-5f(#VVGN`D0N%aF&~teC;$mK(xMUk?@Wxlg8Ii>Olh0 zVxQj;WeGB*S5G>^YP|vS-p3rmflwDV8n9n>J*!(UV-sRgPWib(w5-pG6f7<~_JAhD z3@&wyUWYA-!wlYsDGf)7$WL>-05-3cObOf^O*o?QV}IsXshF*d`QPy%7|<1wYk6^1 zk=j#vc6zx~FQ?9L)Q$7Ct;O~9)y@>Cc$4A=&>FJk{`P?U{0CLaAc3w0XCM%W;DrTe zk2C4vF+%~?Hf+WY|EF{o2IIZ*1un%e>Q@{m_EqD;-Yh6T3}rYd+2#T z4J1!DDvzg@Dj(UEgLniukhM|L?#)Ga2{5MrA`b~Dr{XK(;Y(pvZsHmbTMaVtd@8TQ z(E{CJjA4=y58<(lh?KM8q}ZY3t`)0{$DM{W{CbALC+@WEoTTSgyFx`R%C-{BAenNKp$ftJ`JSJ_ciSg%46V?{H^J0(aLq8R@ zCv0I&JcHwCx$_6_@9ELUY^PHsT?V9iKkHFz!LvdC*Z@Y;eEpg)Th+D~dFB(cTQ%zT z)j4ty;SdGmz?vb#o?AMKENby5t(eE`toe;2Qa>~U%5`d|l;}Q?+@zY+S(+1T;^XyR)8da(- zs^Iomus+B0CqvJ=+ZXajuC=l=UvXIGYF#Lx4O>3?OHQY*$cWg}6iK=732yYM()V8a zPc0IkkLjm&I%4rRm6SL3ZV8VWKIEf}X&E}BOr@~?PYO-Jv1DjID z(9<%|fhUA{MUyVP9WwiQL|Km)87Y^k`Q^Tt)AcM6yc&d5=u-#EWHF25t^yfSN{ zWd$kI0`I@n@JZ*lm#8y03O6gaWd5kr=}X5zT$099-x%z8D%y2FQ_Bm)W56Fwczzc7 zM#N%*UrkCz3wh8hI{%^9F`G#0*-;r!Fa z)xhAl@FVUsS=Q>(TNg~H{1MLP)wH=>C3VM-4d_}>-G`Md)8dyidMFN?~@^4myv z=v*1C?6&X$>u#|&Ix0s;U&q1gjDB`$HvWJkcIXo$eK)iO^D*I^*MVvRtT@y^oO7Ed zV(e0vSZ7639mpVM`#82>zX>(@F!5q-V_eU1GkxWZ5SuJ*b3O~T67pDjcK7nbzMc%K zqAK>2P!Cx0DEvfS+_}(Djn@q8;U{Wd(T9W?kDtj+wbQ_%ePvg9pGchS^*R;@PI93Q zBirVX+3u#l01z2Gr`ZPONf$rmpq&3*5Al9|Ml3sQns~bhHILp_vOG{*`ye)AXwrEa zs=Pij`q>`3Q4Ys@guLBXATz}Wqv&fA@A(T}9<<0aR@X(e>8%~AnfHGY4{`cTb@;h6 z^FhQs#Y4*DXT@&JKkyRJ=^Gb{(~}Vvx)vS)s&OX4oKAgnWth-h#gq;kEgfbY#bK&7 z2a1}KIw^_*LS{8i>X0PK+NdUH5PcPRd>Z0AYU6+Lor!phk1gHb(j7?FfeV);+m%#M zq*5DI9esSH?{Uc*DOUgJ%bFo#|MS?KmlhsuTQA+;pE}!ixzd!%{DIp`<043@eV)r0 z6BZ?r8+K!XL2TyWp!(@SmGn5K%sg8d4KT$;0e4=Al90^(_1o26jhM2++<09pjupki z%cs)p_0blV-QyKA-8vrY6e?~RxhgAGV9p7TTN!~i zL&a^eSVP7M5S1C^Rj0`8mlY`$m@k?05rO6b_EYJwmPUfW+P6?8e+M%){EG=kU}-)lH#Ng59}Lc@+@1vvz0D9Uanl zs<1>UV%`Y|YST|ITE)6p%^CF*?45XCGf7_%1F(UID;D{-e0#v=wS7%?HEG$;lcrLk zA*1Xb1${<%cc%O8lqM?BvTFYkv@-uxgJPk%NA(J?`?r!yhatLyFbal1DW|I9y|0(< zrHV{>qtR%;!u<27GE24HHpp|YrJYo~o#dkNmGDUgs7FJr%J5M;Zu9f}tZ_=D;W*HL zb!hc(5BL>&$4x9#F^70YKdA`#3Oqo0%VCN#2wRvDp}K|x9ozLXDH+!j_CD)yBM#5< zZcg+!-ng(qnG~FyKg9j%sKR^;Fs4q1lhg&5Eww7mVfJhrjW-&YE73=O$g{vf+ZjamOBV<`pKVaGbXb4W`vFPh6bSc@VRj96QCB~K&|fTk`C>g*c13X zjkT}W(khz&91pP=@K8&%s|ko#<$Br)JP8rd(fzi71u*`zF_YOq=!KJw&v8}iUEU^n zi>v<`wS}A}jay`vLdDR#mEPz^jh~zZf$F;x!s=(j?9!(%zL~>IhVD+t+iP!RRO2zx zffS!JtppYR0tFp8&1P(__(szB6!Q{Iue25hkrv0)dn+kAuCV)OAuV|69dBK~_YBz8 z@zleVeFc_OuiE6iTC<|?OTdW@tR{WJUq{Hed`hC8rSO1$l$~lMO#^Q*ZK(7?nwb6^$b}Q|8_q!JPN!3R} zA*Z}v%rY1H(kvuI>Rz|{5>R3_XO58wRl7w!T`w|1{x?RfxpGca+}Ytzs^G>gLv0If zz2zHDQO1EovZ46VL(N3hYWKDus)PlaZ++!ASQ+oDHWMQ_)!*{5ys@$XzdbCTx#XXv zZ>WOOv%}W2o^a(UnGB0B>9{t z!z{32K=0T(hBnYhG`B8LF@=Q5vCDB6*(sBNKUn4KVBGB;5*k{=a6Z~~$!DomWtraR z8r@OESJ{GKTIa=(%$`8*)@0|azQZ&Xkh!J4d4O}^1(!<}n#k2b5Q=qdm#Q)0@q<

sib;NK!WU@V$-m}bRS zw}+O0#K`Hgp-RUbo+EMFL+T;72b44v>b+&eu)N?HCln}`B4=hXsl#uQk`7S+m|!~S zXVXpg55Ok-C3B7$z;$->q0woGQoWM<6~~5FGXhUE%jhJ%Vze#R_1(vEyFpL0=xsaM zn>_-fwVrMqqt0&@kD>i(w==;AQKa%phQPu@mvipvqT}N5)V!?e0V}hCQo^l8oR2$t zZRA47-Z^p<^6_d4FtLiPHsTW4*K^o^)VH`^GfFpjQN+cE?iP$JTPxqPeMhXxnnhX# z7!}&-o%>gq&S&Q)7^`ip@Y}GQ>cK0X%4hVfBI9yXhovREyZHktsco2pJi(*I? zC|{Vfih4>f@Y}od;VxhFj+mL|!4hoqbDpQ+40n=*U7cU-?FJAao`Ndx{;^Y90e5$D zcEKt>@ua}qdU!PN1`k%?BP%f-;}UdrDnI$4K;7tNl6r($O(2&$sK{(GLbdDE_eJV0 zz2zMv{vA(IA@pt9M8?3ya9jykmij-m01lf%=|(%eQUPNg)x;|Gyj<4pYpts%-8~S&*Xca5Z$0WZi3~?7UM)bmVq` zN4mSS2MN1@kF-fwq70^raATqJ4V7RYIFT)6{a_}?C&);+2D_+D$?@ovtxHc5NK#jl z!rpv6qgqnXg>5(J|E!w4)+jQ2A8w*_`X@4UejFD6@8;6N61L7zVBTX%*~S^3&Cd}{ zh>_lOLnNKrXvJ+)+g^4&%^LLKc+ZYJ8 z&|G0*Wi*+K7RdHkm(;$=JF=l4daK6ADdt!@$13$fpAHe%mavPjN1+3~FG^mp6stc9&XLBC*2Z~@$KnLQAW%(W1>$%b%Rc>f z4^`Jz?b{z~9(D1r>BxAVCuW31^|rzT*%|4(dZ8vv<+am#DB>{x`P(BlY)V&b=?^#e zkz{S;-Ljy=yA3{7mWT~kKza4y7_ymS)ivGEYhg!&o6~IcpKH!c$9 z_`0L+w(x{4B>tZMs#1S9(MX1?K}mZ^JH6u19i+Z{HO{RsE* zJffD0&UNm82zlEJn7~e|O_jfDqTj)qm0DPzis*h6Y0DS0HC`G$sK)BeNtsxCx2XCL z6!?*3e;ndI?NVq{AB)$jRBhZ^%aC6pT02ow5IrR#E~O)ZM3(h+FG6(*$SXP|6I zd%=@I8QCGx+hSl*Tw}zQZd$-6vX8%tD_({eD{Kx4l9yW6>kOLpZ~wwtJg7AEMvu^Q z3ifR2*L(iLdUFpZ{ZXVf=$uO1-nzF@>i+3O4({1N^8$cx730Pu8B|XvzfSsFeZc!4 zwtC@)W4>n!nY>6|Yo$dv>eqxcf|FBlMe5xm`-l_?Q>Z#`)h@{XL(7}s_R&$z?WO|e zt2M?RPgmHyT8B}5r=8o~^&d|K*hNi#%Fe8K{1A9Egb)IG;xTpRFQpkp#(q;3ls*8b6g|L_t&I$$SsOTF`FwTWe^$GY0LiMNf)}5F+ zmrPOGPOj<259%S4hVrgJ>PZiTPz)|??y>U~`&KZJjSE<%7(tG{E4byt@i}LrrSq87 zGW5R#i|6>lX|X(&MAf`eH*X-P@{++lhUVr*m`-D0Zj#7r=g<2I{a-mo#0_r+OQ<+Z z{BU%>B&s7We_Gqt{tn!oE~GD{BT-KO%I{fHxNbPrvIp80>$Iw+-|dMktqAa{v9PT> z8>=c!*^TLsNrB$bmbbmg0+c?R%8=Sm-#hKqLDQ3lOwcN6cpgZ1AJGYozJE317KK(Gir^T9SVi1>w&escelzH94llYhlD*-B6XbdvUxZmYJ zYV>V~#B&Z13@_J5kiRA3o?C={YGq=VIg6@$N?2fTP6d4dlwp#Y52ZUco($iO0zw!I zLv4aZ*?A^QJBQCxT6VPoVz=SDO@4MihcuQkFy5CHl*;8B!t5I`WZ`ymD8(|0-zYz{MX!+EB)gL(1T< z`2W~ql-~W(`i8!pgAIoVTNORo^dtG3i!C04?~4B;V&XZjsDtYVzszcLY1!&8eTgGh z#*tiPrmpv*lXV-U1HnB252b62|Jn_hz+oyXFo{>I~a&pPhP;(-c13Y=Dn=$k^eo;|21Ec2z1Ou z%IOZRJz;dhcWwL&a-;y+p()k>zorIMX<9jCImk$-e(oup!J)^-w0OZLj|-tXw9r@K zkoQRYi>)5RF2zL(dev)Wly1(hOf8^r7Ci5TdPJ@#T znXGfZ6zI4P;$I{N7<##xxP@H3xv&3S3<2EmaI9=pLlj2eLtESY2J?#fqlm%?OAV)} zl#1W~4#xxwcj4t?5iGCQ6R{Aa38~h=18{I?bRvY#`4JVU;f!VFQ;kh&zn)MQe2KHY zy5T8OX^hHve!?bqxZ(rfNXrO4)CZZU;oEb2b<&D8TIBa>mQynbqW$j-|F{Nk1k>R_ z%mWE*Y}x#GYWns^`P!-ETRETlTdDtNT~_G>drikD2OocuD* zFY-6d!{NyPetF;v7oYLS^`ORMyP*3r%M5sHFrufWb?#A+GVwl|lyha|lf{)|^GRq^ z{R5-Q`+TLBs8iwa1c(l2yqxyEYG<6|O-HN!3LC$9xZn57X0-nu`M;zZ)8dp@)!mQ2 zVJui8li5xiubgTAWj#lhIIneGXHP=1Ub8lGyLjCO=6{O1PP!mLQSG2DN5>EJ6| ztu;@OnON<;V&u?+5i@(6`PaX=3nI`T1V=UoddSwLE;eWZG==N`vB&$0tp(>}lM}zn*7XPsWaaxBff+e<@8yK+~Edggla*elF5C zz3oWm+{FEtQpIF(UMEmr8M40jl`N^Q4(H9J#QtMib!qL%y!|(CHc{Wb5Neox<2RJa ze+K(U>u{pby&(k9fbIs*(<73TrY6_9oa4mGY35aBscSX!b?N-N+=Q>R4sp70`JAxBuf`bJC80gpmmE?;Y1VM!x93{`oQtOaH7( zlz5!8SrTz@BTK;Z1@>QiGW0HdBFpyYZgYuU&M>x?y2empZ{vc2g!A*8Ls>8B>l-=6 zXkPv>T4!tDy{gBab$p7UO}?`yrdiX!4DBhJr}{SJV9O+-gGz zp_m$yQ@ZFDlB(HQbZBmk$?J7XTlIVD+zp_8B(pbo{D0V$V~*M@8{Kfx-bkch9CV&;No+Fl{wo&zfcK|g5aFZxR_7%FIV=o~kJ7c5b$K6bnblrU_`1;W z8gMH9Crk>e{A(~+dGz2RO)4np(~w(l9D)jvJf>+z>c2$SC?rGk7lX)l-@U~1HTA=06*?pdXbZFtRR{r!7b4P7l; zaPYzH*n-;?pPk?@^6xhJtV(9RIh%FJ#nydB!GT7en0NE8WN-{b<1hF9A4D#5LLJ7b zW0_7nzdWNT?zjKmR@SZ3Vf|`s+{wX`xo<*0bOCC`f6@Q_wQb4gRCi0hfCxeDj*(`Y zP42V(IaInNL8K;qLnaLZWu@z04#8>_%*3=&M4HWqgG_P)n`7me=uu z>9nDlan$55w3dR(RvX4I3<12SEpDSc*|Gl9QO3$zMf|^HsAIi+F&6A)DbkdTclYVC z-8RllBJZHN6}_raLOLgTgmg(ek9x9H9u2;$%U6EB{AbAhPx|zYzB`rnPw`Co>f0Uh zX3kqDVwtWNA>K)z&HBqYU=Qd8xuj&Yjm+(Ki=Iha-QpMR@5UiGmp;xK3TdScL&qn?T)}mpmYNYx7z8Vud zq;8Kh+TsrSkYi~c{cAX&Cmv3Q6-s6U%ivK<9i6^6{PD_jz@xtyx#-}37IIq|)lc95mP zZ?}G^($89WhV`WMD6k2snODZWyl3pU`5^nR#2z47c-=1aK=xRr-G7HPMD-bYp9IgI|i~6oCI!W*}+vzL@muUadIUgf`gU?)tyH zt^fG5v%$fw>J=icOnS=q1I{Pne2N`}AqR7PYGwkmHaEav+J2V*@{&J;%lRAp=`tO& z!}ey5b4QEO<@v}U6AxXIx_}$tE@!*PkuGCJW+obcNCGolz)XP=+L=o^kI)H z^_<^`cT#Zc5P1BsBPfXxgWOnD--?B`JE$F91HW1Z}}t6A5dpx z8|%A2+1%xf%d|1FoM4eWr}3L&yTz|>pi9DUFQLZ+yw-~Zj`W7sqhFa`5u4$@qV;%FIwbfw2_X$~X zQDz?I2Usq1{4R%SfCUoIHh&WzRhTsW^t@ExnmYU^On(yl4}>M?fO9rzIPLHLKK~|# zFyxUHS0K$Vjp^Tt?w|%NU=URLB@=%_4e+?pL+eJkz?7baHl93@auab>Q`n~D?|Fj9 zs?luAa=N6uZ*<4BrQq+s)+?CT+dQ`nTnUctLH~WO@oyA=aN{q{+To%iwlQQQmoOb5 zVnj%#YZJ#Y`JkP$4$SD_9C3mB67%n(!#^OEc}UC5Tb?AW_Z31;n<|4tFxs+y{mwyH z;ysH8zxy7<*#W8fg9K40P+HH}i%WSQ+TWIzR_ArZR|Vv5MKmrX6TcJJouS(Mz-Ryh z6sur-tWVl+dnysF`57sxQgrwWke*Par>!29)|3?8rs)P(+H*D4mVNOn^#7Foz&5qT z{YebCd2*u%1)yH%+<$txPN?8RB(ow8xd_RW_d==cyJun0WySapGuYd*54$wW+vk)h zpd?NHj2=@Z-Y(_I8o8WUTkhBgsRK(gNxIMEwCexCCVv*8S>*M7(12 zX4#!HxGYU44Ga~%!Y&$3vE7td>bnq^g0~D9X}oj&DYozqJlHlq13&rA2Zu#0f&FQO zAsvMZq3yjdhh2tVNAJagiJ$CNut#VUMu|9`N853jF5ya?MF#CWJzX+8rETwll6`5~ z@U=Yngb;vj=$gflu%}0LzZr+jy(CIk$=?#}2AyxD&D6VaBHQKjfjr!40mqWY z3)v(!IPbk%NxhZ7KG}e+%xFzS6N*p_$yO)N2{K24DSpCT4rg0*BiH7vwN~c zcX0}tJDm`E<1v1MMM_k|F+#PMdnr&%?;9)i!5PZMAfP_NSa=5DsPFWqA)4JfG>v%P zh9c|iV{GBw7ybqy0f4gJ2=O>9Vjp6#cH2*zC?oj+6ts6cL(#sv{6N^mPw2jpkG=){ zSZf2Yv@bdH4nV zXD~Hn5Qb1taE;&8R5SdQBMQ-hfjf8oXkT^6vQ7=0@oao^n%-M_3NM~KcP}5w#n!pB zZ3LdOUhz&R5F*_7{m|?V^$B2qge&G7ir0%mCm~Il@u>cBYRSwsj!ODuiRMm&~GQ$2|II%E}4xo=L6cg-_dEU4I97ceI1!LG#WF(Wp8jy=^)MmVk|XU z^u@_#xFEr0zVo=O`}E_|=ByD7*gLv+Ruw2;%%l_Gom~gr5XYNH2j;TQqZd^eB$hW- z>sYV)tX4MRQ9*ly^-R?v1YJedS;IKFP5 z^Y98gSnL~H&)elOOn1=YiwhrChj~%QpO$yTHsheC;+%^uA7$-5!-M?#@znjS^RAE(TM>T}3#&g-`S!iUXLS~p* z$B3Nt=yYqCSx7dmfc`D&@$7Q1CD)R!k+01mwBp%M#`OCzhN3Q5xP@(o^b*UO3dq#< z8nMi0O*RtQ7R~;~dGlGTk)`FARl-BFBLKV@>fnA5xvlsRXVRW8sLhv`g=T&}9BTCDyU zj*{l8uCEV7kF#_oNLS8c$x+OF&$E29FhdFh*y9>e%qv%fmduY!HaZe_rd-7XzO@@; z=HK4UmBjbSO|WXMCVkD2GHK*|MXoilAn#lcdp=Hnr3M?PdxwhL?nRq5km`$(eQtbV zBQ?8q#Dka`m+s`QdT4@3P3|C=E}XFQ0R%C;QfJ+hX351UhE&GgGVvY{LN2wbE4?8! zW0f;xVZfJ(eJ_=VU7LLHbseho&^zbdx8@%Kaag#dv{ZLnru@-m=7aIj#g}Q$OQp(Y z@E1BKL6glp?F39o%@l){7WhN$;3kxSt)|&t>SEoFVT{RG-&q;Ll4{#X@t`_nV`9ZM_IfcAbV2sd1&7Cgs(AjlV$C(QVc@;)m=%#sSEqd-nQ%x&nigmX;u}x$Bgjf_HGrFM~XD8H@+sw zjv~mhASSigP(HU%Hwa{5b@Nm$E*t3yhOe*O<;(+l?z~VY&0{&91br?lN|bf{a)Z#1 zAm*`p6Veke99nJQe83$EXB34RWi$HL%3fIdP2BVSlA>)E0)krFdf>&%lNwS~cMch( z#&%Zjx(t1!UyLVdjp5(KaHuRb$*Azo`j)*lo_7WcTA}zaa_`L6xWKV;CBqrOh#8jC zjIz7tu2v|}(H;@e_?8W2!umJi?COBD-ZHUN$?A!qy|2R%|JE(D9B|prmbKfXbZoY& zw@EyvW-@fDd2?EsBkQW)Vy`Ko&n@Q;g?;x?=94GcDw^?eQqec3s4p15PJ#>)Xbv{?w-H7 zCkOl`RQ2)@jlkmtECTo8*>P`l{wlk$pMz}61kDe2VCt5@ zF8wegt`W#AvC!jyYnXAisJ7K|B*AqFHW`XSt6*SGtDwnwA z?nN7=cy*!+`;y4FYK1+OjduP-+Mf#6fW}6%&ifpWdhp0QCO6ZPYERVb@$nBxP?b_2 zEI-s69XcPoo~XSD3@1!G=3>ZVn>#5{*!WhEP6^*&|A)&V3Akz&p1JR}*vr@|?j}MP zGC6MADtFLYR3)HIM3OKy>p76a_jP@r!mM1)LN#twi$p_rJcW%)E%J7!JP2U&61foj z;YzidlY(CNYB9>sek1RAg9c^?V~ZQ1exp~H5KA(ag<=$a(T^RkeF>eN z_T#>H@1Cs1c@gdt2DsrIVw+aNDGo-d>+!rbr-lo@7Ct{0<~0>Y>A))DL&d<^Tj3V6 zCgwSrz_`Yl2HonjF?y-C(ZZLDe|Pu<*1y#rwNV9!>@x`q|7h8l>@Zqrxv=}?55&KBbAUH-@R1 zVoN^G(%FUW*OLN0@3O({jkz-WrV8h~4#EPO!IV%Wig6vL_`a^e;AWYEXyyGb25^VqY^>Zqrv#({7 zXu~v?Kd_8A33y&U*(R*_(kAREb0so(Q7Gf;{l*eQ%a1riP|re6nnVNYndM3z!>30L zIxSqI5$Y;e70u1P!aGdjyfdWS#3iUCw^w4My~>^m=xUTdHqXt) zDHe9rNc$6HU)Z1~UxqXA2$X@oHcut1d(;crB)+s{&@gjQe6JxOxX^SXLnn=}6IW1! zPwSPD8k~(cNMqdfVFG47X|`}g4c+1K>-#1GeBO(c{)=g^snWu9h2W-xd{fC7&UelBIMF zi%}UPhTl52(&xl)NwNXcf$6C#>~6EXWNU8Ssl$MP6zsmiEV%R%oE8^tIao1gQ5SI( z>UnnhX!4W_snW%;E*DvxjX6^@#my+hD-*PuTrMR2CS+2W4VW3+${nQcX&yimpzKbz zR1>Pq{B#Q}mhl)lCpQ%+<$TiXa;Al-2L6IA7CTxe_V}&Qbc`Gr@wtnCgJ;O`Lf!N< zi^)FX^4aOg=MX$TZ)Orh&?O!Z*1r zaOA`XORnF<6>ebI^$sVM2zYI9<5X#_r-%NmBZhch(OLoc&*|V&>}1o3kWM=5y3`SG z2=3kD!wx-ap-VK`g!IFAY!ZjQIX$*WjHwtW_Yo;=Dj(6~OZl^nO0VG8gDp24V{}%d zNv7^gyg-LMwB_W5scfYH(Q#ymQnu<+j0C%Ie9e80s7OK`gjwn8V9KIH95q|_S3U)s z8#mg_g7LO9Rx^&pAzy&gW31dvNc!tE#~ES-Ew(JG*&Qy?Fk>cmDvDb86ju zM;?H!5tJt@)ciVOwX<?UogJ(Gm&gZZi~8*t(8U}k7Lx*JlsV(=*< zu6MD|yxI|z&a$@6cerU^z4*aLR%0z$g3ql2a`=Xi)<+&0K6d!cNU<={j2E^)zAZI| zIP5uhAM*7pK`e|nPh6PFYSkB+-PGMnRyTPA!HBV-$K@;57t+8ix-eqkB(6k{Op17~ z#i!`&EYFnCtUi63l0^{YKm8sv-lQlZSRV%IPL3AvU;Dl{ko83%hxrHi=#==!Vn>o7 zH*ua)Ylrd9@z|cpa+|KR=A~>gl1m8J2`w-4USRX|ht5ROO-szH#Edk?J(w++cH`Nt zyLY$z!Iz@RTvg2q0d6@pxdOIpD7VTUDTH*VjeIa!KIGutYvE6$*Vkmjdy!2s;(pO3 zww|ZY7;C-)E!}|u{A8z0^P6l8D3;9}>pefOls9}v-G1Q^KOM%RA&3vBpL34qT(J5y z%XU+Vg-79aX5o$T$`~R+($jbdYC#`=w|~D(yyt*k_H?a;{HZCYsL$uk<5q ztmt+UZ^UBmERvTE&V>boLVBYXO=lP&djeRdcvOF4+RC)b7zUk&9| z^1phL-Lm`eN{u87j%+E4o|RFXc&rtsdVziOu|(RKAKDE3hHwzHH6JZd;Y5aNsHNCu zW@fP{`@<9CnN2JxEyMCTSK&NY9aQ)I?v)p{6O_%SbQ# zxCOWH+U00ntTlGxGUo89+NR}mS0u2?6Xn7Q2Op!^RW9(Em46U3>#$gvetJR~C0d8- zIXN!KgL=AhmvEa&Md+bho}JlXqy;an3M_7cg~NjYBC$YtUpa=fVm@8trtW_E7zjYu zYNj45v-fA%YwinujuZR8r#&^6DD)M@3HPDKx}?rLM&&Vd24uI+tMd$Od@Uns7VCL# z8n>T$wvT>Dv_|YYq(Qw!+vsr17*VFUQ7c0&$$aYa)vJ(tPev;S;FM_E%RC}dJ)pLL zRJT(CALFrA%ivv*qwdamY)^|%4-W5sROeOrHsH~*shoqRrnfTE($o!%Ydb<`4<)Zt zNd~L|UD1xNayi=zS2+(Dp?h?Pw^lvl`iY7fO?E6p-_H~1_Z22EPc<^mf5a~mB)Pj?1S z&xCbnQ?tw+B_mgANk)eYHKbuMopM5^`z+ zPu-Y2u>Vb>=Upvv9CNXcr=O_z2xxu3AblC1VM&v4Ti{OPDOS=y@3yGYvs!0aQE@>f z|5-norlM-r#YO56sV*L}|9LHv0jPrEsBCoqx;FOTYDvrVR5_gLYteZgjK5i&prDS? z0*X9?1D@8uxk44m*3vaqp(Ov)oylL;lwb}360 wZcY4uN%#l%e)0IfQuuGi1L$Oo)-;)xwwh4TXz@{s0{Gm#rmI=1ZvDsq0L-ha>i_@% literal 0 HcmV?d00001 diff --git a/docs/en/getting-started/example-datasets/images/superset-github-lines-added-deleted.png b/docs/en/getting-started/example-datasets/images/superset-github-lines-added-deleted.png new file mode 100644 index 0000000000000000000000000000000000000000..48dbad1934d5f4214062efc3ba6e051c266c76dd GIT binary patch literal 70603 zcmeEv2UL?;yDnA028aksRa81i?+^q5MT*ipQbn5d4n`eRT0pu~Wk3Y!y#|%u0@Aw@ zAoSisNOC{sA7^w#^53=YS?8>K{#na0hVR?u-S7MC=Pi5lLS0qy)bR_)@$m3Y-Bgml zi-$)_hKEPcMS2YQW_Q0T0r(5w<*wp&yxdOuIp81imby2sR8;UdfX}3O_|I(chz=eB z{#*e5;NcOb;^Pqlf8!r~mrC&Mr=(=5gx@|BbR9gnCLJ7xhX=yDDKD$(iN82TmS|$> z$BS8cAHOJ}N2O3W06d6DhwSU+;i9xq9Of8jkqq-v zmGek{di@a|J^>La1vBVZAEJy?3pS;CWsiOeXdsn>;>@bTuN(Tak2HCZl4;b*Vuv%h z9=(Xaf@5qH%-3!aJtb0T0WDm`^(ZhnMqE7-q{*jxMQv(`kR2fqpwmfWadC08rZ9%? zp`r6eMn){PHA8icjg9voJP7LV@4xRbUd28;($5u|pe;j7?qEo1J0hZB?O{Z5@?C_= zDaxM4+d)?%XBvX7d0s?j@G^e(I_9Getj5W0=B=Z92ANKfS+JR+wwEFq z%Xyvf$y`%ff=Xh-oLkDIIa_*l>5Doo^TfgByE-oeW@01Qsyw` z&^J6XVm{p(eaElT*Wb9xeaV4|P;AG6b*#zcJZgpk) zbH%%^I-tD$BN*ojp|>_L;sqNxv=uZ#fQ$q0XKE#T&%fIpfuct|cfUS^2XW@IWf^X#-6^ z?~y&vhw#?6kE>>`&6b1yYc=6P;VxosiS9P4zFRg6%8NG8nkbo17eOJLv09BWEE1?w6^>EJU7jy&7;Bc=8_Y!^kQgbLrAK&5%A77NSK z+aqb&+1Z2I3lM$3YMZSmC8xQ!lO*k(=q25~_dCWWodrm{qk&>LOJxZQ@y56f*i+9Z5`eMrNcN79ODt$JW5x)_S)~ zBATXx2$Zif(gy^rcyPQ1JJy$WZhL-+`sBXw>CKt|+yG+2z*Mt>ZG>o#pXE(<_bdo# zjB6=)FjBeOTrr* z+2fJz(S3uaq4%AJc|#Dj6C%l@@y(53_mO~>E3x%_P4rY72-t)@idA~;#qR1e8AHjJ zrk?^&HEwyq-!%VjSgZt2uO66hXc$;iV7$H7<9!KCbcW)H)d)5fc3!RC%8J4#Nsjj>~8 z{*{H5bunf_kJ-85!O_ROFZfOuaB^~<8`XG9%b%a`+vdVaa+^t@K=;(wv?iz!kL0>r z!vi8I$imVV$k$sU7i!;u*2+&g8vcm3YwAGD^(zZm1I9k~Q@;n~GCv%=2dzkuV%>$V zN9WbE${<^e$;lBj znA*BsQh3~BH+vGs$;r~oLAdtL&DR9={&H7r$kLIdfP(fl0k6+c+axe`*+jqXp|v5H zzq<}+!0soe#)2kBRJ9gS7B#l%xfk%AWpyN;;N0Pii;HY#?cYDHw~YKNTH)oH5i<+Fz@k zR^XkduPh86xKI*a_wZ)b>7xzMn^aKvw11_r&T-E+gYh%A{qNa3d;9y(8I(D28zYP` zCYbeDxY3HY4kUXM!>@BPQ+3^OxR`wzipA(wdpadLHJ)FKOxeFY(;iQy$mHj`b{~9_ zoJ+6p_i`)g$uDpwtf%j+98*x9-;!-jN_A7ezF(qtJljg%6r4CLPE;f;r7+c*1jEPY zv)DTsLFi{MV>3TlZu(WPfoQ+5LyfwrJ%@fH!;AMt4#QFe)vljCcBM!yL_|{?uUTB8 zN*rG8^|t;_dcXJlk# zme2<{#yU7S@GMV4zdSw>B!Mz8Fo36E=2C*PWcIotll4?;i!RjQWQ4@R`=U!aq@L&n z6O-bvNh`kAYqO8v3%Ja*AHZJlJZs{^>j3WGP7)X5f$p#Gj_=gcEDYp9c4}7q8n*QV zESOLn%2zvr`$kq4Y&{D6Y%IMnZQ5h!0t$CWCl^w*j1i?=HTEz+sP9a&85V%(Gr_Fw zp-`FS{lRhMu&Ib;Jsi>6L|UD{&i0s#h9N+9 z4f*-4Kedi_ZY+dGx58e<*8?dO9sV&OI2I-?UFu%iRK^tGQ1V-fHlxrgG|ba&zwO1! zdm^IJl?TdBFQqX`Dd;;FIAAYkv9gpXcx&g)Bgq!{4p(?Kf!$D{r4oTyN0lFHn^wYo zF+41fF$&R40m_lUWV>ajxJ*Ab{9wx56v8AU%|F;>P+{Nkd3UgeW~VfJxojGKYR`Z@ z6{?DnX-cGFRSDpU>M|Mou*kH(NBu;ss`6++VD8EWsV46TfyWo{rG$kA1sB~w%xWTC zEf~gF9_?$iN}yCIW}0X>B60^dwk|c|!)&}_4`MdjRs=C~KP5U{FY!_6Xey{q*?e=a zFb2H2wEw9yC?l}&_{kiTxd*wrYAS8M?HpGnCetMQrP$e=o7B)?v2|^BWb{Y*jAQu1!*&Y>mJO~ zT7+?|nxhue%nJvf;`DZ4)XLkSB&>K`hdD`lIWxR2)7xN<>qsnN(PNe)09Nn$$y2Uw zZlO~d%MT39&CMx~-?z69JxR8xaSyj6vm3KxC}cVuceB9abG7GM z`xPgElKIGqd@=ux=-BP_OGowxENrGDxN*;+wx0^8heSwCafUWC${U9^fKoe&pAyac z;tAo}bV*NY0uEv7-FZqJeS^hmD9&VV;P?6-jVOSBrjCCR48HXamw!SuZwOLK?Icm( zI9l@n(h34*bBrUM5vL$sfUd=vKpjVuV1S7HN03KF9Vjwgz%$~G(3vAOGCze_yt2w{ zdRxT_)~MAz-Y52Zwu;Sps^BKr^9p;aw2FJX%a+UY^FfV`U#rH$A0Bg3_wewzQ}z1w zJyS)urI6*EHkSVJGu?gC`R3d2g{<`Jr{HhyEncKNrQ)+Z=4cXYzv%ts!lvi8`%75| zsx!2g}M3s*^Rvmw&kywX9oc zag)oiJU=sF!f%IqL``a|w|T8u+Qg+p=!VQpM*>Y!Qj*hjw9Ywr%(=Ojh)%zeGDk>D zB&%XF+>+Nvx@r8u0cU&0Y4KA(Bt#%5@Vf2i>plfURy)kInnCSSlAUk@IIT6MgN z6L*{62=gKkP=CJyY?}$jSrXtPP7TyL^}GSdw!v~I z^EJ0E-%n^{!oyarT0lUuv z*oKqoHcs!o-`UCwkP>zrHw@`+lEJK9k_^8GQS+PEh`2g5keyq}A#6W<79xY$&9z<3 zqnS%s`Lw+~Hj(C4W2aWjyHXAhGv-afkMiZsHL5BdK>IlC*Rx9Bd8*CN^$?5RS*h@W z`q^IrqZJnFkFtk(6I10Y^!aYQIuw~4t~!^F(Q)E~xsxQ!oRtmST-LnyHs>0q7LkY) zriv&G#D6ob*}iXj(B&~!2fC+~==%BQS{)(z+3eF(3}cR-pXeq{Fw-VMRi9uNyOH$qOa7&W zlDj@1U%+w&qTkprb^ZSH=R0g{t16kaG1nk3i(Gys-S@g4$K2!2O-qoB$iF0SEI=ptgmtogF zykJeH<@IV&o|I$SQ|J4-mmyN~&a=2qYG;(H$f_eK@qKPgBB*l}q{@R}n0UtMeV>wT z7w|DnM&Dt&A^yd26rEpWRdHWYcAzI-5I;<~!M5kY~x|BPKD`o}4-VJTu03Up@>~1e2zlKred)?EUG^-SCUz)Rf%n)o6&bpxaw-U4tnaSX{e2p>zJ~P=$-N?-xQQHUcK( z^Cu?-rr?}e>1AuJ-a~JFc@eGDK4I3i*xKC&-gD>Xw78KBkR3bJ$Uc~x@1YHBiqHgO_J_+N>pu}UaC&;1Gb_D zc_(8w)~5(q3bxu-Tjy$X%c#um6Pf40+Y7~OVCy;L40XZH9YaJRe>|kb4xE?R&V&x>_drXXfGdvL91mS%Zqr3`i zS7X1)9TRb;{_UiuMN2_5W@dLfi_**5FNS`k!Ub_I&j$quHfo-$n-f5E;@nDvq+^(x zY9|84w$cf^rdS)`{25v~2@0D_zQQ`=@=;ocuX(Kl0z}Qsw7|Y<^v=x}BUZrl6Uq1y z!MwG*Uhrr)mn)B}RRiU>>0yTNm`8TETU)6Ekg)+3+3b2UBS}m2VV5pnqi@ZXJ!%&N zK?%HA-KFm`ugv=++e~V+{Z;u97LKZQGh;|Gn*y~SE)H;TDxJXjkJ|SF|UDm}G zrt57@uE3%>DW@*G#V|;?pG|Qw7*9`89z$4c9s~rT8xl~B+YShZ`JHhe$2*QN8Ninx z%8*o3y^HYT9}TDKGj9xG&yCD;DS5izyb3DK+naq{ZPnif6PruMe>@(RXDl+3w552f zWqXE&aQ!sKkeHcuxM(U1@BG=>*;zJ5v2a;PVz1iUlY~I17s|#+9{&1y3VPAHfp2I< z|B4*(5O#hyT`$bbKM6mSre^krQ|}$c1_0iTL}!lUiMbEONGTtqD&$5cx58aQoDJ6t z$n^N1zC^TREJ;)Kokx=4(M>waj;NlCuoR{~iW~iT+v$78naSn>e#@vTvjC%RT)Bo| zkm^S(T4vVW>}{Nh)~F5Dm1;ulQz})qWLa#dN3FoR z*UI^G&CvwTUf9}OFtV}tksb)^hs>(+BLYtu1>%t7xlye+LO| zuDw(9%hDD4YVYV{kQjhLHkgwfu(+NK6;ADAv)N-lVMe)z;U0U4%RYOSbk1*NLS2Lc zeTn3y-7ihh_{1d6!gln>kts`1U?@QpO)sw~2W5FJn?s~ZJuR*WQvCrdeq!!o+citr zoQb_0v#n=#Cb{zJ+(V5gk54@~{^`zBqCmMJp`*xtZ%Liwa`^a&gP^?JqS(!N4=sei zqRW}>;vQAbtKJ)Ks)LB9w&UO=Wh6r0(SPc)ziQ@87QWcK{Z^gY!Z>FCIb{=lF1Hkz z2TTiZU3E)R8Ww%=k)$vzK4rsO&)He*6TX+nj>&$5iM(`*G;6WTN2FGsw^l)j|1rw( z^_a4=RGoRA4V*zPGxB~g$Vr}2?}eNjam*{yFw?r>65C(|%*3_(#u^N}4{N9>WT@bm z4W#Gy40m&$2A>q28rwl4KP&Twp5Q4@y|1HMyCW$*0)Vm_b>ylT*r;l%e5tB{Gq*1v zHEZefS#ZQ3paqQwB>(0PAlnyvO&ky-QGn{7SaF55>=@p-s1wk<*767x23tXouTZ~! z?rR|n?9ccR*LHK~0M!`;{PMcX1uJ!C)_xay0oMqK9!TFv)t5%w+A%C|oWA}>a}Hj9zx$2Z`$8Bp zGf1i_d$`~5&c(b61jcv2?&GcSDMF@QkRd>c8_dbp{bE?R_>{II?L$z%r!6I0}?fFezQw^n}En=WRe%Bt6N z$`FXr0hcpE3)2G?ybIdOuq834L@Km(vX_@kyx6)+u0^=V9OANv>X(^P z!&A($DXkPHt;XKqW7FWQ!SjqQ>EASiw#{}L>A%jL#zhL))a7U8U=5vG*ih_;_)xg3>`WP1Vlp&vH_ zOWNR*Z&x7OxnVW)eNSilJv(F-8-w36o>7Uu*^f42^j-a+Tj^?(=+Y@xMNGCmQzv04oZm?3s5e_~ZFCb0DSKGE=GBTUUXV%wwUlF0&28Z#>J9SJPA?&;|s7b{M!n`&*vf2I~k`79)fp4*2@4+>>l zKfPKVEEx;tRc?`<9z%G&_1<#Wd*!!~H9A!5rRSPgRpERB>!>Z0>DA6CL8t##;kLB& zg+|`Y(d$s`6ACY6HJyyk+2%glRzqlAR<_);ba;fSo}d26Vk8={>A^6ze${*RhL$AW6Bja0*iD1SwhX z#;lD5u81A&QxLlx0knF3xO_{;W6{jhV#s_gJIamgpjC{^PrKtE{YSmG0le6}(I{Jm_V4`MDFkvEeBh)g=?_p=vK%yP-;< zFwx2U)~3lpphdfpapT1iJug2bwN8Tw6rECFvk_yKAam4q{D&gy5x;8dEdbhXv^8;2 zr97UwZM(Pmj?c1n%8VGCNko#Nl5jzT)5N-BJ!`>iIx>4QQ3q2NCwk9!iy{T*;;<9n zDtQSCQwO5jfTHeuR13zF<%2gA)&_h^F6_E zcZMlqT+7M!8uU*4%&fid`W-^^rm#>sawA%mXuJ=<-=s3PwKRwssM+@5$R;zSBbVHL zSUuHCvviIpGugJOvs1V1ofMD@%S!OQ3Q9dq%rD+M*sG2-#%vWS&F7Q9y8Womg@C&w zw4mq{%yNZq|P}#6d*bTL3bj z_7PL|m*E8dQ3mR?adSs9==wq4^iw@=-FKC1eVP;NpGm&qlO1=SfINu*isrh|r+C`{$6}^&ugF ze^w8DH?1GP0sC>Ow}#hA3pS#ha59@rlU@b6u@3&v^OazM&}9-c|_ z%i;a0p{XW7CcyuNCXTEIKEX6lH^8GtOlDHQj3f^8?>hcJgnXuEiwme zB2Dvp84f^9P;|{CHjxLf@i+gn${RPHiab>6q#EK3JEvF@#S=Pb*b~J z45;E#)vj!Sx#n?EdQA8^j$*%j;#)|KcbaEsZF))uM`hXw{Ag4^BX=e_IB@yaJ4nW9 zM@|WG@i#?9oEPZm-c(co;qFguD-Uq&3HeNbrEC*sY2dV!2(~Xyge*>)#V>xfAF&mH zEF%!hxfs+ym+rfst^lcjg;?sCfj*9b(>Z=Q7t=iAQW=IZTgN%9U?c<*xOQSM@9nWa z9>?ueESA*uM|YR(FQAh%fV7}ONQ%ijYay_I!5{k^7$0EWC*yA2_~k;>NmJajdM3I2IUW8U=VfAuqxGD!-J#p9=~MPU=5{`&$BlR zL4MrWznnu>(hG-P{TpWvf8_~KT1HzP&vN*(|NRcg=r~}Q)A=@nIK@*9>?>b7=yCdB z^LGIg=6R-)hRZbTz8ut`InI#c5E8R`ED%j|6SKRZ^7p_G2a;pFKs3!$K=EkU0+@T{ zS)lZ;?OxA)Tzy~j1Xk!nf*8&qS#LT3NJgg<6mV+GhXYPA-cE@V;&_N(Efn^l1-(0l z-MUA6&qlJUaXiQcwt-T+G3{yx8;3AI2{!XiyPulpjZWS)CKH|(n( z6Q$kD--^J|fkGMR-E{&Wz6eRdPPXbIr?TRN-5&52CI~x*X=NwB=u!1?y8TM@qQ1U? z*~i_r0?W4kS)iJYRr`I&h(lHO88%TVpgQbY<-{p(H5_Z5e(pY(#tLvkhR$)13DNL1 zm^^fOdHIcPg0Mr_DLRoxAY053YwUjGyh; z;zf{toNQ_e`(95V&E9S#H%wSizCBKw`?Sn{7f`n4yg!f*eVz{C;@rj}0*uduhK2&v zoA=~Ju#p9>xB}=V@vW_L@Ob0>&%eI_D&=`!$jva&8_#I*gK6z3|6SG<)S=e-Qf%K$&>brv1yGY6K z%6Xt<)R}nQdLxepd!Zt+VoSY@`k z0LCZPtc?zy92g?36R*>HRkrk;kMRDdZ3c$!g7lj83B=v84Y5bbkYx^J&Zs&va2vJQ zjh600Ig?IJ6tRZbpLa@P_fukK;(gd5BvMZ|u{sPCkIA{Zxi`d>BK$XNC#S9JRYWGv ziHv*NwkJuj4mV0-6`e89%Ehp9`wLLR&~nON=7Vu|QAA_4Y9W5SWQ_|KgHcVAUU4@rLBGinV7xl9)js&uIQwiR-|CkU2; z`lyqZW^UasKARdB&|H1r!67VOz`Bu)m7XTn(Anm_{+273;WexmJaR6qlrP=F+t|eZ z8-Mx4S%^PwC<6w@<*YJ3>4ouSY(Lt1izpm0v!@eh@BZ9pe$bRcs zc6{nc5CN=u;H+{eERnCK(9ie%ME=DL)p$d))2@vf9SLTWPhWHoi?7FcXVSTz`enl8 z*wy^ynAvJ<8oJ+Ja?_~$8fx*D6cEDu%e9GGKZkrl#kxq)oL~3*%}BG4PZ6bBv8(aY z_7Cp7Eh*v4Q1fvp0UMEc;l%t~Y(M}L%CX{v6HB7|g8M)EeDMDdg9BR;)tnghZyHKo+oG#VMV!%9o}8WfrO>~L}{Hy{Vv)ugVHA}zEohttuwq`c3ithjk(G1?yx zm&py}H%FR`aT$^N1%R1+KA9f&6Xtv~i$8Zb@GwB1O8gdo2^8O+Pp1R|_Z{lp%+DgkV5-_d>Gnj(mQW#EAK{EAVKL-oJuMe2!SZBl! zROcE9yqu$^uv1hbN6;;L^}^elhmWi^Lw^|JAKMcfL8zQqGv5y{e>af7UI3xV03Tu^ z&V~<{-q5~$)O2d|gFC`#rx75aW{n|pYab`XFE&D$1{75iewM&D&HUFGKz@H%s;Ud3 zJ?tQnJ6K@fJv=D;Chuq^1is@vUpR^ zgWU3}G?2(QUmhLe%do#4LMKxC??p<01g-h|4ArDF$k#YGu0m45m~@ssob;2l7f8fq zE}8TGS{wf*ceYT#Nw*2StB4Z4{K-+89s&j7GQS+TkO2=RwB|DPfL zn$Y|&l_JQ+#=8(A;oQXF(ez0i6)(gIEz89RRhG%&y&f0@J*bQ zwRUl7u>x?&oT&sib=~@dy6UPVieCH=bv?vb|7*D5|5JAT-#o-FLmv;nr6u5!5a7m| zrW|M`%K@gdCO1M3An3Gqs?A&kFYHd|PbZ^UlsrGos3>O7rpR zC-CCfYBr#Hak6w|C4}3c^c;$x_90xQOd;^Z>39*5#t@qv$eo^BxH`E;bTE%=R_$@3 z==8UZiDE_WciOX_9_tieNau66Gca6hvwc&s-VC1Wd3z2O;MAbCHV9X%Z{d|>2f~|G z3$(n4YUp?D0(Ab5`oq)ea&n&>;8Cw1v2s8d%lBT<1yK`qdQEehb&d|*NK$0<61=LG ze3t5Jh|joFU!<$6dqa%9L|RedCGhn}AoTI6fb&51-QA6j8|$IxDVv%M zgc6IO_XajdPuDJ)mzF6S$UJjX)Ug}?%qXE>x}(vv9C2A4*-ejWk$kBjZ$Nrd1&G;6 zlq@eh$fP`LM6eOCC@=@`Sg#c!#hyj%TPshm)WDB*)#Z%1%cBDAqgx=~g%%TgjA~;Q zcTk{}wu|8|8X8wa4g!ltN~UUG`(U{*ICIOz^?{2t7P0=?KrGf?hxtppfNF5naxE`V zzz1%fC+G>%I<_rY`)-}|iIY8dgUrkAQG_i4(Y#{n1?E;0Pqsgb7X4*V#w z3U5K?Z6FMCh}PcxRhpY!+pK(6k1>Lm7cEq!N}^Uown^c+Jkb#y{wP3rM}6%j;O*=( z=%PdP%uyhWKYeu5Wsw_g<-VHhc~s0>JvX?=PT)_p`G67X1T42x_t26bUr`gH*abi9 zOFi12;$<%c!iTrHd^XoaUArHbFO^mWP4yKt(Q7SA>}6hwV4q8Htl$olnrV&`T1YkX z0gfz5a(A5ZmQtzyJ{(bJN?e)nierJOmj?+e&zQ+*mHEiQ^k;+=&YxM)1tah3CH2Apb=@;DEE9-2;33 z7c|vh;-oxKuMVM5KXA4v)jILLbh^6?m%dZ)Ltm-^eA~@sGtM7gKZH@TB>|LgE>QCs zP9s_YcGH@w!CT+ayF(kl#R>#yDlhh3#AWzhRY2D(Jf*lvB;Y_tIVib3KXj(w(eX_n zBG6S8ob%mRhfEo`y3iGv%YVf9tCig3=^8~@8I;)#T{y^9#l+0SUAuQ(P^d-2Q|C~5 z{@oDB??fDO*?d2S*Mzcc)Z1VbTPN2&K0f~M1tD~Ara}tzi|_jb8IY>N#~_X3lF+%d zm=2ul=?_#f?9pbImX_WUP0fMS{2MuGs?~Dv9-RR9_rP(ee#xBya>Zv7p5Qc)|C=~Z zcwP7)z5L&DJG5t_c{LzcjA^_bj4Sk07Y+y@uzm)}5FI^C^#DNgw?wS;apls90xtv6 zwadaJIIn5hqXL{uC|{2muD-8b2Asx=1!6eOxHkm=26SlrIh?uTivTj5e9<9`W7eQl z3?QF*$M)j5!v+1f&`#b2BK}Htr+)>oSQ#G=49iV-rsrt7h-f_zEv6nRshyK=5zII4 zRyP>Dxo^}KtzFo!kALF4aR0HBT-6j`m`-wikpaoH$PDV%J(?D>!2igzlV5U1n=@ik z<&I0%eDwtFP1A&aserF$k{)|VPvo>t+Q^c9WSbpM6%r>< zI5U`$!zhN#5Jj%aCA{yYe(yUw9z`2s1S7GmkV^97e6;o?q$nG;&M1DDvFQr7u zZIJr6S0_xW!pNtZ;N0x7Ah15i`GaAak{rDL=*q9T)E9SRo(2nFfvPljAxNXEJ>wtm zJ*rbBUQgm-#&MM=(8aY^Ko1H;G!-lye_PT3m5$(QqV9p6AZ4|N%u!a>%c!E zPw)vzQF}1TFfIva)##M{^9YD3jJPs3ZI)Zd;cD<^&CBiU^5gH?S%0i&QBbR6ofs(| z8q$F^6v2{%vjon9;%LHKpRTJ|!1n$?h(_qaA1;*Z$9KS>bkSz}y_0F(zUDZW%=0f7gieF;Q01 z@87p)N<@5M<@X5)OTl~z|3G?LC-9{=N-|m+-D+N0{7OHZNdVuW1}y%Lz1Z|WyZug0 z|792wXDKe{%Uv77W%6~I$4Sr7_Qhy8{8C;29ALg2V0jWmkvPwQhGacr2LI0=WLuw={cJ2=b?>qr5mrhVObR})v84m^b3;Jv|^=GLRz@UMY8&C-~DNHWh`HY+LmVZVt%lPV`n@w{B$IRc3!tu zd-aET%qUJSw;G=L>1|+m0N@CF!;-yujD6Rp<5CiWfva|yud+&vObPyIWAjAGkcM4D zcN5D})$t&oB0X7x=vzNGS$4o&=%CTjeWC%FeX9r1iIJA>-(Ch zamagqi2P5Y1djl0B|mh+9FiQe-h`7$zq+K377EMgy({&TMgP$AL3BWHJnfgqF#g&d zX-@!FYu???ar8Z%04#n~B5)`8Y5f1hOCEsO(8k7StKoQ(*5{yK=n1^PMsoNEuoJMW zUgP1;N&e^j{@L?3poES7Gs@>1R43BaIGMtI+k=CE0X1Oou+WMi}SJ%AkY61adSj+O+hV=P7oT+(xQ z%8L;=AarSF#tM#-@#J9io3d~fUvgqrSH!Uv%oL$aH@{3aD6hrCd@o9G&B5$3zRMdX zVLB#KG*}E49jt3?TyWxN(E0HjZ~^6g-|)J*9_z_?ChTSD^{+ckB+{$nyUQLa&wF4g zFG52dHc>TE+daG4P|R&EZf=N5S~lB1*72Xq7xESaHxbFNEuf6BbXweH=w8?|v+)w+ zF{!N{8a^=RT1=>8!wV`8h3viyFey+Du4T0&y1;b#VN*8?pO6qWaF0*yn^=X@x+W9P z-Z``rw&M~PM~k~r>?^|xyrJR`E#nic_#^SauW6JDuCp7nxp0yD zDYG+!y}{a?%HV+9wynF)z{x)F`mmvB6nSw(#<4-~ya7mGpTpUjv%Y=_J>C~OavV5z zUi57_aOI3*w&zQ)`0Ws_8_M|X2<}*`EIEkG*K*wXozVFT1H`WhjuC97uuGSWX><4R zV|?7GPDD_4QC%9#@pj!)_*5BaXobN-UoAx~3JvE%TVG(phZQxw&FL#f5_pNHzI_|G z!h-zD+0C?=b~ z4bE=+<$DUyZ;*zBgNEL5sMSm1%dqr52)bn|0ruS(t~Sa-0wt+|ATRsv(RI_22jR@% z!q8UK>q`q0?)ONRp1El->sF}r%#s#=CJBC0x-#3FHFU*3{Ye#%QtDYYNK-PrU`XN< z@S7T%oRL?~GjTe|uCanC&$1Evx3*_l1U-z{24qs?Urp4=B=gUxI#uIoCe zCu#mx7k4~rU}o;(dms1{qRwOq8f#-uK55`Cv-&wiXZ<@37_g|%L=q-}Tc?q`>#)TAt_X z$<}#XlI2fTyl_BMUz7S6dYkDXdhFX4uIk4hXH^|NzccEh zGwQXkdvIUgtxwju4Gqe%397IC55tA;%pFnAfiUihH6Bt9%6e0DmH}wgqa2OiJ{X!5R*L6wAr)TFxp5V2}c^1GNE=`x8354&cdZ!0_uBT%=`XKg@{Ns58Ugzi0|nF z*pE^a8#0$#j4DBiA}mD|L3j=YFa9!{y4`vZ-Zf3NR100`G={ zHg&Q$lnyB)RA&iq%ofBIf59h!GXpy;y}lj;4k&&EIXjbf6nUfTCH-dB zKFJ3@ft0rp*hxnYh1$#y6bqT5zqlFN&h6e+NP73Ny<=xGwGR>HH4A`r zl-h(_CnIkjgn)Dn4WBnLN}box(D;c~9-;Zr&y?8fqI^Ze*ukDJ8(foB7=uezeX93Lhm=gc@R&tnH+z$e#^1m+H2Y!*E zX;HSYw%G%#FM69CEcfgjoZHc#E9323cOR;LoVU7cYaRH7qdfu|PE*Za;}Q}=wc8$@ z7`YaY>kt~tWdP(78Y_2_tJ3~0E8Gq8{f|ce!pZwj{P~vv1mO4o^Z4_>4UTp4Dtm)h zv9pRYVH$5!AacRMYrE#G<_{h`u;bRm@mtx5UR^Z347MLG7_(^_UA%q#bX*8mTWO2s zKSQj45xj37Xc-Rnw;t`C2@l68nX5R-*;(N6e={io?1TW4g3;&Lc%;uaq_bWM@}JHq zYA90_#wuAdzWC)L{DHlL9Dv1V2SUkCKB;S1xv@5yqOe&No!-K@C=akP!Iwv0;7&(e z?eHa(5%V2YIh@E|-~{l#KqcGk=+OY_t59H18lU}Yg6qN50^q?5BgGsFf77JH`yOXw zR{HN7Bvoime9$*Ob6P(2`K)8FqnL8RtN2`9sb2hPHgC$BzT&msWG#*Up!+I1)8U6s z2>klYs)cvvE4h3B+9tsfG_!e%iD><^H};H#RK}@?_F34~nf;pWE=Yq*V$-nmu^M%X zZW2}+9t!S@G=X{hx4tLVf9<El zoAd6*sjd0n(w8ehcWQ+%guDk zCI(@u!gRsix3KZw9{yWz4o$-ig6mRl2NVnM5Jv9Q=UFAiWp}U>{w0lXAL=pze4>qx zx%d#5g zw9DZ)DEnYyg|{Ddv6W;NejjKvWd)SGwA*_oKBe&+0k9rl_q?s9W*^z}Qzo`9M6gyTeF83L{@P2Ce+@sD&48<6uD%;*balsZGT^dj!ns}T1+`=5M2LD?^KROIz# z5_Hj8@iuBr6Kyo`5C054o1P{1-K$Z_W{~n0ZDsn&2T#38fH1r2|B_o%+lDX0?24pt zmi!Mu0;Z&1eOy6lFWP{UQA$t)kz@@%BrdP3?)gDVr#n#NC*k}P6!JBIC6b_yJnAOV zKi&z+1i0sa2Kj4v{lAc8UH$VIAH@cqpi7lC#`1ef2;__!U?VSHyl5`8Q(Sn7Grt^? zbr8pC#RK=ipcFu`>D_Fu2Yi-6ak4;Btq~!OP2X*R_z7BBWhj;`Mg5ri_!${U7nM@i z^61K4=bU}No-hdmetl(yu8~2oqu)Z6(@cBtT3?=@K>ufGY^=^?MUS6HQ+qnDiRhTwk{b=vJB-N-|QkxP%^rs(4!VrR%z4_Z^pn0N$8NO33^z>%*lxl_-!)s$C=BKsmE_n z-pOS_^HUBofs3J*gmyMA(8Vp1X7)BA@aMh)UzJae1ZkY%=|vWN?TRkqBGz{T{CV!O zrd4?{=D{Q?bA&!#S=kZZDXzQ_g_?|Zf8WOL_d8K5M%Rl)T_SyZCBwYj4b|inQ!3V6 zyy90^-=&Pma*)ryr9sxa$Vcb6Eu(zu;?8OQEs+$avxH-5rwgUM6cj-+KCZ*JZjxz| z0U=lIg_#sN9qzILn^5vIKyeJ8loSIwbnZjkO=#P|2|GL2NL$JkuN?_EiJCNnjPG8& zsP|N}3SDNkI&YrC?;!lKteWxjIoKc&>rzTmah1_Rmm{JuW<7^D&oR~xbBaLk_%V(r z{`74kU2(Fe^+ft}ri_~kd(2re=^GojE5UgS+xB*wW+jpk@_Qs}%%g$!2{|Rkx5*z$ zEpK@jOnUp;c8Juc$XnPh5Hd7bLd|S8L~|D^`lwVgTs}s}zOhd!TWolrx3g11KB%%I z7po)ursR%?_DJpMz}@}DHprryip;&@^yT$PM_+-4V8(Gt)g`~}Jbc$kLPlQ&zfUh+9QT;8mNEFmMnMeY zE`zi$TZ9#G4L;fq$lUC<>OdJGQW&MYU4w-Qh=WL}DVayycJ zVu~VJ^B6lycKZAz?9%C=dUvvr-|M=V+7{mVxNW_@P5rQHENI(qPbpu>m!Uq(8KN@W zXu2fY{h*Xsivo6=BxODR0d0YO)|`5G!r9zSsH>|2+2S_UEOuQwN?;05QhWY5#LoqBLv>%1*^ zU6+UTB7qKlAQCHp{*X+lrc!ElajPYd_|Zw1q$qn2V#5TFfu3Pf`lz}R+YMR!fU2d_ zt5H@}Hx$h=Ksu+9cYpucWYq*p*0A~^pY$$xr{M|tIMa&HM3tGpKhOAff0s>f$rdzf5B+4W?;kX@R)01M`2UuspaHByIVbmiL?shus7&Me$^d2=^nW4cxYXK)N|Vw8vK^~SfKTOb+II-tA9PVCN*dCkL;&xK zoBxMTv#p(4Ubr_ozp^A8gH-+KF8Gh%a|V8L4v2quahsxk?8H=)NZaJO8Tz`tO;J4R z(p4+ljh1ij-Zy5r2n9F9jZF_r{#8eD;o`%u zK9roWKAq`V5*BgAzk1k~1;Rtp%p*kK=&eHE<%imuAq5yuAFhecU8Sp8Q>(rgGTP~j+e;|hr8bHVP1p0?K*>&6} z60ufir%~^-Gy1PvxR5IrXJk3w{Js*X4g`?mV`eW8$^X|apV;)@4^RYrmR!+<0ItGk zm>108TUWk!eor2Fy6Wu6L-b>@6tv@f{7kuKP5-#_tvsJAvRfQ~2i6~X1-kux7XawX zE0F&ZIZnOA16u)=l&M2YBh|xO0~)S=LCLTOZx9AWlwNPWwscsIjsT1X0(vu2S2T+` zagaXooG!9?R5Q;EUJQdMAZw+6kScKV>I3i_nu2X-KS~1)%B6-V-fS4>KPvk&DS$dY zFpCW-v33ds6A#Vg))huQe98>gXZ=?_++U0Mi1@$(T!X&nmakpaweax!nCVq=_b;vb zLwuklV0iXVwY9a)?d`essi^oQ1Q@)M9S`$OxQ}n@uymr0mOkDXzvVctoOU|yha~{C z=IkSe8rJ|VkrD*42`dZWKrOP#c+aq|-QE-P-h5cS0f0(;e7wE$D_rqQk}K-f>qxdv ze{TC)6TPR4wEIlXU}#lK%QL1B)N&Q;RsEc}SGQKs0^gkSA4c;x+pX&&JInbt zO%$7=Q4!QyIL+!`!tZBzh8T$Ae6}=w^yQKL!|w~>Om&F;hsDpl9f4@(3cw`ROlxU9 zv}S&Ad;T=HRDB>+RRZ~;0%@_Lu1uf2(L^QScV~&DP3iRsyD@I1)Vnu-PMm;}K*q7M z;=i)pzYwVFB!Cwk_Z3UNVbMQ<_AfLmAutwESH4VmXp^N>Y?p$J_ijS7Gq{4ACBt$q zz5FXckuO8wosNRr9eUZK;2l%+^|N{Y7Y{N{OT*7DRiQ+(zE8pBlLPj=jX!ktum!lE zS*LLPjOU!d9WE&iqYv?@v@1%f!%BbMq*_-?I)y5vro(x63DIdo0Cd12V1i%H|4%_2 z1#N$;#etptmo#ZQS*M(e_t&Bvd;s>%v6xB%F`y7diR>%iX~3zo&Hsn7 zuYihj>;8su5D=B_?hcg}3F+=mmG15um2T-qy1QEiX^;*n>F$vJp24`+``-8eerwHQ zJ&Tz+bI#uT?D*}y&jE@~z1F+@ce1*tFppw{66r}p18M<&;Lq0a3jLonM6ti~xJ%H@opXDPSWh|a zfdC(4AQR30FX_|Y8xZh0AV6`{L^fiI>xSR{6?JOpVn=8@a7+6vP^tHoDrd3}ORNM} zECW&K)YgHmqL5n~5bs6GFS{aT@=rkGXkAo8RatD$bl6S*>lMfZP>jOj``lrQlgO{) z`KMTFZd`#zH<^xQTN8fd4_lapk(?797-N1ET>mM`fbpP%90?Dn^YI4^0e7O_A!-$m zxq{ZwrcPDg!n&_W(s#5#YBK$HRdj80rZRuQdt{q%0jwK9)%b)UM)B%Q^ zJm7*Eu--R)7W*XN^G$gz&uq^g^uq}0IfIE`JiSodoChb zu8tZVqzUT6a1@Vmn}gP`=C5|I52q9ioZLCfBX3yY5T(XeqJQ2zU|ig4%ZpiJzZ2rA z;}JP-LL9`Zm-H|02e5c@Afc(JpCBRd+h0AAoR{+CymnuCQ#MuLnKOMA>D0Enqx#<5 z?-#B9;6wwoz5pAg==WibY3iWOhz`3Z83y3uS~l&?r3Osq=Z4ceHIPidFTl~`fPe=Z@6j+L<`8%&GqRuRDWc9#6 z7Kwx6;!66df1b0?X$^cGeinT1=T2o6Pa`g>o#4U~*70bBT&7=N^V+=aiISm6dT0Yh}~K0X`MDb_JswO;a@BaA13m{RB6l2i`? zP_6d`$Z0@7*8PP5GFpg!V{?90He28scPzL1{yYCzprc)WG1YG#01p#vkcDZ$ZSrbx z^>C@r;69J&#F|q`mi^2A;R6FVLO%VzJf?=aG-bBwd*AhQhT{TJw9a8p5T6kNfX7WECAm30}-nCj+ zzESw4&HwR%9}Fm+12v8M&njH0P_K|5$awH_Aq0CvbtvBYblo9T^T#D61V0TW3J5?{ z`YQK-X3X_i;g~!*&Y3TBHp-2PI(N>Ge=-Szin`Zz%KqSZNs%v@y7^Rp+GRn_eTken z*ypV7f4Q-(H1{yffMTB=vr%E@H{hM`a!~atA_g^0A;;ILZnFPh`9nzK|Jjc7O9%a% zGx~1?{DK2)nh+BS%>SuoMggxb-{Y_F>EA|!8vM`D6tI+H{{KF)4G6#Gw#onJEg7sL zdg+hY~AwGMFM8+2s=oJFfEnld{!Z!baCGI zPA7F-C%AUCCD?rD+xQBmI+k#z;1hbCM5hYAZuiMPwBEmmr@turmsW3O2L^?9yR>fK z!=N7!wb>8Y7?mOlxvU?&%T$f^(&XTpCeU)Z-0sYrAitONUmP<%$jfJeRTWi}6=PCD zZtvMs|7OB{p+z30)xowQMXfc54U12bPlapNl<_VlJZ0T;AN&1_*CYTJm}Qi4GeSIO zI1tv%R}Sx4@?VEEbscS;j0?3?f=4VgcsN%!BmN(`@d*#ewQJi({M0cPK6VqRTt)ou zuc#LC0;|)n&bs?|sPH$h=ubjzr2xBcrlOMS*H{0GfgoWS0jTL)?}8Ef=Kz1>;?Exe zhKxWuMu7%NEoAlS-7xCVKbsHIoMcwY$fui&7Xi~@BkE(@lPdo%T-4PIen9~ zfrA{|?hLVY^&iP~oW~GP=7nfxp_`+Y0!Fp#6Fz_Q^OV2(SsI|8ffGT|zZ`Jf8VaYg zGmS31DYtC9-M`O?QDm%o8!|#(po@KzZWj@{oMHAR1Weu=Xm9xHqSoq=BP=9t6=;GtL9fg!h9Y<_UkJq znyZHEZU>3G8AK%l$GXzQ*UZAln|B`?L=Fyo?lIUbezfIPVu)_zx}a7lRj7jHK*vW! zO>ZamqbJ}2+FFrFnI}^d6YVivLAJ^{Yfgn)v>+ZV$PMZo;g2AD?0Lvv$N5UmQx`QX zFt2*j1}0oA=t`ZScj9+%M-}fO!~9LQXHDee|^+EA_&CO0hXv)-?P z=c-cr706^Tm6l<&Bi)oWV?l#~qeW%PVHJTaQ>V18@7dm;ZYYO;%RqPEA|UH2i_k};|~k6 zB}WpnF*+yKGZL~hDk0Fo+x?ZacngehA2>GVG0Z!Lf7$Q9Y3wMR)uLt%^7FZV`t>wK%%?%mpq2GUa8vicK+<9w=Y@`lrvKtQMd zs};wi31=&nsnN|kusE1wURk8STCAeKm}Ff57OFvYi*4$q0>1JVdSxFl2R${oK*_l< zHP`HV=k4}R8TWMVKasc}^SpaczigTpt;y42^dP_|u1mvrq{Q%NuzX5AppHXr6_G|Bmgc{iCA{& z?duoG84iUuJ&A@9g{l;=5iq{T_%9sR>jZa`9=F6222@Nc8FXPOo_7q3fqvRgz3*^9 zCs1j5Ia=y<;y_1D%i&dAe0*T4$A#u%@~*Sy(1wX18oCW(+$HGHf^fcJDWBO{fmi3TAA=kM*WQ_IkGJ% zK-zur5t}k0B9~6X=h?C9+M^8rox1K2jKM(n9ck2U!N0dAZ)LQ8Zl}>XGY6bxRZG^y zxS*}(xnHoBrJ&xG`;piAjS$q=imuE~Ki;{?_>;qq45cYmuZlca1Kq1_CsoXtR_Fi& zRC)tVVR1Tc8DDOcxbtvt%kUf@A#>b;E52l27_C@c2+}>5=*?C|XJ@|0xx`^B9ER!Ly9OY<`&2Rm|FV`!dfnA z#RWI7TdX6^wc!0XNCBda?aai7r<>$%WqveHKYk^5@$GwiHvt;rRtq#~A4HL}-r?4w7I%5a5Po>^c(uBNX7Lk>;U+OP{axFX0(0$HPq z{`M>XA+)V1h+3W53t!L5)EL?6B-tVum9ix%tWlsusdaaujR7hlIJ_3e7e?%I7Q?Pf z_yUFpk|!&5>Kh3ZX*u;?d61VEy5oKbQEU+tIGfSISg$bwn!r4nhcrzKa^5zV&4yf` zZ|*kCtYukA+-I>kaxJSis1w`7)i>Hby)a2MaXTBe$X?bkne96b9Y<2gBwT**RZ`qV zqqsP9*>-dJx+<%1rjL7U*LueC#@C`*hs3qxhIVY{Wi-LUyBBiMdQsl zx;?j1KfyFCwI)h_Tp!Z)*k)E$(Rr%UW2fv`MW{&pSk|L;%2BGPSa*CCr`CGhfS}TA znmN_MVtrAoCINDzK2{f7JorXx{L~KK!&8oznW>~h^<3|bxyjL?z6l@RrF+u(!^DA9 zg{B@}(J9rhU`k zx0RS>#|s4dmB)4M*i(rgSbMr}bIosK43$@biYl2cbl}ZodJ;WCwQKK=V6B<@mP;j{ zkQh}7jMqLryUPW?-5>MNn33;tFl5StUUMzZU+o{ioqQK#jZW$$8oQC|3lpkA{^b8$Z4;)Q--~N>aTCzegau z9Np!L9bVni{4g`pI;FoYbQ%BDyUt!wX z9ow#rlY9~vR@yhYTcWB{QP`=ZGgqqJt+lWZmkvI8mxngFmtJd*K6)aiC!wBsaOa|^ zxM(Cc3AeQ@u|F~KYC8Y!0` zXM2rXcTNt+zj{5>44z%OlLsDsIxAE|xXpOV)gZU@URU@jVc0T`kAuhNfwS)1%Zh{@ zLHPJ?WCZq9S*`7-h9+*q>sp$pdO7BKn_+nZpgpHUX9nUUWLMedgmcj{gH-%B!=uBI zZ8lmGcGG#+IW9D$zIQ%h-#imi1n>H{XMlrpTXshz;S55>o z1eEiNsn=SrRs8g!#0}&-br(Vm4885oPLuqR8*iTNPY!s{Lc3d?l4}or*`z8D)ROQs z@$EjP8!90j4Ve*bWZT;Noeg<@J61e2FVU`4Z0^F7-y^itMXBJZP zbv(?tHIQyBxK*Eh*JysZ7`>KygS@Vhg-?@x>D%UchjSlx);&T!GeqlRehubL0u_|gxnrd0G zYZx2lonNZ5Dl)HV7BG}$%(45BwUM-3XK-@PtLOCC^-hvK)l+oU^SaT*TGFcMll=qF zHMxtE3z!PCV~?|PI=k`HWhGwk`|gARYPUXEV1X_dBn~Z=SM^2`%TA{EitJi zs!}byPCP^e70s{8W^r?o)n71=}y+7-Us~>YxOC;FxWFu4^PYugq z8b9UhCfPhYaGnlvlrkvd%Cg*%j$BW1ub73v~xU#SbL_md&5;KNfutSPbtNoD;`36$6j# zE>>&FJbK4l&f!Ywx*T13n}D5BXXV5+>LaI1+x5}BzSweIwQt<_gO+;n)>8s;4?i0` zaT2P#9x16icHPZ_o@|-MUoJ)~OLjyqRJUJMnV2PiI2T>VoEkQm+t(m8Q{@rDbc8k+t7(a#WPU!8I$~Hk=VzoDcZIU!k2G6x6p4~8sksSaD$ zmng;7M(WVI?CvV9M*OWWnLh*0Rh_yX{fP-h`381)O%RVef6B{3{3ecEAS;70A@%`| zdD-t}mV?o9g2tH?kJ7(p*F(W7{X&DfMLHBc1^VIUsTxxhvAJ@!nv)NIGZ_ z<%!&Uhaz&v;UTf4W^S`5&loR()AkE45ra|S=&G!&s6&1*(^AELNj4u2-v*yf-PL4( z%%yh&U27sLmT=H~p@oNiP7dPiVr{%T!TCHASAFu|3zKDB2Zck0jjn)$l` zTRFnxfZ{snnG7wI6_;9gzBWzed+KMrDS52nGF$||j+DW1fq34#razDonk_Drj37$8 zhh9iAmE5*ti_w%dGVqPgVTi(VoQa~CiB zlY&97{gQl+sQ!?m_L_S?d#oXEP}g*V7Zy$nYP5Oh65N#dYBa=uWbJ@^!p_hYmvWuOWezCto{v|l3 znw7<~^^Rd|nu59UMiv`}9m2Dzhb}0htuAySXTke>zc&XjZjuL**WYbEb#rv}(L0 zP^7X*_C*@j8+)oF{i%zWLG9*?~sH-jNEq9S! zH$#^G&~DHjf1hTrX-mzW-K+NTslKbnfE6~(AD;Rl!snZb7EYC8YMTye4ZvRb9? zAk44>jHq-^9H<}x@eWSltC~@wNUke`n2+Of)@gFbIp?u3*s z`{hoOO5q;#$PVC~B%0J!Mn3fR!<+2pZa*Sy3b_&rpa@kOb*)I)W@R8Ep))pKN$L$P(;X;)TP7e2rf`*(G1yjZMJ54lZjvG{`M z&y;wc9*bG@g~<>EJ{{`7<&DYDdI!zg)lh}=DlzF?DqWS^QIjWn9i8~$Gb!+YJTF-P z$~C&H$uPuWSH9jbe4=q|5c#H+2+@4$$)HuAO7SyU7w(9Y@xQgVeMQ2^qaKvK3aA$k z%aAd~9_mr`)^R5bM4HRfncVsmF>{3saGxHqeBP~{KZ?69t~`&5uSY!I*e5E--4>|6 zqT4eye!l7uInCeK%;jfGqQ+FQz3@S$Fjp}qA;kmprx}BV-ggbs z9vpJQ(Z3Zo15G>QBFr}=!k=%9Zd2T#@WX(iuX$TUiplzAVa=NWLYp)q>pJrYBIp|u z2{4*y)}Q+i*OT6Cyu6#Qj?DF5=8YN(&11+NL!{(tvhJKhmIC9aaPKM7GT(}!z`)+V zAnsrVbAfrnf@Gch+nurc8|S+fa2cQTTK13-ykQU+7$E`$fxZqgFDB{qLU5_8o>gZt za-`e^A(5udeNgM1iNL;{)sMOT>)+N`u9IpgB*Nr{Ukya+VZ^ z5_Y!|Q{vuVWCiH!f{=@iITVl8>>siEt@i#O@BJWu3YA{PSI$>Uh4l}&xI;~gNy(E8 zPG6>{K4?ESX;Q(Da7}Nm?5VRMe_rx=QaevXJdJTg247to=0t2%8lWT$q=y=Y64GPj zOYobW&<8ngumZUddLzkuB+?T)OJRpRi6^r%dyt2Z21-dP@5kRejQW*OmEsg0(Q6mMPk@=NMpb*@iPT49^w6;R^yQ1E!WS{#P6>7P7C zd*~5|XH2L`=>5(c4V_E?2D~BwLvMWO?aT1zobbTTkKTI`XUTZUY7&_Bsrs9ijAqk! zfjz7uh8z5XVid4T)-&{#bn|51Ou(U8cZ!FgPHw=lxcH+6_uOEA=h27%J$|lmVwO zt9UuNDWzr$DNcOK>0I}OU~7*^fDA`f$Y>BTYA_WSg$QLMp4e+$5WKqDtvJ5dbfdl< z68@(=-J@XnaJ}HxYbp^}IFXO==!uN_-9=1VR5%dHJT7Rf{=RzSyr?czLRg=YQ^L4M%?m|9X#X^KP|8!(qv= zuU&ApK*bX*Fn+_1iS}c*z+VO~*s|(40-E_UA2npgd3G@nh=?j}67#^iuTjh1?KO43 zsbaq6aW6{md*`}jc}El)RFHOAh`Yjb_q(geA|xUY4BzUpaPpD20arYevA{}-gTQzoHqQZh>0=)aKO316@ zm+96^&IBn&o>%!o(5KE&NwYh_t|1cVb?rMrtlwPC;G)2+!6*;@S}2L!6gZ7SE^vFe zA<@H~XgH?V_=+=;!*VcN5CmnU9O`%7%7}ivzI+M{j*mymDimr3Qa(R5nk=ebyOPnG zheyPlQGFrRwP8|#g{?#Aacc&O2_+SCeFDoBA;wQ|rHPfZ1t<_OB3nt|)-_igSYtt|msy8|rbnncbY33(FagDmBB1V3UM*b4tmc|_i3ab&K} zvuuKAi@P9HuYn4iQ0vZL>VX&)$2qo0wedq@n`sb?$Q4ui6uvvyHzL>x?YL2f!qXb_ zL$2P1`7sBrLX>6H6ZgO*q%a%8(Vg1hZfX6(%cFFqB{&Un>fn3JO?1x9L*{L1e^{v7 zABq!)kpMk$nP&#Huq`Evb+}nJ@8CD^f_~f7w~{KY3z&`Q$X3saXh%(VGAY!LD&f2) zDuUwtA37`KUt02VR+S9fWIWx?CvNv6-snTMI`^4yRc@@#p z_ASewrjX(7%AefHSSQzaRT6I%D?fgB-6YiY!hn!arCIm>@J+WL3EHo5eyAxu0jLa+ zxOW1kXC4!jNAA^~L-;wWoh`puDp4_%m3EjE4J;AnEhw{q#yn;!bL6iH(d)Y(go|D+ z@CnZ=LA;tOpO;?6RY)!!gm|<~deM728c;VEC++lZ(6xk{@Fk$^fBFEe9pL-pGyLxy zhR;&zj0;(E%gG-8eDCr(NxLO8{GHLM4c}gz_6cqVEqzmY7iIy8hgHTJWAo0{G?~(? zO;Z_dB&8J42~))Fqx|p$3@GPQ>=iWMS^J19@&@63vs0&kxBBhn&f&Cd7cOTwwZM5f zdOo${Wgt9#^m-t1o>`C-BF_X?e7f7Rk*&Y6&|*hIG??JzSXq>zy$ zl2nV-32BjQpr#am6nT>~h2;YWixS5HY38T&XxZ?zxPDL;ZBHE*GO;$4exLVkm-j`4^2+)oJ#>G+}R7>HqW4poI-VUNNgs8qygg5!U0q~%I^KJ{#|b-R3w0g#3+@;_pB*UpJjj*^WHrW9;k7jI`M7e zom?s7PQ(TNq~Up~Tr>=_o22MwHHVf$+@%pp zIfihJwBKdu6JIMzGZDQP>eM|JR>AT&8771MX{df104Y?x;ZNj+);u_0SsiHSn3Y9> zV=lJdtG%_Gco0X&ImrHa({W$oVH8*?2?ZYHNje{#OkKZ6k$PvaHW{NmJsSB84r$|u z<7yob2ud2v%AQU^EvoLQsG~d>*J|EQ&==I9_(7LkeDxoK7k%-#-HF)*p=h| zEjvvJAGI9>aY2j2A!)cUNME8{evp)3q%R=O-_@BKk+6IBA@+HNX4+UYMOwW0`MUtk z^@`PgMI$4Q_C?G}SBIxQg092t*I7OZZ*C3#_mEaxY6(86s-d?c;`0JIt}*Yt=fvJO z|B&iyQa!nk_vOh*c5FvTyA!Lwh^jt$UYODvhni0Q{+Jazsgy`x`R2ba!EXhKNunyB2x+Wqg$= z8l^sCmgndBUTNja(%y4!Mo0H?(RuFXzW%h4HTv~?_zSt{dp4eA;dtMg?F2Mmk*l>A zyW>}_ho{*i7Jo@5kdcvralc#^Uz5?C+2z~8Gr3EtAgnT`&iUe}CTR`u z3x8>qU+N_rVWQFBle6^oiKtaVeavpGTQBFRk!ZJ8zqp}p*WpvH1aE#96l*jgZDzUrO<@NQHSx}=lU+6WT$$vDWzD1{DlfLtZxj`E-}Ij2 zcd6~uN^=XvqQMq2Et!N=nW;uz9xBaakJjzgHg|^9)HkP)I&R8CJ%n4HL)|7^$Px?l z!9k`r?)S!5B%farv^i5!3B7K8jDj+HfTSc`g1>d4jBfd@Zw6Nal(E{m?kXPHLy}>N zF2DUC!QTHaEQ(l}hqsfu$?EZZEt+rF*N8;tLo`i22`PUt1P)cK^i!l$x@K;Lgc8hw zpEZdD@nf>$bpC54|9CQ%Spin}*Q>~d#<}+fF@-QUvlaK;KjHF-wOL9=lgg=sm|%oI z2xPE-RC!hUxyW1`m=!B*HP2#cdico|wj>#z8Tl|ryOk%2UZ$_VZ>UMZ#@?yO6H~KY z2v|}QrXWbNr=eC>@+f@ftndWaFbVDx-AOIGU?*Lh21v(>QH$dDR@~!)jmcAJBz^zr zVx#*-jJdc}kLt<3RSx!+68MhI<8hB@Oy0>OVkj@pP;j2fp46opZ+wyfop|U!7w7+k zI7kvs#quK2Uc7gi?sa>jq)(wQjU|Kz*ys-+7%fJ-+zyrFM2gvk4rG;YFx^M;!QGnQ z>4tDQ#ZJU|N8=Uj*fe&j?JR%|=SLHwfx3ZInJd}bJ#0hYZ<>Jc+p5G$0v( zPqG;ufhLJBolb<=a*JJ-4a)(AY4{e18GoubFLb>u$(+>(woGN9Y%-i^%}(RX&ijAA=NiNo?zpVVP{rJ=9!+@~@fM3Hx%*?SOuRd<`itZl>?l&*a zAx+x@s3IxcYN$$Gop*7GqmOGkoF=KP!vp_wsC$(2rM9iP%HrtnsSL2*%!nwJdFkw@ z1^#kjC1v;-#7(858Vt~aL=COGg8blL|Jo5h2!WC6Z5f;Wow61E$86k%^*Zy@i}UfT z9EyXBsgALNPO!v&4=17b8gtqcuRdYce}pzw^M%ivE}wa{l>P6o|77v}Gmsn_jbUOP_2F_{GhJzs{7LfL+n?|xwI0jOY^ma)FEHB^ z?@dg`I)D8b6B-QvCe@DzrdaPt0#dwZMSUw2ehv@qD^K}vBZyyI0i4Mp3LvC^S=|G3 zk?!EUkcAMcEFjXZaUzPZocSCBI@>iw&O+;@Qd%#AFYWYC1_CQ1c|MoLOK7uvlJgyU z^ql8*McxT8UM)*cMfK^21M0l`V&&qAZ$rvo6!|T+(1&?==)zvo=k#CsvsDbxrs}n+ zM3${^1iS?NsvlSp9fPw)r)thpi*^83Z8^=xDarcZhI!oFI zOd`h%ez&#iG#&izRVY0@Cd+FK`D_{n1sU>(hlij0;=ewqvmLjcDw${G_;nV*vOBS8 zZU2p~HMMpr^D^D+NLA4Xs&i>_{&Q4AI>}kxr2U|!qs9adUi5F3?(Uy)V(WH%QpJhF zB4Mk)FVu&1=`N6*O1sM)9tzxt5}qNd6=}@X)%WziLc;#{!xg%7xv@_|-8&CQjI_J1 z%-}SS^XGHT<9VDUw0 z?_Sw0n?yke&!lll%iA|{$Z#5ou$CrKS48iL`Ga7Yfv+A;A|1~LD)k`j?LY!XEt%Q( zmt$~?-0Rw6NUk8AD%{GH3BwzuJ%!m=4rpz!`5|C?Nv8O(3?;@BxW;MY1-ufAU1d@b z+iNXCby1Gh;JU^j^v3vWHG6EreA98m%WT|L^mBw|k$FA+Cxj*X=;H~2+ogEMAyg02 zp>m$41~%t)h@JN#(q>WrVDHkhRFB5w>7I(3af5P^dGgum?%1$LFLkbT0Js?d8Z;?G zz$BJQYVs%>l+EjyxoeM6#11%i#Eda1b0yA>^hy)h_b>cyv;s`d$hN^;9&Vku*Zk^t zGtN!ZV;Z7Oegc>*ULZuBP4}m+ln>k-qh`0Y_3%oPm`%S zqB4g7C1H?EYt{WrEG|M>Gb@Y2vwQOGcoy6gd%OTD6QzewWWR7!Uvv|0nZXhTmMFby zRHe|l#0bJph;%8QSdt9F`?{X!=4#(27uVVIA5}kzN&6%#muO_oBSBx*l9^* zyU&WHY!=2;qB(x7|K%`ci5AluBH%?^hR8jQn&{IRMc;Lyyx7 zG6@rMly70nZxnZ6k(kr-;SwW+8uVM`zQ+Xt)6X@@NZU(QjMW9^ubWoKClpc4CwWCF zcv$5lmb1y2EfNEUQ5vy~q*My*4F4)UF$v;SHQ?mTv{fTWr9oqm07LI-Fjbn1g@7SN z@<{09!s}BYXp2Y?(|Lb*`iME8ZR9>+Z2^_+U4qE@3cIteGod2@mJ&Y079F?flx~>B z_61l!<0f5Mni-D6)T(kYH|agl>n$?jI?80xJhkSk@gZjmmA_RMY%Y(6(~w(S-r2!6 z9u5lt3e;_{TR{l^k56#uEyE7@lo1Z$<(mU;?JSFQ+823&XU$S`$Sn}y$N4Qm3*}=G zchC}EbnL%(Xqj%a#soxB34vccO-Tc^3Z$oX1$0Nwz9R#jt&QRfhfk<)$ zphn;2V5j{Wt)|drx^th6K*hV$z+_wPz#_dUA&Jr5wuy#t?M8L^n}{64{FfRhDlrNx zMfXPJD;~q$S`s}IM1FBS6PHDkpUN9L<`c4np6=F2oJqaGOoiExvJp9NijF*&Euw~` z4yN7|m?WmQrWH3+lU%>|dtDWEuPRp^yeYoRFX@5Q7DEJ~fLk+mCO=kQE0WxLNtxds zXMK`rQ;(t{?P#inm^qf8f!U35D^w5=LTzH;Mo9aSwll%otYnZk86pD1GgzZNn8}Xo zKo7GJ8e+?8FG%po0Kj_yhiK&krvy*)@2XVM(QD!Qf62KMQprm{s*@9tl35f*QFXb8 z(%=dxbu7+H05mV%0%N`EGn}M{fS}xgid8D&L?Ou+Z1Z#>^EYU3Q1i3y&X&Gl-ba8# z;aqH67tisbGOwcg-jVycLIOlPyG3ZE+tN)EgXRpY)k2uth(y-o-tDR7CCGgYbMbC0=m*8V9t48A1JqB6DPu6FI1IsqjF-4Oj5-W5t% z$42DUxSz^kj)Tt+n`(@}`6 z&Y~Vy^Rk^rpljCd^?(U4{sh118_e#ng|?~ zm7<`+9L8S~v4Co=CT-G-Q1M(*?JS(6gzzn=&n^RMV8Oka`be4{jjaSZjL#_idbkJ^ zmqf@tZoM~3ALgBHQsx^@)!^W402zJ)Y*W3v4=-UsB-ZL+E%fOP^y*>@yKy7NwuqjW zL2Y)3!6}p%OLLJ=Yz-~7#xOHp62@y@OOHh#+$d`y+?R`E;^q-oeoK*4_J~am>n z`6AORa)fG;QRjQ^i%~QHYA^A^$oTAd^L5F-CB3l<-Gr=Sh3+xn2`~N3H$liyQ3@#W z(3h9?Y^NADsgf}=M3YBHR)gt~s)M8-0DxTp0@QC3reNvqhMg;@5A#KT)tzG=Ts@d< zJ|o!#m)Gb+O|H0m1)pAa*jC>X+MQ{Ms#nT=J20v#R*<%O*EbF2n`8 zDwCZsV`zv#Lkc2W%ygcP92Uwaxu?5XX<50`-__RP0Aa#6^Gl?xr z3g&*^vM0{oTj*Y?yW;Dc<|(}zH*D$Wi6iSM4SJxf4^-tDX3?jQK80Wy*}3+VP!6Cw z%<$%ZO>U|i;p;C(^+an>p2R|{qnj3~In0BrvH)*egr}J!a4y{^B*)tw8SBh_kq8el z$-(R26Srl$ex6#g|L}2sjp;DyEvOZR2JtLIKyQ9G^C-9IBdd?fu%Be!JWY@vqW=M7 zqB{Es*0K;J%NsY&kL!EI8g`+E>!bQ0Dh1P6!Gh-&_BDK{@5g>>@LO*@NPu5U%XGS> zHmBi5yQ@Iudx~&)T{Foi?eugs1P>KyFnS~J#R4u?<8wV4xsDvdk?awMf_Lfe-GQ-< zSVT+Y*-zxVBtW*@L_tPK9YqbF&_6KRQ;p%7RJS~Kx> zZ7yLeiBCMYV+7J7RQEXYhDq6(0Vi!qKv7du@BkSWm;n#M<;9u@D{qg{(pG}S{qf;X zBEf3OFzJS za25kb)g~Vkyca4j-oh0un`JFLn_Qof?w=PpzT?bfIHn_dHwrwGgi*G^Ar*ltZy?0~ zu(u3GH{4gVobzwe|kstyzw)bx&~@(T>0fTx|j0fhcg49MqSV8v|=)oxCxx zq94pD&LNJT5{A4id=mmH=W~;0S=eNwy%%g)*J`s$u;Bt%-G6ThC=`PMQHst_=0DY@ zZ_Ix^`dMQQPo>)T0XZgR8-(8zh$lM-K65-0KRM&UDx~(4R?dHoo11Nd+>#&VulW#! zUdUF%QJ>70w1FPg_kmsGQzU{4HUE#GkcJJUFtc=Qf%*wIX@;ux z*O}hE?=#+hw}s~9ZLPk6Hk<|vvr~Pn(jNtRyr6$>mJ1{?KI;WJ$Bk39AU&ZJ}FNSS=?|E zLVhcpI-EKgG&o1+F_L zcqCfg_~8ZY*cW4NbR#vp3=r5pBw=HX269_;rzNvOpUjz!1q{5RAq*V1~HeG zQM0gbZqk!lOj|}w-Ip^bqVmbdLO!FmZ^*xKE|fhQSam1YZsVK^rL4jaR2h)5!v$Uy z*r+%_?wKI+C&L~e0){y0X>vc33_L~Y4r}#B&05V38Gk(*Lop^lQx5R=jp%p&Hs+&{ zFYX+;qhF8K!(YyV+ZbPpI%7Uioyl#6FXM!~&q_H*km1(%1?(vz zq@SNW#BVCP>d_s{k=96UH4?d(zriZNL5_L`0VgJ=BCuOO8%=B@RDV$U!6~gRkgB9+V@yl(g}91TnVGv~gGh^k z2c}s;^JZ5r3lRI^qKk!u+ig5u3Qkd0EnmTVJm!^5c1iF zLQ6^1eejj&__z-`EdZ1 zu8k82?(QVGyIX+9p>c0`#g=p4zwbTwzmHL)M^$%GwQ8+eQ@;7lx%6}r=qOC^`v`;! zJo7s4H)J@CRF~l%ZC)B_QBUm_;zexV_lyrFsZM@_j(giv5F44(sDERS1b z6lT?Xe~hN3{_+purD`esK(8x8luDEc>IyEQ$T_a)f;L@?H}Wtf@AzA#2O03n5E%*hJrVjpCZljSKrHIZE^=2yaUvb_PfO{ zuUX6t#)ZU54}t|#u(boKTG(2x)b+}ekz{O#C|3lc7CnIJtDoLioWqNsipYZ71#d!Y1VQxD*&|4m5bEFkK)YdoCPR|&z!1Lc6;zP(vsKjdhl=zd5^Zq7jgox_yXE@eQjO>78@pub3 zL{G7HQrAd@3h~V6v~~~OJmrCQ=VInlEFAV0#ZosK=K;wgj`wd|+dir}ksJ>acDw~# zj@7JgaaFO*xM$RER2Lot`^KWvBY8{NQq{k=4g*aZn=j6Ae2)^h&czLZL*tvx{qQx z-l=Q8ELdPL3_=ls9dkh|lS%WT>h6P;XF=s59}!slMUmG08u{Nr;;Wc;`=WT#lF^YA z(haAXtC-Po&6~vEHcqb#6STlT?Qbe|kZAmJZ3qSbWUc`0!Wds*?(gTW#x+`ZFmmub zWSDkNg$D0}zncWn%XqwFH+!ek6*5afk3!E7Z$J$IdS!|#l$b||@IlBhTRI8og5@5r zk?=%XXI>cHeDx%%TVL>~ZU>qZYq5c#4oR_JjA#K5ZU2@V!Y{#EGIO$gq1yS%AV@wP zrab&w{g_1L*SYX|+C`h%QD;R$3AWVsdc3xaM(_Zak-f{v zO1tpG=jJnWP^9J({G;oeDxx7y$PfDh->ovw#`t2rI7BoYmSGBy7$KeVRToC?T7P` zG`=6m#TSNnr|8uv96At*mj@m2+nwH#M&2h3D{uPC2M}HPwu7^nws#8I7_A6j zD>e!>hn!gk_b&htQ9__Er)SE(*UF^ug7pJ20TbC8gp!%e$_Q|>u6#vfwx$`>ytD^b zugICkj3$r7f<$qF$=>e|*S%NyQ3^<1*i;t1lOqeY&ij&UfQWWn&|2c@pzbsM;ogkl zM{3U#rfroWuYSgmjPyMOmg;wp<_{oj;1G<{z3sYcoM>}bLT}4e712bWK7NZzedLrK z>K?PZjg}@eM#VH zrJEIXX3S-l@SRYvx?9z-;^^;Zpfn7JW;!Z0Q6j1KJ|Ab%Ovs6H-HgFeDRXCbvp*6AG=47a>z|Yd7y%0f8QpP_rD#$+hQ~Xup2TrU zd6aDL!qa7^lwfIl_+dOP^Qi!~?a*<2%oTKt)F+~`>VnlVg6P03$X6NMLB|5>5JHNh zYGYlY>=qdJ0ePj`zz0iKE674Fe-%7C*i;nIpAETu8}#h>u`gE{5R@##Q{{s~u%GNY z{4i(0`cB)sD1g%Nih{`pBp9E>Zw{iLZ^74PL=olhfA-m#T85~wxySv@DIn``## zwOn6q&ddG2Z6(4{;_sHhlcV}lfzY@KZXy!M?vUKeJHrk~w z?zY7)pb$c!bQsr%g@Rxv*1qE%zaOStxcBp;_)DE^DrbVZ#vZ0;l>)l!PiW80!|&4o=($DagMVV^z1kV; zZjDrYR~fq0Hh58Mm$P72tIBaF*t@cOrvx`c&zZ)rjkC=3)SN1LeCw{EyErQ-f}U$D zyi4*=Q=AQZ8+~%@uo1ahas^xGtrg#tgYB0)j@kg>axMA+LXIa+O*JvxOlp5>xBj4Z zL?&e*YsDz{g2F5pA&b!RT^@a#%(7#aQToU^|4pv5ESN+|a=zflg~?pv=ACVMO#AyH zBX@LOIwaOd5h)stLVe?wDUjtI+^59rG~G7oA_|z-?btY@@~rZD>X@h1Eg zEZ08Xpdf8Rtf1v+Q+Q)?EVfV6?CJHUpn~e7{-LOJLV4xMs8e?TiMKpKwb3Rx#?Key z8MSmDsb*6l&U2~nA)RJRCxta^1|Fianb+oo*=|wQDMX4R|d=c{}JLvI||FMbwM(j)jb02;E(S6|mM=)HNVsyk7ccim) zbAGrshaE>$H#5|90Sz9}Q_}a6tcwNAzr8lq&_W3Qz{FqMjg;DPyuTqXUyk{7c-GC~ zo3}i8@}p01{6N)uP2_+T<)?+sVv_$I*x?b;)DL*7z_ApIag!IPJzj5DG=mp9VFUeP}5a=TKZ)A@M4``PI zq=e!SJ(tw(wA{c5llTT*R?(_vkfF~^Q{~JSjB4?Q8&toEwymvmD~&Ld z5vXho1eDHpaDH^3eG-fVa7H)6y-l!7%-vf6kkIx-NK=<>LaSk@$^4{B)P?`>yPgWr zz|ts3#t}jtR+*KUg(%YszvuZ*g(^pcvWH*hQ(OJw3EnN$`nNy#=g`CI_&Qzn&ZRJJUYD>gS z6abuQNfH~Pq7@K1^2U4Iym=Ji>=$# zHTi99Fsj_wNq2tp@^6KFB`a^zJXgfv{xBc$VnvO8_a|1Uw#JPP^Z`=1I5=++o3~h1 zFE~X{Lr^SBUwOQAc%+=%06v3lztod+>#wm3 z@33B^gEbk}0P?$;v@eUJI)Q$e)t=qDL-lj5l z;mWkHL-1x1@89lyMAqq4vcQy!5-rn_9qF>owvW*1N%_nol%(N(><%`jX^fN7U)p#x zelg22;2&i8XnLgh(!WwZ>J!j^mO_y_G$HmE)dD~#p7;SdiUq%3sSER&2X3EQhqmbh z_J%hA1i`mm`XWtc)LCuE;pF5v2cEDp@hM`R6|9Bxy9SJ~s9)wxLS&8aJ;JPj z=!Dr9L}0!{-hS|pM>MF@RITI6E}nF*3M_?0m;&;Ioke%!nVsr^W8ueMbf$sw@#LLC zg}C{Q5Itp!^@x7Ynqs{pU#ze1nqsgYWmJENXe7Z8sDwyi$$07N`}wXF>6Ylv2PJ>; zG}_k@-!-?z8lxKax1YH9&g8!-h^rF?f(JL&v5nSTJM{z~0zc)8<4aAp6#)LjO0Dz| zaM3^IW&amn`ooB-3=w^xk`B$E0b#(q_4Q>APUE)P7IGBVS0wlCQEO ztT%f({w$tIxOpyo7O)e5hx130fG^uK;^%hXFRjSlO;5d1oX6IY2LstPJ0b{yGB`I} ziVya>DTHr<04VP|!=v@_pEnu>@~(tnr;d$@>O5Agwa3Ls}G4WEpP@yE%h$k}~vma-;YSrd-H%=RiZFIAoJ?49y)-#W^_ zZwT~4`4!}dBmAy;k^I_v343^has;vhbC3@>`}bLYw+}G&xsgdD7c*Jj%L6mFrytQ$ zns$-C}@Zz&Jr!iyB2%Jb=3;9e+vL^{`6db77XD5A(DcTimOp7WIB z{-23H@R&y^elOXsMTiy)OAg}wXd6S9xJ6|}!=69Rm&dIg2k~a;xjVOKO>Hr*IaNtU z4+Zx@L?Q!x9AW~BASvo&6ZlYg864i-D5?Yu@kA5!yP*QySO@CY)?c=RHgTPLi)S>U zWfr-v3m>x#$2WDfEbev0DBP3QfSwOQ^7+e``G?l}`!J5;5i~Q($(94|LrVMlh#S0m zjZAqllk?nsNr__`R>}xamH&Pb|FzSfC4;}jh+<`~B|P@*YeNJjMzLXuGELcbS@gtcRq*w=xbVL* z#Xo`SBRao0qjB9VG$(FaM!4P2iCDRekJOsKAK_1_uSW}Su}=v^$^Gfa%7zK_Wv6R= z3Y4l|Q~=n-({0y32xuEruYb1MNtu9WA?>b!j6D~v544Q`!|VQgsQ(;ra+OjJ4}Z=t1!+|EmYU+i#zl8^sy{U1jQ9|W2DjzzxkVWuzigF`caAZ3+SHt`Q?`R6o# zG1ho5p98~NhWIT7?QE%CiI5Xp4sk};<_!E-X!j2t{O2S7H=lmFbfZ}LxfxpCsr>$l zaTs=C)7ZpJOjx)q*Z^YqHjL=eAZhn}_KKR{EVn?5JL85N&>{cdKKbXNZ^kof;sEyp za?spv$@cS}1!u(Pa|7&hlWx_GX8rYb{MPdwwKjEM^FGBGZF|+nv!kp3jRX8^GQ9!853t?5SP9EeC?35i zhD40$m}m#4>R;Xdk5TxqhpS!)FexP1oBFh`;r3=`yKn7po|uI~Le6qy+L7s$#xQ)& z94&2azmJWnx^HYLkJ<^|xrKIRJL*z!Nxg9f=vlYOKSaan8y#%D_2B>K0rL_sPug;UuraCX!!zWE3FR%Y%3@x5(f z#~ZgpAqSr0aEm68DK`VBmrnmdy|pqFb7db&Q^E zj71E8PPt14U$VlDKUjHYi{n7%Q>m-~q&Bj3MdikA)5(LE!Mpy#4SlFICr)=>Ceg*Y zE&4*HE%MOrq%GAbbjUO#c1y8FdYE_ue@Qr9qvz)dWPAO@x(=GZFlPRB4n{j9Rj-uD zFk`0%`rfBB9FjM;(I4HKa9A_}H3?l{gOOU1_RejEYb?5eAaRJ?k?fAU+U>8R3QU}B z#-SB?#d>t$ThImP(q+BxrKK%N0|{4T?1`*Mv>hD5PUply{xI`-AL|cr865X`-9}p5 z-LdvOyR?T4`;;VTkGH9~(S`Sy)JdNYcIDp3@~^a)qL=Qa4`w3DYq|T47L8tuBwOPn z;C`+4+a#-EhmH-lY+dV5(gaOU5`N#Ot=(&>7^Yc2CDXr+fRYDYiX*=w(;Ywm#rN27;)Cv4izc1>$%ks_Y0vf{6+1ai6V+2h&@N zKJ* z&AfqDN)7u{h_w&M&*HK{9I~j;s$T1{DcV=@aj?2-5!SSYL zT#$33r>ff|x~9f#Xe0JS@oT}c_GPcm%G8-_YJ>cJhOHs44l9*_3<;|1LoVx_4ZSk` z_p&@lPVxbpgEGY_Zt|&_$b=2X?`$KgOP2*7j}bi&t&{Wo6$p% zRmNGS4eGd;4=t)%F=3E)SCrD9KJyTSu_W@jiFu{kF+m1di*5}7o~tBl+E;FmV!1lx zo~=Ov{rkcCl!kX71@dlrXY3u&$*(7@;F1L(_Y8NHX2{@yL~k<3A0w?PELwf%Z@~L6 z8+$gACh?0xLZr%$BiEgM_9jvS{Y?}PmGjJ6%huf38Er6&%>;Y#bPLlV zt)<$T+K0i*mH0?Kg3p_2b)nI%N#R^9c29AT+?3z>1BEq_X#zrSLQmQCK~xT9Lklv1LNP_e||TU2nc3RrV2T zuNw#|q=mbz#7x!<}wthI># zy|+^}*)!SABI`7j;rgRPI+cl@n8~++ueVCmu6io1#MY*Q^xL<+Fpk@$&mk{+CrKjd z1{m9ZT$)YVP+TU$O7?}*OLJ~bArI!4Q#%)0=8h{_`}}#guD@ZM4qHW~o}D)7Zh6I9 z`2DnYBvTrLEg6mkCd1(i_$h2f-(p1crSBytkiN)&jo5JXnQ8tNV;ibC?a9S50LUX~ zP&2BO`EONQO$5K0*ys5ml$TEIVMi^{he;6Uk*?u2cABtbg7;BC#+cp1#jjG3-`vQY z$Nn?e=Mkni>@Bri0wK39l0P$RDR#)4lZ@0jdG~g^HS=dlwll zM>gSu3Z?VNcstAl=khwny;VDC3ErF{{o$ZyC4bJ@Jv&zPdxxny_rO5L08uzpUU zrG+-!{4ghXC;|`Cd8^lhXHNLro1DQ+5L)o&lkZ-^u|7%`w;2nbp^wCcDZ@OR?u-MFB+O-Pe1Of&AjF0 z^Bvdjc(1@t%c<4>Q8)hIVt*}In7#)&M(t{YFx>!e!@vB zq?H?F_U7l-(eVP2Z@kV>KskAc z6HbQ>5o~d!8@p-XQ%8E1aX8WPW;93Uq}UDbSIDJPRpxl^QGfYGQM(z><(kLV?tt1xf+=>de`R1h~_XHMvk|%w)@hE}e8t>}j?`*#` z6v;;2&kd|hrZa+W<>xhon6(rYL_PUPGzh-jl zc-E=L^}R(~56WQ&Imis90fz5d1%Hkid)6i!fb zC;T*ar?NkZibILwbc&0a%xr_vExp&5BVh!6>|-a4yD4mgnBgQTTp*d;Bz|L}C6a?D zhVLD%j6lCfF?gUaRc|A8-!5pd$jM7@yzaP;iocs9RmSY3s zVbO`Ox&L4R+sUFFPhG;Xx~cE}=WjjTMx<0U)Naf}G+>JnElxMHA+b$}qb^4*B}{Fj zu&~&fcq_7GP{1iG{nkZ)+Lf7ORmWC(>1NEDV07~W8pp*k#GlzqSmd5(hR$G!UuR?( z*ld3t5RT7bYicSg%W#r+HZ?#}^aWD!o#;J9n0{C*0o@66NO0qClbqMJc=)z2?{(LA_J_*RyLQiUMs~ zS;ZA{(|k?F&3H8ZIiW+*TxkW#Ch^5jPE!?dO^M4Op!lZ-hJA=`kr1z!$p*Nl5!R9` zgO|tDBU-@x9fX*A#|f348cnX$F!!jNF_kQoK3@3TN^lw-jo8{{)0sCuZW_O2m0potAlKX47*c7 zX`Bm&twoI>Bibw znGf2}oHmevPVB$LgM@1?EaxlM`Io2?-lP{Dv|+wL+8&%U(6Tbwe&=q!>AXu!U%F7A zT)I@wAU(J!-_qiB7WN}Oht2zURzq|>0)m<7>d=zM())ep3cunPquI}Cq{unbeMr{b zLTX-ws=YvXxL3aCgcO7s<#0?Mdvy?)RGN*#4_;It>T z4x;(?uo1wYozuLHKxSe0gL*&EogFtpSKqpOh9biw_D8{WLpiL~)fwpET?E@=drtYo zxA@>84YLai%F048W`O2=qyrS7vYwZ@y&%cNH*}BqM(UOE{(Z_~tOAgMlz)xPlSZFE zIG)+(VyK4yTUg-7;qj>NsKm3_pSsi)5AwB%g-ZnIFT<%A# zjY&(I|3K50@%M{d8ov@0s}TfyHr^MFWA=4-!}mp+cn9z zTOusT>T7mDJ2S$o5exW^3^N5$yb90i;@I zjj+8p_vXlh4@;&j*~F+Kb^awNSr%^nQi1fU$P?v*V20g7!sgx5jS2c`NsQKM$k&1X zc&3lPZn&!L*Pj-a8e2?-VRjNVd%oxdYGXc&qk>;B5@CtA_qHhPuW$#lX%(>AU4?<@ z$0Jt&R$HdqG>^;VNuf~_! zjk)xt+7McK@WSX1)d~^6mu>BBxnjuGOsSNLqsWle+JI?kY4swU2P#i4F2$iCPUI?# zh#V%>adaNUBYGox@b)k@3YFtCRm05DD@$R~n0 zy&B{CzgGIT8)J{2gF<>d%9A?n%W`SStv1)!aZGjcnrusVM+^?3RRig6b%8EfpyAUiJb|HrB*}zG-oC%XzwXov$%n{%_3`t7u?Cj)Z{Wb!yJU+8aN__k}WIult8J{5< zyMg&NaMNWYti|Vpuccc$!81geX&uva_llfYMq~C+DaV1ms+BN*)GuGiRU{BXB*sa4 zlv8;jjT>n!WiOA%>bZcDNq$;}Ua@A;SO#nb8K>5Bda$WeHZ_yCLIa#xe8=^`s48W) z`S==mL(t|Q4#oJr?-0}4M#VJZ<4gC`iuNm6sVY6_H%jhVp}XLw6=$>8I%?Pr2bg7P zq=K$VH^@)*UCyiYAu=h4KpFnkPT5Nm{N#Pi0lOIJ^HfI208fG8O;5`uWwiBprF7$ z=)=*EH*enTUs%n3yy@3!l@Yl+rkWfZgBVz?Mf_m%`b~l7eESi)aQktkHK5wDr=}V- zu(^g$tPJPph8cC=Y5^5?Ump^pj-4IxI-EGv=co+5dpRe$j6^U4c}W_!DF~n1(7_)m zO@;b6kf+7KVsv|UWAO{M^9DI@amdhNlVoOFTWY znOsEyp>kV5clmy8i!9saqGivP+-&3J;hegmmOQWu0@WLS{!La*s9E5}%H28JU?1Cw zJ1V->%c{B0P&e#X1SGjV8LnS(honF4O*7>KhNtO%t&I*lKHsAKjGDrPQ%Uy`f#RjR zR9^CyNi-Tj;A&?oYY`)wF3Vx!w)>pH2IH^7r#@9t#h$G+i!C~;XGZFLEA}BB@%w}K z7MVAB4)>D#0PFH(?KXa!>o;5^)^S+IReh5^>nBqi$qT{Um4|%*7#AXYpw|aMrrm@= zJ5?ai<)~E|G>>Q5HSDlui&d*ZF^-bd<;ffKMEd=XAAHtWz0*l*VZ> zT)}$ebUVewz z-P5zvBXUhQUvEUB?{P47w&=(ed<>VEs96}{8TNrmn#^RR~OPK262`xEMZg$Q5;B$kv zTLFxTx`&08s{P7D@!~NiO(fErGZ?YYSDuuM(FRBS?Vo{4H;ZLn(lmm;+w<~PRlKSycImZM61UmdWf|1jJV$| zdk|>(rTONkHAEgSgY5ofYx}kCq-!7sLz*DSUCBFo2HLV^(rSSUM+*IR4hM&WBr74N zo-3Uh!16&N$H;&PV**QcfNye z`5dT~e^GZI3c=oarqI6>dF)4}>gu+7AK7FTWQ--n{cZ_vPEQ)^m_}w5wEPCTznkZC zQByV=T3d~9(~oO&X84fNmQ|$9if;wKa@_-M+S{pHa!1uoX>GUK#NVE?Fx;3pLOV3~ zK+&{PisqknNZ!W?cE}rKq9zt}?W{pK>SFWk)^(ct=9x}ujk}QmwsT_9 zd7iPmE2o7edJ-ZzV|Y7Ar}N3@R~*7=G!^u8^MUAWOu*J;k8x}nwuUc!kiu}V2mQh5Offn$qfAlDINDk{ z#DzthQVfFyoGrAS(`XrOj%MYQlI!+e`y5nde1EvhxSu6!SCJpV#)RmPnI@vdDWpok zTR+@iUP*vLA#wIv1gr1{sSUje9K)633d+inMQ~M)J)pzxLJOvE{p?j}@I*E+%bAvd1lK%V1eSIc6^3sozhxfiib{P{xncdo-!%!3U13Xm8pBjNyK#vBMVmqN zUTKbDhD99&s@eN`g`v}N+UfyKDNG+6VACheCWBpN=poh9Ww^)}oif>LLQ)u#&%6*Z>5&lv?%+qzm#oofyZoDdPyW2odVaBjwyA?b>jc zcZ{{VmQixGHs_o&;6U3fOOUj&z`?AOStC-WwTA8stlB2EjyyLwIAB`*etC@21lpUf ze}7=C_?Fgy;}eXYO@-3eSL<0Y1QmQ+-@Dm>uyLZN{&~m>QGzvkVxlr32IZGEOoo}f zi!uA`VgWjiXCTuQsOVK&_Fnca^Djfwf!8M_!4+-Jz{8m#s2=82K@MydN?=$faopTEDJ2-xtJ#Dca72D6!LJio{x9b}AQ^m3m4;?vZ7@ zBN<0P6P(J?O|;n~!uz|QF~e;e{_;H&lVGV~-Mkmw?yPZ|N(G9{d4pT4t0Do9RU?2LvdvAT;pustQZ$|4N~vQ%cWf zpT>)wjti7V$v&n)J;Ig{ka*(*Ok8=yd;D7&tf~*$C=6u2udNi#JcIh+XrKsyXiE0| zDBRzpvDhP%k>E2v6G<#Ag`#O;@+=z-}_B!Pr9lzBb3vbB#k*J9&4DXZWoqK3}e|Bms4Y^!|^eizre220s znClh05z0L#@Pw(|MyaSij~z#3Ft z34fM)I$A;#$Sh~WiBg>W9c`3ufnM*WyF1=NByZeckD9R&t;S1`tdBQfIz)p*o^2ts zYEgiVxhDmP9Mz|o5H?k%kyPQ>^S9UfgO{Y6ZUspuZMUV7^_s*MTNx>336{C4tcKXG zJ6$u2Ls^ZD?=4p&D0kM`ny(56elDG@nmOvYU=avYHN81Okq4yAb9g@0J5*r`&$HRv zQaz4gj*u>7X_{Hj%wc_9(`#Z@qL%S&dJAEMy;+XNeV_iUhVU1{gI0{$qq@3koHZDX zr>YIC&)Xp6v0>+5KJ)&>V`8fZ3Oex^x5%~{5ui6Dp_QLwR`@#qI-jWz>NPtY7kRx( zJG6nrW4R{2Yv3&FgRlED?wo;dgNr>{GbX{JTL zaVv0exMMP5m;oA+J3G5jyza&MErmy;oX)?VzF1U}=B;DuhzpO{EU`M6TxHF&E)=Sz zS-oTj_L)8Gb&ipeO>Wl zWO%*)qxjP&V(#ZtJUi~fUa+5GWhzlMFo|a|NTdW%&i7shB_jR$%ZR3&P7i(o*1MKT zL!KHF%H9|B;1Ww0{J9mkEjXd$PR@N{+TWIKJlxz-SA;l@2SMm$W?=+$dYnJ%st9C& z84p6>Fe=%S>{a&J#i6dOqi^@z8+kC)+-+IKgE4TsxGw}bkM-gF*ha4BfxMrUsIDBhj4tDbW<+f6K-7lo zw#<7@LlZyIa7N#S>()x7-Ju`(EPzaCt&t#q&581~_bUz-y18ouY@lV|7}un>xy7WX zW^!>+)Z=2`(d--*pMOzuUCR~SeRonvi+z*j+d`EK8PoGccDlBD8sBDvL!lns^Vka4 z2fxc%yHgEj@Cp>qptO(J@x0KPzt6#gt*jsn#fmRlr%SGNCrbG9R>^#J^*zmp`pUW_ z)I^-=w{L%`|KbRv=L-!Cukx%vJLg$hix7dPf4{I>$ljbov~6{x6S}yXIBay==y*NH z5OjYCeTbeeF?@Sj-X%q_Kz4n-ULDHfDRer3Kg+oDl&6W_uP@Ja^EK!~sO6}Mn6cpZ z+rYAxRxr<98OSFT1ji@JG}D=KS4w1;q=sw%_F>yZ*`fJ2>4)g+Yu1FR;Ui*$MfPI- zrj%t1rX&}yeRP)}SKAfqr5tJXn|$pnN~-rSzwsP4dxz?OiN~#6_qpg@uMS{2VRkOR zPx@uNQmqH2OKYksS~!n(>cS;j>T)_xq$Ud(E+xINpWnM@-Q;GRd((+8^6<%pN_Oyz zjWG{m<<+-GnMCe&`?_g-`|!{nHlWvauVacCC#LABSyxljC|n{LE61&C*8b>zU}^sX z=V6UkJwm@3L!SW!vLd3h@`07$;d_b^3w6uFd|WuB+?J0X*eIJ^YuG%YFgUdXVxU?E^GM5PK~((b2_=Wz(yd!E3^c>N1X=ff#z}qk&r? z6~d9t28HKR4R?~Of=rFQ?tMZoD?6n>g7uc$t>_(u`oIM(o})@H5gYlNmRh~fH5LF& z+p*g1REoQT_!6Iaa=Hq!vrCls<)&iG+e0IM`%_my4?6bj+Zt%ijj$4%nb<-4x#!!# z1=-E4hxuj^Kt1bb3DNG)6_Z!OP|)F0Qw9rfEA$Xa=n77ktfcTeTt>0Tpe6~Vd})a1 z;PcE2h$skyDWImB`!wY0Ih#hJQiia+)z#~)!78odk6GWo*}6wysD}tTGnM7`cXSBZ zL0#hEIQDV^5?LFbW9`j#T2k-y;#1b$$ouav9RNkL4QkgM-v19+lc)^Qub)`?c#jk@O;dLDT(C28_>U z#T)k#6mVHhMAHED28SOzHv^S7g;|!q@C)pXmo+a`I>_S;nip&7Zxs;M2An>Bj^dhQ zvba)=QQn}raABO`gm3NPpgx)a%x=T5+ZV};Qj|x(_okdB8m16Uu}(%FP8QCFe2zis zu1MtIB72HyDn#Jj_ACqAF+kx+4-;?fg_M8Zh?bIIuK})rQLdLRH*bx`u)t5%;Hbt| zLkG@FzAm_9dHyuHtp)?A)kn{ckiAbyYiZU&V&WFUg&tb<5wUY9NBiv!byIB^;FnLE z05;J8B1dVY>H2FNQ#3DQ{TQoola)f}@qss=4tn@;0N ze#4*BQ(8|9uwGu6>5;S6IK26kWkWA_`yvp-=GgnaUov*_GsasCj>|q=r{jhbygPD0IWasSc>{PZz&}7wc-_4Ch28@m(NsS-4X`Ww-s$)Nrbii0R87a+Uce_lq~}2W zPTe>kT6n%^EC~m|C5U^`DSJk|*(%ptEu|NN1vDEh3PX)|hQjNmWcV&=wU z^{U_CW==ldY4SZH?|WnAlQ-F|Gm`re>^=HI1j5>=YPBw daO)%5*GT(YZL7n*rEtKXtfZ1esrU!~{{zKCWa0n- literal 0 HcmV?d00001 From c4f522b41467aa8514cd7f57d1331419e0447d45 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 16:32:15 +0100 Subject: [PATCH 329/526] Support arbintrary structure collections --- src/Common/ErrorCodes.cpp | 1 + src/Storages/NamedCollections.cpp | 432 ++++++++++++++---- src/Storages/NamedCollections.h | 52 +-- .../tests/gtest_named_collections.cpp | 79 +++- 4 files changed, 434 insertions(+), 130 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 26f837f9fe5..ce96b1a9917 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -638,6 +638,7 @@ M(667, NOT_INITIALIZED) \ M(668, INVALID_STATE) \ M(669, UNKNOWN_NAMED_COLLECTION) \ + M(669, NAMED_COLLECTION_ALREADY_EXISTS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 94870507b59..f598a302516 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -1,12 +1,15 @@ #include "NamedCollections.h" +#include #include +#include #include #include #include #include #include #include +#include #include #include #include @@ -15,18 +18,63 @@ namespace DB { -static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections."; +static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; + extern const int NAMED_COLLECTION_ALREADY_EXISTS; } namespace { std::string getCollectionPrefix(const std::string & collection_name) { - return NAMED_COLLECTIONS_CONFIG_PREFIX + collection_name; + return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); + } + + /// Enumerate keys paths of the config recursively. + /// E.g. if `enumerate_paths` = {"root.key1"} and config like + /// + /// + /// + /// + /// + /// + /// + /// + /// + /// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4" + void collectKeys( + const Poco::Util::AbstractConfiguration & config, + std::queue & enumerate_paths, + std::set & result) + { + if (enumerate_paths.empty()) + return; + + auto initial_paths = std::move(enumerate_paths); + enumerate_paths = {}; + while (!initial_paths.empty()) + { + auto path = initial_paths.front(); + initial_paths.pop(); + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(path, keys); + + if (keys.empty()) + { + result.insert(path); + } + else + { + for (const auto & key : keys) + enumerate_paths.emplace(path + '.' + key); + } + } + + collectKeys(config, enumerate_paths, result); } } @@ -36,35 +84,48 @@ NamedCollectionFactory & NamedCollectionFactory::instance() return instance; } -void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & server_config) +void NamedCollectionFactory::initialize( + const Poco::Util::AbstractConfiguration & server_config) { std::lock_guard lock(mutex); if (is_initialized) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Named collection factory is already initialzied"); + "Named collection factory already initialized"); } config = &server_config; is_initialized = true; } -void NamedCollectionFactory::assertInitialized(std::lock_guard & /* lock */) const +void NamedCollectionFactory::assertInitialized( + std::lock_guard & /* lock */) const { if (!is_initialized) { throw Exception( ErrorCodes::LOGICAL_ERROR, - "Named collection factory must be initialized before used"); + "Named collection factory must be initialized before being used"); } } bool NamedCollectionFactory::exists(const std::string & collection_name) const { std::lock_guard lock(mutex); + return existsUnlocked(collection_name, lock); +} + +bool NamedCollectionFactory::existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const +{ assertInitialized(lock); - return config->has(getCollectionPrefix(collection_name)); + /// Named collections can be added via SQL command or via config. + /// Named collections from config are loaded on first access, + /// therefore it might not be in `named_collections` map yet. + return named_collections.contains(collection_name) + || config->has(getCollectionPrefix(collection_name)); } NamedCollectionPtr NamedCollectionFactory::get( @@ -74,11 +135,13 @@ NamedCollectionPtr NamedCollectionFactory::get( std::lock_guard lock(mutex); assertInitialized(lock); - if (!exists(collection_name)) + if (!existsUnlocked(collection_name, lock)) + { throw Exception( ErrorCodes::UNKNOWN_NAMED_COLLECTION, - "There is no named collection `{}` in config", + "There is no named collection `{}`", collection_name); + } return getImpl(collection_name, collection_info, lock); } @@ -90,7 +153,7 @@ NamedCollectionPtr NamedCollectionFactory::tryGet( std::lock_guard lock(mutex); assertInitialized(lock); - if (!exists(collection_name)) + if (!existsUnlocked(collection_name, lock)) return nullptr; return getImpl(collection_name, collection_info, lock); @@ -104,87 +167,192 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( auto it = named_collections.find(collection_name); if (it == named_collections.end()) { - const auto collection_prefix = getCollectionPrefix(collection_name); - const auto collection_view = config->createView(collection_prefix); - - auto collection = std::make_unique(); - collection->initialize(*collection_view, collection_info); - it = named_collections.emplace(collection_name, std::move(collection)).first; + it = named_collections.emplace( + collection_name, + std::make_unique( + *config, collection_name, collection_info)).first; } return it->second; } +void NamedCollectionFactory::add( + const std::string & collection_name, + NamedCollectionPtr collection) +{ + std::lock_guard lock(mutex); + auto [it, inserted] = named_collections.emplace(collection_name, collection); + if (!inserted) + { + throw Exception( + ErrorCodes::NAMED_COLLECTION_ALREADY_EXISTS, + "A named collection `{}` already exists", + collection_name); + } +} + +void NamedCollectionFactory::remove(const std::string & collection_name) +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + if (!existsUnlocked(collection_name, lock)) + { + throw Exception( + ErrorCodes::UNKNOWN_NAMED_COLLECTION, + "There is no named collection `{}`", + collection_name); + } + + if (config->has(collection_name)) + { + throw Exception( + ErrorCodes::NOT_IMPLEMENTED, + "Collection {} is defined in config and cannot be removed", + collection_name); + } + + [[maybe_unused]] auto removed = named_collections.erase(collection_name); + assert(removed); +} + +NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const +{ + std::lock_guard lock(mutex); + assertInitialized(lock); + + NamedCollections result(named_collections); + + Poco::Util::AbstractConfiguration::Keys config_collections_names; + config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); + + for (const auto & name : config_collections_names) + { + if (result.contains(name)) + continue; + + const auto collection_prefix = getCollectionPrefix(name); + std::queue enumerate_input; + std::set enumerate_result; + + enumerate_input.push(collection_prefix); + collectKeys(*config, enumerate_input, enumerate_result); + + NamedCollectionInfo collection_info; + + /// Collection does not have any keys. + /// (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1; + if (!collection_is_empty) + { + for (const auto & path : enumerate_result) + { + collection_info.emplace( + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + path.substr(std::strlen(collection_prefix.data()) + 1), + NamedCollectionValueInfo{}); + } + } + + result.emplace( + name, std::make_unique(*config, name, collection_info)); + } + + return result; +} + struct NamedCollection::Impl { - std::unordered_map collection; +private: + using IConfigurationPtr = Poco::AutoPtr; + using ConfigurationPtr = Poco::AutoPtr; - ImplPtr copy() const + /// Named collection configuration + /// + /// ... + /// + ConfigurationPtr config; + /// Information about the values of keys. Key is a path to the + /// value represented as a dot concatenated list of keys. + const CollectionInfo collection_info; + +public: + Impl(const Poco::Util::AbstractConfiguration & config_, + const std::string & collection_name_, + const NamedCollectionInfo & collection_info_) + : config(createEmptyConfiguration(collection_name_)) + , collection_info(collection_info_) { - auto impl = std::make_unique(); - impl->collection = collection; - return impl; + auto collection_path = getCollectionPrefix(collection_name_); + for (const auto & [key, value_info] : collection_info) + copyConfigValue( + config_, collection_path + '.' + key, *config, key, value_info.type); } Value get(const Key & key) const { - auto it = collection.find(key); - if (it == collection.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "There is no key: {}", key); - return it->second; + auto value_info = collection_info.at(key); + return getConfigValue(*config, key, value_info.type, value_info.is_required); } - void replace(const Key & key, const Value & value) + void set(const Key & key, const Value & value) { - auto it = collection.find(key); - if (it == collection.end()) - collection.emplace(key, value); - else - it->second = value; + setConfigValue(*config, key, value); } - void initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) + /// Get a string representation of the collection structure. + /// Used for debugging and tests. + std::string toString() const { - for (const auto & [key, key_info] : collection_info) + /// Convert a collection config like + /// + /// value0 + /// + /// value2 + /// + /// value3 + /// + /// + /// + /// to a string: + /// "key0: value0 + /// key1: + /// key2: value2 + /// key3: + /// key4: value3" + WriteBufferFromOwnString wb; + for (const auto & [key, value_info] : collection_info) { - const auto & default_value = key_info.default_value; - const bool has_value = config.has(key); + Strings key_parts; + splitInto<'.'>(key_parts, key); + size_t tab_cnt = 0; - if (!default_value && !has_value) - continue; - - Field value; - switch (key_info.type) + for (auto it = key_parts.begin(); it != key_parts.end(); ++it) { - case Field::Types::Which::String: - value = has_value ? config.getString(key) : default_value->get(); - break; - case Field::Types::Which::UInt64: - value = has_value ? config.getUInt64(key) : default_value->get(); - break; - case Field::Types::Which::Int64: - value = has_value ? config.getInt64(key) : default_value->get(); - break; - case Field::Types::Which::Float64: - value = has_value ? config.getDouble(key) : default_value->get(); - break; - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unsupported type: {}", toString(key_info.type)); + if (it != key_parts.begin()) + wb << '\n' << std::string(tab_cnt++, '\t'); + wb << *it << ':'; } - - collection.emplace(key, value); + wb << '\t' << convertFieldToString(get(key)) << '\n'; } + return wb.str(); } +private: static void validate( const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) + const std::string & collection_path, + const NamedCollectionInfo & collection_info_) { Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys("", config_keys); + config.keys(collection_path, config_keys); + checkKeys(config_keys, collection_info_); + } + static void checkKeys( + const Poco::Util::AbstractConfiguration::Keys & config_keys, + const NamedCollectionInfo & collection_info) + + { auto get_suggestion = [&](bool only_required_keys) { std::string suggestion; @@ -228,16 +396,121 @@ struct NamedCollection::Impl fmt::join(required_keys, ", "), get_suggestion(true)); } } + + static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) + { + using DocumentPtr = Poco::AutoPtr; + DocumentPtr xml_document(new Poco::XML::Document()); + xml_document->appendChild(xml_document->createElement(root_name)); + ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document)); + return config; + } + + using ConfigValueType = Field::Types::Which; + static void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path, + ConfigValueType type) + { + using Type = Field::Types::Which; + switch (type) + { + case Type::String: + to_config.setString(to_path, from_config.getString(from_path)); + break; + case Type::UInt64: + to_config.setUInt64(to_path, from_config.getUInt64(from_path)); + break; + case Type::Int64: + to_config.setInt64(to_path, from_config.getInt64(from_path)); + break; + case Type::Float64: + to_config.setDouble(to_path, from_config.getDouble(from_path)); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + } + + static void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const Field & value) + { + using Type = Field::Types::Which; + switch (value.getType()) + { + case Type::String: + config.setString(path, value.safeGet()); + break; + case Type::UInt64: + config.setUInt64(path, value.safeGet()); + break; + case Type::Int64: + config.setInt64(path, value.safeGet()); + break; + case Type::Float64: + config.setDouble(path, value.safeGet()); + break; + default: + throw Exception( + ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + } + + static Field getConfigValue( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + ConfigValueType type, + bool throw_not_found, + std::optional default_value = std::nullopt) + { + const bool has_value = config.has(path); + if (!has_value) + { + if (throw_not_found) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Failed to find key `{}` in config, but this key is required", + path); + } + else if (!default_value) + return Null{}; + } + + Field value; + + using Type = Field::Types::Which; + switch (type) + { + case Type::String: + value = has_value ? config.getString(path) : default_value->get(); + break; + case Type::UInt64: + value = has_value ? config.getUInt64(path) : default_value->get(); + break; + case Type::Int64: + value = has_value ? config.getInt64(path) : default_value->get(); + break; + case Type::Float64: + value = has_value ? config.getDouble(path) : default_value->get(); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); + } + return value; + } }; -NamedCollection::NamedCollection() +NamedCollection::NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const CollectionInfo & collection_info) + : pimpl(std::make_unique(config, collection_path, collection_info)) { - pimpl = std::make_unique(); -} - -NamedCollection::NamedCollection(ImplPtr pimpl_) -{ - pimpl = std::move(pimpl_); } NamedCollection::Value NamedCollection::get(const Key & key) const @@ -245,28 +518,9 @@ NamedCollection::Value NamedCollection::get(const Key & key) const return pimpl->get(key); } -std::shared_ptr NamedCollection::copy() const +std::string NamedCollection::toString() const { - return std::make_shared(pimpl->copy()); -} - -void NamedCollection::validate( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) const -{ - pimpl->validate(config, collection_info); -} - -void NamedCollection::initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) -{ - pimpl->initialize(config, collection_info); -} - -void NamedCollection::replace(const Key & key, const Value & value) -{ - pimpl->replace(key, value); + return pimpl->toString(); } } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 870d167889d..cfc7ef5ef2f 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -10,7 +10,7 @@ namespace DB class NamedCollection; using NamedCollectionPtr = std::shared_ptr; struct NamedCollectionValueInfo; -using NamedCollectionInfo = std::unordered_map; +using NamedCollectionInfo = std::map; /** * A factory of immutable named collections. @@ -28,6 +28,8 @@ using NamedCollectionInfo = std::unordered_map; + NamedCollections getAll() const; private: NamedCollectionPtr getImpl( @@ -48,10 +57,14 @@ private: const NamedCollectionInfo & collection_info, std::lock_guard & lock) const; - using NamedCollections = std::unordered_map; + bool existsUnlocked( + const std::string & collection_name, + std::lock_guard & lock) const; + mutable NamedCollections named_collections; private: + /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; void assertInitialized(std::lock_guard & lock) const; @@ -74,31 +87,17 @@ private: public: using Key = std::string; using Value = Field; + using ValueInfo = NamedCollectionValueInfo; + using CollectionInfo = NamedCollectionInfo; + + NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const CollectionInfo & collection_info); Value get(const Key & key) const; - void replace(const Key & key, const Value & value); - - /// Copy current named collection to allow modification as - /// NamedConnectionFactory returns immutable collections. - std::shared_ptr copy() const; - - NamedCollection(); - explicit NamedCollection(ImplPtr pimpl_); - -protected: - /// Initialize from config. `config` must be a view to the required collection, - /// e.g. the root of `config` is the root of collection. - void initialize( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info); - - /// Validate named collection in config. - /// Throws exception if named collection keys in config are not the same as - /// expected (contains unknown keys or misses required keys) - void validate( - const Poco::Util::AbstractConfiguration & config, - const NamedCollectionInfo & collection_info) const; + std::string toString() const; }; @@ -109,7 +108,8 @@ protected: struct NamedCollectionValueInfo { /// Type of the value. One of: String, UInt64, Int64, Double. - Field::Types::Which type; + using Type = Field::Types::Which; + Type type = Type::String; /// Optional default value for the case if there is no such key in config. std::optional default_value; /// Is this value required or optional? Throw exception if the value is diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 669e7bae25b..34dfe5c1087 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -6,15 +6,15 @@ using namespace DB; -TEST(NamedCollections, Simple) +TEST(NamedCollections, SimpleConfig) { std::string xml(R"CONFIG( value1 2 - -3 - 4.4 + 3.3 + -4 value4 @@ -34,6 +34,18 @@ TEST(NamedCollections, Simple) ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3", {}) == nullptr); + auto collections = NamedCollectionFactory::instance().getAll(); + ASSERT_EQ(collections.size(), 2); + ASSERT_TRUE(collections.contains("collection1")); + ASSERT_TRUE(collections.contains("collection2")); + + ASSERT_EQ(collections["collection1"]->toString(), + R"CONFIG(key1: value1 +key2: 2 +key3: 3.3 +key4: -4 +)CONFIG"); + using ValueInfo = NamedCollectionValueInfo; ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; ValueInfo uint_def{Field::Types::Which::UInt64, std::nullopt, true}; @@ -43,16 +55,17 @@ TEST(NamedCollections, Simple) NamedCollectionInfo collection1_info; collection1_info.emplace("key1", string_def); collection1_info.emplace("key2", uint_def); - collection1_info.emplace("key3", int_def); - collection1_info.emplace("key4", double_def); + collection1_info.emplace("key3", double_def); + collection1_info.emplace("key4", int_def); auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); ASSERT_TRUE(collection1 != nullptr); + ASSERT_TRUE(collection1->get("key1").safeGet() == "value1"); ASSERT_TRUE(collection1->get("key2").safeGet() == 2); - ASSERT_TRUE(collection1->get("key3").safeGet() == -3); - ASSERT_TRUE(collection1->get("key4").safeGet() == 4.4); + ASSERT_TRUE(collection1->get("key3").safeGet() == 3.3); + ASSERT_TRUE(collection1->get("key4").safeGet() == -4); NamedCollectionInfo collection2_info; collection2_info.emplace("key4", string_def); @@ -64,12 +77,48 @@ TEST(NamedCollections, Simple) ASSERT_TRUE(collection2->get("key4").safeGet() == "value4"); ASSERT_TRUE(collection2->get("key5").safeGet() == 5); - ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); - - auto mutable_collection2 = collection2->copy(); - mutable_collection2->replace("key4", UInt64(4)); - mutable_collection2->replace("key7", "value7"); - - ASSERT_TRUE(mutable_collection2->get("key4").safeGet() == 4); - ASSERT_TRUE(mutable_collection2->get("key7").safeGet() == "value7"); + ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); } + +// TEST(NamedCollections, NestedConfig) +// { +// std::string xml(R"CONFIG( +// +// +// +// value1 +// +// +// value2_1 +// +// +// value2_4 +// value2_5 +// +// +// +// +// +// )CONFIG"); +// +// Poco::XML::DOMParser dom_parser; +// Poco::AutoPtr document = dom_parser.parseString(xml); +// Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); +// +// ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); +// +// using ValueInfo = NamedCollectionValueInfo; +// ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; +// +// NamedCollectionInfo collection1_info; +// collection1_info.emplace("key1.key1_1", string_def); +// collection1_info.emplace("key2.key2_1", string_def); +// collection1_info.emplace("key2.key2_2.key2_3.key2_4", string_def); +// collection1_info.emplace("key2.key2_2.key2_3.key2_5", string_def); +// +// auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); +// ASSERT_TRUE(collection1 != nullptr); +// +// ASSERT_TRUE(collection1->get("key1.key1_1").safeGet() == "value1"); +// +// } From b1e942e14c6f7bcc5494f5150c5684cbd87877f2 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 15:59:11 +0000 Subject: [PATCH 330/526] Do not run global test with sanitizers --- .../configs/global_overcommit_tracker.xml | 0 .../test_global_overcommit_tracker/test.py | 61 +++++++++++++++++++ .../test_overcommit_tracker/test.py | 37 +---------- 3 files changed, 62 insertions(+), 36 deletions(-) rename tests/integration/{test_overcommit_tracker => test_global_overcommit_tracker}/configs/global_overcommit_tracker.xml (100%) create mode 100644 tests/integration/test_global_overcommit_tracker/test.py diff --git a/tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml b/tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml similarity index 100% rename from tests/integration/test_overcommit_tracker/configs/global_overcommit_tracker.xml rename to tests/integration/test_global_overcommit_tracker/configs/global_overcommit_tracker.xml diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py new file mode 100644 index 00000000000..654f11be45a --- /dev/null +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -0,0 +1,61 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance( + "node", main_configs=["configs/global_overcommit_tracker.xml"] +) + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + + +GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" +GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" + + +def test_global_overcommit(): + # NOTE: another option is to increase waiting time. + if ( + node.is_built_with_thread_sanitizer() + or node.is_built_with_address_sanitizer() + or node.is_built_with_memory_sanitizer() + ): + pytest.skip("doesn't fit in memory limits") + + node.query("CREATE USER IF NOT EXISTS A") + node.query("GRANT ALL ON *.* TO A") + node.query("CREATE USER IF NOT EXISTS B") + node.query("GRANT ALL ON *.* TO B") + + responses_A = list() + responses_B = list() + for i in range(100): + if i % 2 == 0: + responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) + else: + responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) + + overcommited_killed = False + for response in responses_A: + _, err = response.get_answer_and_error() + if "MEMORY_LIMIT_EXCEEDED" in err: + overcommited_killed = True + finished = False + for response in responses_B: + _, err = response.get_answer_and_error() + if err == "": + finished = True + + assert overcommited_killed, "no overcommited task was killed" + assert finished, "all tasks are killed" + + node.query("DROP USER IF EXISTS A") + node.query("DROP USER IF EXISTS B") diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 5c18ee950d5..50af8525b99 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -5,7 +5,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance( - "node", main_configs=["configs/global_overcommit_tracker.xml"] + "node" ) @@ -18,9 +18,6 @@ def start_cluster(): cluster.shutdown() -GLOBAL_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=1" -GLOBAL_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS memory_overcommit_ratio_denominator_for_user=80000000" - USER_TEST_QUERY_A = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=1" USER_TEST_QUERY_B = "SELECT groupArray(number) FROM numbers(2500000) SETTINGS max_memory_usage_for_user=2000000000,memory_overcommit_ratio_denominator=80000000" @@ -52,35 +49,3 @@ def test_user_overcommit(): assert finished, "all tasks are killed" node.query("DROP USER IF EXISTS A") - - -def test_global_overcommit(): - node.query("CREATE USER IF NOT EXISTS A") - node.query("GRANT ALL ON *.* TO A") - node.query("CREATE USER IF NOT EXISTS B") - node.query("GRANT ALL ON *.* TO B") - - responses_A = list() - responses_B = list() - for i in range(100): - if i % 2 == 0: - responses_A.append(node.get_query_request(GLOBAL_TEST_QUERY_A, user="A")) - else: - responses_B.append(node.get_query_request(GLOBAL_TEST_QUERY_B, user="B")) - - overcommited_killed = False - for response in responses_A: - _, err = response.get_answer_and_error() - if "MEMORY_LIMIT_EXCEEDED" in err: - overcommited_killed = True - finished = False - for response in responses_B: - _, err = response.get_answer_and_error() - if err == "": - finished = True - - assert overcommited_killed, "no overcommited task was killed" - assert finished, "all tasks are killed" - - node.query("DROP USER IF EXISTS A") - node.query("DROP USER IF EXISTS B") From 4e15d648e7028c220ebacf823b249b380b472419 Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Thu, 10 Nov 2022 16:02:45 +0000 Subject: [PATCH 331/526] Add test_global_overcommit_tracker/__init__.py --- tests/integration/test_global_overcommit_tracker/__init__.py | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_global_overcommit_tracker/__init__.py diff --git a/tests/integration/test_global_overcommit_tracker/__init__.py b/tests/integration/test_global_overcommit_tracker/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From bed7ac9442d90e1abc29a5fc0c5630b0a0b0119a Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 17:05:04 +0100 Subject: [PATCH 332/526] Support arbitrary structure collections --- programs/local/LocalServer.cpp | 3 ++ programs/server/Server.cpp | 3 ++ src/Access/Common/AccessType.h | 1 + .../System/StorageSystemNamedCollections.cpp | 38 +++++++++++++++++++ .../System/StorageSystemNamedCollections.h | 21 ++++++++++ src/Storages/System/attachSystemTables.cpp | 2 + 6 files changed, 68 insertions(+) create mode 100644 src/Storages/System/StorageSystemNamedCollections.cpp create mode 100644 src/Storages/System/StorageSystemNamedCollections.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 3ac9c1e7c37..8514e0f11af 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -118,6 +119,8 @@ void LocalServer::initialize(Poco::Util::Application & self) config().getUInt("max_io_thread_pool_size", 100), config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); + + NamedCollectionFactory::instance().initialize(config()); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..079dae52f15 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -60,6 +60,7 @@ #include #include #include +#include #include #include #include @@ -732,6 +733,8 @@ int Server::main(const std::vector & /*args*/) config().getUInt("max_io_thread_pool_free_size", 0), config().getUInt("io_thread_pool_queue_size", 10000)); + NamedCollectionFactory::instance().initialize(config()); + /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) { diff --git a/src/Access/Common/AccessType.h b/src/Access/Common/AccessType.h index 5c85c93c98f..8263f50d1b0 100644 --- a/src/Access/Common/AccessType.h +++ b/src/Access/Common/AccessType.h @@ -130,6 +130,7 @@ enum class AccessType M(SHOW_ROW_POLICIES, "SHOW POLICIES, SHOW CREATE ROW POLICY, SHOW CREATE POLICY", TABLE, SHOW_ACCESS) \ M(SHOW_QUOTAS, "SHOW CREATE QUOTA", GLOBAL, SHOW_ACCESS) \ M(SHOW_SETTINGS_PROFILES, "SHOW PROFILES, SHOW CREATE SETTINGS PROFILE, SHOW CREATE PROFILE", GLOBAL, SHOW_ACCESS) \ + M(SHOW_NAMED_COLLECTIONS, "SHOW NAMED COLLECTIONS", GLOBAL, SHOW_ACCESS) \ M(SHOW_ACCESS, "", GROUP, ACCESS_MANAGEMENT) \ M(ACCESS_MANAGEMENT, "", GROUP, ALL) \ \ diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp new file mode 100644 index 00000000000..a53b6dc9bd3 --- /dev/null +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -0,0 +1,38 @@ +#include "StorageSystemNamedCollections.h" +#include +#include +#include +#include +#include + + +namespace DB +{ + +NamesAndTypesList StorageSystemNamedCollections::getNamesAndTypes() +{ + return { + {"name", std::make_shared()}, + {"collection", std::make_shared()}, + }; +} + +StorageSystemNamedCollections::StorageSystemNamedCollections(const StorageID & table_id_) + : IStorageSystemOneBlock(table_id_) +{ +} + +void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const +{ + context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); + + auto collections = NamedCollectionFactory::instance().getAll(); + + for (const auto & [name, collection] : collections) + { + res_columns[0]->insert(name); + res_columns[1]->insert(collection->toString()); + } +} + +} diff --git a/src/Storages/System/StorageSystemNamedCollections.h b/src/Storages/System/StorageSystemNamedCollections.h new file mode 100644 index 00000000000..d20fa62d30b --- /dev/null +++ b/src/Storages/System/StorageSystemNamedCollections.h @@ -0,0 +1,21 @@ +#pragma once + +#include + +namespace DB +{ + +class StorageSystemNamedCollections final : public IStorageSystemOneBlock +{ +public: + explicit StorageSystemNamedCollections(const StorageID & table_id_); + + std::string getName() const override { return "SystemNamedCollections"; } + + static NamesAndTypesList getNamesAndTypes(); + +protected: + void fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo & query_info) const override; +}; + +} diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index d3b81f4d1f9..068f7ddce46 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -72,6 +72,7 @@ #include #include #include +#include #include #include #include @@ -174,6 +175,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b attach(context, system_database, "filesystem_cache"); attach(context, system_database, "remote_data_paths"); attach(context, system_database, "certificates"); + attach(context, system_database, "named_collections"); if (has_zookeeper) attach(context, system_database, "zookeeper"); From a47aac3e2bec561364cdea6318c833bfa3b4d265 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 17:07:17 +0100 Subject: [PATCH 333/526] Better system table structure --- src/Storages/NamedCollections.cpp | 15 +++++++++-- src/Storages/NamedCollections.h | 4 +++ .../System/StorageSystemNamedCollections.cpp | 25 +++++++++++++++++-- 3 files changed, 40 insertions(+), 4 deletions(-) diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index f598a302516..e7c86f0e549 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -299,8 +299,14 @@ public: setConfigValue(*config, key, value); } - /// Get a string representation of the collection structure. - /// Used for debugging and tests. + std::map dumpStructure() + { + std::map result; + for (const auto & [key, _] : collection_info) + result.emplace(key, get(key)); + return result; + } + std::string toString() const { /// Convert a collection config like @@ -518,6 +524,11 @@ NamedCollection::Value NamedCollection::get(const Key & key) const return pimpl->get(key); } +std::map NamedCollection::dumpStructure() const +{ + return pimpl->dumpStructure(); +} + std::string NamedCollection::toString() const { return pimpl->toString(); diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index cfc7ef5ef2f..75884d96ce9 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -97,6 +97,10 @@ public: Value get(const Key & key) const; + std::map dumpStructure() const; + + /// Get a string representation of the collection structure. + /// Used for debugging and tests. std::string toString() const; }; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index a53b6dc9bd3..3fc9c5c8313 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -1,8 +1,13 @@ #include "StorageSystemNamedCollections.h" + +#include #include +#include #include +#include #include #include +#include #include @@ -13,7 +18,7 @@ NamesAndTypesList StorageSystemNamedCollections::getNamesAndTypes() { return { {"name", std::make_shared()}, - {"collection", std::make_shared()}, + {"collection", std::make_shared(std::make_shared(), std::make_shared())}, }; } @@ -31,7 +36,23 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte for (const auto & [name, collection] : collections) { res_columns[0]->insert(name); - res_columns[1]->insert(collection->toString()); + + auto * column_map = typeid_cast(res_columns[1].get()); + + auto & offsets = column_map->getNestedColumn().getOffsets(); + auto & tuple_column = column_map->getNestedData(); + auto & key_column = tuple_column.getColumn(0); + auto & value_column = tuple_column.getColumn(1); + + size_t size = 0; + for (const auto & [key, value] : collection->dumpStructure()) + { + key_column.insertData(key.data(), key.size()); + value_column.insert(convertFieldToString(value)); + size++; + } + + offsets.push_back(offsets.back() + size); } } From fe283af82bf7d168710440d69d28b4b6d5dbd194 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 10 Nov 2022 16:08:05 +0000 Subject: [PATCH 334/526] Automatic style fix --- tests/integration/test_global_overcommit_tracker/test.py | 1 + tests/integration/test_overcommit_tracker/test.py | 4 +--- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_global_overcommit_tracker/test.py b/tests/integration/test_global_overcommit_tracker/test.py index 654f11be45a..6bedc03a30e 100644 --- a/tests/integration/test_global_overcommit_tracker/test.py +++ b/tests/integration/test_global_overcommit_tracker/test.py @@ -8,6 +8,7 @@ node = cluster.add_instance( "node", main_configs=["configs/global_overcommit_tracker.xml"] ) + @pytest.fixture(scope="module", autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_overcommit_tracker/test.py b/tests/integration/test_overcommit_tracker/test.py index 50af8525b99..50979526e6a 100644 --- a/tests/integration/test_overcommit_tracker/test.py +++ b/tests/integration/test_overcommit_tracker/test.py @@ -4,9 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) -node = cluster.add_instance( - "node" -) +node = cluster.add_instance("node") @pytest.fixture(scope="module", autouse=True) From 5c147262e9b3338af46dfca67d22b5d544a5b44b Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Thu, 10 Nov 2022 16:36:30 +0000 Subject: [PATCH 335/526] add toc --- .../example-datasets/github.md | 99 +++++++++++++------ 1 file changed, 71 insertions(+), 28 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 6a5c9eab2f5..c8530f4dc92 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -20,6 +20,43 @@ As of November 8th, 2022, each tsv is approximately the following size and numbe - `file_changes` - 53M - 266,051 rows - `line_changes` - 2.7G - 7,535,157 rows +# Table of Contents + +- [ClickHouse GitHub data](#clickhouse-github-data) +- [Table of Contents](#table-of-contents) +- [Generating the data](#generating-the-data) +- [Downloading and inserting the data](#downloading-and-inserting-the-data) +- [Queries](#queries) + - [History of a single file](#history-of-a-single-file) + - [Find the current active files](#find-the-current-active-files) + - [List files with most modifications](#list-files-with-most-modifications) + - [What day of the week do commits usually occur?](#what-day-of-the-week-do-commits-usually-occur) + - [History of subdirectory/file - number of lines, commits and contributors over time](#history-of-subdirectoryfile---number-of-lines-commits-and-contributors-over-time) + - [List files with maximum number of authors](#list-files-with-maximum-number-of-authors) + - [Oldest lines of code in the repository](#oldest-lines-of-code-in-the-repository) + - [Files with longest history](#files-with-longest-history) + - [Distribution of contributors with respect to docs and code over the month](#distribution-of-contributors-with-respect-to-docs-and-code-over-the-month) + - [Authors with the most diverse impact](#authors-with-the-most-diverse-impact) + - [Favorite files for an author](#favorite-files-for-an-author) + - [Largest files with lowest number of authors](#largest-files-with-lowest-number-of-authors) + - [Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories](#commits-and-lines-of-code-distribution-by-time-by-weekday-by-author-for-specific-subdirectories) + - [Matrix of authors that shows what authors tends to rewrite another authors code](#matrix-of-authors-that-shows-what-authors-tends-to-rewrite-another-authors-code) + - [Who is the highest percentage contributor per day of week?](#who-is-the-highest-percentage-contributor-per-day-of-week) + - [Distribution of code age across repository](#distribution-of-code-age-across-repository) + - [What percentage of code for an author has been removed by other authors?](#what-percentage-of-code-for-an-author-has-been-removed-by-other-authors) + - [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) + - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) + - [Files sorted by average code age](#files-sorted-by-average-code-age) + - [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) + - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) + - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) + - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) + - [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) + - [Most consecutive days of commits by an author](#most-consecutive-days-of-commits-by-an-author) + - [Line by line commit history of a file](#line-by-line-commit-history-of-a-file) +- [Unsolved Questions](#unsolved-questions) + - [Git blame](#git-blame) + # Generating the data ```bash @@ -202,34 +239,40 @@ FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commit The tool suggests several queries via its help output. We have answered these in addition to some additional supplementary questions of interest: -- [History of a single file](#history-of-a-single-file) -- [Find the current active files](#find-the-current-active-files) -- [List files with most modifications](#list-files-with-most-modifications) -- [What day of the week do commits usually occur?](#what-day-of-the-week-do-commits-usually-occur) -- [List files with maximum number of authors](#list-files-with-maximum-number-of-authors) -- [Oldest lines of code in the repository](#oldest-lines-of-code-in-the-repository) -- [Files with longest history](#files-with-longest-history) -- [Distribution of contributors with respect to docs and code over the month](#distribution-of-contributors-with-respect-to-docs-and-code-over-the-month) -- [Authors with the most diverse impact](#authors-with-the-most-diverse-impact) -- [Favorite files for an author](#favorite-files-for-an-author) -- [Largest files with lowest number of authors](#largest-files-with-lowest-number-of-authors) -- [Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories](#commits-and-lines-of-code-distribution-by-time-by-weekday-by-author-for-specific-subdirectories) -- [Matrix of authors that shows what authors tends to rewrite another authors code](#matrix-of-authors-that-shows-what-authors-tends-to-rewrite-another-authors-code) -- [Who is the highest percentage contributor per day of week?](#who-is-the-highest-percentage-contributor-per-day-of-week) -- [Distribution of code age across repository](#distribution-of-code-age-across-repository) -- [What percentage of code for an author has been removed by other authors?](#what-percentage-of-code-for-an-author-has-been-removed-by-other-authors) -- [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) -- [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) -- [History of subdirectory/file - number of lines, commits and contributors over time](#history-of-subdirectoryfile---number-of-lines-commits-and-contributors-over-time) -- [Files sorted by average code age](#files-sorted-by-average-code-age) -- [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) -- [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) -- [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) -- [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) -- [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) -- [Most consecutive days of commits by an author](#most-consecutive-days-of-commits-by-an-author) -- [Line by line commit history of a file](#line-by-line-commit-history-of-a-file) -- [Show Git blame for a file](#git-blame) +- [ClickHouse GitHub data](#clickhouse-github-data) +- [Table of Contents](#table-of-contents) +- [Generating the data](#generating-the-data) +- [Downloading and inserting the data](#downloading-and-inserting-the-data) +- [Queries](#queries) + - [History of a single file](#history-of-a-single-file) + - [Find the current active files](#find-the-current-active-files) + - [List files with most modifications](#list-files-with-most-modifications) + - [What day of the week do commits usually occur?](#what-day-of-the-week-do-commits-usually-occur) + - [History of subdirectory/file - number of lines, commits and contributors over time](#history-of-subdirectoryfile---number-of-lines-commits-and-contributors-over-time) + - [List files with maximum number of authors](#list-files-with-maximum-number-of-authors) + - [Oldest lines of code in the repository](#oldest-lines-of-code-in-the-repository) + - [Files with longest history](#files-with-longest-history) + - [Distribution of contributors with respect to docs and code over the month](#distribution-of-contributors-with-respect-to-docs-and-code-over-the-month) + - [Authors with the most diverse impact](#authors-with-the-most-diverse-impact) + - [Favorite files for an author](#favorite-files-for-an-author) + - [Largest files with lowest number of authors](#largest-files-with-lowest-number-of-authors) + - [Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories](#commits-and-lines-of-code-distribution-by-time-by-weekday-by-author-for-specific-subdirectories) + - [Matrix of authors that shows what authors tends to rewrite another authors code](#matrix-of-authors-that-shows-what-authors-tends-to-rewrite-another-authors-code) + - [Who is the highest percentage contributor per day of week?](#who-is-the-highest-percentage-contributor-per-day-of-week) + - [Distribution of code age across repository](#distribution-of-code-age-across-repository) + - [What percentage of code for an author has been removed by other authors?](#what-percentage-of-code-for-an-author-has-been-removed-by-other-authors) + - [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) + - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) + - [Files sorted by average code age](#files-sorted-by-average-code-age) + - [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) + - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) + - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) + - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) + - [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) + - [Most consecutive days of commits by an author](#most-consecutive-days-of-commits-by-an-author) + - [Line by line commit history of a file](#line-by-line-commit-history-of-a-file) +- [Unsolved Questions](#unsolved-questions) + - [Git blame](#git-blame) These queries are of approximately increasing complexity vs. the tool's arbitrary order. From e415fc6e95fcaaf4d718cd7ce5d637a199a5cefd Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Nov 2022 19:38:55 +0300 Subject: [PATCH 336/526] Revert "Optimize TTL merge, completely expired parts can be removed in time" --- src/Interpreters/PartLog.cpp | 3 --- src/Interpreters/PartLog.h | 2 -- .../MergeTree/MergeFromLogEntryTask.cpp | 8 +++---- .../MergeTree/MergeTreeDataMergerMutator.cpp | 23 ++++--------------- src/Storages/MergeTree/MergeType.cpp | 2 +- src/Storages/MergeTree/MergeType.h | 2 -- src/Storages/StorageMergeTree.cpp | 7 +----- .../02293_part_log_has_merge_reason.reference | 2 +- 8 files changed, 11 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/PartLog.cpp b/src/Interpreters/PartLog.cpp index f1b29a8a9b2..b35ee50b98e 100644 --- a/src/Interpreters/PartLog.cpp +++ b/src/Interpreters/PartLog.cpp @@ -31,8 +31,6 @@ PartLogElement::MergeReasonType PartLogElement::getMergeReasonType(MergeType mer return TTL_DELETE_MERGE; case MergeType::TTLRecompress: return TTL_RECOMPRESS_MERGE; - case MergeType::TTLDrop: - return TTL_DROP_MERGE; } throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Unknown MergeType {}", static_cast(merge_type)); @@ -74,7 +72,6 @@ NamesAndTypesList PartLogElement::getNamesAndTypes() {"RegularMerge", static_cast(REGULAR_MERGE)}, {"TTLDeleteMerge", static_cast(TTL_DELETE_MERGE)}, {"TTLRecompressMerge", static_cast(TTL_RECOMPRESS_MERGE)}, - {"TTLDropMerge", static_cast(TTL_DROP_MERGE)}, } ); diff --git a/src/Interpreters/PartLog.h b/src/Interpreters/PartLog.h index 6180963908d..2ce0dfd76de 100644 --- a/src/Interpreters/PartLog.h +++ b/src/Interpreters/PartLog.h @@ -41,8 +41,6 @@ struct PartLogElement TTL_DELETE_MERGE = 3, /// Merge with recompression TTL_RECOMPRESS_MERGE = 4, - /// Merge assigned to drop parts (with TTLMergeSelector) - TTL_DROP_MERGE = 5, }; String query_id; diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 0711ed157a5..9a9b8a4a6bb 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -152,9 +152,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() } /// Start to make the main work - size_t need_total_size = 0; - if (entry.merge_type != MergeType::TTLDrop) - need_total_size = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts); + size_t estimated_space_for_merge = MergeTreeDataMergerMutator::estimateNeededDiskSpace(parts); /// Can throw an exception while reserving space. IMergeTreeDataPart::TTLInfos ttl_infos; @@ -182,7 +180,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() std::optional tagger; ReservationSharedPtr reserved_space = storage.balancedReservation( metadata_snapshot, - need_total_size, + estimated_space_for_merge, max_volume_index, future_merged_part->name, future_merged_part->part_info, @@ -192,7 +190,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() if (!reserved_space) reserved_space = storage.reserveSpacePreferringTTLRules( - metadata_snapshot, need_total_size, ttl_infos, time(nullptr), max_volume_index); + metadata_snapshot, estimated_space_for_merge, ttl_infos, time(nullptr), max_volume_index); future_merged_part->uuid = entry.new_part_uuid; future_merged_part->updatePath(storage, reserved_space.get()); diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 0b2f17c0d27..fcc1b4cb3e2 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -314,31 +314,18 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMerge( if (metadata_snapshot->hasAnyTTL() && merge_with_ttl_allowed && !ttl_merges_blocker.isCancelled()) { /// TTL delete is preferred to recompression - TTLDeleteMergeSelector drop_ttl_selector( + TTLDeleteMergeSelector delete_ttl_selector( next_delete_ttl_merge_times_by_partition, current_time, data_settings->merge_with_ttl_timeout, - true); + data_settings->ttl_only_drop_parts); - parts_to_merge = drop_ttl_selector.select(parts_ranges,data_settings->max_bytes_to_merge_at_max_space_in_pool); + parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); if (!parts_to_merge.empty()) { - future_part->merge_type = MergeType::TTLDrop; + future_part->merge_type = MergeType::TTLDelete; } - else if (!data_settings->ttl_only_drop_parts) - { - TTLDeleteMergeSelector delete_ttl_selector( - next_delete_ttl_merge_times_by_partition, - current_time, - data_settings->merge_with_ttl_timeout, - false); - - parts_to_merge = delete_ttl_selector.select(parts_ranges, max_total_size_to_merge); - if (!parts_to_merge.empty()) - future_part->merge_type = MergeType::TTLDelete; - } - - if (parts_to_merge.empty() && metadata_snapshot->hasAnyRecompressionTTL()) + else if (metadata_snapshot->hasAnyRecompressionTTL()) { TTLRecompressMergeSelector recompress_ttl_selector( next_recompress_ttl_merge_times_by_partition, diff --git a/src/Storages/MergeTree/MergeType.cpp b/src/Storages/MergeTree/MergeType.cpp index 96862b6cca1..045114578d0 100644 --- a/src/Storages/MergeTree/MergeType.cpp +++ b/src/Storages/MergeTree/MergeType.cpp @@ -20,7 +20,7 @@ MergeType checkAndGetMergeType(UInt32 merge_type) bool isTTLMergeType(MergeType merge_type) { - return merge_type == MergeType::TTLDelete || merge_type == MergeType::TTLRecompress || merge_type == MergeType::TTLDrop; + return merge_type == MergeType::TTLDelete || merge_type == MergeType::TTLRecompress; } } diff --git a/src/Storages/MergeTree/MergeType.h b/src/Storages/MergeTree/MergeType.h index fa5df87b25a..ce9a40c5931 100644 --- a/src/Storages/MergeTree/MergeType.h +++ b/src/Storages/MergeTree/MergeType.h @@ -19,8 +19,6 @@ enum class MergeType TTLDelete = 2, /// Merge with recompression TTLRecompress = 3, - /// Merge assigned to drop parts (with TTLMergeSelector) - TTLDrop = 4, }; /// Check parsed merge_type from raw int and get enum value. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 805242fbec9..a450a9ef3a9 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -929,12 +929,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMerge( if (isTTLMergeType(future_part->merge_type)) getContext()->getMergeList().bookMergeWithTTL(); - /// If merge_type is TTLDrop, no need to reserve disk space - size_t need_total_size = 0; - if (future_part->merge_type != MergeType::TTLDrop) - need_total_size = MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts); - - merging_tagger = std::make_unique(future_part, need_total_size, *this, metadata_snapshot, false); + merging_tagger = std::make_unique(future_part, MergeTreeDataMergerMutator::estimateNeededDiskSpace(future_part->parts), *this, metadata_snapshot, false); return std::make_shared(future_part, std::move(merging_tagger), std::make_shared()); } diff --git a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference index 37156ee37e7..220107cf15b 100644 --- a/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference +++ b/tests/queries/0_stateless/02293_part_log_has_merge_reason.reference @@ -1 +1 @@ -MergeParts TTLDropMerge +MergeParts TTLDeleteMerge From afc6c8bb7389dc94fee9d0e9c69049ef00bd83ee Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 2 Nov 2022 16:05:59 +0100 Subject: [PATCH 337/526] tests: add ATTACH FROM s3_plain for wide part Signed-off-by: Azat Khuzhin --- .../configs/disk_s3.xml | 29 +++++++++--- .../test_attach_backup_from_s3_plain/test.py | 45 ++++++++++++++++--- 2 files changed, 60 insertions(+), 14 deletions(-) diff --git a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml index 67278694d39..e4f481d3325 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml +++ b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml @@ -9,23 +9,38 @@ minio123 33554432 - + s3_plain - - http://minio1:9001/root/data/disks/disk_s3_plain/backup/ + + http://minio1:9001/root/data/disks/disk_s3_plain/backup_compact/ minio minio123 33554432 - + + + s3_plain + + http://minio1:9001/root/data/disks/disk_s3_plain/backup_wide/ + minio + minio123 + 33554432 + - +

- attach_disk_s3_plain + s3_backup_compact
- + + + +
+ s3_backup_wide +
+
+
diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py index 35d53d5b8bd..f544a0c6e0a 100644 --- a/tests/integration/test_attach_backup_from_s3_plain/test.py +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -21,20 +21,51 @@ def start_cluster(): cluster.shutdown() -def test_attach_backup(): +@pytest.mark.parametrize( + "table_name,backup_name,storage_policy,min_bytes_for_wide_part", + [ + pytest.param( + "compact", "backup_compact", "s3_backup_compact", int(1e9), id="compact" + ), + pytest.param("wide", "backup_wide", "s3_backup_wide", int(0), id="wide"), + ], +) +def test_attach_compact_part( + table_name, backup_name, storage_policy, min_bytes_for_wide_part +): node.query( f""" + -- Catch any errors (NOTE: warnings are ok) + set send_logs_level='error'; + -- BACKUP writes Ordinary like structure set allow_deprecated_database_ordinary=1; - create database ordinary engine=Ordinary; - create table ordinary.test_backup_attach engine=MergeTree() order by tuple() as select * from numbers(100); + create database ordinary_db engine=Ordinary; + + create table ordinary_db.{table_name} engine=MergeTree() order by tuple() as select * from numbers(100); -- NOTE: name of backup ("backup") is significant. - backup table ordinary.test_backup_attach TO Disk('backup_disk_s3_plain', 'backup'); + backup table ordinary_db.{table_name} TO Disk('backup_disk_s3_plain', '{backup_name}'); - drop table ordinary.test_backup_attach; - attach table ordinary.test_backup_attach (number UInt64) engine=MergeTree() order by tuple() settings storage_policy='attach_policy_s3_plain'; + drop table ordinary_db.{table_name}; + attach table ordinary_db.{table_name} (number UInt64) + engine=MergeTree() + order by tuple() + settings + min_bytes_for_wide_part={min_bytes_for_wide_part}, + storage_policy='{storage_policy}'; """ ) - assert int(node.query("select count() from ordinary.test_backup_attach")) == 100 + assert int(node.query(f"select count() from ordinary_db.{table_name}")) == 100 + + node.query( + f""" + -- NOTE: be aware not to DROP the table, but DETACH first to keep it in S3. + detach table ordinary_db.{table_name}; + + -- NOTE: DROP DATABASE cannot be done w/o this due to metadata leftovers + set force_remove_data_recursively_on_drop=1; + drop database ordinary_db sync; + """ + ) From 434b9c14d80612ff085deeca77eeaf4345f13b8f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 05:34:39 +0100 Subject: [PATCH 338/526] Handle all entries for azure blob storage (not only first 5k) Signed-off-by: Azat Khuzhin --- .../AzureBlobStorage/AzureObjectStorage.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index c3549701ec1..81bfdc6d5ff 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -149,10 +149,17 @@ void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWit blobs_list_options.Prefix = path; auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); - auto blobs_list = blobs_list_response.Blobs; + for (;;) + { + auto blobs_list = blobs_list_response.Blobs; - for (const auto & blob : blobs_list) - children.emplace_back(blob.Name, blob.BlobSize); + for (const auto & blob : blobs_list) + children.emplace_back(blob.Name, blob.BlobSize); + + if (!blobs_list_response.HasPage()) + break; + blobs_list_response.MoveToNextPage(); + } } /// Remove file. Throws exception if file doesn't exists or it's a directory. From 7b7ae175dff578b44a96b88fa3960126b0c6c4cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 05:38:19 +0100 Subject: [PATCH 339/526] Add max_keys for IObjectStorage::findAllFiles() v2: Fix google-default-arguments for IObjectStorage::findAllFiles() v3: Update max_keys for S3 requests in S3ObjectStorage::findAllFiles() loop v4: Clarify things about max_keys vs list_object_keys_size in S3ObjectStorage::findAllFiles() Signed-off-by: Azat Khuzhin --- .../AzureBlobStorage/AzureObjectStorage.cpp | 6 +++++- .../AzureBlobStorage/AzureObjectStorage.h | 2 +- .../ObjectStorages/Cached/CachedObjectStorage.cpp | 4 ++-- .../ObjectStorages/Cached/CachedObjectStorage.h | 2 +- ...skObjectStorageRemoteMetadataRestoreHelper.cpp | 4 ++-- src/Disks/ObjectStorages/IObjectStorage.cpp | 2 +- src/Disks/ObjectStorages/IObjectStorage.h | 6 +++++- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 15 +++++++++++++-- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 2 +- 9 files changed, 31 insertions(+), 12 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 81bfdc6d5ff..a99fafec71b 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -141,12 +141,14 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO return std::make_unique(std::move(buffer), std::move(finalize_callback), object.absolute_path); } -void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const +void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const { auto client_ptr = client.get(); Azure::Storage::Blobs::ListBlobsOptions blobs_list_options; blobs_list_options.Prefix = path; + if (max_keys) + blobs_list_options.PageSizeHint = max_keys; auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); for (;;) @@ -156,6 +158,8 @@ void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWit for (const auto & blob : blobs_list) children.emplace_back(blob.Name, blob.BlobSize); + if (max_keys && children.size() >= static_cast(max_keys)) + break; if (!blobs_list_response.HasPage()) break; blobs_list_response.MoveToNextPage(); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 6fd41dae2ec..32b69a9ea78 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -84,7 +84,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index f3d3f049dc1..505b26ebb3a 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -282,9 +282,9 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const +void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const { - object_storage->findAllFiles(path, children); + object_storage->findAllFiles(path, children, max_keys); } ObjectMetadata CachedObjectStorage::getObjectMetadata(const std::string & path) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 64e6eed45bb..b84382a762a 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -72,7 +72,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 4ea42616ba2..2ae5e46eb66 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -390,7 +390,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * }; RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path, children); + source_object_storage->findAllFiles(restore_information.source_path, children, /* max_keys= */ 0); restore_files(children); @@ -540,7 +540,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject }; RelativePathsWithSize children; - source_object_storage->findAllFiles(restore_information.source_path + "operations/", children); + source_object_storage->findAllFiles(restore_information.source_path + "operations/", children, /* max_keys= */ 0); restore_file_operations(children); if (restore_information.detached) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 3f8ac566603..45ecad35747 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -14,7 +14,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &) const +void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &, int) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "findAllFiles() is not supported"); } diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 9451ae31b07..c570dfb6e9b 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -73,13 +73,17 @@ public: /// find . -type f /// /// @param children - out files (relative paths) with their sizes. + /// @param max_keys - return not more then max_keys children + /// NOTE: max_keys is not the same as list_object_keys_size (disk property) + /// - if max_keys is set not more then max_keys keys should be returned + /// - however list_object_keys_size determine the size of the batch and should return all keys /// /// NOTE: It makes sense only for real object storages (S3, Azure), since /// it is used only for one of the following: /// - send_metadata (to restore metadata) /// - see DiskObjectStorage::restoreMetadataIfNeeded() /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk - virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children) const; + virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const; /// Analog of directory content for object storage (object storage does not /// have "directory" definition, but it can be emulated with usage of diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 0c421ee03d7..6b4c7bb3e12 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -248,7 +248,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN std::move(s3_buffer), std::move(finalize_callback), object.absolute_path); } -void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const +void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -256,7 +256,10 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi Aws::S3::Model::ListObjectsV2Request request; request.SetBucket(bucket); request.SetPrefix(path); - request.SetMaxKeys(settings_ptr->list_object_keys_size); + if (max_keys) + request.SetMaxKeys(max_keys); + else + request.SetMaxKeys(settings_ptr->list_object_keys_size); Aws::S3::Model::ListObjectsV2Outcome outcome; do @@ -275,6 +278,14 @@ void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSi for (const auto & object : objects) children.emplace_back(object.GetKey(), object.GetSize()); + if (max_keys) + { + int keys_left = max_keys - static_cast(children.size()); + if (keys_left <= 0) + break; + request.SetMaxKeys(keys_left); + } + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); } while (outcome.GetResult().GetIsTruncated()); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 56f1c895924..447ca034aac 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -105,7 +105,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children, int max_keys) const override; void getDirectoryContents(const std::string & path, RelativePathsWithSize & files, std::vector & directories) const override; From 41884d3b8848c3931bbe32424ff2b335e5dcf484 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 05:39:53 +0100 Subject: [PATCH 340/526] Optimize MetadataStorageFromPlainObjectStorage::getFileSize() Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 259f6e01fd7..456e7582307 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -66,7 +66,7 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { RelativePathsWithSize children; - object_storage->findAllFiles(getAbsolutePath(path), children); + object_storage->findAllFiles(getAbsolutePath(path), children, 1); if (children.empty()) return 0; if (children.size() != 1) From 2cbc61df1807cfb5efda0431a8c0bed68797015c Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 05:40:57 +0100 Subject: [PATCH 341/526] Fix ATTACH FROM s3_plain for wide part Previously because of incorrect MetadataStorageFromPlainObjectStorage::exists(), that used S3ObjectStorage::exists() before, which works only for existing keys, not some intermediate path. Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 456e7582307..8ace42547f3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -39,8 +39,11 @@ std::filesystem::path MetadataStorageFromPlainObjectStorage::getAbsolutePath(con bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) const { - auto object = StoredObject::create(*object_storage, getAbsolutePath(path)); - return object_storage->exists(object); + RelativePathsWithSize children; + /// NOTE: exists() cannot be used here since it works only for existing + /// key, and does not work for some intermediate path. + object_storage->findAllFiles(getAbsolutePath(path), children, 1); + return !children.empty(); } bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) const From 9296bfb161715c5e720dbce05b21104a028b485f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 08:25:42 +0100 Subject: [PATCH 342/526] Handle list_object_keys_size for Azure Signed-off-by: Azat Khuzhin --- .../ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp | 3 ++- .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp | 2 ++ .../ObjectStorages/AzureBlobStorage/AzureObjectStorage.h | 5 ++++- 3 files changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp index 590433e91e5..53712639f46 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureBlobStorageAuth.cpp @@ -147,7 +147,8 @@ std::unique_ptr getAzureBlobStorageSettings(const Po config.getUInt64(config_prefix + ".max_single_part_upload_size", 100 * 1024 * 1024), config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".max_single_read_retries", 3), - config.getInt(config_prefix + ".max_single_download_retries", 3) + config.getInt(config_prefix + ".max_single_download_retries", 3), + config.getInt(config_prefix + ".list_object_keys_size", 1000) ); } diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index a99fafec71b..15ddbe551da 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -149,6 +149,8 @@ void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWit blobs_list_options.Prefix = path; if (max_keys) blobs_list_options.PageSizeHint = max_keys; + else + blobs_list_options.PageSizeHint = settings.get()->list_object_keys_size; auto blobs_list_response = client_ptr->ListBlobs(blobs_list_options); for (;;) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 32b69a9ea78..648016fb732 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -29,11 +29,13 @@ struct AzureObjectStorageSettings uint64_t max_single_part_upload_size_, uint64_t min_bytes_for_seek_, int max_single_read_retries_, - int max_single_download_retries_) + int max_single_download_retries_, + int list_object_keys_size_) : max_single_part_upload_size(max_single_part_upload_size_) , min_bytes_for_seek(min_bytes_for_seek_) , max_single_read_retries(max_single_read_retries_) , max_single_download_retries(max_single_download_retries_) + , list_object_keys_size(list_object_keys_size_) { } @@ -41,6 +43,7 @@ struct AzureObjectStorageSettings uint64_t min_bytes_for_seek; size_t max_single_read_retries; size_t max_single_download_retries; + int list_object_keys_size; }; using AzureClient = Azure::Storage::Blobs::BlobContainerClient; From d3e0f1687336826e1db7cf1fd61772a069c103fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 08:51:48 +0100 Subject: [PATCH 343/526] Remove common root path in MetadataStorageFromPlainObjectStorage::listDirectory() This path should not leak into users/outside, since later it can be passed to other APIs, i.e. exists() and so on. Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 8ace42547f3..34a9ae021b7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -88,6 +88,11 @@ std::vector MetadataStorageFromPlainObjectStorage::listDirectory(co result.push_back(path_size.relative_path); for (const auto & directory : directories) result.push_back(directory); + for (auto & row : result) + { + chassert(row.starts_with(object_storage_root_path)); + row.erase(0, object_storage_root_path.size()); + } return result; } From ac27bc0193039a87e3b1289f5866cdfacfaa5c0d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 3 Nov 2022 11:48:21 +0100 Subject: [PATCH 344/526] Add trailing slash (/) in S3ObjectStorage::getDirectoryContents() Otherwise it returns only the directory itself. Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 6b4c7bb3e12..099a7d458d0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -299,7 +299,12 @@ void S3ObjectStorage::getDirectoryContents(const std::string & path, Aws::S3::Model::ListObjectsV2Request request; request.SetBucket(bucket); - request.SetPrefix(path); + /// NOTE: if you do "ls /foo" instead of "ls /foo/" over S3 with this API + /// it will return only "/foo" itself without any underlying nodes. + if (path.ends_with("/")) + request.SetPrefix(path); + else + request.SetPrefix(path + "/"); request.SetMaxKeys(settings_ptr->list_object_keys_size); request.SetDelimiter("/"); From e220906c9e8ff9345d17ea8b282dfb4d67ee60a7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Nov 2022 13:51:19 +0100 Subject: [PATCH 345/526] Analyzer added identifier typo corrections --- src/Access/SettingsConstraints.cpp | 2 +- src/Analyzer/Identifier.h | 25 +++ src/Analyzer/Passes/QueryAnalysisPass.cpp | 196 +++++++++++++++++++++- src/Common/NamePrompter.cpp | 6 +- src/Common/NamePrompter.h | 8 +- 5 files changed, 220 insertions(+), 17 deletions(-) diff --git a/src/Access/SettingsConstraints.cpp b/src/Access/SettingsConstraints.cpp index d97a78c78ab..0317e43f8d1 100644 --- a/src/Access/SettingsConstraints.cpp +++ b/src/Access/SettingsConstraints.cpp @@ -147,7 +147,7 @@ bool SettingsConstraints::checkImpl(const Settings & current_settings, SettingCh { if (const auto hints = current_settings.getHints(change.name); !hints.empty()) { - e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); + e.addMessage(fmt::format("Maybe you meant {}", toString(hints))); } } throw; diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index 2252ce2854f..abfee7cafc2 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -152,6 +152,11 @@ public: return popFirst(1); } + void pop_front() /// NOLINT + { + return popFirst(); + } + void popLast(size_t parts_to_remove_size) { assert(parts_to_remove_size <= parts.size()); @@ -365,6 +370,26 @@ inline std::ostream & operator<<(std::ostream & stream, const IdentifierView & i } +template <> +struct std::hash +{ + size_t operator()(const DB::Identifier & identifier) const + { + std::hash hash; + return hash(identifier.getFullName()); + } +}; + +template <> +struct std::hash +{ + size_t operator()(const DB::IdentifierView & identifier) const + { + std::hash hash; + return hash(identifier.getFullName()); + } +}; + /// See https://fmt.dev/latest/api.html#formatting-user-defined-types template <> diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 5dbc8ffdb3c..76a51c05243 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1,5 +1,7 @@ #include +#include + #include #include #include @@ -1056,6 +1058,27 @@ private: const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name); + static void getTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + const QueryTreeNodePtr & table_expression, + const TableExpressionData & table_expression_data, + std::unordered_set & valid_identifiers_result); + + static void getScopeValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + const IdentifierResolveScope & scope, + bool allow_expression_identifiers, + bool allow_function_identifiers, + bool allow_table_expression_identifiers, + std::unordered_set & valid_identifiers_result); + + static void getScopeWithParentScopesValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + const IdentifierResolveScope & scope, + bool allow_expression_identifiers, + bool allow_function_identifiers, + bool allow_table_expression_identifiers, + std::unordered_set & valid_identifiers_result); + + static std::vector getIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers); + static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); static QueryTreeNodePtr tryGetLambdaFromSQLUserDefinedFunctions(const std::string & function_name, ContextPtr context); @@ -1358,6 +1381,140 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN return sort_column_projection_name_buffer.str(); } +/// Get valid identifiers for typo correction from table expression +void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + const QueryTreeNodePtr & table_expression, + const TableExpressionData & table_expression_data, + std::unordered_set & valid_identifiers_result) +{ + for (const auto & [column_name, _] : table_expression_data.column_name_to_column_node) + { + Identifier column_identifier(column_name); + if (unresolved_identifier.getPartsSize() == column_identifier.getPartsSize()) + valid_identifiers_result.insert(column_identifier); + + if (table_expression->hasAlias()) + { + Identifier column_identifier_with_alias({table_expression->getAlias()}); + for (const auto & column_identifier_part : column_identifier) + column_identifier_with_alias.push_back(column_identifier_part); + + if (unresolved_identifier.getPartsSize() == column_identifier_with_alias.getPartsSize()) + valid_identifiers_result.insert(column_identifier_with_alias); + } + + if (!table_expression_data.table_name.empty()) + { + Identifier column_identifier_with_table_name({table_expression_data.table_name}); + for (const auto & column_identifier_part : column_identifier) + column_identifier_with_table_name.push_back(column_identifier_part); + + if (unresolved_identifier.getPartsSize() == column_identifier_with_table_name.getPartsSize()) + valid_identifiers_result.insert(column_identifier_with_table_name); + } + + if (!table_expression_data.database_name.empty() && !table_expression_data.table_name.empty()) + { + Identifier column_identifier_with_table_name_and_database_name({table_expression_data.database_name, table_expression_data.table_name}); + for (const auto & column_identifier_part : column_identifier) + column_identifier_with_table_name_and_database_name.push_back(column_identifier_part); + + if (unresolved_identifier.getPartsSize() == column_identifier_with_table_name_and_database_name.getPartsSize()) + valid_identifiers_result.insert(column_identifier_with_table_name_and_database_name); + } + } +} + +/// Get valid identifiers for typo correction from scope without looking at parent scopes +void QueryAnalyzer::getScopeValidIdentifiersForTypoCorrection( + const Identifier & unresolved_identifier, + const IdentifierResolveScope & scope, + bool allow_expression_identifiers, + bool allow_function_identifiers, + bool allow_table_expression_identifiers, + std::unordered_set & valid_identifiers_result) +{ + if (allow_expression_identifiers) + { + if (unresolved_identifier.isShort()) + { + for (const auto & [name, _] : scope.alias_name_to_expression_node) + valid_identifiers_result.insert(Identifier(name)); + } + + for (const auto & [table_expression, table_expression_data] : scope.table_expression_node_to_data) + { + getTableExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + table_expression, + table_expression_data, + valid_identifiers_result); + } + + } + + if (allow_function_identifiers && unresolved_identifier.isShort()) + { + for (const auto & [name, _] : scope.alias_name_to_expression_node) + valid_identifiers_result.insert(Identifier(name)); + } + + if (allow_table_expression_identifiers && unresolved_identifier.isShort()) + { + for (const auto & [name, _] : scope.alias_name_to_table_expression_node) + valid_identifiers_result.insert(Identifier(name)); + } + + if (unresolved_identifier.isShort()) + { + for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node) + { + auto expression_node_type = expression->getNodeType(); + + if (allow_expression_identifiers && isExpressionNodeType(expression_node_type)) + valid_identifiers_result.insert(Identifier(argument_name)); + else if (allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type)) + valid_identifiers_result.insert(Identifier(argument_name)); + else if (allow_table_expression_identifiers && isTableExpressionNodeType(expression_node_type)) + valid_identifiers_result.insert(Identifier(argument_name)); + } + } +} + +void QueryAnalyzer::getScopeWithParentScopesValidIdentifiersForTypoCorrection( + const Identifier & unresolved_identifier, + const IdentifierResolveScope & scope, + bool allow_expression_identifiers, + bool allow_function_identifiers, + bool allow_table_expression_identifiers, + std::unordered_set & valid_identifiers_result) +{ + const IdentifierResolveScope * current_scope = &scope; + + while (current_scope) + { + getScopeValidIdentifiersForTypoCorrection(unresolved_identifier, + *current_scope, + allow_expression_identifiers, + allow_function_identifiers, + allow_table_expression_identifiers, + valid_identifiers_result); + + current_scope = current_scope->parent_scope; + } +} + +std::vector QueryAnalyzer::getIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers) +{ + std::vector prompting_strings; + prompting_strings.reserve(valid_identifiers.size()); + + for (const auto & valid_identifier : valid_identifiers) + prompting_strings.push_back(valid_identifier.getFullName()); + + NamePrompter<1> prompter; + return prompter.getHints(unresolved_identifier.getFullName(), prompting_strings); +} + /** Wrap expression node in tuple element function calls for nested paths. * Example: Expression node: compound_expression. Nested path: nested_path_1.nested_path_2. * Result: tupleElement(tupleElement(compound_expression, 'nested_path_1'), 'nested_path_2'). @@ -2107,13 +2264,24 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id } if (!result_column || (!match_full_identifier && !compound_identifier)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Identifier '{}' cannot be resolved from {}{}. In scope {}", + { + std::string error_message = fmt::format("Identifier '{}' cannot be resolved from {}{}. In scope {}", identifier.getFullName(), table_expression_data.table_expression_description, table_expression_data.table_expression_name.empty() ? "" : " with name " + table_expression_data.table_expression_name, scope.scope_node->formatASTForErrorMessage()); + std::unordered_set valid_identifiers; + getTableExpressionValidIdentifiersForTypoCorrection(identifier, + table_expression_node, + table_expression_data, + valid_identifiers); + auto hints = getIdentifierTypoHints(identifier, valid_identifiers); + appendHintsMessage(error_message, hints); + + throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message); + } + QueryTreeNodePtr result_expression = result_column; bool clone_is_needed = true; @@ -4309,12 +4477,24 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id if (allow_table_expression) message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION); - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, - "Unknown {}{} identifier '{}' in scope {}", + std::string error_message = fmt::format("Unknown {}{} identifier '{}' in scope {}", toStringLowercase(IdentifierLookupContext::EXPRESSION), message_clarification, unresolved_identifier.getFullName(), scope.scope_node->formatASTForErrorMessage()); + + std::unordered_set valid_identifiers; + getScopeWithParentScopesValidIdentifiersForTypoCorrection(unresolved_identifier, + scope, + true, + allow_lambda_expression, + allow_table_expression, + valid_identifiers); + + auto hints = getIdentifierTypoHints(unresolved_identifier, valid_identifiers); + appendHintsMessage(error_message, hints); + + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, error_message); } if (node->getNodeType() == QueryTreeNodeType::LIST) @@ -4876,17 +5056,15 @@ void QueryAnalyzer::initializeTableExpressionColumns(const QueryTreeNodePtr & ta { table_expression_data.table_name = query_node ? query_node->getCTEName() : union_node->getCTEName(); table_expression_data.table_expression_description = "subquery"; - - if (table_expression_node->hasAlias()) - table_expression_data.table_expression_name = table_expression_node->getAlias(); } else if (table_function_node) { table_expression_data.table_expression_description = "table_function"; - if (table_function_node->hasAlias()) - table_expression_data.table_expression_name = table_function_node->getAlias(); } + if (table_expression_node->hasAlias()) + table_expression_data.table_expression_name = table_expression_node->getAlias(); + if (table_node || table_function_node) { const auto & storage_snapshot = table_node ? table_node->getStorageSnapshot() : table_function_node->getStorageSnapshot(); diff --git a/src/Common/NamePrompter.cpp b/src/Common/NamePrompter.cpp index c5a2224dcb4..ea42b801ee2 100644 --- a/src/Common/NamePrompter.cpp +++ b/src/Common/NamePrompter.cpp @@ -1,9 +1,10 @@ #include #include -namespace DB::detail +namespace DB { -void appendHintsMessageImpl(String & message, const std::vector & hints) + +void appendHintsMessage(String & message, const std::vector & hints) { if (hints.empty()) { @@ -12,4 +13,5 @@ void appendHintsMessageImpl(String & message, const std::vector & hints) message += ". Maybe you meant: " + toString(hints); } + } diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 962a89a8e76..8b69dd100ee 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -12,6 +12,7 @@ namespace DB { + template class NamePrompter { @@ -90,10 +91,7 @@ private: } }; -namespace detail -{ -void appendHintsMessageImpl(String & message, const std::vector & hints); -} +void appendHintsMessage(String & message, const std::vector & hints); template class IHints @@ -109,7 +107,7 @@ public: void appendHintsMessage(String & message, const String & name) const { auto hints = getHints(name); - detail::appendHintsMessageImpl(message, hints); + DB::appendHintsMessage(message, hints); } IHints() = default; From c627251f9e52347eaeddd9c55ca4ababfe04ce1b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Nov 2022 16:31:08 +0100 Subject: [PATCH 346/526] Added tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 ++++- .../02475_analyzer_subquery_compound_expression.reference | 3 +++ .../02475_analyzer_subquery_compound_expression.sql | 7 +++++++ 3 files changed, 14 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.reference create mode 100644 tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 76a51c05243..04fb3f59cff 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2289,7 +2289,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id { IdentifierView nested_path(identifier_view); nested_path.popFirst(); - auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, identifier_view); + auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, nested_path); resolveFunction(tuple_element_result, scope); result_expression = std::move(tuple_element_result); clone_is_needed = false; @@ -3758,6 +3758,9 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod */ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { + std::cout << "QueryAnalyzer::resolveFunction " << std::endl; + std::cout << node->dumpTree() << std::endl; + FunctionNodePtr function_node_ptr = std::static_pointer_cast(node); auto function_name = function_node_ptr->getFunctionName(); diff --git a/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.reference b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.reference new file mode 100644 index 00000000000..e26ee49e323 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.reference @@ -0,0 +1,3 @@ +(1,2) 1 2 +-- +1 2 diff --git a/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql new file mode 100644 index 00000000000..f96c834c057 --- /dev/null +++ b/tests/queries/0_stateless/02475_analyzer_subquery_compound_expression.sql @@ -0,0 +1,7 @@ +SET allow_experimental_analyzer=1; + +SELECT cast(tuple(1, 2), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value, value.value_1, value.value_2; + +SELECT '--'; + +SELECT value.value_1, value.value_2 FROM (SELECT cast(tuple(1, 2), 'Tuple(value_1 UInt64, value_2 UInt64)') AS value); From 66683dd17524df0fd40c9d71749c5ed0366c3148 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 7 Nov 2022 17:05:06 +0100 Subject: [PATCH 347/526] Added function typo corrections --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 44 +++++++++++++++++++---- 1 file changed, 38 insertions(+), 6 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 04fb3f59cff..88086959d14 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4112,11 +4112,15 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi if (function_node.isWindowFunction()) { if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) - throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, - "Aggregate function with name {} does not exists. In scope {}", + { + std::string error_message = fmt::format("Aggregate function with name {} does not exists. In scope {}", function_name, scope.scope_node->formatASTForErrorMessage()); + AggregateFunctionFactory::instance().appendHintsMessage(error_message, function_name); + throw Exception(ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION, error_message); + } + AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); @@ -4141,10 +4145,38 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi if (!function) { if (!AggregateFunctionFactory::instance().isAggregateFunctionName(function_name)) - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, - "Function with name {} does not exists. In scope {}", - function_name, - scope.scope_node->formatASTForErrorMessage()); + { + std::vector possible_function_names; + + auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context); + possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end()); + + function_names = UserDefinedSQLFunctionFactory::instance().getAllRegisteredNames(); + possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end()); + + function_names = FunctionFactory::instance().getAllRegisteredNames(); + possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end()); + + function_names = AggregateFunctionFactory::instance().getAllRegisteredNames(); + possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end()); + + for (auto & [name, lambda_node] : scope.alias_name_to_lambda_node) + { + if (lambda_node->getNodeType() == QueryTreeNodeType::LAMBDA) + possible_function_names.push_back(name); + } + + std::string error_message = fmt::format("Function with name {} does not exists. In scope {}", + function_name, + scope.scope_node->formatASTForErrorMessage()); + + NamePrompter<2> name_prompter; + auto hints = name_prompter.getHints(function_name, possible_function_names); + + appendHintsMessage(error_message, hints); + + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, error_message); + } AggregateFunctionProperties properties; auto aggregate_function = AggregateFunctionFactory::instance().get(function_name, argument_types, parameters, properties); From a6d2069c2b9c8e17995bcd8bf52b73be85914265 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 8 Nov 2022 19:36:43 +0100 Subject: [PATCH 348/526] Added compound types typo corrections --- src/Analyzer/Identifier.h | 12 +- src/Analyzer/Passes/QueryAnalysisPass.cpp | 218 +++++++++++++++------- src/Analyzer/Utils.cpp | 29 +++ src/Analyzer/Utils.h | 8 + src/Analyzer/tests/gtest_identifier.cpp | 16 ++ src/Common/NamePrompter.cpp | 13 +- src/Common/NamePrompter.h | 8 +- 7 files changed, 220 insertions(+), 84 deletions(-) diff --git a/src/Analyzer/Identifier.h b/src/Analyzer/Identifier.h index abfee7cafc2..71c5d784464 100644 --- a/src/Analyzer/Identifier.h +++ b/src/Analyzer/Identifier.h @@ -182,23 +182,21 @@ public: void push_back(std::string && part) /// NOLINT { - parts.push_back(std::move(part)); - full_name += '.'; - full_name += parts.back(); + emplace_back(std::move(part)); } void push_back(const std::string & part) /// NOLINT { - parts.push_back(part); - full_name += '.'; - full_name += parts.back(); + emplace_back(part); } template void emplace_back(Args&&... args) /// NOLINT { parts.emplace_back(std::forward(args)...); - full_name += '.'; + bool was_not_empty = parts.size() != 1; + if (was_not_empty) + full_name += '.'; full_name += parts.back(); } private: diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 88086959d14..a14d693c348 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1058,6 +1058,11 @@ private: const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name); + static void getCompoundExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + const DataTypePtr & compound_expression_type, + const Identifier & compound_expression_identifier, + std::unordered_set & valid_identifiers_result); + static void getTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const QueryTreeNodePtr & table_expression, const TableExpressionData & table_expression_data, @@ -1099,6 +1104,12 @@ private: static QueryTreeNodePtr tryResolveTableIdentifierFromDatabaseCatalog(const Identifier & table_identifier, ContextPtr context); + QueryTreeNodePtr tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier, + size_t identifier_bind_size, + const QueryTreeNodePtr & compound_expression, + String compound_expression_source, + IdentifierResolveScope & scope); + QueryTreeNodePtr tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); static bool tryBindIdentifierToAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope); @@ -1381,18 +1392,67 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN return sort_column_projection_name_buffer.str(); } +/// Get valid identifiers for typo correction from compound expression +void QueryAnalyzer::getCompoundExpressionValidIdentifiersForTypoCorrection( + const Identifier & unresolved_identifier, + const DataTypePtr & compound_expression_type, + const Identifier & compound_expression_identifier, + std::unordered_set & valid_identifiers_result) +{ + std::vector> identifiers_with_types_to_process; + identifiers_with_types_to_process.emplace_back(compound_expression_identifier, compound_expression_type.get()); + + while (!identifiers_with_types_to_process.empty()) + { + auto [identifier, type] = identifiers_with_types_to_process.back(); + identifiers_with_types_to_process.pop_back(); + + while (const DataTypeArray * array = checkAndGetDataType(type)) + type = array->getNestedType().get(); + + const DataTypeTuple * tuple = checkAndGetDataType(type); + + if (!tuple) + continue; + + const auto & tuple_element_names = tuple->getElementNames(); + size_t tuple_element_names_size = tuple_element_names.size(); + + for (size_t i = 0; i < tuple_element_names_size; ++i) { + const auto & element_name = tuple_element_names[i]; + const auto & element_type = tuple->getElements()[i]; + + identifier.push_back(element_name); + + if (identifier.getPartsSize() > unresolved_identifier.getPartsSize()) + break; + + valid_identifiers_result.insert(identifier); + identifiers_with_types_to_process.emplace_back(identifier, element_type.get()); + + identifier.pop_back(); + } + } +} + /// Get valid identifiers for typo correction from table expression -void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, +void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( + const Identifier & unresolved_identifier, const QueryTreeNodePtr & table_expression, const TableExpressionData & table_expression_data, std::unordered_set & valid_identifiers_result) { - for (const auto & [column_name, _] : table_expression_data.column_name_to_column_node) + for (const auto & [column_name, column_node] : table_expression_data.column_name_to_column_node) { Identifier column_identifier(column_name); if (unresolved_identifier.getPartsSize() == column_identifier.getPartsSize()) valid_identifiers_result.insert(column_identifier); + getCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + column_node->getColumnType(), + column_identifier, + valid_identifiers_result); + if (table_expression->hasAlias()) { Identifier column_identifier_with_alias({table_expression->getAlias()}); @@ -1858,6 +1918,55 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con return std::make_shared(std::move(storage), storage_lock, storage_snapshot); } +/// Resolve identifier from compound expression. +QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier, + size_t identifier_bind_size, + const QueryTreeNodePtr & compound_expression, + String compound_expression_source, + IdentifierResolveScope & scope) +{ + Identifier compound_expression_identifier; + for (size_t i = 0; i < identifier_bind_size; ++i) + compound_expression_identifier.push_back(expression_identifier[i]); + + IdentifierView nested_path(expression_identifier); + nested_path.popFirst(identifier_bind_size); + + auto expression_type = compound_expression->getResultType(); + + if (!nestedIdentifierCanBeResolved(expression_type, nested_path)) + { + std::unordered_set valid_identifiers; + getCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier, + expression_type, + compound_expression_identifier, + valid_identifiers); + + auto hints = getIdentifierTypoHints(expression_identifier, valid_identifiers); + + String compound_expression_from_error_message; + if (!compound_expression_source.empty()) + { + compound_expression_from_error_message += " from "; + compound_expression_from_error_message += compound_expression_source; + } + + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Identifier {} nested path {} cannot be resolved from type {}{}. In scope {}{}", + expression_identifier, + nested_path, + expression_type->getName(), + compound_expression_from_error_message, + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); + } + + auto tuple_element_result = wrapExpressionNodeInTupleElement(compound_expression, nested_path); + resolveFunction(tuple_element_result, scope); + + return tuple_element_result; +} + /** Resolve identifier from expression arguments. * * Expression arguments can be initialized during lambda analysis or they could be provided externally. @@ -1873,8 +1982,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con * It is important to support edge cases, where we lookup for table or function node, but argument has same name. * Example: WITH (x -> x + 1) AS func, (func -> func(1) + func) AS lambda SELECT lambda(1); * - * 3. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap node - * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. + * 3. If identifier is compound and identifier lookup is in expression context use `tryResolveIdentifierFromCompoundExpression`. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope) { @@ -1899,15 +2007,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromExpressionArguments(cons return {}; if (!resolve_full_identifier && identifier_lookup.identifier.isCompound() && identifier_lookup.isExpressionLookup()) - { - auto nested_path = IdentifierView(identifier_lookup.identifier); - nested_path.popFirst(); - - auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); - resolveFunction(tuple_element_result, scope); - - return tuple_element_result; - } + return tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, 1 /*identifier_bind_size*/, it->second, {}, scope); return it->second; } @@ -1974,11 +2074,8 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi * If identifier is resolved, depending on lookup context, erase entry from expression or lambda map. Check QueryExpressionsAliasVisitor documentation. * Pop node from current scope expressions in resolve process stack. * - * 5. If identifier is compound and identifier lookup is in expression context, pop first part from identifier lookup and wrap alias node - * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. - * - * Example: SELECT value AS alias, alias.nested_path. - * Result: SELECT value AS alias, tupleElement(value, 'nested_path') value.nested_path. + * 5. Pop node from current expressions to resolve. + * 6. If identifier is compound and identifier lookup is in expression context, use `tryResolveIdentifierFromCompoundExpression`. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { @@ -2069,13 +2166,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier { if (identifier_lookup.isExpressionLookup()) { - auto nested_path = IdentifierView(identifier_lookup.identifier); - nested_path.popFirst(); - - auto tuple_element_result = wrapExpressionNodeInTupleElement(result, nested_path); - resolveFunction(tuple_element_result, scope); - - result = tuple_element_result; + return tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, 1 /*identifier_bind_size*/, it->second, {}, scope); } else if (identifier_lookup.isFunctionLookup() || identifier_lookup.isTableExpressionLookup()) { @@ -2096,9 +2187,9 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier * 2. If identifier full name match table column use column. Save information that we resolve identifier using full name. * 3. Else if identifier binds to table column, use column. * 4. Try to resolve column ALIAS expression if it exists. - * 5. If identifier was compound and was not resolved using full name during step 1 pop first part from identifier lookup and wrap column node - * using nested parts of identifier using `wrapExpressionNodeInTupleElement` function. + * 5. If identifier was compound and was not resolved using full name during step 1 use `tryResolveIdentifierFromCompoundExpression`. * This can be the case with compound ALIAS columns. + * * Example: * CREATE TABLE test_table (id UInt64, value Tuple(id UInt64, value String), alias_value ALIAS value.id) ENGINE=TinyLog; */ @@ -2124,15 +2215,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableColumns(const Ident QueryTreeNodePtr result = it->second; if (!full_column_name_match && identifier.isCompound()) - { - auto nested_path = IdentifierView(identifier_lookup.identifier); - nested_path.popFirst(); - - auto tuple_element_result = wrapExpressionNodeInTupleElement(it->second, nested_path); - resolveFunction(tuple_element_result, scope); - - result = tuple_element_result; - } + return tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, 1 /*identifier_bind_size*/, it->second, {}, scope); return result; } @@ -2263,36 +2346,38 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id result_column = it->second; } - if (!result_column || (!match_full_identifier && !compound_identifier)) - { - std::string error_message = fmt::format("Identifier '{}' cannot be resolved from {}{}. In scope {}", - identifier.getFullName(), - table_expression_data.table_expression_description, - table_expression_data.table_expression_name.empty() ? "" : " with name " + table_expression_data.table_expression_name, - scope.scope_node->formatASTForErrorMessage()); + QueryTreeNodePtr result_expression; + bool clone_is_needed = true; + String table_expression_source = table_expression_data.table_expression_description; + if (!table_expression_data.table_expression_name.empty()) + table_expression_source += " with name " + table_expression_data.table_expression_name; + + if (!match_full_identifier && compound_identifier) + { + size_t identifier_bind_size = identifier_column_qualifier_parts + 1; + result_expression = tryResolveIdentifierFromCompoundExpression(identifier_lookup.identifier, identifier_bind_size, result_column, table_expression_source, scope); + clone_is_needed = false; + } + else + { + result_expression = result_column; + } + + if (!result_expression) + { std::unordered_set valid_identifiers; getTableExpressionValidIdentifiersForTypoCorrection(identifier, table_expression_node, table_expression_data, valid_identifiers); auto hints = getIdentifierTypoHints(identifier, valid_identifiers); - appendHintsMessage(error_message, hints); - throw Exception(ErrorCodes::BAD_ARGUMENTS, error_message); - } - - QueryTreeNodePtr result_expression = result_column; - bool clone_is_needed = true; - - if (!match_full_identifier && compound_identifier) - { - IdentifierView nested_path(identifier_view); - nested_path.popFirst(); - auto tuple_element_result = wrapExpressionNodeInTupleElement(result_expression, nested_path); - resolveFunction(tuple_element_result, scope); - result_expression = std::move(tuple_element_result); - clone_is_needed = false; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Identifier '{}' cannot be resolved from {}. In scope {}{}", + identifier.getFullName(), + table_expression_source, + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); } if (clone_is_needed) @@ -3758,9 +3843,6 @@ ProjectionNames QueryAnalyzer::resolveLambda(const QueryTreeNodePtr & lambda_nod */ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, IdentifierResolveScope & scope) { - std::cout << "QueryAnalyzer::resolveFunction " << std::endl; - std::cout << node->dumpTree() << std::endl; - FunctionNodePtr function_node_ptr = std::static_pointer_cast(node); auto function_name = function_node_ptr->getFunctionName(); @@ -4512,12 +4594,6 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id if (allow_table_expression) message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION); - std::string error_message = fmt::format("Unknown {}{} identifier '{}' in scope {}", - toStringLowercase(IdentifierLookupContext::EXPRESSION), - message_clarification, - unresolved_identifier.getFullName(), - scope.scope_node->formatASTForErrorMessage()); - std::unordered_set valid_identifiers; getScopeWithParentScopesValidIdentifiersForTypoCorrection(unresolved_identifier, scope, @@ -4527,9 +4603,13 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id valid_identifiers); auto hints = getIdentifierTypoHints(unresolved_identifier, valid_identifiers); - appendHintsMessage(error_message, hints); - throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, error_message); + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {}{} identifier '{}' in scope {}{}", + toStringLowercase(IdentifierLookupContext::EXPRESSION), + message_clarification, + unresolved_identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); } if (node->getNodeType() == QueryTreeNodeType::LIST) diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 4e277fa8624..c854be59666 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -5,6 +5,11 @@ #include #include +#include +#include + +#include + #include #include #include @@ -289,4 +294,28 @@ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_nod return result; } +bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, IdentifierView nested_identifier) +{ + const IDataType * current_type = compound_type.get(); + + for (const auto & identifier_part : nested_identifier) + { + while (const DataTypeArray * array = checkAndGetDataType(current_type)) + current_type = array->getNestedType().get(); + + const DataTypeTuple * tuple = checkAndGetDataType(current_type); + + if (!tuple) + return false; + + auto position = tuple->tryGetPositionByName(identifier_part); + if (!position) + return false; + + current_type = tuple->getElements()[*position].get(); + } + + return true; +} + } diff --git a/src/Analyzer/Utils.h b/src/Analyzer/Utils.h index dbb2e7d5b59..c1a3abd0db7 100644 --- a/src/Analyzer/Utils.h +++ b/src/Analyzer/Utils.h @@ -31,4 +31,12 @@ QueryTreeNodes extractTableExpressions(const QueryTreeNodePtr & join_tree_node); */ QueryTreeNodes buildTableExpressionsStack(const QueryTreeNodePtr & join_tree_node); +/** Returns true if nested identifier can be resolved from compound type. + * Compound type can be tuple or array of tuples. + * + * Example: Compound type: Tuple(nested_path Tuple(nested_path_2 UInt64)). Nested identifier: nested_path_1.nested_path_2. + * Result: true. + */ +bool nestedIdentifierCanBeResolved(const DataTypePtr & compound_type, IdentifierView nested_identifier); + } diff --git a/src/Analyzer/tests/gtest_identifier.cpp b/src/Analyzer/tests/gtest_identifier.cpp index 4f224dfbb07..054a66f2792 100644 --- a/src/Analyzer/tests/gtest_identifier.cpp +++ b/src/Analyzer/tests/gtest_identifier.cpp @@ -70,6 +70,22 @@ TEST(Identifier, IdentifierBasics) } } +TEST(Identifier, IdentifierPushParts) +{ + { + Identifier identifier; + + identifier.push_back("value1"); + ASSERT_EQ(identifier.getFullName(), "value1"); + identifier.push_back("value2"); + ASSERT_EQ(identifier.getFullName(), "value1.value2"); + identifier.push_back("value3"); + ASSERT_EQ(identifier.getFullName(), "value1.value2.value3"); + ASSERT_FALSE(identifier.isEmpty()); + } +} + + TEST(Identifier, IdentifierPopParts) { { diff --git a/src/Common/NamePrompter.cpp b/src/Common/NamePrompter.cpp index ea42b801ee2..047e6f28145 100644 --- a/src/Common/NamePrompter.cpp +++ b/src/Common/NamePrompter.cpp @@ -4,14 +4,17 @@ namespace DB { -void appendHintsMessage(String & message, const std::vector & hints) +String getHintsErrorMessageSuffix(const std::vector & hints) { if (hints.empty()) - { - return; - } + return {}; - message += ". Maybe you meant: " + toString(hints); + return ". Maybe you meant: " + toString(hints); +} + +void appendHintsMessage(String & message, const std::vector & hints) +{ + message += getHintsErrorMessageSuffix(hints); } } diff --git a/src/Common/NamePrompter.h b/src/Common/NamePrompter.h index 8b69dd100ee..4a087548a72 100644 --- a/src/Common/NamePrompter.h +++ b/src/Common/NamePrompter.h @@ -91,7 +91,9 @@ private: } }; -void appendHintsMessage(String & message, const std::vector & hints); +String getHintsErrorMessageSuffix(const std::vector & hints); + +void appendHintsMessage(String & error_message, const std::vector & hints); template class IHints @@ -104,10 +106,10 @@ public: return prompter.getHints(name, getAllRegisteredNames()); } - void appendHintsMessage(String & message, const String & name) const + void appendHintsMessage(String & error_message, const String & name) const { auto hints = getHints(name); - DB::appendHintsMessage(message, hints); + DB::appendHintsMessage(error_message, hints); } IHints() = default; From 7fb3f8f7c5daa369d5fcc947ff7e5052dcccc2c6 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Nov 2022 13:29:50 +0100 Subject: [PATCH 349/526] Updated tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 134 ++++++++----- .../02476_analyzer_identifier_hints.reference | 1 + .../02476_analyzer_identifier_hints.sh | 184 ++++++++++++++++++ 3 files changed, 274 insertions(+), 45 deletions(-) create mode 100644 tests/queries/0_stateless/02476_analyzer_identifier_hints.reference create mode 100755 tests/queries/0_stateless/02476_analyzer_identifier_hints.sh diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index a14d693c348..1e741bc6d7e 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1058,31 +1058,31 @@ private: const ProjectionName & fill_to_expression_projection_name, const ProjectionName & fill_step_expression_projection_name); - static void getCompoundExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + static void collectCompoundExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const DataTypePtr & compound_expression_type, const Identifier & compound_expression_identifier, std::unordered_set & valid_identifiers_result); - static void getTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + static void collectTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const QueryTreeNodePtr & table_expression, const TableExpressionData & table_expression_data, std::unordered_set & valid_identifiers_result); - static void getScopeValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + static void collectScopeValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const IdentifierResolveScope & scope, bool allow_expression_identifiers, bool allow_function_identifiers, bool allow_table_expression_identifiers, std::unordered_set & valid_identifiers_result); - static void getScopeWithParentScopesValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, + static void collectScopeWithParentScopesValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const IdentifierResolveScope & scope, bool allow_expression_identifiers, bool allow_function_identifiers, bool allow_table_expression_identifiers, std::unordered_set & valid_identifiers_result); - static std::vector getIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers); + static std::vector collectIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers); static QueryTreeNodePtr wrapExpressionNodeInTupleElement(QueryTreeNodePtr expression_node, IdentifierView nested_path); @@ -1393,7 +1393,7 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN } /// Get valid identifiers for typo correction from compound expression -void QueryAnalyzer::getCompoundExpressionValidIdentifiersForTypoCorrection( +void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( const Identifier & unresolved_identifier, const DataTypePtr & compound_expression_type, const Identifier & compound_expression_identifier, @@ -1436,7 +1436,7 @@ void QueryAnalyzer::getCompoundExpressionValidIdentifiersForTypoCorrection( } /// Get valid identifiers for typo correction from table expression -void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( +void QueryAnalyzer::collectTableExpressionValidIdentifiersForTypoCorrection( const Identifier & unresolved_identifier, const QueryTreeNodePtr & table_expression, const TableExpressionData & table_expression_data, @@ -1448,7 +1448,7 @@ void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( if (unresolved_identifier.getPartsSize() == column_identifier.getPartsSize()) valid_identifiers_result.insert(column_identifier); - getCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, column_node->getColumnType(), column_identifier, valid_identifiers_result); @@ -1461,6 +1461,11 @@ void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( if (unresolved_identifier.getPartsSize() == column_identifier_with_alias.getPartsSize()) valid_identifiers_result.insert(column_identifier_with_alias); + + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + column_node->getColumnType(), + column_identifier_with_alias, + valid_identifiers_result); } if (!table_expression_data.table_name.empty()) @@ -1471,6 +1476,11 @@ void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( if (unresolved_identifier.getPartsSize() == column_identifier_with_table_name.getPartsSize()) valid_identifiers_result.insert(column_identifier_with_table_name); + + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + column_node->getColumnType(), + column_identifier_with_table_name, + valid_identifiers_result); } if (!table_expression_data.database_name.empty() && !table_expression_data.table_name.empty()) @@ -1481,12 +1491,17 @@ void QueryAnalyzer::getTableExpressionValidIdentifiersForTypoCorrection( if (unresolved_identifier.getPartsSize() == column_identifier_with_table_name_and_database_name.getPartsSize()) valid_identifiers_result.insert(column_identifier_with_table_name_and_database_name); + + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + column_node->getColumnType(), + column_identifier_with_table_name_and_database_name, + valid_identifiers_result); } } } /// Get valid identifiers for typo correction from scope without looking at parent scopes -void QueryAnalyzer::getScopeValidIdentifiersForTypoCorrection( +void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( const Identifier & unresolved_identifier, const IdentifierResolveScope & scope, bool allow_expression_identifiers, @@ -1494,53 +1509,81 @@ void QueryAnalyzer::getScopeValidIdentifiersForTypoCorrection( bool allow_table_expression_identifiers, std::unordered_set & valid_identifiers_result) { + bool identifier_is_short = unresolved_identifier.isShort(); + bool identifier_is_compound = unresolved_identifier.isCompound(); + if (allow_expression_identifiers) { - if (unresolved_identifier.isShort()) + for (const auto & [name, expression] : scope.alias_name_to_expression_node) + { + auto expression_identifier = Identifier(name); + valid_identifiers_result.insert(expression_identifier); + + auto expression_node_type = expression->getNodeType(); + + if (identifier_is_compound && isExpressionNodeType(expression_node_type)) + { + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + expression->getResultType(), + expression_identifier, + valid_identifiers_result); + } + } + + for (const auto & [table_expression, table_expression_data] : scope.table_expression_node_to_data) + { + collectTableExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + table_expression, + table_expression_data, + valid_identifiers_result); + } + } + + if (identifier_is_short) + { + if (allow_function_identifiers) { for (const auto & [name, _] : scope.alias_name_to_expression_node) valid_identifiers_result.insert(Identifier(name)); } - for (const auto & [table_expression, table_expression_data] : scope.table_expression_node_to_data) + if (allow_table_expression_identifiers) { - getTableExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, - table_expression, - table_expression_data, - valid_identifiers_result); + for (const auto & [name, _] : scope.alias_name_to_table_expression_node) + valid_identifiers_result.insert(Identifier(name)); } - } - if (allow_function_identifiers && unresolved_identifier.isShort()) + for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node) { - for (const auto & [name, _] : scope.alias_name_to_expression_node) - valid_identifiers_result.insert(Identifier(name)); - } + auto expression_node_type = expression->getNodeType(); - if (allow_table_expression_identifiers && unresolved_identifier.isShort()) - { - for (const auto & [name, _] : scope.alias_name_to_table_expression_node) - valid_identifiers_result.insert(Identifier(name)); - } - - if (unresolved_identifier.isShort()) - { - for (const auto & [argument_name, expression] : scope.expression_argument_name_to_node) + if (allow_expression_identifiers && isExpressionNodeType(expression_node_type)) { - auto expression_node_type = expression->getNodeType(); + auto expression_identifier = Identifier(argument_name); - if (allow_expression_identifiers && isExpressionNodeType(expression_node_type)) - valid_identifiers_result.insert(Identifier(argument_name)); - else if (allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type)) - valid_identifiers_result.insert(Identifier(argument_name)); - else if (allow_table_expression_identifiers && isTableExpressionNodeType(expression_node_type)) - valid_identifiers_result.insert(Identifier(argument_name)); + if (identifier_is_compound) + { + collectCompoundExpressionValidIdentifiersForTypoCorrection(unresolved_identifier, + expression->getResultType(), + expression_identifier, + valid_identifiers_result); + } + + valid_identifiers_result.insert(expression_identifier); + } + else if (identifier_is_short && allow_function_identifiers && isFunctionExpressionNodeType(expression_node_type)) + { + valid_identifiers_result.insert(Identifier(argument_name)); + } + else if (allow_table_expression_identifiers && isTableExpressionNodeType(expression_node_type)) + { + valid_identifiers_result.insert(Identifier(argument_name)); } } } -void QueryAnalyzer::getScopeWithParentScopesValidIdentifiersForTypoCorrection( +void QueryAnalyzer::collectScopeWithParentScopesValidIdentifiersForTypoCorrection( const Identifier & unresolved_identifier, const IdentifierResolveScope & scope, bool allow_expression_identifiers, @@ -1552,7 +1595,7 @@ void QueryAnalyzer::getScopeWithParentScopesValidIdentifiersForTypoCorrection( while (current_scope) { - getScopeValidIdentifiersForTypoCorrection(unresolved_identifier, + collectScopeValidIdentifiersForTypoCorrection(unresolved_identifier, *current_scope, allow_expression_identifiers, allow_function_identifiers, @@ -1563,7 +1606,7 @@ void QueryAnalyzer::getScopeWithParentScopesValidIdentifiersForTypoCorrection( } } -std::vector QueryAnalyzer::getIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers) +std::vector QueryAnalyzer::collectIdentifierTypoHints(const Identifier & unresolved_identifier, const std::unordered_set & valid_identifiers) { std::vector prompting_strings; prompting_strings.reserve(valid_identifiers.size()); @@ -1937,12 +1980,12 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const if (!nestedIdentifierCanBeResolved(expression_type, nested_path)) { std::unordered_set valid_identifiers; - getCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier, + collectCompoundExpressionValidIdentifiersForTypoCorrection(expression_identifier, expression_type, compound_expression_identifier, valid_identifiers); - auto hints = getIdentifierTypoHints(expression_identifier, valid_identifiers); + auto hints = collectIdentifierTypoHints(expression_identifier, valid_identifiers); String compound_expression_from_error_message; if (!compound_expression_source.empty()) @@ -2367,11 +2410,12 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromTableExpression(const Id if (!result_expression) { std::unordered_set valid_identifiers; - getTableExpressionValidIdentifiersForTypoCorrection(identifier, + collectTableExpressionValidIdentifiersForTypoCorrection(identifier, table_expression_node, table_expression_data, valid_identifiers); - auto hints = getIdentifierTypoHints(identifier, valid_identifiers); + + auto hints = collectIdentifierTypoHints(identifier, valid_identifiers); throw Exception(ErrorCodes::BAD_ARGUMENTS, "Identifier '{}' cannot be resolved from {}. In scope {}{}", identifier.getFullName(), @@ -4595,14 +4639,14 @@ ProjectionNames QueryAnalyzer::resolveExpressionNode(QueryTreeNodePtr & node, Id message_clarification = std::string(" or ") + toStringLowercase(IdentifierLookupContext::TABLE_EXPRESSION); std::unordered_set valid_identifiers; - getScopeWithParentScopesValidIdentifiersForTypoCorrection(unresolved_identifier, + collectScopeWithParentScopesValidIdentifiersForTypoCorrection(unresolved_identifier, scope, true, allow_lambda_expression, allow_table_expression, valid_identifiers); - auto hints = getIdentifierTypoHints(unresolved_identifier, valid_identifiers); + auto hints = collectIdentifierTypoHints(unresolved_identifier, valid_identifiers); throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {}{} identifier '{}' in scope {}{}", toStringLowercase(IdentifierLookupContext::EXPRESSION), diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.reference b/tests/queries/0_stateless/02476_analyzer_identifier_hints.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh new file mode 100755 index 00000000000..cd99d1fbd02 --- /dev/null +++ b/tests/queries/0_stateless/02476_analyzer_identifier_hints.sh @@ -0,0 +1,184 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS test_table; + CREATE TABLE test_table + ( + id UInt64, + value String + ) ENGINE=MergeTree ORDER BY id; + + INSERT INTO test_table VALUES (0, 'Value'); +"; + +$CLICKHOUSE_CLIENT -q "SELECT value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_tabl.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_alias.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_alias.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_alia.value_ FROM test_table AS test_table_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_alias.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT value_ FROM (SELECT 1 AS value) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT subquery.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['subquery.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT subquer.value_ FROM (SELECT 1 AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['subquery.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquer.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alias.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery_alias.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT 1 AS value) SELECT cte_subquery_alia.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery_alias.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, constant_valu SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT 1 AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_valu, [1, 2, 3]) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH 1 AS constant_value SELECT (SELECT constant_valu) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS test_table_compound; + CREATE TABLE test_table_compound + ( + id UInt64, + value Tuple(value_1 String) + ) ENGINE=MergeTree ORDER BY id; + + INSERT INTO test_table_compound VALUES (0, tuple('Value_1')); +"; + +$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_tabl_compound.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alias.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound_alias.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_compound_alia.value.value_ FROM test_table_compound AS test_table_compound_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_compound_alias.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT subquery.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['subquery.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT subquer.value.value_ FROM (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) AS subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['subquery.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery.value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquer.value.value_ FROM cte_subquery SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alias.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery_alias.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cte_subquery AS (SELECT cast(tuple(1), 'Tuple(value_1 String)') AS value) SELECT cte_subquery_alia.value.value_ FROM cte_subquery AS cte_subquery_alias SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['cte_subquery_alias.value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_value.value_ SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, constant_valu.value_ SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT cast(tuple(1), 'Tuple(value_1 String)') AS constant_value, arrayMap(lambda_argument -> lambda_argument + constant_value.value_, [1, 2, 3]) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "WITH cast(tuple(1), 'Tuple(value_1 String)') AS constant_value SELECT (SELECT constant_value.value_) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['constant_value.value_1'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE IF EXISTS test_table_1; + CREATE TABLE test_table_1 + ( + id UInt64, + value String + ) ENGINE=MergeTree ORDER BY id; + + INSERT INTO test_table_1 VALUES (0, 'Value'); + + DROP TABLE IF EXISTS test_table_2; + CREATE TABLE test_table_2 + ( + id UInt64, + value String + ) ENGINE=MergeTree ORDER BY id; + + INSERT INTO test_table_2 VALUES (0, 'Value'); +"; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_1.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_1.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT test_table_2.value_ FROM test_table_1 INNER JOIN test_table_2 ON test_table_1.id = test_table_2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_table_2.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT t1.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['t1.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT t2.value_ FROM test_table_1 AS t1 INNER JOIN test_table_2 AS t2 ON t1.id = t2.id SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['t2.value'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT 1"; + +$CLICKHOUSE_CLIENT -n -q " + DROP TABLE test_table; + DROP TABLE test_table_compound; + DROP TABLE test_table_1; + DROP TABLE test_table_2; +"; From 8adc4c81634ab3638793b5c304126e01f1802c4c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Nov 2022 18:32:53 +0100 Subject: [PATCH 350/526] Updated tests --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 30 +++++++------------ .../02477_analyzer_function_hints.reference | 1 + .../02477_analyzer_function_hints.sh | 26 ++++++++++++++++ 3 files changed, 38 insertions(+), 19 deletions(-) create mode 100644 tests/queries/0_stateless/02477_analyzer_function_hints.reference create mode 100755 tests/queries/0_stateless/02477_analyzer_function_hints.sh diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 1e741bc6d7e..c2d35efe8cb 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1060,7 +1060,7 @@ private: static void collectCompoundExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, const DataTypePtr & compound_expression_type, - const Identifier & compound_expression_identifier, + const Identifier & valid_identifier_prefix, std::unordered_set & valid_identifiers_result); static void collectTableExpressionValidIdentifiersForTypoCorrection(const Identifier & unresolved_identifier, @@ -1396,17 +1396,20 @@ ProjectionName QueryAnalyzer::calculateSortColumnProjectionName(const QueryTreeN void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( const Identifier & unresolved_identifier, const DataTypePtr & compound_expression_type, - const Identifier & compound_expression_identifier, + const Identifier & valid_identifier_prefix, std::unordered_set & valid_identifiers_result) { std::vector> identifiers_with_types_to_process; - identifiers_with_types_to_process.emplace_back(compound_expression_identifier, compound_expression_type.get()); + identifiers_with_types_to_process.emplace_back(valid_identifier_prefix, compound_expression_type.get()); while (!identifiers_with_types_to_process.empty()) { auto [identifier, type] = identifiers_with_types_to_process.back(); identifiers_with_types_to_process.pop_back(); + if (identifier.getPartsSize() + 1 > unresolved_identifier.getPartsSize()) + continue; + while (const DataTypeArray * array = checkAndGetDataType(type)) type = array->getNestedType().get(); @@ -1424,9 +1427,6 @@ void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( identifier.push_back(element_name); - if (identifier.getPartsSize() > unresolved_identifier.getPartsSize()) - break; - valid_identifiers_result.insert(identifier); identifiers_with_types_to_process.emplace_back(identifier, element_type.get()); @@ -2199,12 +2199,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier QueryTreeNodePtr result = it->second; - /** If identifier is compound and it is expression identifier lookup, wrap compound expression into - * tuple elements functions. - * - * Example: SELECT compound_expression AS alias, alias.first.second; - * Result: SELECT compound_expression AS alias, tupleElement(tupleElement(compound_expression, 'first'), 'second'); - */ if (identifier_lookup.identifier.isCompound() && result) { if (identifier_lookup.isExpressionLookup()) @@ -4292,16 +4286,14 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi possible_function_names.push_back(name); } - std::string error_message = fmt::format("Function with name {} does not exists. In scope {}", - function_name, - scope.scope_node->formatASTForErrorMessage()); - NamePrompter<2> name_prompter; auto hints = name_prompter.getHints(function_name, possible_function_names); - appendHintsMessage(error_message, hints); - - throw Exception(ErrorCodes::UNKNOWN_FUNCTION, error_message); + throw Exception(ErrorCodes::UNKNOWN_FUNCTION, + "Function with name {} does not exists. In scope {}{}", + function_name, + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); } AggregateFunctionProperties properties; diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.reference b/tests/queries/0_stateless/02477_analyzer_function_hints.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.sh b/tests/queries/0_stateless/02477_analyzer_function_hints.sh new file mode 100755 index 00000000000..ee9645d381f --- /dev/null +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['plus'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['uniqExact'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function AS x -> x + 1;" +$CLICKHOUSE_CLIENT -q "SELECT test_user_defined_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['test_user_defined_function'\]" &>/dev/null; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION test_user_defined_function"; + +$CLICKHOUSE_CLIENT -q "WITH (x -> x + 1) AS lambda_function SELECT lambda_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['lambda_function'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT row_numbe() OVER (PARTITION BY 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ + | grep "Maybe you meant: \['row_number'\]" &>/dev/null; + +$CLICKHOUSE_CLIENT -q "SELECT 1"; From 9398b96dde5676b6bca95507fad8a348541f1748 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 9 Nov 2022 18:39:30 +0100 Subject: [PATCH 351/526] Fixed style check --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index c2d35efe8cb..d6636c26787 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1421,7 +1421,8 @@ void QueryAnalyzer::collectCompoundExpressionValidIdentifiersForTypoCorrection( const auto & tuple_element_names = tuple->getElementNames(); size_t tuple_element_names_size = tuple_element_names.size(); - for (size_t i = 0; i < tuple_element_names_size; ++i) { + for (size_t i = 0; i < tuple_element_names_size; ++i) + { const auto & element_name = tuple_element_names[i]; const auto & element_type = tuple->getElements()[i]; From 46f5d0ef93addf20082f6e77eba14876aebe4ae7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 10:45:56 +0100 Subject: [PATCH 352/526] Fixed tests --- tests/queries/0_stateless/02477_analyzer_function_hints.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02477_analyzer_function_hints.sh b/tests/queries/0_stateless/02477_analyzer_function_hints.sh index ee9645d381f..d49c20cab75 100755 --- a/tests/queries/0_stateless/02477_analyzer_function_hints.sh +++ b/tests/queries/0_stateless/02477_analyzer_function_hints.sh @@ -1,5 +1,7 @@ #!/usr/bin/env bash +# Tags: no-parallel + set -e CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) @@ -12,6 +14,7 @@ $CLICKHOUSE_CLIENT -q "SELECT plu(1, 1) SETTINGS allow_experimental_analyzer = 1 $CLICKHOUSE_CLIENT -q "SELECT uniqExac(1, 1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['uniqExact'\]" &>/dev/null; +$CLICKHOUSE_CLIENT -q "DROP FUNCTION IF EXISTS test_user_defined_function;" $CLICKHOUSE_CLIENT -q "CREATE FUNCTION test_user_defined_function AS x -> x + 1;" $CLICKHOUSE_CLIENT -q "SELECT test_user_defined_functio(1) SETTINGS allow_experimental_analyzer = 1;" 2>&1 \ | grep "Maybe you meant: \['test_user_defined_function'\]" &>/dev/null; From 057cef469d5b8a6534aa658f1cfe0bfc4e7e238e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 11:22:50 +0100 Subject: [PATCH 353/526] Fixed comments --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index d6636c26787..24b88a729be 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1962,7 +1962,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con return std::make_shared(std::move(storage), storage_lock, storage_snapshot); } -/// Resolve identifier from compound expression. +/// Resolve identifier from compound expression QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromCompoundExpression(const Identifier & expression_identifier, size_t identifier_bind_size, const QueryTreeNodePtr & compound_expression, @@ -2118,8 +2118,7 @@ bool QueryAnalyzer::tryBindIdentifierToAliases(const IdentifierLookup & identifi * If identifier is resolved, depending on lookup context, erase entry from expression or lambda map. Check QueryExpressionsAliasVisitor documentation. * Pop node from current scope expressions in resolve process stack. * - * 5. Pop node from current expressions to resolve. - * 6. If identifier is compound and identifier lookup is in expression context, use `tryResolveIdentifierFromCompoundExpression`. + * 5. If identifier is compound and identifier lookup is in expression context, use `tryResolveIdentifierFromCompoundExpression`. */ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const IdentifierLookup & identifier_lookup, IdentifierResolveScope & scope, IdentifierResolveSettings identifier_resolve_settings) { From 978f567d422b805ed6cbbebdf8a5d9f5b02b5bee Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 10 Nov 2022 13:47:35 -0500 Subject: [PATCH 354/526] broken img markdown? --- docs/en/getting-started/example-datasets/github.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index c8530f4dc92..d681e91850c 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -578,11 +578,11 @@ This data visualizes well. Below we use Superset. **For lines added and deleted:** -image +![](./images/superset-github-lines-added-deleted.png) **For commits and authors:** -image +![](./images/superset-commits-authors.png) ## List files with maximum number of authors @@ -2287,4 +2287,4 @@ FORMAT PrettyCompactMonoBlock This is particularly difficult to get an exact result due to the inability to currently keep state in array functions. This will be possible with an `arrayFold` or `arrayReduce`, which allows state to be held on each iteration. -We welcome solutions here. \ No newline at end of file +We welcome solutions here. From 312875627b811e9e2a1cc6b49c257bbd88d34b0e Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 10 Nov 2022 13:55:39 -0500 Subject: [PATCH 355/526] spelling --- .../example-datasets/github.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index d681e91850c..0ae3f8a4885 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -1,7 +1,7 @@ --- slug: /en/getting-started/example-datasets/github -sidebar_label: Github Repo Analysis -description: Analyze the ClickHouse Github repo or any repository of your choosing +sidebar_label: GitHub Repo Analysis +description: Analyze the ClickHouse GitHub repo or any repository of your choosing --- # ClickHouse GitHub data @@ -14,7 +14,7 @@ The generated data provides a `tsv` file for each of the following tables: - `file_changes` - files changed in every commit with the info about the change and statistics; - `line_changes` - every changed line in every changed file in every commit with full info about the line and the information about the previous change of this line. -As of November 8th, 2022, each tsv is approximately the following size and number of rows: +As of November 8th, 2022, each TSV is approximately the following size and number of rows: - `commits` - 7.8M - 266,051 rows - `file_changes` - 53M - 266,051 rows @@ -47,7 +47,7 @@ As of November 8th, 2022, each tsv is approximately the following size and numbe - [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - [Files sorted by average code age](#files-sorted-by-average-code-age) - - [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) + - [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) @@ -264,7 +264,7 @@ The tool suggests several queries via its help output. We have answered these in - [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - [Files sorted by average code age](#files-sorted-by-average-code-age) - - [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) + - [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) @@ -390,7 +390,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.085 sec. Processed 532.10 thousand rows, 8.68 MB (6.30 million rows/s., 102.64 MB/s.) ``` -Note that this allows for files to be renamed and then re-renamed to their original values. First we aggregare `old_path` for a list of deleted files as a result of renaming. We union this with the last operation for every `path`. Finally, we filter this list to those where the final event is not a `Delete`. +Note that this allows for files to be renamed and then re-renamed to their original values. First we aggregate `old_path` for a list of deleted files as a result of renaming. We union this with the last operation for every `path`. Finally, we filter this list to those where the final event is not a `Delete`. ```sql SELECT uniq(path) @@ -1280,7 +1280,7 @@ LIMIT 1 BY day_of_week 7 rows in set. Elapsed: 0.012 sec. Processed 62.78 thousand rows, 395.47 KB (5.44 million rows/s., 34.27 MB/s.) ``` -Ok, some possible advantages here to the longest contributor - our founder Alexey. Lets limit our analysis to the last year. +OK, some possible advantages here to the longest contributor - our founder Alexey. Lets limit our analysis to the last year. ```sql @@ -1311,7 +1311,7 @@ LIMIT 1 BY day_of_week 7 rows in set. Elapsed: 0.004 sec. Processed 21.82 thousand rows, 140.02 KB (4.88 million rows/s., 31.29 MB/s.) ``` -This is still alittle simple and doesn't reflect people's work. +This is still a little simple and doesn't reflect people's work. A better metric might be who is the top contributor each day as a fraction of the total work performed in the last year. Note that we treat the deletion and adding code equally. @@ -1725,7 +1725,7 @@ LIMIT 10 10 rows in set. Elapsed: 3.134 sec. Processed 16.13 million rows, 1.83 GB (5.15 million rows/s., 582.99 MB/s.) ``` -## Who tends to write more tests / cpp code / comments? +## Who tends to write more tests / CPP code / comments? There are a few ways we can address this question. Focusing on the code to test ratio, this query is relatively simple - count the number of contributions to folders containing `tests` and compute the ratio to total contributions. @@ -1840,7 +1840,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.136 sec. Processed 7.54 million rows, 31.57 MB (55.33 million rows/s., 231.83 MB/s.) ``` -Surprisingly high % for all our contributors and part of what makes our code soo readable. +Surprisingly high % for all our contributors and part of what makes our code so readable. ## How does an authors commits change over time with respect to code/comments percentage? @@ -1880,7 +1880,7 @@ LIMIT 10 Ideally, however, we want to see how this changes in aggregate across all authors from the first day they start committing. Do they slowly reduce the number of comments they write? -To compute this, we first work out each author's comments ratio over time - similar to [Who tends to write more tests / cpp code / comments?](#who-tends-to-write-more-tests--cpp-code--comments). This is joined against each author's start date, allowing us to calculate the comment ratio by week offset. +To compute this, we first work out each author's comments ratio over time - similar to [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments). This is joined against each author's start date, allowing us to calculate the comment ratio by week offset. After calculating the average by-week offset across all authors, we sample these results by selecting every 10th week. From a5d06cd90967185b35a55af426169a7b7cdd09f3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 19:58:07 +0100 Subject: [PATCH 356/526] Better Impl --- src/Storages/NamedCollections.cpp | 389 ++++++++---------- src/Storages/NamedCollections.h | 110 ++--- .../System/StorageSystemNamedCollections.cpp | 4 +- .../tests/gtest_named_collections.cpp | 45 +- 4 files changed, 239 insertions(+), 309 deletions(-) diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index e7c86f0e549..3dc34b699d7 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -128,9 +128,7 @@ bool NamedCollectionFactory::existsUnlocked( || config->has(getCollectionPrefix(collection_name)); } -NamedCollectionPtr NamedCollectionFactory::get( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const +NamedCollectionPtr NamedCollectionFactory::get(const std::string & collection_name) const { std::lock_guard lock(mutex); assertInitialized(lock); @@ -143,12 +141,10 @@ NamedCollectionPtr NamedCollectionFactory::get( collection_name); } - return getImpl(collection_name, collection_info, lock); + return getImpl(collection_name, lock); } -NamedCollectionPtr NamedCollectionFactory::tryGet( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const +NamedCollectionPtr NamedCollectionFactory::tryGet(const std::string & collection_name) const { std::lock_guard lock(mutex); assertInitialized(lock); @@ -156,12 +152,11 @@ NamedCollectionPtr NamedCollectionFactory::tryGet( if (!existsUnlocked(collection_name, lock)) return nullptr; - return getImpl(collection_name, collection_info, lock); + return getImpl(collection_name, lock); } NamedCollectionPtr NamedCollectionFactory::getImpl( const std::string & collection_name, - const NamedCollectionInfo & collection_info, std::lock_guard & /* lock */) const { auto it = named_collections.find(collection_name); @@ -169,8 +164,7 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( { it = named_collections.emplace( collection_name, - std::make_unique( - *config, collection_name, collection_info)).first; + NamedCollection::create(*config, collection_name)).first; } return it->second; } @@ -225,36 +219,12 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const Poco::Util::AbstractConfiguration::Keys config_collections_names; config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); - for (const auto & name : config_collections_names) + for (const auto & collection_name : config_collections_names) { - if (result.contains(name)) + if (result.contains(collection_name)) continue; - const auto collection_prefix = getCollectionPrefix(name); - std::queue enumerate_input; - std::set enumerate_result; - - enumerate_input.push(collection_prefix); - collectKeys(*config, enumerate_input, enumerate_result); - - NamedCollectionInfo collection_info; - - /// Collection does not have any keys. - /// (`enumerate_result` == ). - const bool collection_is_empty = enumerate_result.size() == 1; - if (!collection_is_empty) - { - for (const auto & path : enumerate_result) - { - collection_info.emplace( - /// Skip collection prefix and add +1 to avoid '.' in the beginning. - path.substr(std::strlen(collection_prefix.data()) + 1), - NamedCollectionValueInfo{}); - } - } - - result.emplace( - name, std::make_unique(*config, name, collection_info)); + result.emplace(collection_name, NamedCollection::create(*config, collection_name)); } return result; @@ -271,43 +241,46 @@ private: /// ... /// ConfigurationPtr config; - /// Information about the values of keys. Key is a path to the - /// value represented as a dot concatenated list of keys. - const CollectionInfo collection_info; + Keys keys; public: Impl(const Poco::Util::AbstractConfiguration & config_, const std::string & collection_name_, - const NamedCollectionInfo & collection_info_) + const Keys & keys_) : config(createEmptyConfiguration(collection_name_)) - , collection_info(collection_info_) + , keys(keys_) { auto collection_path = getCollectionPrefix(collection_name_); - for (const auto & [key, value_info] : collection_info) - copyConfigValue( - config_, collection_path + '.' + key, *config, key, value_info.type); + for (const auto & key : keys) + copyConfigValue(config_, collection_path + '.' + key, *config, key); } - Value get(const Key & key) const + ImplPtr copy() const { - auto value_info = collection_info.at(key); - return getConfigValue(*config, key, value_info.type, value_info.is_required); + return std::make_unique(*this); } - void set(const Key & key, const Value & value) + template T get(const Key & key) const { - setConfigValue(*config, key, value); + return getConfigValue(*config, key); } - std::map dumpStructure() + template T getOrDefault(const Key & key, const T & default_value) const { - std::map result; - for (const auto & [key, _] : collection_info) - result.emplace(key, get(key)); - return result; + return getConfigValueOrDefault(*config, key, default_value); } - std::string toString() const + template void set(const Key & key, const T & value) + { + setConfigValue(*config, key, value); + } + + Keys getKeys() const + { + return keys; + } + + std::string dumpStructure() const { /// Convert a collection config like /// @@ -326,7 +299,7 @@ public: /// key3: /// key4: value3" WriteBufferFromOwnString wb; - for (const auto & [key, value_info] : collection_info) + for (const auto & key : keys) { Strings key_parts; splitInto<'.'>(key_parts, key); @@ -338,69 +311,98 @@ public: wb << '\n' << std::string(tab_cnt++, '\t'); wb << *it << ':'; } - wb << '\t' << convertFieldToString(get(key)) << '\n'; + wb << '\t' << get(key) << '\n'; } return wb.str(); } private: - static void validate( + template static T getConfigValue( const Poco::Util::AbstractConfiguration & config, - const std::string & collection_path, - const NamedCollectionInfo & collection_info_) + const std::string & path) { - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(collection_path, config_keys); - checkKeys(config_keys, collection_info_); + return getConfigValueOrDefault(config, path); } - static void checkKeys( - const Poco::Util::AbstractConfiguration::Keys & config_keys, - const NamedCollectionInfo & collection_info) - + template static T getConfigValueOrDefault( + const Poco::Util::AbstractConfiguration & config, + const std::string & path, + const std::optional & default_value = std::nullopt) { - auto get_suggestion = [&](bool only_required_keys) + const bool has_value = config.has(path); + if (!has_value) { - std::string suggestion; - for (const auto & [key, info] : collection_info) - { - if (only_required_keys && info.is_required) - continue; - - if (!suggestion.empty()) - suggestion += ", "; - - suggestion += key; - } - return suggestion; - }; - - std::set required_keys; - for (const auto & [key, info] : collection_info) - { - if (info.is_required) - required_keys.insert(key); + if (!default_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", path); + return *default_value; } - for (const auto & key : config_keys) - { - if (!collection_info.contains(key)) - { - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Unknown key `{}`, expected one of: {}", - key, get_suggestion(false)); - } - required_keys.erase(key); - } - - if (!required_keys.empty()) - { + if constexpr (std::is_same_v) + return config.getString(path); + else if constexpr (std::is_same_v) + return config.getUInt64(path); + else if constexpr (std::is_same_v) + return config.getInt64(path); + else if constexpr (std::is_same_v) + return config.getDouble(path); + else throw Exception( ErrorCodes::BAD_ARGUMENTS, - "Keys `{}` are required, but was not found in config. List of required keys: {}", - fmt::join(required_keys, ", "), get_suggestion(true)); - } + "Unsupported type in getConfigValueOrDefault(). " + "Supported types are String, UInt64, Int64, Float64"); + } + + template static void setConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path, + const T & value) + { + const bool has_value = config.has(path); + if (has_value) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", path); + + if constexpr (std::is_same_v) + config.setString(path, value); + else if constexpr (std::is_same_v) + config.setUInt64(path, value); + else if constexpr (std::is_same_v) + config.setInt64(path, value); + else if constexpr (std::is_same_v) + config.setDouble(path, value); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unsupported type in setConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); + } + + template static void copyConfigValue( + const Poco::Util::AbstractConfiguration & from_config, + const std::string & from_path, + Poco::Util::AbstractConfiguration & to_config, + const std::string & to_path) + { + if (!from_config.has(from_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", from_path); + + if (to_config.has(to_path)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", to_path); + + if constexpr (std::is_same_v) + to_config.setString(to_path, from_config.getString(from_path)); + else if constexpr (std::is_same_v) + to_config.setString(to_path, from_config.getString(from_path)); + else if constexpr (std::is_same_v) + to_config.setUInt64(to_path, from_config.getUInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setInt64(to_path, from_config.getInt64(from_path)); + else if constexpr (std::is_same_v) + to_config.setDouble(to_path, from_config.getDouble(from_path)); + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Unsupported type in copyConfigValue(). " + "Supported types are String, UInt64, Int64, Float64"); } static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) @@ -411,127 +413,88 @@ private: ConfigurationPtr config(new Poco::Util::XMLConfiguration(xml_document)); return config; } - - using ConfigValueType = Field::Types::Which; - static void copyConfigValue( - const Poco::Util::AbstractConfiguration & from_config, - const std::string & from_path, - Poco::Util::AbstractConfiguration & to_config, - const std::string & to_path, - ConfigValueType type) - { - using Type = Field::Types::Which; - switch (type) - { - case Type::String: - to_config.setString(to_path, from_config.getString(from_path)); - break; - case Type::UInt64: - to_config.setUInt64(to_path, from_config.getUInt64(from_path)); - break; - case Type::Int64: - to_config.setInt64(to_path, from_config.getInt64(from_path)); - break; - case Type::Float64: - to_config.setDouble(to_path, from_config.getDouble(from_path)); - break; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - } - - static void setConfigValue( - Poco::Util::AbstractConfiguration & config, - const std::string & path, - const Field & value) - { - using Type = Field::Types::Which; - switch (value.getType()) - { - case Type::String: - config.setString(path, value.safeGet()); - break; - case Type::UInt64: - config.setUInt64(path, value.safeGet()); - break; - case Type::Int64: - config.setInt64(path, value.safeGet()); - break; - case Type::Float64: - config.setDouble(path, value.safeGet()); - break; - default: - throw Exception( - ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - } - - static Field getConfigValue( - const Poco::Util::AbstractConfiguration & config, - const std::string & path, - ConfigValueType type, - bool throw_not_found, - std::optional default_value = std::nullopt) - { - const bool has_value = config.has(path); - if (!has_value) - { - if (throw_not_found) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Failed to find key `{}` in config, but this key is required", - path); - } - else if (!default_value) - return Null{}; - } - - Field value; - - using Type = Field::Types::Which; - switch (type) - { - case Type::String: - value = has_value ? config.getString(path) : default_value->get(); - break; - case Type::UInt64: - value = has_value ? config.getUInt64(path) : default_value->get(); - break; - case Type::Int64: - value = has_value ? config.getInt64(path) : default_value->get(); - break; - case Type::Float64: - value = has_value ? config.getDouble(path) : default_value->get(); - break; - default: - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unsupported type"); - } - return value; - } }; NamedCollection::NamedCollection( const Poco::Util::AbstractConfiguration & config, const std::string & collection_path, - const CollectionInfo & collection_info) - : pimpl(std::make_unique(config, collection_path, collection_info)) + const Keys & keys) + : pimpl(std::make_unique(config, collection_path, keys)) { } -NamedCollection::Value NamedCollection::get(const Key & key) const +NamedCollection::NamedCollection(ImplPtr pimpl_) + : pimpl(std::move(pimpl_)) { - return pimpl->get(key); } -std::map NamedCollection::dumpStructure() const +NamedCollectionPtr NamedCollection::create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name) +{ + const auto collection_prefix = getCollectionPrefix(collection_name); + std::queue enumerate_input; + std::set enumerate_result; + + enumerate_input.push(collection_prefix); + collectKeys(config, enumerate_input, enumerate_result); + + /// Collection does not have any keys. + /// (`enumerate_result` == ). + const bool collection_is_empty = enumerate_result.size() == 1; + std::set keys; + if (!collection_is_empty) + { + /// Skip collection prefix and add +1 to avoid '.' in the beginning. + for (const auto & path : enumerate_result) + keys.emplace(path.substr(std::strlen(collection_prefix.data()) + 1)); + } + return std::make_unique(config, collection_name, keys); +} + +template T NamedCollection::get(const Key & key) const +{ + return pimpl->get(key); +} + +template T NamedCollection::getOrDefault(const Key & key, const T & default_value) const +{ + return pimpl->getOrDefault(key, default_value); +} + +template void NamedCollection::set(const Key & key, const T & value) +{ + pimpl->set(key, value); +} + +NamedCollectionPtr NamedCollection::duplicate() const +{ + return std::make_shared(pimpl->copy()); +} + +NamedCollection::Keys NamedCollection::getKeys() const +{ + return pimpl->getKeys(); +} + +std::string NamedCollection::dumpStructure() const { return pimpl->dumpStructure(); } -std::string NamedCollection::toString() const -{ - return pimpl->toString(); -} +template String NamedCollection::get(const NamedCollection::Key & key) const; +template UInt64 NamedCollection::get(const NamedCollection::Key & key) const; +template Int64 NamedCollection::get(const NamedCollection::Key & key) const; +template Float64 NamedCollection::get(const NamedCollection::Key & key) const; + +template String NamedCollection::getOrDefault(const NamedCollection::Key & key, const String & default_value) const; +template UInt64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const UInt64 & default_value) const; +template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Int64 & default_value) const; +template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; + +template void NamedCollection::set(const NamedCollection::Key & key, const String & value); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 75884d96ce9..0056b255025 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -9,21 +9,55 @@ namespace DB class NamedCollection; using NamedCollectionPtr = std::shared_ptr; -struct NamedCollectionValueInfo; -using NamedCollectionInfo = std::map; /** - * A factory of immutable named collections. - * Named collections are defined in server config as arbitrary - * structure configurations: + * Class to represent arbitrary-structured named collection object. + * It can be defined via config or via SQL command. * * * ... * * ... * - * In order to get a named collection, you need to know it's name - * and expected structure of the collection defined as NamedCollectionInfo. + */ +class NamedCollection +{ +private: + struct Impl; + using ImplPtr = std::unique_ptr; + + ImplPtr pimpl; + +public: + using Key = std::string; + using Keys = std::set; + + static NamedCollectionPtr create( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_name); + + NamedCollection( + const Poco::Util::AbstractConfiguration & config, + const std::string & collection_path, + const Keys & keys); + + explicit NamedCollection(ImplPtr pimpl_); + + template T get(const Key & key) const; + + template T getOrDefault(const Key & key, const T & default_value) const; + + template void set(const Key & key, const T & value); + + NamedCollectionPtr duplicate() const; + + Keys getKeys() const; + + std::string dumpStructure() const; +}; + +/** + * A factory of immutable named collections. */ class NamedCollectionFactory : boost::noncopyable { @@ -34,13 +68,9 @@ public: bool exists(const std::string & collection_name) const; - NamedCollectionPtr get( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const; + NamedCollectionPtr get(const std::string & collection_name) const; - NamedCollectionPtr tryGet( - const std::string & collection_name, - const NamedCollectionInfo & collection_info) const; + NamedCollectionPtr tryGet(const std::string & collection_name) const; void add( const std::string & collection_name, @@ -52,9 +82,10 @@ public: NamedCollections getAll() const; private: + void assertInitialized(std::lock_guard & lock) const; + NamedCollectionPtr getImpl( const std::string & collection_name, - const NamedCollectionInfo & collection_info, std::lock_guard & lock) const; bool existsUnlocked( @@ -63,62 +94,11 @@ private: mutable NamedCollections named_collections; -private: /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; - void assertInitialized(std::lock_guard & lock) const; - bool is_initialized = false; mutable std::mutex mutex; }; - -class NamedCollection -{ -friend class NamedCollectionFactory; - -private: - struct Impl; - using ImplPtr = std::unique_ptr; - - ImplPtr pimpl; - -public: - using Key = std::string; - using Value = Field; - using ValueInfo = NamedCollectionValueInfo; - using CollectionInfo = NamedCollectionInfo; - - NamedCollection( - const Poco::Util::AbstractConfiguration & config, - const std::string & collection_path, - const CollectionInfo & collection_info); - - Value get(const Key & key) const; - - std::map dumpStructure() const; - - /// Get a string representation of the collection structure. - /// Used for debugging and tests. - std::string toString() const; -}; - - -/** - * Named collection info which allows to parse config. - * Contains a mapping key_path -> value_info. - */ -struct NamedCollectionValueInfo -{ - /// Type of the value. One of: String, UInt64, Int64, Double. - using Type = Field::Types::Which; - Type type = Type::String; - /// Optional default value for the case if there is no such key in config. - std::optional default_value; - /// Is this value required or optional? Throw exception if the value is - /// required, but is not specified in config. - bool is_required = true; -}; - } diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 3fc9c5c8313..8c3092b948e 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -45,10 +45,10 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte auto & value_column = tuple_column.getColumn(1); size_t size = 0; - for (const auto & [key, value] : collection->dumpStructure()) + for (const auto & key : collection->getKeys()) { key_column.insertData(key.data(), key.size()); - value_column.insert(convertFieldToString(value)); + value_column.insert(collection->get(key)); size++; } diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 34dfe5c1087..6a3057df329 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -32,52 +32,39 @@ TEST(NamedCollections, SimpleConfig) ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2")); - ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3", {}) == nullptr); + ASSERT_TRUE(NamedCollectionFactory::instance().tryGet("collection3") == nullptr); auto collections = NamedCollectionFactory::instance().getAll(); ASSERT_EQ(collections.size(), 2); ASSERT_TRUE(collections.contains("collection1")); ASSERT_TRUE(collections.contains("collection2")); - ASSERT_EQ(collections["collection1"]->toString(), + ASSERT_EQ(collections["collection1"]->dumpStructure(), R"CONFIG(key1: value1 key2: 2 key3: 3.3 key4: -4 +)CONFIG"); + ASSERT_EQ(collections["collection2"]->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 )CONFIG"); - using ValueInfo = NamedCollectionValueInfo; - ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; - ValueInfo uint_def{Field::Types::Which::UInt64, std::nullopt, true}; - ValueInfo int_def{Field::Types::Which::Int64, std::nullopt, true}; - ValueInfo double_def{Field::Types::Which::Float64, std::nullopt, true}; - - NamedCollectionInfo collection1_info; - collection1_info.emplace("key1", string_def); - collection1_info.emplace("key2", uint_def); - collection1_info.emplace("key3", double_def); - collection1_info.emplace("key4", int_def); - - auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); + auto collection1 = NamedCollectionFactory::instance().get("collection1"); ASSERT_TRUE(collection1 != nullptr); + ASSERT_TRUE(collection1->get("key1") == "value1"); + ASSERT_TRUE(collection1->get("key2") == 2); + ASSERT_TRUE(collection1->get("key3") == 3.3); + ASSERT_TRUE(collection1->get("key4") == -4); - ASSERT_TRUE(collection1->get("key1").safeGet() == "value1"); - ASSERT_TRUE(collection1->get("key2").safeGet() == 2); - ASSERT_TRUE(collection1->get("key3").safeGet() == 3.3); - ASSERT_TRUE(collection1->get("key4").safeGet() == -4); - - NamedCollectionInfo collection2_info; - collection2_info.emplace("key4", string_def); - collection2_info.emplace("key5", uint_def); - collection2_info.emplace("key6", double_def); - - auto collection2 = NamedCollectionFactory::instance().get("collection2", collection2_info); + auto collection2 = NamedCollectionFactory::instance().get("collection2"); ASSERT_TRUE(collection2 != nullptr); - ASSERT_TRUE(collection2->get("key4").safeGet() == "value4"); - ASSERT_TRUE(collection2->get("key5").safeGet() == 5); - ASSERT_TRUE(collection2->get("key6").safeGet() == 6.6); + ASSERT_TRUE(collection2->get("key4") == "value4"); + ASSERT_TRUE(collection2->get("key5") == 5); + ASSERT_TRUE(collection2->get("key6") == 6.6); } // TEST(NamedCollections, NestedConfig) From 790a3b94fa04e2b065a3dc8b6574baddc3a75c86 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Thu, 10 Nov 2022 14:01:52 -0500 Subject: [PATCH 357/526] bad code indent --- .../example-datasets/github.md | 49 +++++++++---------- 1 file changed, 24 insertions(+), 25 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 0ae3f8a4885..fbc65d36ba9 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -440,32 +440,31 @@ The difference here is caused by a few factors: - A rename can occur alongside other modifications to the file. These are listed as separate events in file_changes but with the same time. The `argMax` function has no way of distinguishing these - it picks the first value. The natural ordering of the inserts (the only means of knowing the correct order) is not maintained across the union so modified events can be selected. For example, below the `src/Functions/geometryFromColumn.h` file has several modifications before being renamed to `src/Functions/geometryConverters.h`. Our current solution may pick a Modify event as the latest change causing `src/Functions/geometryFromColumn.h` to be retained. + ```sql + SELECT + change_type, + path, + old_path, + time, + commit_hash + FROM git.file_changes + WHERE (path = 'src/Functions/geometryFromColumn.h') OR (old_path = 'src/Functions/geometryFromColumn.h') - ```sql - SELECT - change_type, - path, - old_path, - time, - commit_hash - FROM git.file_changes - WHERE (path = 'src/Functions/geometryFromColumn.h') OR (old_path = 'src/Functions/geometryFromColumn.h') - - ┌─change_type─┬─path───────────────────────────────┬─old_path───────────────────────────┬────────────────time─┬─commit_hash──────────────────────────────┐ - │ Add │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 9376b676e9a9bb8911b872e1887da85a45f7479d │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 6d59be5ea4768034f6526f7f9813062e0c369f7b │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 33acc2aa5dc091a7cb948f78c558529789b2bad8 │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 78e0db268ceadc42f82bc63a77ee1a4da6002463 │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 14a891057d292a164c4179bfddaef45a74eaf83a │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ d0d6e6953c2a2af9fb2300921ff96b9362f22edb │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ fe8382521139a58c0ba277eb848e88894658db66 │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 3be3d5cde8788165bc0558f1e2a22568311c3103 │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ afad9bf4d0a55ed52a3f55483bc0973456e10a56 │ - │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ e3290ecc78ca3ea82b49ebcda22b5d3a4df154e6 │ - │ Rename │ src/Functions/geometryConverters.h │ src/Functions/geometryFromColumn.h │ 2021-03-11 12:08:16 │ 125945769586baf6ffd15919b29565b1b2a63218 │ - └─────────────┴────────────────────────────────────┴────────────────────────────────────┴─────────────────────┴──────────────────────────────────────────┘ - 11 rows in set. Elapsed: 0.030 sec. Processed 266.05 thousand rows, 6.61 MB (8.89 million rows/s., 220.82 MB/s.) - ``` + ┌─change_type─┬─path───────────────────────────────┬─old_path───────────────────────────┬────────────────time─┬─commit_hash──────────────────────────────┐ + │ Add │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 9376b676e9a9bb8911b872e1887da85a45f7479d │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 6d59be5ea4768034f6526f7f9813062e0c369f7b │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 33acc2aa5dc091a7cb948f78c558529789b2bad8 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 78e0db268ceadc42f82bc63a77ee1a4da6002463 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 14a891057d292a164c4179bfddaef45a74eaf83a │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ d0d6e6953c2a2af9fb2300921ff96b9362f22edb │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ fe8382521139a58c0ba277eb848e88894658db66 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ 3be3d5cde8788165bc0558f1e2a22568311c3103 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ afad9bf4d0a55ed52a3f55483bc0973456e10a56 │ + │ Modify │ src/Functions/geometryFromColumn.h │ │ 2021-03-11 12:08:16 │ e3290ecc78ca3ea82b49ebcda22b5d3a4df154e6 │ + │ Rename │ src/Functions/geometryConverters.h │ src/Functions/geometryFromColumn.h │ 2021-03-11 12:08:16 │ 125945769586baf6ffd15919b29565b1b2a63218 │ + └─────────────┴────────────────────────────────────┴────────────────────────────────────┴─────────────────────┴──────────────────────────────────────────┘ + 11 rows in set. Elapsed: 0.030 sec. Processed 266.05 thousand rows, 6.61 MB (8.89 million rows/s., 220.82 MB/s.) + ``` - Broken commit history - missing delete events. Source and cause TBD. These differences shouldn't meaningfully impact our analysis. **We welcome improved versions of this query**. From bb00a246f10ec993695cce4e0a8fadf50a0b2960 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 10 Nov 2022 20:53:05 +0100 Subject: [PATCH 358/526] priority for DROP_RANGEs, assertion for others --- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 28 +++++++++++++++---- 1 file changed, 22 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index e60403149b5..fec312fae2f 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1032,6 +1032,14 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( [[maybe_unused]] bool called_for_broken_part = !covering_entry; assert(currently_executing_drop_replace_ranges.contains(part_info) || called_from_alter_query_directly || called_for_broken_part || !fetch_entry_znode.empty()); + auto is_simple_part_producing_op = [](const ReplicatedMergeTreeLogEntryData & data) + { + return data.type == LogEntry::GET_PART || + data.type == LogEntry::ATTACH_PART || + data.type == LogEntry::MERGE_PARTS || + data.type == LogEntry::MUTATE_PART; + }; + for (Queue::iterator it = queue.begin(); it != queue.end();) { /// Skipping currently processing entry @@ -1041,18 +1049,21 @@ void ReplicatedMergeTreeQueue::removePartProducingOpsInRange( continue; } - auto type = (*it)->type; - bool is_simple_producing_op = type == LogEntry::GET_PART || - type == LogEntry::ATTACH_PART || - type == LogEntry::MERGE_PARTS || - type == LogEntry::MUTATE_PART; + bool is_simple_producing_op = is_simple_part_producing_op(**it); bool simple_op_covered = is_simple_producing_op && part_info.contains(MergeTreePartInfo::fromPartName((*it)->new_part_name, format_version)); bool replace_range_covered = covering_entry && checkReplaceRangeCanBeRemoved(part_info, *it, *covering_entry); if (simple_op_covered || replace_range_covered) { if ((*it)->currently_executing) + { + bool is_covered_by_simple_op = covering_entry && is_simple_part_producing_op(*covering_entry); + bool is_fetching_covering_part = !fetch_entry_znode.empty(); + if (is_covered_by_simple_op || is_fetching_covering_part) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot remove covered entry {} producing parts {}, it's a bug", + (*it)->znode_name, fmt::join((*it)->getVirtualPartNames(format_version), ", ")); to_wait.push_back(*it); + } auto code = zookeeper->tryRemove(fs::path(replica_path) / "queue" / (*it)->znode_name); if (code != Coordination::Error::ZOK) LOG_INFO(log, "Couldn't remove {}: {}", (fs::path(replica_path) / "queue" / (*it)->znode_name).string(), Coordination::errorMessage(code)); @@ -1118,7 +1129,12 @@ bool ReplicatedMergeTreeQueue::isCoveredByFuturePartsImpl(const LogEntry & entry /// Parts are not disjoint. They can be even intersecting and it's not a problem, /// because we may have two queue entries producing intersecting parts if there's DROP_RANGE between them (so virtual_parts are ok). - /// We cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) + /// Give priority to DROP_RANGEs and allow processing them even if covered entries are currently executing. + /// DROP_RANGE will cancel covered operations and will wait for them in removePartProducingOpsInRange. + if (result_part.isFakeDropRangePart() && result_part.contains(future_part)) + continue; + + /// In other cases we cannot execute `entry` (or upgrade its actual_part_name to `new_part_name`) /// while any covered or covering parts are processed. /// But we also cannot simply return true and postpone entry processing, because it may lead to kind of livelock. /// Since queue is processed in multiple threads, it's likely that there will be at least one thread From 953457de281d1167f51d91de9c3ca8df32780b30 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 10 Nov 2022 22:41:26 +0100 Subject: [PATCH 359/526] Remove POCO_CLICKHOUSE_PATCH Signed-off-by: Azat Khuzhin --- programs/keeper/Keeper.cpp | 12 ------------ programs/server/Server.cpp | 12 ------------ src/Common/Exception.h | 6 +----- src/Daemon/BaseDaemon.h | 6 +----- src/IO/HTTPCommon.cpp | 11 ----------- 5 files changed, 2 insertions(+), 45 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index e1d03b40b66..a1bf324f482 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -149,19 +149,7 @@ std::string getUserName(uid_t user_id) Poco::Net::SocketAddress Keeper::socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, [[maybe_unused]] bool secure) const { auto address = makeSocketAddress(host, port, &logger()); -#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100 - if (secure) - /// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl - /// https://github.com/pocoproject/poco/pull/2257 - socket.bind(address, /* reuseAddress = */ true); - else -#endif -#if POCO_VERSION < 0x01080000 - socket.bind(address, /* reuseAddress = */ true); -#else socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); -#endif - socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64)); return address; diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..0622303a093 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -341,19 +341,7 @@ Poco::Net::SocketAddress Server::socketBindListen( [[maybe_unused]] bool secure) const { auto address = makeSocketAddress(host, port, &logger()); -#if !defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION < 0x01090100 - if (secure) - /// Bug in old (<1.9.1) poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl - /// https://github.com/pocoproject/poco/pull/2257 - socket.bind(address, /* reuseAddress = */ true); - else -#endif -#if POCO_VERSION < 0x01080000 - socket.bind(address, /* reuseAddress = */ true); -#else socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config.getBool("listen_reuse_port", false)); -#endif - /// If caller requests any available port from the OS, discover it after binding. if (port == 0) { diff --git a/src/Common/Exception.h b/src/Common/Exception.h index c5259d157b2..e26a9690f67 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -121,11 +121,7 @@ public: } - std::string displayText() const -#if defined(POCO_CLICKHOUSE_PATCH) - override -#endif - ; + std::string displayText() const override; ssize_t getLineNumber() const { return line_number; } void setLineNumber(int line_number_) { line_number = line_number_;} diff --git a/src/Daemon/BaseDaemon.h b/src/Daemon/BaseDaemon.h index d248ad9cec9..ae64651caed 100644 --- a/src/Daemon/BaseDaemon.h +++ b/src/Daemon/BaseDaemon.h @@ -136,11 +136,7 @@ protected: /// fork the main process and watch if it was killed void setupWatchdog(); - void waitForTerminationRequest() -#if defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION >= 0x02000000 // in old upstream poco not vitrual - override -#endif - ; + void waitForTerminationRequest() override; /// thread safe virtual void onInterruptSignals(int signal_id); diff --git a/src/IO/HTTPCommon.cpp b/src/IO/HTTPCommon.cpp index f33b2399492..c015d4566d6 100644 --- a/src/IO/HTTPCommon.cpp +++ b/src/IO/HTTPCommon.cpp @@ -49,11 +49,7 @@ namespace { void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) { -#if defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION >= 0x02000000 session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); -#else - session.setTimeout(std::max({timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout})); -#endif session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout); } @@ -93,12 +89,7 @@ namespace ProfileEvents::increment(ProfileEvents::CreatedHTTPConnections); /// doesn't work properly without patch -#if defined(POCO_CLICKHOUSE_PATCH) session->setKeepAlive(keep_alive); -#else - (void)keep_alive; // Avoid warning: unused parameter -#endif - return session; } @@ -122,12 +113,10 @@ namespace session->setProxyHost(proxy_host); session->setProxyPort(proxy_port); -#if defined(POCO_CLICKHOUSE_PATCH) session->setProxyProtocol(proxy_scheme); /// Turn on tunnel mode if proxy scheme is HTTP while endpoint scheme is HTTPS. session->setProxyTunnel(!proxy_https && https); -#endif } return session; } From 63de577172ee024a08e76db69f5000568673db48 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 10 Nov 2022 22:44:26 +0100 Subject: [PATCH 360/526] Better, add tests --- programs/server/Server.cpp | 1 + src/Storages/NamedCollections.cpp | 127 ++++++++++++------ src/Storages/NamedCollections.h | 15 ++- .../System/StorageSystemNamedCollections.cpp | 1 - .../tests/gtest_named_collections.cpp | 126 ++++++++++------- 5 files changed, 175 insertions(+), 95 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 079dae52f15..d4e623b49b3 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1282,6 +1282,7 @@ int Server::main(const std::vector & /*args*/) #if USE_SSL CertificateReloader::instance().tryLoad(*config); #endif + NamedCollectionFactory::instance().reload(*config); ProfileEvents::increment(ProfileEvents::MainConfigLoads); /// Must be the last. diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 3dc34b699d7..60c11c90b0a 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -24,6 +24,8 @@ namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; extern const int NAMED_COLLECTION_ALREADY_EXISTS; + extern const int BAD_ARGUMENTS; + extern const int NOT_IMPLEMENTED; } namespace @@ -84,8 +86,7 @@ NamedCollectionFactory & NamedCollectionFactory::instance() return instance; } -void NamedCollectionFactory::initialize( - const Poco::Util::AbstractConfiguration & server_config) +void NamedCollectionFactory::initialize(const Poco::Util::AbstractConfiguration & config_) { std::lock_guard lock(mutex); if (is_initialized) @@ -95,10 +96,17 @@ void NamedCollectionFactory::initialize( "Named collection factory already initialized"); } - config = &server_config; + config = &config_; is_initialized = true; } +void NamedCollectionFactory::reload(const Poco::Util::AbstractConfiguration & config_) +{ + std::lock_guard lock(mutex); + config = &config_; + loaded_named_collections.clear(); +} + void NamedCollectionFactory::assertInitialized( std::lock_guard & /* lock */) const { @@ -124,7 +132,7 @@ bool NamedCollectionFactory::existsUnlocked( /// Named collections can be added via SQL command or via config. /// Named collections from config are loaded on first access, /// therefore it might not be in `named_collections` map yet. - return named_collections.contains(collection_name) + return loaded_named_collections.contains(collection_name) || config->has(getCollectionPrefix(collection_name)); } @@ -159,10 +167,10 @@ NamedCollectionPtr NamedCollectionFactory::getImpl( const std::string & collection_name, std::lock_guard & /* lock */) const { - auto it = named_collections.find(collection_name); - if (it == named_collections.end()) + auto it = loaded_named_collections.find(collection_name); + if (it == loaded_named_collections.end()) { - it = named_collections.emplace( + it = loaded_named_collections.emplace( collection_name, NamedCollection::create(*config, collection_name)).first; } @@ -174,7 +182,7 @@ void NamedCollectionFactory::add( NamedCollectionPtr collection) { std::lock_guard lock(mutex); - auto [it, inserted] = named_collections.emplace(collection_name, collection); + auto [it, inserted] = loaded_named_collections.emplace(collection_name, collection); if (!inserted) { throw Exception( @@ -205,7 +213,7 @@ void NamedCollectionFactory::remove(const std::string & collection_name) collection_name); } - [[maybe_unused]] auto removed = named_collections.erase(collection_name); + [[maybe_unused]] auto removed = loaded_named_collections.erase(collection_name); assert(removed); } @@ -214,7 +222,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const std::lock_guard lock(mutex); assertInitialized(lock); - NamedCollections result(named_collections); + NamedCollections result(loaded_named_collections); Poco::Util::AbstractConfiguration::Keys config_collections_names; config->keys(NAMED_COLLECTIONS_CONFIG_PREFIX, config_collections_names); @@ -230,7 +238,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const return result; } -struct NamedCollection::Impl +class NamedCollection::Impl { private: using IConfigurationPtr = Poco::AutoPtr; @@ -255,11 +263,6 @@ public: copyConfigValue(config_, collection_path + '.' + key, *config, key); } - ImplPtr copy() const - { - return std::make_unique(*this); - } - template T get(const Key & key) const { return getConfigValue(*config, key); @@ -270,9 +273,18 @@ public: return getConfigValueOrDefault(*config, key, default_value); } - template void set(const Key & key, const T & value) + template void set(const Key & key, const T & value, bool update_if_exists) { - setConfigValue(*config, key, value); + setConfigValue(*config, key, value, update_if_exists); + if (!keys.contains(key)) + keys.insert(key); + } + + void remove(const Key & key) + { + removeConfigValue(*config, key); + [[maybe_unused]] auto removed = keys.erase(key); + assert(removed); } Keys getKeys() const @@ -280,6 +292,11 @@ public: return keys; } + ImplPtr copy() const + { + return std::make_unique(*this); + } + std::string dumpStructure() const { /// Convert a collection config like @@ -299,19 +316,34 @@ public: /// key3: /// key4: value3" WriteBufferFromOwnString wb; + Strings prev_key_parts; for (const auto & key : keys) { Strings key_parts; splitInto<'.'>(key_parts, key); size_t tab_cnt = 0; - for (auto it = key_parts.begin(); it != key_parts.end(); ++it) + auto it = key_parts.begin(); + auto prev_key_parts_it = prev_key_parts.begin(); + while (it != key_parts.end() + && prev_key_parts_it != prev_key_parts.end() + && *it == *prev_key_parts_it) { - if (it != key_parts.begin()) - wb << '\n' << std::string(tab_cnt++, '\t'); + ++it; + ++prev_key_parts_it; + ++tab_cnt; + } + + auto start_it = it; + for (; it != key_parts.end(); ++it) + { + if (it != start_it) + wb << '\n'; + wb << std::string(tab_cnt++, '\t'); wb << *it << ':'; } wb << '\t' << get(key) << '\n'; + prev_key_parts = key_parts; } return wb.str(); } @@ -329,11 +361,10 @@ private: const std::string & path, const std::optional & default_value = std::nullopt) { - const bool has_value = config.has(path); - if (!has_value) + if (!config.has(path)) { if (!default_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); return *default_value; } @@ -347,7 +378,7 @@ private: return config.getDouble(path); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in getConfigValueOrDefault(). " "Supported types are String, UInt64, Int64, Float64"); } @@ -355,11 +386,11 @@ private: template static void setConfigValue( Poco::Util::AbstractConfiguration & config, const std::string & path, - const T & value) + const T & value, + bool update = false) { - const bool has_value = config.has(path); - if (has_value) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", path); + if (!update && config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", path); if constexpr (std::is_same_v) config.setString(path, value); @@ -371,7 +402,7 @@ private: config.setDouble(path, value); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in setConfigValue(). " "Supported types are String, UInt64, Int64, Float64"); } @@ -383,10 +414,10 @@ private: const std::string & to_path) { if (!from_config.has(from_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "No such key `{}`", from_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", from_path); if (to_config.has(to_path)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Key `{}` already exists", to_path); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Key `{}` already exists", to_path); if constexpr (std::is_same_v) to_config.setString(to_path, from_config.getString(from_path)); @@ -400,11 +431,20 @@ private: to_config.setDouble(to_path, from_config.getDouble(from_path)); else throw Exception( - ErrorCodes::BAD_ARGUMENTS, + ErrorCodes::NOT_IMPLEMENTED, "Unsupported type in copyConfigValue(). " "Supported types are String, UInt64, Int64, Float64"); } + static void removeConfigValue( + Poco::Util::AbstractConfiguration & config, + const std::string & path) + { + if (!config.has(path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "No such key `{}`", path); + config.remove(path); + } + static ConfigurationPtr createEmptyConfiguration(const std::string & root_name) { using DocumentPtr = Poco::AutoPtr; @@ -419,7 +459,7 @@ NamedCollection::NamedCollection( const Poco::Util::AbstractConfiguration & config, const std::string & collection_path, const Keys & keys) - : pimpl(std::make_unique(config, collection_path, keys)) + : NamedCollection(std::make_unique(config, collection_path, keys)) { } @@ -462,12 +502,17 @@ template T NamedCollection::getOrDefault(const Key & key, const T & return pimpl->getOrDefault(key, default_value); } -template void NamedCollection::set(const Key & key, const T & value) +template void NamedCollection::set(const Key & key, const T & value, bool update_if_exists) { - pimpl->set(key, value); + pimpl->set(key, value, update_if_exists); } -NamedCollectionPtr NamedCollection::duplicate() const +void NamedCollection::remove(const Key & key) +{ + pimpl->remove(key); +} + +std::shared_ptr NamedCollection::duplicate() const { return std::make_shared(pimpl->copy()); } @@ -492,9 +537,9 @@ template UInt64 NamedCollection::getOrDefault(const NamedCollection::Key template Int64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Int64 & default_value) const; template Float64 NamedCollection::getOrDefault(const NamedCollection::Key & key, const Float64 & default_value) const; -template void NamedCollection::set(const NamedCollection::Key & key, const String & value); -template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value); -template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value); -template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value); +template void NamedCollection::set(const NamedCollection::Key & key, const String & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const UInt64 & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const Int64 & value, bool update_if_exists); +template void NamedCollection::set(const NamedCollection::Key & key, const Float64 & value, bool update_if_exists); } diff --git a/src/Storages/NamedCollections.h b/src/Storages/NamedCollections.h index 0056b255025..83bb1dd964e 100644 --- a/src/Storages/NamedCollections.h +++ b/src/Storages/NamedCollections.h @@ -23,7 +23,7 @@ using NamedCollectionPtr = std::shared_ptr; class NamedCollection { private: - struct Impl; + class Impl; using ImplPtr = std::unique_ptr; ImplPtr pimpl; @@ -47,9 +47,11 @@ public: template T getOrDefault(const Key & key, const T & default_value) const; - template void set(const Key & key, const T & value); + template void set(const Key & key, const T & value, bool update_if_exists = false); - NamedCollectionPtr duplicate() const; + void remove(const Key & key); + + std::shared_ptr duplicate() const; Keys getKeys() const; @@ -64,7 +66,9 @@ class NamedCollectionFactory : boost::noncopyable public: static NamedCollectionFactory & instance(); - void initialize(const Poco::Util::AbstractConfiguration & server_config); + void initialize(const Poco::Util::AbstractConfiguration & config_); + + void reload(const Poco::Util::AbstractConfiguration & config_); bool exists(const std::string & collection_name) const; @@ -92,9 +96,8 @@ private: const std::string & collection_name, std::lock_guard & lock) const; - mutable NamedCollections named_collections; + mutable NamedCollections loaded_named_collections; - /// FIXME: this will be invalid when config is reloaded const Poco::Util::AbstractConfiguration * config; bool is_initialized = false; diff --git a/src/Storages/System/StorageSystemNamedCollections.cpp b/src/Storages/System/StorageSystemNamedCollections.cpp index 8c3092b948e..6f4078369d2 100644 --- a/src/Storages/System/StorageSystemNamedCollections.cpp +++ b/src/Storages/System/StorageSystemNamedCollections.cpp @@ -32,7 +32,6 @@ void StorageSystemNamedCollections::fillData(MutableColumns & res_columns, Conte context->checkAccess(AccessType::SHOW_NAMED_COLLECTIONS); auto collections = NamedCollectionFactory::instance().getAll(); - for (const auto & [name, collection] : collections) { res_columns[0]->insert(name); diff --git a/src/Storages/tests/gtest_named_collections.cpp b/src/Storages/tests/gtest_named_collections.cpp index 6a3057df329..5ba9156bcd9 100644 --- a/src/Storages/tests/gtest_named_collections.cpp +++ b/src/Storages/tests/gtest_named_collections.cpp @@ -44,11 +44,6 @@ TEST(NamedCollections, SimpleConfig) key2: 2 key3: 3.3 key4: -4 -)CONFIG"); - ASSERT_EQ(collections["collection2"]->dumpStructure(), - R"CONFIG(key4: value4 -key5: 5 -key6: 6.6 )CONFIG"); auto collection1 = NamedCollectionFactory::instance().get("collection1"); @@ -59,53 +54,90 @@ key6: 6.6 ASSERT_TRUE(collection1->get("key3") == 3.3); ASSERT_TRUE(collection1->get("key4") == -4); + ASSERT_EQ(collections["collection2"]->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 +)CONFIG"); + auto collection2 = NamedCollectionFactory::instance().get("collection2"); ASSERT_TRUE(collection2 != nullptr); ASSERT_TRUE(collection2->get("key4") == "value4"); ASSERT_TRUE(collection2->get("key5") == 5); ASSERT_TRUE(collection2->get("key6") == 6.6); + + auto collection2_copy = collections["collection2"]->duplicate(); + NamedCollectionFactory::instance().add("collection2_copy", collection2_copy); + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection2_copy")); + ASSERT_EQ(NamedCollectionFactory::instance().get("collection2_copy")->dumpStructure(), + R"CONFIG(key4: value4 +key5: 5 +key6: 6.6 +)CONFIG"); + + collection2_copy->set("key4", "value44", true); + ASSERT_TRUE(collection2_copy->get("key4") == "value44"); + ASSERT_TRUE(collection2->get("key4") == "value4"); + + collection2_copy->remove("key4"); + ASSERT_TRUE(collection2_copy->getOrDefault("key4", "N") == "N"); + ASSERT_TRUE(collection2->getOrDefault("key4", "N") == "value4"); + + collection2_copy->set("key4", "value45"); + ASSERT_TRUE(collection2_copy->getOrDefault("key4", "N") == "value45"); + + NamedCollectionFactory::instance().remove("collection2_copy"); + ASSERT_FALSE(NamedCollectionFactory::instance().exists("collection2_copy")); + + config.reset(); } -// TEST(NamedCollections, NestedConfig) -// { -// std::string xml(R"CONFIG( -// -// -// -// value1 -// -// -// value2_1 -// -// -// value2_4 -// value2_5 -// -// -// -// -// -// )CONFIG"); -// -// Poco::XML::DOMParser dom_parser; -// Poco::AutoPtr document = dom_parser.parseString(xml); -// Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); -// -// ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); -// -// using ValueInfo = NamedCollectionValueInfo; -// ValueInfo string_def{Field::Types::Which::String, std::nullopt, true}; -// -// NamedCollectionInfo collection1_info; -// collection1_info.emplace("key1.key1_1", string_def); -// collection1_info.emplace("key2.key2_1", string_def); -// collection1_info.emplace("key2.key2_2.key2_3.key2_4", string_def); -// collection1_info.emplace("key2.key2_2.key2_3.key2_5", string_def); -// -// auto collection1 = NamedCollectionFactory::instance().get("collection1", collection1_info); -// ASSERT_TRUE(collection1 != nullptr); -// -// ASSERT_TRUE(collection1->get("key1.key1_1").safeGet() == "value1"); -// -// } +TEST(NamedCollections, NestedConfig) +{ + std::string xml(R"CONFIG( + + + + value1 + + + value2_1 + + + 4 + 5 + + + + + +)CONFIG"); + + Poco::XML::DOMParser dom_parser; + Poco::AutoPtr document = dom_parser.parseString(xml); + Poco::AutoPtr config = new Poco::Util::XMLConfiguration(document); + NamedCollectionFactory::instance().reload(*config); + + ASSERT_TRUE(NamedCollectionFactory::instance().exists("collection1")); + + auto collection1 = NamedCollectionFactory::instance().get("collection1"); + ASSERT_TRUE(collection1 != nullptr); + + ASSERT_EQ(collection1->dumpStructure(), + R"CONFIG(key1: + key1_1: value1 +key2: + key2_1: value2_1 + key2_2: + key2_3: + key2_4: 4 + key2_5: 5 +)CONFIG"); + + ASSERT_EQ(collection1->get("key1.key1_1"), "value1"); + ASSERT_EQ(collection1->get("key2.key2_1"), "value2_1"); + ASSERT_EQ(collection1->get("key2.key2_2.key2_3.key2_4"), 4); + ASSERT_EQ(collection1->get("key2.key2_2.key2_3.key2_5"), 5); + +} From 9020631826f29540304918528c3efb0062782cbf Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Wed, 9 Nov 2022 13:30:22 +0100 Subject: [PATCH 361/526] Ignore SAVEPOINT queries in MaterializedMySQL --- src/Core/MySQL/MySQLReplication.cpp | 3 ++- src/Core/MySQL/MySQLReplication.h | 3 ++- .../materialize_with_ddl.py | 17 +++++++++++++++++ .../test_materialized_mysql_database/test.py | 7 +++++++ 4 files changed, 28 insertions(+), 2 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 6f3ac1b40e9..b211d746a2f 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -118,7 +118,7 @@ namespace MySQLReplication } else if (query.starts_with("SAVEPOINT")) { - throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR); + typ = QUERY_SAVEPOINT; } } @@ -941,6 +941,7 @@ namespace MySQLReplication { case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: + case QUERY_SAVEPOINT: { event = std::make_shared(std::move(query->header)); break; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index 8900eee0102..5825924d10b 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -368,7 +368,8 @@ namespace MySQLReplication { QUERY_EVENT_DDL = 0, QUERY_EVENT_MULTI_TXN_FLAG = 1, - QUERY_EVENT_XA = 2 + QUERY_EVENT_XA = 2, + QUERY_SAVEPOINT = 3, }; class QueryEvent : public EventBase diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 22d4633685e..c6f2f80c2fe 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -2151,3 +2151,20 @@ def materialized_database_mysql_date_type_to_date32( "SELECT b from test_database.a order by a FORMAT TSV", "1970-01-01\n1971-02-16\n2101-05-16\n2022-02-16\n" + "2104-06-06\n", ) + + +def savepoint(clickhouse_node, mysql_node, mysql_host): + db = "savepoint" + clickhouse_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"DROP DATABASE IF EXISTS {db}") + mysql_node.query(f"CREATE DATABASE {db}") + mysql_node.query(f"CREATE TABLE {db}.t1 (id INT PRIMARY KEY)") + clickhouse_node.query( + f"CREATE DATABASE {db} ENGINE = MaterializeMySQL('{mysql_host}:3306', '{db}', 'root', 'clickhouse')" + ) + mysql_node.query("BEGIN") + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (1)") + mysql_node.query("SAVEPOINT savepoint_1") + mysql_node.query(f"INSERT INTO {db}.t1 VALUES (2)") + mysql_node.query("ROLLBACK TO savepoint_1") + mysql_node.query("COMMIT") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index a672ec72275..0e33c01a6c9 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -509,3 +509,10 @@ def test_materialized_database_mysql_date_type_to_date32( materialize_with_ddl.materialized_database_mysql_date_type_to_date32( clickhouse_node, started_mysql_5_7, "mysql57" ) + + +def test_savepoint_query( + started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node +): + materialize_with_ddl.savepoint(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.savepoint(clickhouse_node, started_mysql_5_7, "mysql57") From 0ed1d983fd6dd55c34d3f6c9ad0b6d82acc4c9c4 Mon Sep 17 00:00:00 2001 From: canenoneko <53267815+canenoneko@users.noreply.github.com> Date: Fri, 11 Nov 2022 09:53:27 +0800 Subject: [PATCH 362/526] Update distinctive-features.md --- docs/zh/introduction/distinctive-features.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/introduction/distinctive-features.md b/docs/zh/introduction/distinctive-features.md index c97ab082c09..a9097e0ecaa 100644 --- a/docs/zh/introduction/distinctive-features.md +++ b/docs/zh/introduction/distinctive-features.md @@ -67,7 +67,7 @@ ClickHouse提供各种各样在允许牺牲数据精度的情况下对查询进 2. 基于数据的部分样本进行近似查询。这时,仅会从磁盘检索少部分比例的数据。 3. 不使用全部的聚合条件,通过随机选择有限个数据聚合条件进行聚合。这在数据聚合条件满足某些分布条件下,在提供相当准确的聚合结果的同时降低了计算资源的使用。 -## Adaptive Join Algorithm {#adaptive-join-algorithm} +## 自适应连接算法 {#adaptive-join-algorithm} ClickHouse支持自定义[JOIN](../sql-reference/statements/select/join.md)多个表,它更倾向于散列连接算法,如果有多个大表,则使用合并-连接算法 From 331c1cbeb7dcea4982ec323a87b7f69c550c1c64 Mon Sep 17 00:00:00 2001 From: Tiaonmmn Date: Fri, 11 Nov 2022 10:00:59 +0800 Subject: [PATCH 363/526] Update CompressionCodecDeflateQpl.cpp Modify the data type to resolve warning of clang-tidy "implicit conversion". --- src/Compression/CompressionCodecDeflateQpl.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/CompressionCodecDeflateQpl.cpp b/src/Compression/CompressionCodecDeflateQpl.cpp index 87c0ead0795..348496a2753 100644 --- a/src/Compression/CompressionCodecDeflateQpl.cpp +++ b/src/Compression/CompressionCodecDeflateQpl.cpp @@ -240,7 +240,7 @@ Int32 HardwareCodecDeflateQpl::doDecompressDataAsynchronous(const char * source, void HardwareCodecDeflateQpl::flushAsynchronousDecompressRequests() { - UInt32 n_jobs_processing = decomp_async_job_map.size(); + auto n_jobs_processing = decomp_async_job_map.size(); std::map::iterator it = decomp_async_job_map.begin(); while (n_jobs_processing) From 6f6a1f5d76ec75af7e589d1766e608ed8713e495 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 11 Nov 2022 02:30:05 +0000 Subject: [PATCH 364/526] fix --- src/TableFunctions/TableFunctionDelta.cpp | 2 +- src/TableFunctions/TableFunctionHudi.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 8456878be76..aaf89b75adc 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -158,7 +158,7 @@ StoragePtr TableFunctionDelta::executeImpl( void registerTableFunctionDelta(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index 6be5dc2d089..ea682b84ea3 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -158,7 +158,7 @@ StoragePtr TableFunctionHudi::executeImpl( void registerTableFunctionHudi(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction({.documentation = {}, .allow_readonly = true}); } } From eebc5349ccfb26253171b45a498715416b0399f8 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 11 Nov 2022 02:42:18 +0000 Subject: [PATCH 365/526] add documentation --- src/TableFunctions/TableFunctionDelta.cpp | 7 ++++++- src/TableFunctions/TableFunctionHudi.cpp | 8 ++++++-- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index aaf89b75adc..93502c00116 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -158,7 +158,12 @@ StoragePtr TableFunctionDelta::executeImpl( void registerTableFunctionDelta(TableFunctionFactory & factory) { - factory.registerFunction({.documentation = {}, .allow_readonly = true}); + factory.registerFunction( + {.documentation + = {R"(The table function can be used to read the DeltaLake table stored on object store.)", + Documentation::Examples{{"hudi", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)"}}, + Documentation::Categories{"DataLake"}}, + .allow_readonly = true}); } } diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index ea682b84ea3..b59953a2699 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -158,9 +158,13 @@ StoragePtr TableFunctionHudi::executeImpl( void registerTableFunctionHudi(TableFunctionFactory & factory) { - factory.registerFunction({.documentation = {}, .allow_readonly = true}); + factory.registerFunction( + {.documentation + = {R"(The table function can be used to read the Hudi table stored on object store.)", + Documentation::Examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)"}}, + Documentation::Categories{"DataLake"}}, + .allow_readonly = true}); } - } #endif From d4e270b522bd4d869a6af329ac96938b24c1be59 Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 10:44:16 +0800 Subject: [PATCH 366/526] to MaterializedMySQL_improvement_bug_fix-fix codestyle --- src/Core/MySQL/MySQLReplication.cpp | 6 +++--- src/Core/MySQL/MySQLReplication.h | 2 +- .../materialize_with_ddl.py | 12 +++++++++--- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp index 45437ea5843..f4785875be0 100644 --- a/src/Core/MySQL/MySQLReplication.cpp +++ b/src/Core/MySQL/MySQLReplication.cpp @@ -119,7 +119,7 @@ namespace MySQLReplication else if (query.starts_with("SAVEPOINT") || query.starts_with("ROLLBACK") || query.starts_with("RELEASE SAVEPOINT")) { - typ = QUERY_EVENT_OTHER; + typ = QUERY_SAVEPOINT; } } @@ -942,8 +942,8 @@ namespace MySQLReplication { case QUERY_EVENT_MULTI_TXN_FLAG: case QUERY_EVENT_XA: - /// Ignore queries that have no impact on the data - case QUERY_EVENT_OTHER: + /// Ignore queries that have no impact on the data. + case QUERY_SAVEPOINT: { event = std::make_shared(std::move(query->header)); break; diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h index d4bb3fb86f9..e603ecb42bf 100644 --- a/src/Core/MySQL/MySQLReplication.h +++ b/src/Core/MySQL/MySQLReplication.h @@ -369,7 +369,7 @@ namespace MySQLReplication QUERY_EVENT_DDL = 0, QUERY_EVENT_MULTI_TXN_FLAG = 1, QUERY_EVENT_XA = 2, - QUERY_EVENT_OTHER = 3 + QUERY_SAVEPOINT = 3 }; class QueryEvent : public EventBase diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index bfb6d2bc6dd..8bc1b13c7a4 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -875,9 +875,15 @@ def alter_rename_table_with_materialized_mysql_database( "1\n2\n3\n4\n5\n", ) - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5") - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6") - mysql_node.query("ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7") + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" ++ ) + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" ++ ) + mysql_node.query( ++ "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" ++ ) check_query( clickhouse_node, From 661c541e57be992f32e417ea200fdad167238c1f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 10 Nov 2022 18:43:19 +0800 Subject: [PATCH 367/526] Fix Nullable keys in hyperrectangle. --- src/Storages/Hive/HiveFile.cpp | 14 +++--- src/Storages/MergeTree/KeyCondition.cpp | 46 +++++++++++++------ src/Storages/MergeTree/KeyCondition.h | 31 ++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 ++- ...llable_primary_key_second_column.reference | 1 + ...479_nullable_primary_key_second_column.sql | 9 ++++ 6 files changed, 73 insertions(+), 34 deletions(-) create mode 100644 tests/queries/0_stateless/02479_nullable_primary_key_second_column.reference create mode 100644 tests/queries/0_stateless/02479_nullable_primary_key_second_column.sql diff --git a/src/Storages/Hive/HiveFile.cpp b/src/Storages/Hive/HiveFile.cpp index 8f5b1b5f5fd..219fe562f2c 100644 --- a/src/Storages/Hive/HiveFile.cpp +++ b/src/Storages/Hive/HiveFile.cpp @@ -54,7 +54,7 @@ Range createRangeFromOrcStatistics(const StatisticsType * stats) } else { - return Range(); + return Range::createWholeUniverseWithoutNull(); } } @@ -64,14 +64,14 @@ Range createRangeFromParquetStatistics(std::shared_ptr stats) /// We must check if there are minimum or maximum values in statistics in case of /// null values or NaN/Inf values of double type. if (!stats->HasMinMax()) - return Range(); + return Range::createWholeUniverseWithoutNull(); return Range(FieldType(stats->min()), true, FieldType(stats->max()), true); } Range createRangeFromParquetStatistics(std::shared_ptr stats) { if (!stats->HasMinMax()) - return Range(); + return Range::createWholeUniverseWithoutNull(); String min_val(reinterpret_cast(stats->min().ptr), stats->min().len); String max_val(reinterpret_cast(stats->max().ptr), stats->max().len); return Range(min_val, true, max_val, true); @@ -116,7 +116,7 @@ void IHiveFile::loadSplitMinMaxIndexes() Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { if (!col_stats || col_stats->hasNull()) - return {}; + return Range::createWholeUniverseWithoutNull(); if (const auto * int_stats = dynamic_cast(col_stats)) { @@ -155,7 +155,7 @@ Range HiveORCFile::buildRange(const orc::ColumnStatistics * col_stats) { return createRangeFromOrcStatistics(date_stats); } - return {}; + return Range::createWholeUniverseWithoutNull(); } void HiveORCFile::prepareReader() @@ -194,7 +194,7 @@ std::unique_ptr HiveORCFile::buildMinMaxIndex(c size_t range_num = index_names_and_types.size(); auto idx = std::make_unique(); - idx->hyperrectangle.resize(range_num); + idx->hyperrectangle.resize(range_num, Range::createWholeUniverseWithoutNull()); size_t i = 0; for (const auto & name_type : index_names_and_types) @@ -308,7 +308,7 @@ void HiveParquetFile::loadSplitMinMaxIndexesImpl() { auto row_group_meta = meta->RowGroup(static_cast(i)); split_minmax_idxes[i] = std::make_shared(); - split_minmax_idxes[i]->hyperrectangle.resize(num_cols); + split_minmax_idxes[i]->hyperrectangle.resize(num_cols, Range::createWholeUniverseWithoutNull()); size_t j = 0; auto it = index_names_and_types.begin(); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 1d688427a57..f053c072dfa 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -438,8 +438,8 @@ const KeyCondition::AtomMap KeyCondition::atom_map [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_IS_NOT_NULL; - // isNotNull means (-Inf, +Inf), which is the default Range - out.range = Range(); + // isNotNull means (-Inf, +Inf) + out.range = Range::createWholeUniverseWithoutNull(); return true; } }, @@ -448,9 +448,10 @@ const KeyCondition::AtomMap KeyCondition::atom_map [] (RPNElement & out, const Field &) { out.function = RPNElement::FUNCTION_IS_NULL; - // isNull means +Inf (NULLS_LAST) or -Inf (NULLS_FIRST), - // which is equivalent to not in Range (-Inf, +Inf) - out.range = Range(); + // isNull means +Inf (NULLS_LAST) or -Inf (NULLS_FIRST), We don't support discrete + // ranges, instead will use the inverse of (-Inf, +Inf). The inversion happens in + // checkInHyperrectangle. + out.range = Range::createWholeUniverseWithoutNull(); return true; } } @@ -1938,6 +1939,7 @@ static BoolMask forAnyHyperrectangle( bool left_bounded, bool right_bounded, std::vector & hyperrectangle, + const DataTypes & data_types, size_t prefix_size, BoolMask initial_mask, F && callback) @@ -1981,12 +1983,17 @@ static BoolMask forAnyHyperrectangle( if (left_bounded && right_bounded) hyperrectangle[prefix_size] = Range(left_keys[prefix_size], false, right_keys[prefix_size], false); else if (left_bounded) - hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], false); + hyperrectangle[prefix_size] = Range::createLeftBounded(left_keys[prefix_size], false, data_types[prefix_size]->isNullable()); else if (right_bounded) - hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], false); + hyperrectangle[prefix_size] = Range::createRightBounded(right_keys[prefix_size], false, data_types[prefix_size]->isNullable()); for (size_t i = prefix_size + 1; i < key_size; ++i) - hyperrectangle[i] = Range(); + { + if (data_types[i]->isNullable()) + hyperrectangle[i] = Range::createWholeUniverse(); + else + hyperrectangle[i] = Range::createWholeUniverseWithoutNull(); + } BoolMask result = initial_mask; @@ -2004,7 +2011,9 @@ static BoolMask forAnyHyperrectangle( if (left_bounded) { hyperrectangle[prefix_size] = Range(left_keys[prefix_size]); - result = result | forAnyHyperrectangle(key_size, left_keys, right_keys, true, false, hyperrectangle, prefix_size + 1, initial_mask, callback); + result = result + | forAnyHyperrectangle( + key_size, left_keys, right_keys, true, false, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback); if (result.isComplete()) return result; } @@ -2014,7 +2023,9 @@ static BoolMask forAnyHyperrectangle( if (right_bounded) { hyperrectangle[prefix_size] = Range(right_keys[prefix_size]); - result = result | forAnyHyperrectangle(key_size, left_keys, right_keys, false, true, hyperrectangle, prefix_size + 1, initial_mask, callback); + result = result + | forAnyHyperrectangle( + key_size, left_keys, right_keys, false, true, hyperrectangle, data_types, prefix_size + 1, initial_mask, callback); if (result.isComplete()) return result; } @@ -2030,7 +2041,16 @@ BoolMask KeyCondition::checkInRange( const DataTypes & data_types, BoolMask initial_mask) const { - std::vector key_ranges(used_key_size, Range()); + std::vector key_ranges; + + key_ranges.reserve(used_key_size); + for (size_t i = 0; i < used_key_size; ++i) + { + if (data_types[i]->isNullable()) + key_ranges.push_back(Range::createWholeUniverse()); + else + key_ranges.push_back(Range::createWholeUniverseWithoutNull()); + } // std::cerr << "Checking for: ["; // for (size_t i = 0; i != used_key_size; ++i) @@ -2041,7 +2061,7 @@ BoolMask KeyCondition::checkInRange( // std::cerr << (i != 0 ? ", " : "") << applyVisitor(FieldVisitorToString(), right_keys[i]); // std::cerr << "]\n"; - return forAnyHyperrectangle(used_key_size, left_keys, right_keys, true, true, key_ranges, 0, initial_mask, + return forAnyHyperrectangle(used_key_size, left_keys, right_keys, true, true, key_ranges, data_types, 0, initial_mask, [&] (const std::vector & key_ranges_hyperrectangle) { auto res = checkInHyperrectangle(key_ranges_hyperrectangle, data_types); @@ -2193,7 +2213,7 @@ BoolMask KeyCondition::checkInHyperrectangle( const Range * key_range = &hyperrectangle[element.key_column]; /// The case when the column is wrapped in a chain of possibly monotonic functions. - Range transformed_range; + Range transformed_range = Range::createWholeUniverse(); if (!element.monotonic_functions_chain.empty()) { std::optional new_range = applyMonotonicFunctionsChainToRange( diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 138dce83db9..6aa0ae737c8 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -60,13 +60,10 @@ private: static bool less(const Field & lhs, const Field & rhs); public: - FieldRef left = NEGATIVE_INFINITY; /// the left border - FieldRef right = POSITIVE_INFINITY; /// the right border - bool left_included = false; /// includes the left border - bool right_included = false; /// includes the right border - - /// The whole universe (not null). - Range() {} /// NOLINT + FieldRef left; /// the left border + FieldRef right; /// the right border + bool left_included; /// includes the left border + bool right_included; /// includes the right border /// One point. Range(const FieldRef & point) /// NOLINT @@ -82,9 +79,19 @@ public: shrinkToIncludedIfPossible(); } - static Range createRightBounded(const FieldRef & right_point, bool right_included) + static Range createWholeUniverse() { - Range r; + return Range(NEGATIVE_INFINITY, true, POSITIVE_INFINITY, true); + } + + static Range createWholeUniverseWithoutNull() + { + return Range(NEGATIVE_INFINITY, false, POSITIVE_INFINITY, false); + } + + static Range createRightBounded(const FieldRef & right_point, bool right_included, bool with_null = false) + { + Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); r.right = right_point; r.right_included = right_included; r.shrinkToIncludedIfPossible(); @@ -94,9 +101,9 @@ public: return r; } - static Range createLeftBounded(const FieldRef & left_point, bool left_included) + static Range createLeftBounded(const FieldRef & left_point, bool left_included, bool with_null = false) { - Range r; + Range r = with_null ? createWholeUniverse() : createWholeUniverseWithoutNull(); r.left = left_point; r.left_included = left_included; r.shrinkToIncludedIfPossible(); @@ -367,7 +374,7 @@ private: Function function = FUNCTION_UNKNOWN; /// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE. - Range range; + Range range = Range::createWholeUniverse(); size_t key_column = 0; /// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET using MergeTreeSetIndexPtr = std::shared_ptr; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 59171165704..0318fc0648c 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -686,7 +686,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_lower_limit) { - if (!key_condition.addCondition(sampling_key.column_names[0], Range::createLeftBounded(lower, true))) + if (!key_condition.addCondition( + sampling_key.column_names[0], Range::createLeftBounded(lower, true, sampling_key.data_types[0]->isNullable()))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); @@ -703,7 +704,8 @@ MergeTreeDataSelectSamplingData MergeTreeDataSelectExecutor::getSampling( if (has_upper_limit) { - if (!key_condition.addCondition(sampling_key.column_names[0], Range::createRightBounded(upper, false))) + if (!key_condition.addCondition( + sampling_key.column_names[0], Range::createRightBounded(upper, false, sampling_key.data_types[0]->isNullable()))) throw Exception("Sampling column not in primary key", ErrorCodes::ILLEGAL_COLUMN); ASTPtr args = std::make_shared(); diff --git a/tests/queries/0_stateless/02479_nullable_primary_key_second_column.reference b/tests/queries/0_stateless/02479_nullable_primary_key_second_column.reference new file mode 100644 index 00000000000..f0227e1a41e --- /dev/null +++ b/tests/queries/0_stateless/02479_nullable_primary_key_second_column.reference @@ -0,0 +1 @@ +a \N diff --git a/tests/queries/0_stateless/02479_nullable_primary_key_second_column.sql b/tests/queries/0_stateless/02479_nullable_primary_key_second_column.sql new file mode 100644 index 00000000000..ad0c09222c2 --- /dev/null +++ b/tests/queries/0_stateless/02479_nullable_primary_key_second_column.sql @@ -0,0 +1,9 @@ +drop table if exists test_table; + +create table test_table (A Nullable(String), B Nullable(String)) engine MergeTree order by (A,B) settings index_granularity = 1, allow_nullable_key=1; + +insert into test_table values ('a', 'b'), ('a', null), (null, 'b'); + +select * from test_table where B is null; + +drop table test_table; From 0e614aec122d048dde4f53052fad198003dbcbed Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 11:24:29 +0800 Subject: [PATCH 368/526] to MaterializedMySQL_improvement_bug_fix-fix code sytle --- .../materialize_with_ddl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index 8bc1b13c7a4..a994863417e 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -877,13 +877,13 @@ def alter_rename_table_with_materialized_mysql_database( mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" -+ ) + ) mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" -+ ) + ) mysql_node.query( + "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" -+ ) + ) check_query( clickhouse_node, From a9a208c3cdc9ccc370a529429158e047e42c238a Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 11:46:22 +0800 Subject: [PATCH 369/526] to MaterializedMySQL_improvement_bug_fix-fix code style --- .../materialize_with_ddl.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index a994863417e..cdb8ba464af 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -876,13 +876,13 @@ def alter_rename_table_with_materialized_mysql_database( ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" + "ALTER TABLE test_database_rename_table.test_table_4 RENAME test_database_rename_table.test_table_5" ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" + "ALTER TABLE test_database_rename_table.test_table_5 RENAME TO test_database_rename_table.test_table_6" ) mysql_node.query( -+ "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" + "ALTER TABLE test_database_rename_table.test_table_6 RENAME AS test_database_rename_table.test_table_7" ) check_query( From d96d11899b87cb2ea9e001604ca6f64006a53bb1 Mon Sep 17 00:00:00 2001 From: flynn Date: Fri, 11 Nov 2022 06:14:51 +0000 Subject: [PATCH 370/526] fix --- src/TableFunctions/TableFunctionDelta.cpp | 5 ++--- src/TableFunctions/TableFunctionHudi.cpp | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDelta.cpp index 93502c00116..25ea2aaa77f 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDelta.cpp @@ -81,6 +81,8 @@ void TableFunctionDelta::parseArgumentsImpl( if (args_to_idx.contains("format")) base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + else + base_configuration.format = "Parquet"; if (args_to_idx.contains("structure")) base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); @@ -96,9 +98,6 @@ void TableFunctionDelta::parseArgumentsImpl( if (args_to_idx.contains("secret_access_key")) base_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); - - if (base_configuration.format == "auto") - base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); } void TableFunctionDelta::parseArguments(const ASTPtr & ast_function, ContextPtr context) diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index b59953a2699..b1db90da550 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -81,6 +81,8 @@ void TableFunctionHudi::parseArgumentsImpl( if (args_to_idx.contains("format")) base_configuration.format = checkAndGetLiteralArgument(args[args_to_idx["format"]], "format"); + else + base_configuration.format = "Parquet"; if (args_to_idx.contains("structure")) base_configuration.structure = checkAndGetLiteralArgument(args[args_to_idx["structure"]], "structure"); @@ -96,9 +98,6 @@ void TableFunctionHudi::parseArgumentsImpl( if (args_to_idx.contains("secret_access_key")) base_configuration.auth_settings.secret_access_key = checkAndGetLiteralArgument(args[args_to_idx["secret_access_key"]], "secret_access_key"); - - if (base_configuration.format == "auto") - base_configuration.format = FormatFactory::instance().getFormatFromFileName(base_configuration.url, true); } void TableFunctionHudi::parseArguments(const ASTPtr & ast_function, ContextPtr context) From 3835373644c851b646c0b770f2bfa74f953ccacd Mon Sep 17 00:00:00 2001 From: zzsmdfj Date: Fri, 11 Nov 2022 16:40:10 +0800 Subject: [PATCH 371/526] to add_oss_function_and_StorageOSS --- .../maskSensitiveInfoInQueryForLogging.cpp | 4 ++-- src/Storages/StorageS3.cpp | 5 +++++ src/Storages/registerStorages.cpp | 2 ++ src/TableFunctions/TableFunctionS3.cpp | 5 +++++ src/TableFunctions/TableFunctionS3.h | 12 ++++++++++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + ...14_all_new_table_functions_must_be_documented.sql | 2 +- 8 files changed, 29 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index c69f91394b9..fe05283eef5 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -164,7 +164,7 @@ namespace /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) wipePasswordFromArgument(*storage.engine, data, 4); } - else if (engine_name == "S3" || engine_name == "COSN") + else if (engine_name == "S3" || engine_name == "COSN" || engine_name == "OSS") { /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) wipePasswordFromS3TableEngineArguments(*storage.engine, data); @@ -222,7 +222,7 @@ namespace /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) wipePasswordFromArgument(function, data, 4); } - else if (function.name == "s3" || function.name == "cosn") + else if (function.name == "s3" || function.name == "cosn" || function.name == "oss") { /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ false); diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index d759c339dea..ab9b71f5ff3 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -1314,6 +1314,11 @@ void registerStorageCOS(StorageFactory & factory) return registerStorageS3Impl("COSN", factory); } +void registerStorageOSS(StorageFactory & factory) +{ + return registerStorageS3Impl("OSS", factory); +} + NamesAndTypesList StorageS3::getVirtuals() const { return virtual_columns; diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index fd81b55ef61..200b8e637da 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -32,6 +32,7 @@ void registerStorageMeiliSearch(StorageFactory& factory); #if USE_AWS_S3 void registerStorageS3(StorageFactory & factory); void registerStorageCOS(StorageFactory & factory); +void registerStorageOSS(StorageFactory & factory); void registerStorageHudi(StorageFactory & factory); void registerStorageDelta(StorageFactory & factory); #endif @@ -120,6 +121,7 @@ void registerStorages() #if USE_AWS_S3 registerStorageS3(factory); registerStorageCOS(factory); + registerStorageOSS(factory); registerStorageHudi(factory); registerStorageDelta(factory); #endif diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 4c0b5352545..7f07dc16025 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -183,6 +183,11 @@ void registerTableFunctionCOS(TableFunctionFactory & factory) factory.registerFunction(); } +void registerTableFunctionOSS(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} + } #endif diff --git a/src/TableFunctions/TableFunctionS3.h b/src/TableFunctions/TableFunctionS3.h index 5c12c2a3975..e13fb7f88d9 100644 --- a/src/TableFunctions/TableFunctionS3.h +++ b/src/TableFunctions/TableFunctionS3.h @@ -62,6 +62,18 @@ private: const char * getStorageTypeName() const override { return "COSN"; } }; +class TableFunctionOSS : public TableFunctionS3 +{ +public: + static constexpr auto name = "oss"; + std::string getName() const override + { + return name; + } +private: + const char * getStorageTypeName() const override { return "OSS"; } +}; + } #endif diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index 9328c12c122..89701f3bf70 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -27,6 +27,7 @@ void registerTableFunctions() registerTableFunctionS3(factory); registerTableFunctionS3Cluster(factory); registerTableFunctionCOS(factory); + registerTableFunctionOSS(factory); #endif #if USE_HDFS diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index 3bc9e3a85da..afbda491e9a 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -24,6 +24,7 @@ void registerTableFunctionMeiliSearch(TableFunctionFactory & factory); void registerTableFunctionS3(TableFunctionFactory & factory); void registerTableFunctionS3Cluster(TableFunctionFactory & factory); void registerTableFunctionCOS(TableFunctionFactory & factory); +void registerTableFunctionOSS(TableFunctionFactory & factory); #endif #if USE_HDFS diff --git a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql index 5d43ec6f0c2..a73993f6a5a 100644 --- a/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql +++ b/tests/queries/0_stateless/02414_all_new_table_functions_must_be_documented.sql @@ -2,5 +2,5 @@ -- Please help shorten this list down to zero elements. SELECT name FROM system.table_functions WHERE length(description) < 10 AND name NOT IN ( - 'cosn', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite' -- these functions are not enabled in fast test + 'cosn', 'oss', 'hdfs', 'hdfsCluster', 'hive', 'mysql', 'postgresql', 's3', 's3Cluster', 'sqlite' -- these functions are not enabled in fast test ) ORDER BY name; From 41056bb99063a41ace8a55c2482475578b466535 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Wed, 9 Nov 2022 18:54:49 +0100 Subject: [PATCH 372/526] Vuln WIP --- .../AggregateFunctionAggThrow.cpp | 2 +- ...ateFunctionCategoricalInformationValue.cpp | 2 +- .../AggregateFunctionGroupArray.h | 4 +- .../AggregateFunctionGroupArrayMoving.h | 2 +- .../AggregateFunctionHistogram.h | 2 +- .../AggregateFunctionMaxIntersections.h | 2 +- .../AggregateFunctionMinMaxAny.h | 4 +- .../AggregateFunctionSequenceNextNode.h | 2 +- .../AggregateFunctionUniqUpTo.h | 2 +- src/AggregateFunctions/QuantileExact.h | 2 +- src/AggregateFunctions/QuantileTDigest.h | 2 +- src/AggregateFunctions/StatCommon.h | 4 +- .../0_stateless/02477_invalid_reads.reference | 0 .../0_stateless/02477_invalid_reads.sql | 47 +++++++++++++++++++ 14 files changed, 62 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/02477_invalid_reads.reference create mode 100644 tests/queries/0_stateless/02477_invalid_reads.sql diff --git a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp index ae1bbfddf75..432b1f39f84 100644 --- a/src/AggregateFunctions/AggregateFunctionAggThrow.cpp +++ b/src/AggregateFunctions/AggregateFunctionAggThrow.cpp @@ -95,7 +95,7 @@ public: void deserialize(AggregateDataPtr __restrict /* place */, ReadBuffer & buf, std::optional /* version */, Arena *) const override { char c = 0; - buf.read(c); + buf.readStrict(c); } void insertResultInto(AggregateDataPtr __restrict, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp index 89ffdfa6109..93b5de0c5ab 100644 --- a/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp +++ b/src/AggregateFunctions/AggregateFunctionCategoricalInformationValue.cpp @@ -118,7 +118,7 @@ public: void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - buf.read(place, sizeOfData()); + buf.readStrict(place, sizeOfData()); } DataTypePtr getReturnType() const override diff --git a/src/AggregateFunctions/AggregateFunctionGroupArray.h b/src/AggregateFunctions/AggregateFunctionGroupArray.h index 6888c113556..89b382de819 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArray.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArray.h @@ -270,7 +270,7 @@ public: auto & value = this->data(place).value; value.resize(size, arena); - buf.read(reinterpret_cast(value.data()), size * sizeof(value[0])); + buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); if constexpr (Trait::sampler == Sampler::RNG) { @@ -343,7 +343,7 @@ struct GroupArrayNodeBase Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; - buf.read(node->data(), size); + buf.readStrict(node->data(), size); return node; } }; diff --git a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h index 1fa568024af..40867b1949a 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h +++ b/src/AggregateFunctions/AggregateFunctionGroupArrayMoving.h @@ -144,7 +144,7 @@ public: { auto & value = this->data(place).value; value.resize(size, arena); - buf.read(reinterpret_cast(value.data()), size * sizeof(value[0])); + buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); this->data(place).sum = value.back(); } } diff --git a/src/AggregateFunctions/AggregateFunctionHistogram.h b/src/AggregateFunctions/AggregateFunctionHistogram.h index 18bfc085ba3..fbd92aa8220 100644 --- a/src/AggregateFunctions/AggregateFunctionHistogram.h +++ b/src/AggregateFunctions/AggregateFunctionHistogram.h @@ -293,7 +293,7 @@ public: if (size > max_bins * 2) throw Exception("Too many bins", ErrorCodes::TOO_LARGE_ARRAY_SIZE); - buf.read(reinterpret_cast(points), size * sizeof(WeightedValue)); + buf.readStrict(reinterpret_cast(points), size * sizeof(WeightedValue)); } }; diff --git a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h index 19547bdd247..d2f553172c9 100644 --- a/src/AggregateFunctions/AggregateFunctionMaxIntersections.h +++ b/src/AggregateFunctions/AggregateFunctionMaxIntersections.h @@ -130,7 +130,7 @@ public: auto & value = this->data(place).value; value.resize(size, arena); - buf.read(reinterpret_cast(value.data()), size * sizeof(value[0])); + buf.readStrict(reinterpret_cast(value.data()), size * sizeof(value[0])); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index f8d252cf8e9..46be7331195 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -515,7 +515,7 @@ public: size = rhs_size; if (size > 0) - buf.read(small_data, size); + buf.readStrict(small_data, size); } else { @@ -527,7 +527,7 @@ public: } size = rhs_size; - buf.read(large_data, size); + buf.readStrict(large_data, size); } } else diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c29055ae8db..87ccd0d8a19 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -81,7 +81,7 @@ struct NodeBase Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; - buf.read(node->data(), size); + buf.readStrict(node->data(), size); readBinary(node->event_time, buf); UInt64 ulong_bitset; diff --git a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h index 48b4c0f2c68..99f36b664d7 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqUpTo.h +++ b/src/AggregateFunctions/AggregateFunctionUniqUpTo.h @@ -108,7 +108,7 @@ struct AggregateFunctionUniqUpToData readBinary(count, rb); if (count <= threshold) - rb.read(data_ptr, count * sizeof(T)); + rb.readStrict(data_ptr, count * sizeof(T)); } /// ALWAYS_INLINE is required to have better code layout for uniqUpTo function diff --git a/src/AggregateFunctions/QuantileExact.h b/src/AggregateFunctions/QuantileExact.h index eb1865d0355..bec7bd4c6c3 100644 --- a/src/AggregateFunctions/QuantileExact.h +++ b/src/AggregateFunctions/QuantileExact.h @@ -55,7 +55,7 @@ struct QuantileExactBase size_t size = 0; readVarUInt(size, buf); array.resize(size); - buf.read(reinterpret_cast(array.data()), size * sizeof(array[0])); + buf.readStrict(reinterpret_cast(array.data()), size * sizeof(array[0])); } Value get(Float64 level) diff --git a/src/AggregateFunctions/QuantileTDigest.h b/src/AggregateFunctions/QuantileTDigest.h index 5e9261919bd..109a98815ae 100644 --- a/src/AggregateFunctions/QuantileTDigest.h +++ b/src/AggregateFunctions/QuantileTDigest.h @@ -316,7 +316,7 @@ public: centroids.resize(size); // From now, TDigest will be in invalid state if exception is thrown. - buf.read(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); + buf.readStrict(reinterpret_cast(centroids.data()), size * sizeof(centroids[0])); for (const auto & c : centroids) { diff --git a/src/AggregateFunctions/StatCommon.h b/src/AggregateFunctions/StatCommon.h index ff824ca11b8..407c1a1cd67 100644 --- a/src/AggregateFunctions/StatCommon.h +++ b/src/AggregateFunctions/StatCommon.h @@ -112,8 +112,8 @@ struct StatisticalSample readVarUInt(size_y, buf); x.resize(size_x, arena); y.resize(size_y, arena); - buf.read(reinterpret_cast(x.data()), size_x * sizeof(x[0])); - buf.read(reinterpret_cast(y.data()), size_y * sizeof(y[0])); + buf.readStrict(reinterpret_cast(x.data()), size_x * sizeof(x[0])); + buf.readStrict(reinterpret_cast(y.data()), size_y * sizeof(y[0])); } }; diff --git a/tests/queries/0_stateless/02477_invalid_reads.reference b/tests/queries/0_stateless/02477_invalid_reads.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02477_invalid_reads.sql b/tests/queries/0_stateless/02477_invalid_reads.sql new file mode 100644 index 00000000000..1e488ee5627 --- /dev/null +++ b/tests/queries/0_stateless/02477_invalid_reads.sql @@ -0,0 +1,47 @@ +-- MIN, MAX AND FAMILY should check for errors in its input +SELECT finalizeAggregation(CAST(unhex('0F00000030'), 'AggregateFunction(min, String)')); -- { serverError 33 } +SELECT finalizeAggregation(CAST(unhex('FFFF000030'), 'AggregateFunction(min, String)')); -- { serverError 33 } + +-- aggThrow should check for errors in its input +SELECT finalizeAggregation(CAST('', 'AggregateFunction(aggThrow(0.), UInt8)')); -- { serverError 32 } + +-- categoricalInformationValue should check for errors in its input +SELECT finalizeAggregation(CAST(unhex('01000000000000000100000000000000'), + 'AggregateFunction(categoricalInformationValue, UInt8, UInt8)')); -- { serverError 33 } +SELECT finalizeAggregation(CAST(unhex('0101000000000000000100000000000000020000000000000001000000000000'), + 'AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)')); -- { serverError 33 } + +-- groupArray should check for errors in its input +SELECT finalizeAggregation(CAST(unhex('5FF3001310132'), 'AggregateFunction(groupArray, String)')); -- { serverError 33 } +SELECT finalizeAggregation(CAST(unhex('FF000000000000000001000000000000000200000000000000'), 'AggregateFunction(groupArray, UInt64)')); -- { serverError 33 } + +-- Same for groupArrayMovingXXXX +SELECT finalizeAggregation(CAST(unhex('0FF00000000000000001000000000000000300000000000000'), 'AggregateFunction(groupArrayMovingSum, UInt64)')); -- { serverError 33 } +SELECT finalizeAggregation(CAST(unhex('0FF00000000000000001000000000000000300000000000000'), 'AggregateFunction(groupArrayMovingAvg, UInt64)')); -- { serverError 33 } + +-- Histogram +SELECT finalizeAggregation(CAST(unhex('00000000000024C000000000000018C00500000000000024C0000000000000F03F00000000000022C0000000000000F03F00000000000020C0000000000000'), + 'AggregateFunction(histogram(5), Int64)')); -- { serverError 33 } + +-- StatisticalSample +SELECT finalizeAggregation(CAST(unhex('0F01000000000000244000000000000026400000000000002840000000000000244000000000000026400000000000002840000000000000F03F'), + 'AggregateFunction(mannWhitneyUTest, Float64, UInt8)')); -- { serverError 33 } + +-- maxIntersections +SELECT finalizeAggregation(CAST(unhex('0F010000000000000001000000000000000300000000000000FFFFFFFFFFFFFFFF03340B9B047F000001000000000000000500000065000000FFFFFFFFFFFFFFFF'), + 'AggregateFunction(maxIntersections, UInt8, UInt8)')); -- { serverError 33 } + +-- sequenceNextNode (This was fine because it would fail in the next readBinary call, but better to add a test) +SELECT finalizeAggregation(CAST(unhex('FFFFFFF014181056F38010000000000000001FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF'), + 'AggregateFunction(sequenceNextNode(''forward'', ''head''), DateTime, Nullable(String), UInt8, Nullable(UInt8))')) + SETTINGS allow_experimental_funnel_functions=1; -- { serverError 33 } + +-- uniqUpTo +SELECT finalizeAggregation(CAST(unhex('04128345AA2BC97190'), + 'AggregateFunction(uniqUpTo(10), String)')); -- { serverError 33 } + +-- quantiles +SELECT finalizeAggregation(CAST(unhex('0F0000000000000000'), + 'AggregateFunction(quantileExact, UInt64)')); -- { serverError 33 } +SELECT finalizeAggregation(CAST(unhex('0F000000000000803F'), + 'AggregateFunction(quantileTDigest, UInt64)')); -- { serverError 33 } From 4ce261dfd77da5ca5fbd249c9aa961ddba7e4143 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 19:17:10 +0100 Subject: [PATCH 373/526] Analyzer improve JOIN with constants --- src/Planner/PlannerJoinTree.cpp | 6 ++- src/Planner/PlannerJoins.cpp | 40 +++++++++++++++++-- ...479_analyzer_join_with_constants.reference | 15 +++++++ .../02479_analyzer_join_with_constants.sql | 27 +++++++++++++ 4 files changed, 84 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02479_analyzer_join_with_constants.reference create mode 100644 tests/queries/0_stateless/02479_analyzer_join_with_constants.sql diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 205c6c5e740..2dc6e7fa678 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -227,7 +227,11 @@ QueryPlan buildQueryPlanForJoinNode(QueryTreeNodePtr join_tree_node, JoinClausesAndActions join_clauses_and_actions; JoinKind join_kind = join_node.getKind(); - auto join_constant = tryExtractConstantFromJoinNode(join_tree_node); + std::optional join_constant; + + if (join_node.getStrictness() == JoinStrictness::All) + join_constant = tryExtractConstantFromJoinNode(join_tree_node); + if (join_constant) { /** If there is JOIN with always true constant, we transform it to cross. diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index f62517eaaad..53b9cfc5d99 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -20,6 +20,7 @@ #include #include +#include #include #include #include @@ -76,6 +77,23 @@ void JoinClause::dump(WriteBuffer & buffer) const if (!right_filter_condition_nodes.empty()) buffer << " right_condition_nodes: " + dump_dag_nodes(right_filter_condition_nodes); + + if (!asof_conditions.empty()) + { + buffer << " asof_conditions: "; + size_t asof_conditions_size = asof_conditions.size(); + + for (size_t i = 0; i < asof_conditions_size; ++i) + { + const auto & asof_condition = asof_conditions[i]; + + buffer << "key_index: " << asof_condition.key_index; + buffer << "inequality: " << toString(asof_condition.asof_inequality); + + if (i + 1 != asof_conditions_size) + buffer << ','; + } + } } String JoinClause::dump() const @@ -249,9 +267,7 @@ void buildJoinClause(ActionsDAGPtr join_expression_dag, join_node); if (!expression_side_optional) - throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, - "JOIN {} with constants is not supported", - join_node.formatASTForErrorMessage()); + expression_side_optional = JoinTableSide::Right; auto expression_side = *expression_side_optional; join_clause.addCondition(expression_side, join_expressions_actions_node); @@ -277,6 +293,22 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & for (const auto & node : join_expression_actions_nodes) join_expression_dag_input_nodes.insert(&node); + auto * function_node = join_node.getJoinExpression()->as(); + if (!function_node) + throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, + "JOIN {} join expression expected function", + join_node.formatASTForErrorMessage()); + + /** It is possible to have constant value in JOIN ON section, that we need to ignore during DAG construction. + * If we do not ignore it, this function will be replaced by underlying constant. + * For example ASOF JOIN does not support JOIN with constants, and we should process it like ordinary JOIN. + * + * Example: SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 + * ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t1.value); + */ + auto constant_value = function_node->getConstantValueOrNull(); + function_node->performConstantFolding({}); + PlannerActionsVisitor join_expression_visitor(planner_context); auto join_expression_dag_node_raw_pointers = join_expression_visitor.visit(join_expression_actions, join_node.getJoinExpression()); if (join_expression_dag_node_raw_pointers.size() != 1) @@ -284,6 +316,8 @@ JoinClausesAndActions buildJoinClausesAndActions(const ColumnsWithTypeAndName & "JOIN {} ON clause contains multiple expressions", join_node.formatASTForErrorMessage()); + function_node->performConstantFolding(std::move(constant_value)); + const auto * join_expressions_actions_root_node = join_expression_dag_node_raw_pointers[0]; if (!join_expressions_actions_root_node->function) throw Exception(ErrorCodes::INVALID_JOIN_ON_EXPRESSION, diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference b/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference new file mode 100644 index 00000000000..2a428d5d927 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.reference @@ -0,0 +1,15 @@ +1 1 +-- +-- +-- +1 2 +-- +1 1 1 1 +-- +1 1 0 0 +-- +0 +0 +0 +0 +0 diff --git a/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql new file mode 100644 index 00000000000..0cc3ff3ab00 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_join_with_constants.sql @@ -0,0 +1,27 @@ +SET allow_experimental_analyzer = 1; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id AND 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 1 AS id) AS t2 ON t1.id = t2.id AND 0; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id) AS t1 INNER JOIN (SELECT 2 AS id) AS t2 ON t1.id = t2.id OR 1; + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 == 1 AND (t1.value >= t2.value); + +SELECT '--'; + +SELECT * FROM (SELECT 1 AS id, 1 AS value) AS t1 ASOF LEFT JOIN (SELECT 1 AS id, 1 AS value) AS t2 ON (t1.id = t2.id) AND 1 != 1 AND (t1.value >= t2.value); + +SELECT '--'; + +SELECT b.dt FROM (SELECT NULL > NULL AS pk, 1 AS dt FROM numbers(5)) AS a ASOF LEFT JOIN (SELECT NULL AS pk, 1 AS dt) AS b ON (a.pk = b.pk) AND 1 != 1 AND (a.dt >= b.dt); From a7f0a2255fed546d15302e3f06f81ade83ce892b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 12:50:56 +0100 Subject: [PATCH 374/526] Analyzer aggregation crash fix --- .../Passes/NormalizeCountVariantsPass.cpp | 19 ++++++++++++++----- ...02479_analyzer_aggregation_crash.reference | 2 ++ .../02479_analyzer_aggregation_crash.sql | 13 +++++++++++++ 3 files changed, 29 insertions(+), 5 deletions(-) create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql diff --git a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp index 8c92ecc3900..b91fc94e6cd 100644 --- a/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp +++ b/src/Analyzer/Passes/NormalizeCountVariantsPass.cpp @@ -33,18 +33,27 @@ public: if (function_node->getFunctionName() == "count" && !first_argument_constant_literal.isNull()) { + resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); } - else if (function_node->getFunctionName() == "sum" && first_argument_constant_literal.getType() == Field::Types::UInt64 && + else if (function_node->getFunctionName() == "sum" && + first_argument_constant_literal.getType() == Field::Types::UInt64 && first_argument_constant_literal.get() == 1) { - auto result_type = function_node->getResultType(); - AggregateFunctionProperties properties; - auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); - function_node->resolveAsAggregateFunction(std::move(aggregate_function), std::move(result_type)); + resolveAsCountAggregateFunction(*function_node); function_node->getArguments().getNodes().clear(); } } +private: + static inline void resolveAsCountAggregateFunction(FunctionNode & function_node) + { + auto function_result_type = function_node.getResultType(); + + AggregateFunctionProperties properties; + auto aggregate_function = AggregateFunctionFactory::instance().get("count", {}, {}, properties); + + function_node.resolveAsAggregateFunction(std::move(aggregate_function), std::move(function_result_type)); + } }; } diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference new file mode 100644 index 00000000000..73d811f75f3 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.reference @@ -0,0 +1,2 @@ +10 123456789 +10 123456789 diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql new file mode 100644 index 00000000000..c931a3ab634 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_crash.sql @@ -0,0 +1,13 @@ +SET allow_experimental_analyzer = 1; +SET compile_aggregate_expressions = 1; +SET min_count_to_compile_aggregate_expression = 0; + +DROP TABLE IF EXISTS lc_00906__fuzz_46; +CREATE TABLE lc_00906__fuzz_46 (`b` Int64) ENGINE = MergeTree ORDER BY b; +INSERT INTO lc_00906__fuzz_46 SELECT '0123456789' FROM numbers(10); + +SELECT count(3.4028234663852886e38), b FROM lc_00906__fuzz_46 GROUP BY b; + +SELECT count(1), b FROM lc_00906__fuzz_46 GROUP BY b; + +DROP TABLE lc_00906__fuzz_46; From 54db7c6520f72529f4624862ee8bdab719c64ed9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 11 Nov 2022 10:56:18 +0100 Subject: [PATCH 375/526] Enforce checking read output --- src/Common/ZooKeeper/ZooKeeperIO.cpp | 5 ++++- src/Common/ZooKeeper/ZooKeeperIO.h | 2 +- src/Compression/CompressedReadBuffer.h | 2 +- src/Compression/CompressedReadBufferFromFile.h | 2 +- src/Coordination/KeeperStateManager.cpp | 2 +- src/Core/PostgreSQLProtocol.h | 2 +- src/DataTypes/Serializations/SerializationIP.cpp | 4 ++-- src/IO/MySQLPacketPayloadReadBuffer.cpp | 2 +- src/IO/ReadBuffer.h | 9 +++------ src/Interpreters/TraceCollector.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexFullText.cpp | 3 +-- .../MergeTree/MergeTreeIndexGranuleBloomFilter.cpp | 2 +- 13 files changed, 19 insertions(+), 20 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperIO.cpp b/src/Common/ZooKeeper/ZooKeeperIO.cpp index c84a8624d78..3bfa5585d87 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.cpp +++ b/src/Common/ZooKeeper/ZooKeeperIO.cpp @@ -143,7 +143,10 @@ void read(std::string & s, ReadBuffer & in) throw Exception("Too large string size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); s.resize(size); - in.read(s.data(), size); + size_t read_bytes = in.read(s.data(), size); + if (read_bytes != static_cast(size)) + throw Exception( + Error::ZMARSHALLINGERROR, "Buffer size read from Zookeeper is not big enough. Expected {}. Got {}", size, read_bytes); } void read(ACL & acl, ReadBuffer & in) diff --git a/src/Common/ZooKeeper/ZooKeeperIO.h b/src/Common/ZooKeeper/ZooKeeperIO.h index ec77b46f3d9..2c5fdd5d8a3 100644 --- a/src/Common/ZooKeeper/ZooKeeperIO.h +++ b/src/Common/ZooKeeper/ZooKeeperIO.h @@ -67,7 +67,7 @@ void read(std::array & s, ReadBuffer & in) read(size, in); if (size != N) throw Exception("Unexpected array size while reading from ZooKeeper", Error::ZMARSHALLINGERROR); - in.read(s.data(), N); + in.readStrict(s.data(), N); } template diff --git a/src/Compression/CompressedReadBuffer.h b/src/Compression/CompressedReadBuffer.h index 4148f4fe4d4..1d338303c84 100644 --- a/src/Compression/CompressedReadBuffer.h +++ b/src/Compression/CompressedReadBuffer.h @@ -21,7 +21,7 @@ public: { } - size_t readBig(char * to, size_t n) override; + [[nodiscard]] size_t readBig(char * to, size_t n) override; /// The compressed size of the current block. size_t getSizeCompressed() const diff --git a/src/Compression/CompressedReadBufferFromFile.h b/src/Compression/CompressedReadBufferFromFile.h index 719959b96f4..d307503fb99 100644 --- a/src/Compression/CompressedReadBufferFromFile.h +++ b/src/Compression/CompressedReadBufferFromFile.h @@ -53,7 +53,7 @@ public: /// we store this offset inside nextimpl_working_buffer_offset. void seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) override; - size_t readBig(char * to, size_t n) override; + [[nodiscard]] size_t readBig(char * to, size_t n) override; void setProfileCallback(const ReadBufferFromFileBase::ProfileCallback & profile_callback_, clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE) { diff --git a/src/Coordination/KeeperStateManager.cpp b/src/Coordination/KeeperStateManager.cpp index 9b6aab5533e..9a3b423d4ac 100644 --- a/src/Coordination/KeeperStateManager.cpp +++ b/src/Coordination/KeeperStateManager.cpp @@ -349,7 +349,7 @@ nuraft::ptr KeeperStateManager::read_state() auto buffer_size = content_size - sizeof read_checksum - sizeof version; auto state_buf = nuraft::buffer::alloc(buffer_size); - read_buf.read(reinterpret_cast(state_buf->data_begin()), buffer_size); + read_buf.readStrict(reinterpret_cast(state_buf->data_begin()), buffer_size); SipHash hash; hash.update(version); diff --git a/src/Core/PostgreSQLProtocol.h b/src/Core/PostgreSQLProtocol.h index 994494fc92f..a20151ec167 100644 --- a/src/Core/PostgreSQLProtocol.h +++ b/src/Core/PostgreSQLProtocol.h @@ -175,7 +175,7 @@ public: FrontMessageType receiveMessageType() { char type = 0; - in->read(type); + in->readStrict(type); return static_cast(type); } diff --git a/src/DataTypes/Serializations/SerializationIP.cpp b/src/DataTypes/Serializations/SerializationIP.cpp index ed0e9d54415..2aea08f9b62 100644 --- a/src/DataTypes/Serializations/SerializationIP.cpp +++ b/src/DataTypes/Serializations/SerializationIP.cpp @@ -47,7 +47,7 @@ void SerializationIPv4::deserializeText(IColumn & column, ReadBuffer & istr, con } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; - istr.read(buffer, sizeof(buffer) - 1); + istr.readStrict(buffer, sizeof(buffer) - 1); UInt32 ipv4_value = 0; bool parse_result = parseIPv4(buffer, reinterpret_cast(&ipv4_value)); @@ -90,7 +90,7 @@ void SerializationIPv6::deserializeText(IColumn & column, ReadBuffer & istr, con } char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'}; - istr.read(buffer, sizeof(buffer) - 1); + [[maybe_unused]] size_t read_bytes = istr.read(buffer, sizeof(buffer) - 1); std::string ipv6_value(IPV6_BINARY_LENGTH, '\0'); diff --git a/src/IO/MySQLPacketPayloadReadBuffer.cpp b/src/IO/MySQLPacketPayloadReadBuffer.cpp index 9ca7845b2ae..ab58624d0fa 100644 --- a/src/IO/MySQLPacketPayloadReadBuffer.cpp +++ b/src/IO/MySQLPacketPayloadReadBuffer.cpp @@ -30,7 +30,7 @@ bool MySQLPacketPayloadReadBuffer::nextImpl() "Received packet with payload larger than max_packet_size: {}", payload_length); size_t packet_sequence_id = 0; - in.read(reinterpret_cast(packet_sequence_id)); + in.readStrict(reinterpret_cast(packet_sequence_id)); if (packet_sequence_id != sequence_id) throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT, "Received packet with wrong sequence-id: {}. Expected: {}.", packet_sequence_id, static_cast(sequence_id)); diff --git a/src/IO/ReadBuffer.h b/src/IO/ReadBuffer.h index 8d697710081..182eb0b7105 100644 --- a/src/IO/ReadBuffer.h +++ b/src/IO/ReadBuffer.h @@ -149,7 +149,7 @@ public: } /// Reads a single byte. - bool ALWAYS_INLINE read(char & c) + [[nodiscard]] bool ALWAYS_INLINE read(char & c) { if (peek(c)) { @@ -168,7 +168,7 @@ public: } /** Reads as many as there are, no more than n bytes. */ - size_t read(char * to, size_t n) + [[nodiscard]] size_t read(char * to, size_t n) { size_t bytes_copied = 0; @@ -197,10 +197,7 @@ public: * By default - the same as read. * Don't use for small reads. */ - virtual size_t readBig(char * to, size_t n) - { - return read(to, n); - } + [[nodiscard]] virtual size_t readBig(char * to, size_t n) { return read(to, n); } /** Do something to allow faster subsequent call to 'nextImpl' if possible. * It's used for asynchronous readers with double-buffering. diff --git a/src/Interpreters/TraceCollector.cpp b/src/Interpreters/TraceCollector.cpp index d277763a141..41a7fcf8389 100644 --- a/src/Interpreters/TraceCollector.cpp +++ b/src/Interpreters/TraceCollector.cpp @@ -72,7 +72,7 @@ void TraceCollector::run() UInt8 query_id_size = 0; readBinary(query_id_size, in); query_id.resize(query_id_size); - in.read(query_id.data(), query_id_size); + in.readStrict(query_id.data(), query_id_size); UInt8 trace_size = 0; readIntBinary(trace_size, in); diff --git a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp index 052834358bb..743bb504dbd 100644 --- a/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexAnnoy.cpp @@ -44,7 +44,7 @@ void AnnoyIndex::deserialize(ReadBuffer& istr) readIntBinary(Base::_seed, istr); readVectorBinary(Base::_roots, istr); Base::_nodes = realloc(Base::_nodes, Base::_s * Base::_n_nodes); - istr.read(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); + istr.readStrict(reinterpret_cast(Base::_nodes), Base::_s * Base::_n_nodes); Base::_fd = 0; // set flags diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index b96d40f5759..03335d9ca98 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -59,8 +59,7 @@ void MergeTreeIndexGranuleFullText::deserializeBinary(ReadBuffer & istr, MergeTr for (auto & bloom_filter : bloom_filters) { - istr.read(reinterpret_cast( - bloom_filter.getFilter().data()), params.filter_size); + istr.readStrict(reinterpret_cast(bloom_filter.getFilter().data()), params.filter_size); } has_elems = true; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp index 7efaf0866db..deed9b3f071 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranuleBloomFilter.cpp @@ -96,7 +96,7 @@ void MergeTreeIndexGranuleBloomFilter::deserializeBinary(ReadBuffer & istr, Merg static size_t atom_size = 8; size_t bytes_size = (bits_per_row * total_rows + atom_size - 1) / atom_size; filter = std::make_shared(bytes_size, hash_functions, 0); - istr.read(reinterpret_cast(filter->getFilter().data()), bytes_size); + istr.readStrict(reinterpret_cast(filter->getFilter().data()), bytes_size); } } From 5d53ec9da88b0095fe693f441b85186ec792bbf2 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 11 Nov 2022 11:07:05 +0100 Subject: [PATCH 376/526] Cleaned and replace some Strings by using chars directly --- src/Functions/FunctionsJSON.cpp | 28 ++++++++++++++++------------ 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index b4b15a25047..f8d5f357549 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -893,9 +893,17 @@ struct JSONExtractTree // For the non low cardinality case of FixedString, the padding is done in the FixedString Column implementation. // In order to avoid having to pass the data to a FixedString Column and read it back (which would slow down the execution) // the data is padded here and written directly to the Low Cardinality Column - auto padded_str = str.data() + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); + if (str.size() == fixed_length) + { + assert_cast(dest).insertData(str.data(), str.size()); + } + else + { + String padded_str(str); + padded_str.resize(fixed_length, '\0'); - assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + } return true; } @@ -1230,8 +1238,7 @@ struct JSONExtractTree auto fixed_length = typeid_cast(dictionary_type.get())->getN(); return std::make_unique(fixed_length); } - auto impl = build(function_name, dictionary_type); - return impl; + return build(function_name, dictionary_type); } case TypeIndex::Decimal256: return std::make_unique>(type); case TypeIndex::Decimal128: return std::make_unique>(type); @@ -1387,12 +1394,9 @@ public: { ColumnString::Chars chars; WriteBufferFromVector buf(chars, AppendModeTag()); - chars.push_back(0); traverse(element, buf); buf.finalize(); - std::string str = reinterpret_cast(chars.data()); - chars.push_back(0); - assert_cast(dest).insertData(str.data(), str.size()); + assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); } else { @@ -1423,7 +1427,6 @@ public: chars.push_back(0); std::string str = reinterpret_cast(chars.data()); - auto padded_str = str + std::string(col_str.getN() - std::min(col_str.getN(), str.length()), '\0'); col_str.insertData(str.data(), str.size()); @@ -1441,10 +1444,11 @@ public: traverse(element, buf); buf.finalize(); chars.push_back(0); - std::string str = reinterpret_cast(chars.data()); - auto padded_str = str + std::string(fixed_length - std::min(fixed_length, str.length()), '\0'); - assert_cast(dest).insertData(padded_str.data(), padded_str.size()); + for (unsigned long i = 0; i < fixed_length - chars.size(); ++i) + chars.push_back(0); + + assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); return true; } From d4a0b76abf9e7b1e2ff74e9e563e4ea1b6a5b6ee Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 11 Nov 2022 12:08:43 +0100 Subject: [PATCH 377/526] Fix compilation --- src/IO/tests/gtest_file_encryption.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_file_encryption.cpp b/src/IO/tests/gtest_file_encryption.cpp index 5353faa6086..6a090ff0810 100644 --- a/src/IO/tests/gtest_file_encryption.cpp +++ b/src/IO/tests/gtest_file_encryption.cpp @@ -251,7 +251,7 @@ TEST(FileEncryptionPositionUpdateTest, Decryption) rb.seek(0, SEEK_SET); ASSERT_EQ(rb.getPosition(), 0); res.resize(5); - rb.read(res.data(), res.size()); + ASSERT_EQ(rb.read(res.data(), res.size()), 5); ASSERT_EQ(res, data.substr(0, 5)); res.clear(); From 547cd7142397cd727a3289e1d47bfae298cea3bf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 11 Nov 2022 12:47:14 +0100 Subject: [PATCH 378/526] Address some sanitizer issues --- src/AggregateFunctions/AggregateFunctionNull.h | 4 +++- .../AggregateFunctionSequenceNextNode.h | 10 ++++++++++ tests/queries/0_stateless/02477_invalid_reads.sql | 10 ++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index deed06b8bf2..ebc91ea5100 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -153,7 +153,9 @@ public: void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - bool flag = true; + /// When deserializing a bool it might trigger UBSAN if the input is not 0 or 1, so it's better to use a Int8 + static_assert(sizeof(bool) == sizeof(Int8)); + Int8 flag = true; if constexpr (serialize_flag) readBinary(flag, buf); if (flag) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 87ccd0d8a19..90caaee4d94 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -29,6 +29,11 @@ namespace DB { struct Settings; +namespace ErrorCodes +{ + extern const int TOO_LARGE_ARRAY_SIZE; +} + enum class SequenceDirection { Forward, @@ -43,6 +48,9 @@ enum SequenceBase LastMatch, }; +/// This is for security +static const UInt64 max_node_size_deserialize = 0xFFFFFF; + /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template struct NodeBase @@ -78,6 +86,8 @@ struct NodeBase { UInt64 size; readVarUInt(size, buf); + if unlikely (size > max_node_size_deserialize) + throw Exception("Too large node state size", ErrorCodes::TOO_LARGE_ARRAY_SIZE); Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; diff --git a/tests/queries/0_stateless/02477_invalid_reads.sql b/tests/queries/0_stateless/02477_invalid_reads.sql index 1e488ee5627..37fa8ab098e 100644 --- a/tests/queries/0_stateless/02477_invalid_reads.sql +++ b/tests/queries/0_stateless/02477_invalid_reads.sql @@ -36,6 +36,16 @@ SELECT finalizeAggregation(CAST(unhex('FFFFFFF014181056F38010000000000000001FFFF 'AggregateFunction(sequenceNextNode(''forward'', ''head''), DateTime, Nullable(String), UInt8, Nullable(UInt8))')) SETTINGS allow_experimental_funnel_functions=1; -- { serverError 33 } +-- Fuzzer (ALL) +SELECT finalizeAggregation(CAST(unhex('FFFFFFF014181056F38010000000000000001FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF014181056F38010000000000000001FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF'), + 'AggregateFunction(sequenceNextNode(\'forward\', \'head\'), DateTime, Nullable(String), UInt8, Nullable(UInt8))')) + SETTINGS allow_experimental_funnel_functions = 1; -- { serverError 128 } + +-- Fuzzer 2 (UBSAN) +SELECT finalizeAggregation(CAST(unhex('FFFFFFF014181056F38010000000000000001FFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFFF'), + 'AggregateFunction(sequenceNextNode(\'forward\', \'head\'), DateTime, Nullable(String), UInt8, Nullable(UInt8))')) + SETTINGS allow_experimental_funnel_functions = 1; -- { serverError 33 } + -- uniqUpTo SELECT finalizeAggregation(CAST(unhex('04128345AA2BC97190'), 'AggregateFunction(uniqUpTo(10), String)')); -- { serverError 33 } From 0c30325aa44cd612d989dd185357943c6ccdec96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 11 Nov 2022 12:58:19 +0100 Subject: [PATCH 379/526] Went too far --- src/DataTypes/Serializations/SerializationIP.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/DataTypes/Serializations/SerializationIP.cpp b/src/DataTypes/Serializations/SerializationIP.cpp index 2aea08f9b62..c89c2d7c8ac 100644 --- a/src/DataTypes/Serializations/SerializationIP.cpp +++ b/src/DataTypes/Serializations/SerializationIP.cpp @@ -47,7 +47,7 @@ void SerializationIPv4::deserializeText(IColumn & column, ReadBuffer & istr, con } char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'}; - istr.readStrict(buffer, sizeof(buffer) - 1); + [[maybe_unused]] size_t read_bytes = istr.read(buffer, sizeof(buffer) - 1); UInt32 ipv4_value = 0; bool parse_result = parseIPv4(buffer, reinterpret_cast(&ipv4_value)); From f24991fa56f93e1ff55e37069e0aa58ab9380303 Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Fri, 11 Nov 2022 13:02:30 +0100 Subject: [PATCH 380/526] Remove pushing a last 0 in FixedStrings --- src/Functions/FunctionsJSON.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index f8d5f357549..0973c1f3e39 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1424,7 +1424,6 @@ public: if (chars.size() > col_str.getN()) return false; - chars.push_back(0); std::string str = reinterpret_cast(chars.data()); col_str.insertData(str.data(), str.size()); @@ -1443,7 +1442,6 @@ public: WriteBufferFromVector buf(chars, AppendModeTag()); traverse(element, buf); buf.finalize(); - chars.push_back(0); for (unsigned long i = 0; i < fixed_length - chars.size(); ++i) chars.push_back(0); From 444ab24792ff9ad328d166b3d2f40270e591d478 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 11 Nov 2022 20:12:14 +0800 Subject: [PATCH 381/526] Fix gtest --- src/Interpreters/tests/gtest_merge_tree_set_index.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/tests/gtest_merge_tree_set_index.cpp b/src/Interpreters/tests/gtest_merge_tree_set_index.cpp index 4bdbc9c9fc5..bae29fb4f26 100644 --- a/src/Interpreters/tests/gtest_merge_tree_set_index.cpp +++ b/src/Interpreters/tests/gtest_merge_tree_set_index.cpp @@ -82,11 +82,11 @@ TEST(MergeTreeSetIndex, checkInRangeTuple) std::vector ranges = {Range(1), Range("a", true, "c", true)}; ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, true) << "Range(1), Range('a', true, 'c', true)"; - ranges = {Range(1, false, 3, false), Range()}; - ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, false) << "Range(1, false, 3, false), Range()"; + ranges = {Range(1, false, 3, false), Range::createWholeUniverseWithoutNull()}; + ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, false) << "Range(1, false, 3, false), Range::createWholeUniverseWithoutNull()"; - ranges = {Range(2, false, 5, false), Range()}; - ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, true) << "Range(2, false, 5, false), Range()"; + ranges = {Range(2, false, 5, false), Range::createWholeUniverseWithoutNull()}; + ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, true) << "Range(2, false, 5, false), Range::createWholeUniverseWithoutNull()"; ranges = {Range(3), Range::createLeftBounded("a", true)}; ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, true) << "Range(3), Range::createLeftBounded('a', true)"; @@ -106,7 +106,7 @@ TEST(MergeTreeSetIndex, checkInRangeTuple) ranges = {Range(1), Range("c")}; ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, false) << "Range(1), Range('c')"; - ranges = {Range(2, true, 3, true), Range()}; + ranges = {Range(2, true, 3, true), Range::createWholeUniverseWithoutNull()}; ASSERT_EQ(set->checkInRange(ranges, types).can_be_true, true) << "Range(2, true, 3, true), Range('x', true, 'z', true)"; ranges = {Range(2), Range("a", true, "z", true)}; From e064fb38cc65a115baa140b1f5e7c87aacfeee7b Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 11 Nov 2022 12:34:23 +0000 Subject: [PATCH 382/526] edits to github repo analysis --- .../example-datasets/github.md | 327 ++++++++++-------- .../images/superset-authors-matrix.png | Bin 0 -> 323772 bytes .../images/superset-authors-matrix_v3.png | Bin 0 -> 324610 bytes 3 files changed, 176 insertions(+), 151 deletions(-) create mode 100644 docs/en/getting-started/example-datasets/images/superset-authors-matrix.png create mode 100644 docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index fbc65d36ba9..fdc0212bfe5 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -10,8 +10,8 @@ This dataset contains all of the commits and changes for the ClickHouse reposito The generated data provides a `tsv` file for each of the following tables: -- `commits` - commits with statistics; -- `file_changes` - files changed in every commit with the info about the change and statistics; +- `commits` - commits with statistics. +- `file_changes` - files changed in every commit with the info about the change and statistics. - `line_changes` - every changed line in every changed file in every commit with full info about the line and the information about the previous change of this line. As of November 8th, 2022, each TSV is approximately the following size and number of rows: @@ -48,7 +48,6 @@ As of November 8th, 2022, each TSV is approximately the following size and numbe - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - [Files sorted by average code age](#files-sorted-by-average-code-age) - [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) - - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) - [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) @@ -65,7 +64,7 @@ cd ClickHouse clickhouse git-import --skip-paths 'generated\.cpp|^(contrib|docs?|website|libs/(libcityhash|liblz4|libdivide|libvectorclass|libdouble-conversion|libcpuid|libzstd|libfarmhash|libmetrohash|libpoco|libwidechar_width))/' --skip-commits-with-messages '^Merge branch ' ``` -This will take around 3 minutes (as of November 8th 2022) to complete for the ClickHouse repository. +This will take around 3 minutes (as of November 8th 2022 on a MacBook Pro 2021) to complete for the ClickHouse repository. A full list of available options can be obtained from the tools native help. @@ -103,13 +102,13 @@ CREATE TABLE git.commits Generated files for the following repositories can be found below: - ClickHouse (Nov 8th 2022) - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz - 2.5 MB - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz - 4.5MB - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz - 127.4 MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz - 2.5 MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz - 4.5MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz - 127.4 MB - Linux (Nov 8th 2022) - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/commits.tsv.xz - 2.5 MB - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/file_changes.tsv.xz - 4.5MB - - https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/linux/line_changes.tsv.xz - 127.4 MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/linux/commits.tsv.xz - 44 MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/linux/file_changes.tsv.xz - 467MB + - https://datasets-documentation.s3.amazonaws.com/github/commits/linux/line_changes.tsv.xz - 1.1G To insert this data, prepare the database by executing the following queries: @@ -212,7 +211,7 @@ Insert the data using `INSERT INTO SELECT` and the [s3 function](https://clickho ```sql INSERT INTO git.commits SELECT * -FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz', 'TSV', 'hash String,author LowCardinality(String), time DateTime, message String, files_added UInt32, files_deleted UInt32, files_renamed UInt32, files_modified UInt32, lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32') +FROM s3('https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/commits.tsv.xz', 'TSV', 'hash String,author LowCardinality(String), time DateTime, message String, files_added UInt32, files_deleted UInt32, files_renamed UInt32, files_modified UInt32, lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32') 0 rows in set. Elapsed: 1.826 sec. Processed 62.78 thousand rows, 8.50 MB (34.39 thousand rows/s., 4.66 MB/s.) ``` @@ -221,7 +220,7 @@ FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commit ```sql INSERT INTO git.file_changes SELECT * -FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz', 'TSV', 'change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6), path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32, commit_hash String, author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') +FROM s3('https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/file_changes.tsv.xz', 'TSV', 'change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6), path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), lines_added UInt32, lines_deleted UInt32, hunks_added UInt32, hunks_removed UInt32, hunks_changed UInt32, commit_hash String, author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') 0 rows in set. Elapsed: 2.688 sec. Processed 266.05 thousand rows, 48.30 MB (98.97 thousand rows/s., 17.97 MB/s.) ``` @@ -230,51 +229,14 @@ FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commit ```sql INSERT INTO git.line_changes SELECT * -FROM s3('https://datasets-documentation.s3.eu-west-3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz', 'TSV', ' sign Int8, line_number_old UInt32, line_number_new UInt32, hunk_num UInt32, hunk_start_line_number_old UInt32, hunk_start_line_number_new UInt32, hunk_lines_added UInt32,\n hunk_lines_deleted UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, line_type Enum(\'Empty\' = 0, \'Comment\' = 1, \'Punct\' = 2, \'Code\' = 3), prev_commit_hash String, prev_author LowCardinality(String), prev_time DateTime, file_change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6),\n path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), file_lines_added UInt32, file_lines_deleted UInt32, file_hunks_added UInt32, file_hunks_removed UInt32, file_hunks_changed UInt32, commit_hash String,\n author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') +FROM s3('https://datasets-documentation.s3.amazonaws.com/github/commits/clickhouse/line_changes.tsv.xz', 'TSV', ' sign Int8, line_number_old UInt32, line_number_new UInt32, hunk_num UInt32, hunk_start_line_number_old UInt32, hunk_start_line_number_new UInt32, hunk_lines_added UInt32,\n hunk_lines_deleted UInt32, hunk_context LowCardinality(String), line LowCardinality(String), indent UInt8, line_type Enum(\'Empty\' = 0, \'Comment\' = 1, \'Punct\' = 2, \'Code\' = 3), prev_commit_hash String, prev_author LowCardinality(String), prev_time DateTime, file_change_type Enum(\'Add\' = 1, \'Delete\' = 2, \'Modify\' = 3, \'Rename\' = 4, \'Copy\' = 5, \'Type\' = 6),\n path LowCardinality(String), old_path LowCardinality(String), file_extension LowCardinality(String), file_lines_added UInt32, file_lines_deleted UInt32, file_hunks_added UInt32, file_hunks_removed UInt32, file_hunks_changed UInt32, commit_hash String,\n author LowCardinality(String), time DateTime, commit_message String, commit_files_added UInt32, commit_files_deleted UInt32, commit_files_renamed UInt32, commit_files_modified UInt32, commit_lines_added UInt32, commit_lines_deleted UInt32, commit_hunks_added UInt32, commit_hunks_removed UInt32, commit_hunks_changed UInt32') 0 rows in set. Elapsed: 50.535 sec. Processed 7.54 million rows, 2.09 GB (149.11 thousand rows/s., 41.40 MB/s.) ``` # Queries -The tool suggests several queries via its help output. We have answered these in addition to some additional supplementary questions of interest: - -- [ClickHouse GitHub data](#clickhouse-github-data) -- [Table of Contents](#table-of-contents) -- [Generating the data](#generating-the-data) -- [Downloading and inserting the data](#downloading-and-inserting-the-data) -- [Queries](#queries) - - [History of a single file](#history-of-a-single-file) - - [Find the current active files](#find-the-current-active-files) - - [List files with most modifications](#list-files-with-most-modifications) - - [What day of the week do commits usually occur?](#what-day-of-the-week-do-commits-usually-occur) - - [History of subdirectory/file - number of lines, commits and contributors over time](#history-of-subdirectoryfile---number-of-lines-commits-and-contributors-over-time) - - [List files with maximum number of authors](#list-files-with-maximum-number-of-authors) - - [Oldest lines of code in the repository](#oldest-lines-of-code-in-the-repository) - - [Files with longest history](#files-with-longest-history) - - [Distribution of contributors with respect to docs and code over the month](#distribution-of-contributors-with-respect-to-docs-and-code-over-the-month) - - [Authors with the most diverse impact](#authors-with-the-most-diverse-impact) - - [Favorite files for an author](#favorite-files-for-an-author) - - [Largest files with lowest number of authors](#largest-files-with-lowest-number-of-authors) - - [Commits and lines of code distribution by time; by weekday, by author; for specific subdirectories](#commits-and-lines-of-code-distribution-by-time-by-weekday-by-author-for-specific-subdirectories) - - [Matrix of authors that shows what authors tends to rewrite another authors code](#matrix-of-authors-that-shows-what-authors-tends-to-rewrite-another-authors-code) - - [Who is the highest percentage contributor per day of week?](#who-is-the-highest-percentage-contributor-per-day-of-week) - - [Distribution of code age across repository](#distribution-of-code-age-across-repository) - - [What percentage of code for an author has been removed by other authors?](#what-percentage-of-code-for-an-author-has-been-removed-by-other-authors) - - [List files that were rewritten most number of times?](#list-files-that-were-rewritten-most-number-of-times) - - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - - [Files sorted by average code age](#files-sorted-by-average-code-age) - - [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) - - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) - - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) - - [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) - - [Most consecutive days of commits by an author](#most-consecutive-days-of-commits-by-an-author) - - [Line by line commit history of a file](#line-by-line-commit-history-of-a-file) -- [Unsolved Questions](#unsolved-questions) - - [Git blame](#git-blame) - -These queries are of approximately increasing complexity vs. the tool's arbitrary order. +The tool suggests several queries via its help output. We have answered these in addition to some additional supplementary questions of interest. These queries are of approximately increasing complexity vs. the tool's arbitrary order. ## History of a single file @@ -761,7 +723,7 @@ FROM ( SELECT day, - countIf(file_extension IN ('h', 'cpp', 'sql', 'md')) AS code, + countIf(file_extension IN ('h', 'cpp', 'sql')) AS code, countIf(file_extension = 'md') AS docs, docs / (code + docs) AS docs_ratio FROM git.line_changes @@ -769,41 +731,41 @@ FROM GROUP BY dayOfMonth(time) AS day ) -┌─day─┬─bar──────────────────────────────────────────────────────────┐ -│ 1 │ ██████████████████████████████████▎ │ -│ 2 │ ███████████████████████▏ │ -│ 3 │ ███████████████████████████████▋ │ -│ 4 │ ████████████▊ │ -│ 5 │ ████████████████████▊ │ -│ 6 │ ███████▊ │ -│ 7 │ ███▋ │ -│ 8 │ ████████▍ │ -│ 9 │ ██████████████ │ -│ 10 │ ████████████████▋ │ -│ 11 │ █████████████▏ │ -│ 12 │ ██████████████████████████████████▌ │ -│ 13 │ ████████████████████████████▌ │ -│ 14 │ ██████▋ │ -│ 15 │ ████████████████████████████████████████▎ │ -│ 16 │ ██████████▏ │ -│ 17 │ █████████████████████████████████████▏ │ -│ 18 │ ████████████████████████████████▍ │ -│ 19 │ ██████████▊ │ -│ 20 │ ████████████████████████████████▋ │ -│ 21 │ █████ │ -│ 22 │ ███████████████████████▏ │ -│ 23 │ ██████████████████████████▋ │ -│ 24 │ ███████▌ │ -│ 25 │ █████████████████████████████████▏ │ -│ 26 │ ███████████ │ -│ 27 │ ███████████████████████████████████████████████████████████▎ │ -│ 28 │ █████████████████████████████████████████████████▌ │ -│ 29 │ ███▌ │ -│ 30 │ ██████████████████████████████████████▋ │ -│ 31 │ ████████████████████████████████▏ │ -└─────┴──────────────────────────────────────────────────────────────┘ +┌─day─┬─bar─────────────────────────────────────────────────────────────┐ +│ 1 │ ███████████████████████████████████▍ │ +│ 2 │ ███████████████████████▋ │ +│ 3 │ ████████████████████████████████▋ │ +│ 4 │ █████████████ │ +│ 5 │ █████████████████████▎ │ +│ 6 │ ████████ │ +│ 7 │ ███▋ │ +│ 8 │ ████████▌ │ +│ 9 │ ██████████████▎ │ +│ 10 │ █████████████████▏ │ +│ 11 │ █████████████▎ │ +│ 12 │ ███████████████████████████████████▋ │ +│ 13 │ █████████████████████████████▎ │ +│ 14 │ ██████▋ │ +│ 15 │ █████████████████████████████████████████▊ │ +│ 16 │ ██████████▎ │ +│ 17 │ ██████████████████████████████████████▋ │ +│ 18 │ █████████████████████████████████▌ │ +│ 19 │ ███████████ │ +│ 20 │ █████████████████████████████████▊ │ +│ 21 │ █████ │ +│ 22 │ ███████████████████████▋ │ +│ 23 │ ███████████████████████████▌ │ +│ 24 │ ███████▌ │ +│ 25 │ ██████████████████████████████████▎ │ +│ 26 │ ███████████▏ │ +│ 27 │ ███████████████████████████████████████████████████████████████ │ +│ 28 │ ████████████████████████████████████████████████████▏ │ +│ 29 │ ███▌ │ +│ 30 │ ████████████████████████████████████████▎ │ +│ 31 │ █████████████████████████████████▏ │ +└─────┴─────────────────────────────────────────────────────────────────┘ -31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.56 million rows/s., 949.58 MB/s.) +31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.71 million rows/s., 950.40 MB/s.) ``` Maybe a little more near the end of the month, but overall we keep a good even distribution. @@ -838,6 +800,49 @@ LIMIT 10 10 rows in set. Elapsed: 0.041 sec. Processed 266.05 thousand rows, 4.92 MB (6.56 million rows/s., 121.21 MB/s.) ``` +Let's see who has the most diverse commits in their recent work. Rather than limit by date, we'll restrict to an author's last N commits (in this case, we've used 3 but feel free to modify): + +```sql +SELECT + author, + sum(num_files_commit) AS num_files +FROM +( + SELECT + author, + commit_hash, + uniq(path) AS num_files_commit, + max(time) AS commit_time + FROM git.file_changes + WHERE (change_type IN ('Add', 'Modify')) AND (file_extension IN ('h', 'cpp', 'sql')) + GROUP BY + author, + commit_hash + ORDER BY + author ASC, + commit_time DESC + LIMIT 3 BY author +) +GROUP BY author +ORDER BY num_files DESC +LIMIT 10 + +┌─author───────────────┬─num_files─┐ +│ Mikhail │ 782 │ +│ Li Yin │ 553 │ +│ Roman Peshkurov │ 119 │ +│ Vladimir Smirnov │ 88 │ +│ f1yegor │ 65 │ +│ maiha │ 54 │ +│ Vitaliy Lyudvichenko │ 53 │ +│ Pradeep Chhetri │ 40 │ +│ Orivej Desh │ 38 │ +│ liyang │ 36 │ +└──────────────────────┴───────────┘ + +10 rows in set. Elapsed: 0.106 sec. Processed 266.05 thousand rows, 21.04 MB (2.52 million rows/s., 198.93 MB/s.) +``` + ## Favorite files for an author Here we select our founder [Alexey Milovidov](https://github.com/alexey-milovidov) and limit our analysis to current files. @@ -870,23 +875,23 @@ SELECT path, count() AS c FROM git.file_changes -WHERE (author = 'alexey-milovidov') AND (path IN (current_files)) +WHERE (author = 'Alexey Milovidov') AND (path IN (current_files)) GROUP BY path ORDER BY c DESC LIMIT 10 -┌─path────────────────────────────────────────────┬───c─┐ -│ CHANGELOG.md │ 174 │ -│ CMakeLists.txt │ 22 │ -│ src/Common/HashTable/HashTable.h │ 8 │ -│ .github/PULL_REQUEST_TEMPLATE.md │ 8 │ -│ src/Core/Settings.h │ 8 │ -│ src/Storages/StorageReplicatedMergeTree.cpp │ 7 │ -│ README.md │ 7 │ -│ docker/test/fuzzer/run-fuzzer.sh │ 7 │ -│ programs/install/Install.cpp │ 7 │ -│ src/Dictionaries/ExecutableDictionarySource.cpp │ 6 │ -└─────────────────────────────────────────────────┴─────┘ +┌─path────────────────────────────────────────┬───c─┐ +│ CMakeLists.txt │ 165 │ +│ CHANGELOG.md │ 126 │ +│ programs/server/Server.cpp │ 73 │ +│ src/Storages/MergeTree/MergeTreeData.cpp │ 71 │ +│ src/Storages/StorageReplicatedMergeTree.cpp │ 68 │ +│ src/Core/Settings.h │ 65 │ +│ programs/client/Client.cpp │ 57 │ +│ programs/server/play.html │ 48 │ +│ .gitmodules │ 47 │ +│ programs/install/Install.cpp │ 37 │ +└─────────────────────────────────────────────┴─────┘ 10 rows in set. Elapsed: 0.106 sec. Processed 798.15 thousand rows, 13.97 MB (7.51 million rows/s., 131.41 MB/s.) ``` @@ -899,23 +904,23 @@ SELECT base, count() AS c FROM git.file_changes -WHERE (author = 'alexey-milovidov') AND (file_extension IN ('h', 'cpp', 'sql')) +WHERE (author = 'Alexey Milovidov') AND (file_extension IN ('h', 'cpp', 'sql')) GROUP BY basename(path) AS base ORDER BY c DESC LIMIT 10 -┌─base───────────────────────────┬──c─┐ -│ StorageReplicatedMergeTree.cpp │ 22 │ -│ Settings.h │ 22 │ -│ InterpreterSelectQuery.cpp │ 19 │ -│ MergeTreeData.cpp │ 18 │ -│ Client.cpp │ 17 │ -│ Context.cpp │ 17 │ -│ Server.cpp │ 12 │ -│ ExecutableDictionarySource.cpp │ 12 │ -│ ExpressionAnalyzer.cpp │ 12 │ -│ PODArray.h │ 12 │ -└────────────────────────────────┴────┘ +┌─base───────────────────────────┬───c─┐ +│ StorageReplicatedMergeTree.cpp │ 393 │ +│ InterpreterSelectQuery.cpp │ 299 │ +│ Aggregator.cpp │ 297 │ +│ Client.cpp │ 280 │ +│ MergeTreeData.cpp │ 274 │ +│ Server.cpp │ 264 │ +│ ExpressionAnalyzer.cpp │ 259 │ +│ StorageMergeTree.cpp │ 239 │ +│ Settings.h │ 225 │ +│ TCPHandler.cpp │ 205 │ +└────────────────────────────────┴─────┘ 10 rows in set. Elapsed: 0.032 sec. Processed 266.05 thousand rows, 5.68 MB (8.22 million rows/s., 175.50 MB/s.) ``` @@ -1210,8 +1215,8 @@ The `sign = -1` indicates a code deletion. We exclude punctuation and the insert ```sql SELECT - prev_author, - author, + prev_author || '(a)' as add_author, + author || '(d)' as delete_author, count() AS c FROM git.line_changes WHERE (sign = -1) AND (file_extension IN ('h', 'cpp')) AND (line_type NOT IN ('Punct', 'Empty')) AND (author != prev_author) AND (prev_author != '') @@ -1220,7 +1225,7 @@ GROUP BY author ORDER BY c DESC LIMIT 1 BY prev_author -LIMIT 20 +LIMIT 100 ┌─prev_author──────────┬─author───────────┬─────c─┐ │ Ivan │ Alexey Milovidov │ 18554 │ @@ -1248,6 +1253,16 @@ LIMIT 20 20 rows in set. Elapsed: 0.098 sec. Processed 7.54 million rows, 42.16 MB (76.67 million rows/s., 428.99 MB/s.) ``` +A Sankey chart (SuperSet) allows this to be visualized nicely. Note we increase our `LIMIT BY` to 3, to get the top 3 code removers for each author, to improve the variety in the visual. + + +![](./images/superset-authors-matrix.png) + + +Alexey clearly likes removing other peoples code. Lets exclude him for a more balanced view of code removal. + +![](./images/superset-authors-matrix_v3.png) + ## Who is the highest percentage contributor per day of week? If we consider by just number of commits: @@ -1519,7 +1534,7 @@ WITH HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) ORDER BY path ASC ), - file_changes AS + changes AS ( SELECT path, @@ -1549,7 +1564,7 @@ WITH sum(num_added - num_deleted) OVER (PARTITION BY path ORDER BY max_time ASC) AS current_size, if(current_size > 0, num_added / current_size, 0) AS percent_add, if(current_size > 0, num_deleted / current_size, 0) AS percent_delete - FROM file_changes + FROM changes ) SELECT path, @@ -1733,7 +1748,7 @@ Note we limit to users with more than 20 changes to focus on regular committers ```sql SELECT author, - countIf((file_extension NOT IN ('h', 'cpp')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension NOT IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, code / (code + test) AS ratio_code FROM git.file_changes @@ -1743,26 +1758,26 @@ ORDER BY code DESC LIMIT 20 ┌─author───────────────┬─test─┬──code─┬─────────ratio_code─┐ -│ Alexey Milovidov │ 9016 │ 41799 │ 0.8225720751746531 │ -│ Nikolai Kochetov │ 1376 │ 13361 │ 0.9066295718260161 │ -│ alesapin │ 3704 │ 8796 │ 0.70368 │ -│ kssenii │ 1257 │ 6769 │ 0.843384001993521 │ -│ Maksim Kita │ 1352 │ 5862 │ 0.8125866370945384 │ -│ Alexander Tokmakov │ 2011 │ 5727 │ 0.740113724476609 │ -│ Vitaly Baranov │ 2245 │ 5521 │ 0.7109193922225083 │ -│ Ivan Lezhankin │ 803 │ 4698 │ 0.8540265406289765 │ -│ Anton Popov │ 1056 │ 4346 │ 0.8045168456127361 │ -│ Ivan │ 4937 │ 4269 │ 0.4637193134912014 │ -│ Azat Khuzhin │ 2651 │ 3697 │ 0.5823881537492124 │ -│ Amos Bird │ 702 │ 2901 │ 0.8051623646960866 │ -│ proller │ 1662 │ 2377 │ 0.5885120079227532 │ -│ chertus │ 706 │ 2359 │ 0.7696574225122349 │ -│ alexey-milovidov │ 330 │ 2321 │ 0.8755186721991701 │ +│ Alexey Milovidov │ 7346 │ 41799 │ 0.8505239597110591 │ +│ Nikolai Kochetov │ 1106 │ 13361 │ 0.9235501486140872 │ +│ alesapin │ 1981 │ 8796 │ 0.8161826111162661 │ +│ kssenii │ 660 │ 6769 │ 0.9111589715977925 │ +│ Maksim Kita │ 1148 │ 5862 │ 0.8362339514978602 │ +│ Alexander Tokmakov │ 1135 │ 5727 │ 0.8345963276012824 │ +│ Vitaly Baranov │ 1283 │ 5521 │ 0.8114344503233392 │ +│ Ivan Lezhankin │ 726 │ 4698 │ 0.8661504424778761 │ +│ Anton Popov │ 831 │ 4346 │ 0.8394823256712381 │ +│ Ivan │ 4257 │ 4269 │ 0.5007037297677692 │ +│ Azat Khuzhin │ 1756 │ 3697 │ 0.6779754263708051 │ +│ Amos Bird │ 624 │ 2901 │ 0.8229787234042554 │ +│ proller │ 1226 │ 2377 │ 0.6597280044407439 │ +│ chertus │ 696 │ 2359 │ 0.772176759410802 │ +│ alexey-milovidov │ 254 │ 2321 │ 0.9013592233009708 │ │ Alexey Arno │ 207 │ 2310 │ 0.9177592371871275 │ -│ Vitaliy Lyudvichenko │ 479 │ 2283 │ 0.8265749456915279 │ -│ Robert Schulze │ 328 │ 2196 │ 0.8700475435816165 │ -│ CurtizJ │ 812 │ 2158 │ 0.7265993265993266 │ -│ Alexander Kuzmenkov │ 1198 │ 2092 │ 0.6358662613981763 │ +│ Vitaliy Lyudvichenko │ 294 │ 2283 │ 0.8859138533178114 │ +│ Robert Schulze │ 251 │ 2196 │ 0.8974254188802615 │ +│ CurtizJ │ 705 │ 2158 │ 0.7537548026545582 │ +│ Alexander Kuzmenkov │ 1094 │ 2092 │ 0.6566227244193346 │ └──────────────────────┴──────┴───────┴────────────────────┘ 20 rows in set. Elapsed: 0.034 sec. Processed 266.05 thousand rows, 4.65 MB (7.93 million rows/s., 138.76 MB/s.) @@ -1813,7 +1828,7 @@ What about who adds the most comments when contributing code? ```sql SELECT author, - countIf((line_type = 'Comment') OR (line_type = 'Punct')) AS comments, + countIf(line_type = 'Comment') AS comments, countIf(line_type = 'Code') AS code, comments / (comments + code) AS ratio_comments FROM git.line_changes @@ -1823,24 +1838,34 @@ HAVING code > 20 ORDER BY code DESC LIMIT 10 -┌─author─────────────┬─comments─┬───code─┬──────ratio_comments─┐ -│ Alexey Milovidov │ 100869 │ 356978 │ 0.22031158880586746 │ -│ Nikolai Kochetov │ 34057 │ 113261 │ 0.2311801680717903 │ -│ Vitaly Baranov │ 24994 │ 84504 │ 0.22825987689272864 │ -│ Maksim Kita │ 23338 │ 78778 │ 0.22854400877433506 │ -│ alesapin │ 21976 │ 72279 │ 0.23315473980160203 │ -│ kssenii │ 19465 │ 61852 │ 0.23937184106644366 │ -│ Alexey Arno │ 16469 │ 61674 │ 0.21075464213045314 │ -│ Alexander Tokmakov │ 10760 │ 41964 │ 0.20408163265306123 │ -│ Anton Popov │ 9158 │ 38448 │ 0.1923707095744234 │ -│ Ivan │ 6480 │ 33711 │ 0.161230126147645 │ -└────────────────────┴──────────┴────────┴─────────────────────┘ +┌─author─────────────┬─comments─┬───code─┬───────ratio_comments─┐ +│ Alexey Milovidov │ 30867 │ 356978 │ 0.07958591705449342 │ +│ Nikolai Kochetov │ 11128 │ 113261 │ 0.08946128676973045 │ +│ Vitaly Baranov │ 5120 │ 84504 │ 0.05712755511916451 │ +│ Maksim Kita │ 6184 │ 78778 │ 0.07278548056778325 │ +│ alesapin │ 7456 │ 72279 │ 0.09350975105035429 │ +│ kssenii │ 5804 │ 61852 │ 0.08578692207638643 │ +│ Alexey Arno │ 4430 │ 61674 │ 0.0670156117632821 │ +│ Alexander Tokmakov │ 4022 │ 41964 │ 0.08746140129604663 │ +│ Anton Popov │ 2067 │ 38448 │ 0.051018141429100335 │ +│ Ivan │ 947 │ 33711 │ 0.02732413872698944 │ +└────────────────────┴──────────┴────────┴──────────────────────┘ -10 rows in set. Elapsed: 0.136 sec. Processed 7.54 million rows, 31.57 MB (55.33 million rows/s., 231.83 MB/s.) +10 rows in set. Elapsed: 0.047 sec. Processed 7.54 million rows, 31.57 MB (161.75 million rows/s., 677.77 MB/s.) ``` Surprisingly high % for all our contributors and part of what makes our code so readable. + +``sql + +SELECT author, avg(ratio_comments) as avg_ratio_comments, sum(code) as code FROM ( +SELECT author, commit_hash, countIf(line_type = 'Comment') AS comments, countIf(line_type = 'Code') AS code, if(comments > 0, comments / (comments + code), 0) AS ratio_comments FROM git.line_changes GROUP BY author, commit_hash +) GROUP BY author ORDER BY code DESC LIMIT 10 +``` + + + ## How does an authors commits change over time with respect to code/comments percentage? To compute this by author is trivial, diff --git a/docs/en/getting-started/example-datasets/images/superset-authors-matrix.png b/docs/en/getting-started/example-datasets/images/superset-authors-matrix.png new file mode 100644 index 0000000000000000000000000000000000000000..8195749099bfbfe82fb40539c1f23912010c96fb GIT binary patch literal 323772 zcmbSzbwHE<+BYBwP7zc(RSZHzNy!lcN;lHoDP3bAAQmCrDIG&%G)O2V9RnsQJw}fh z4BngH`=0Zh^PKZMN8Uexi5t7`@Avv%*Y$~O?^Tp!NkMcVJUl$omoJ{H;o+S>BNqz1>4Bkvhv><(+=LiC+Z;Re0AeHKMB=0q1!OrrcICYHfgGwyu`#{}X}7v=8JmI>h{n%44icShVukL>&g#=n;lCdBCF z;>~@9uEaKf} zy%hc3rHkeuLm@hg{<*jHr;y*DXooZJD?f1%c_CkkUO4(l0e);=JzLtH1g3q zvf1`rOCQ@=mowK>6BED7)>2x?^UA*2Nf6^X;u30Ywt2w8SF4{H_0X19;7#+32VJv| zabV_-Gx*PI-xuhX@}t#rT0Y;phWTO@!mg7d^3X3z`lQP*|Dom$2A(bvTH)7GNBUZp`ut*1_5FAsX@*68Z(Fyra*vIXQzFx`nA}H( zwE|7wz{czkWfHkQAG0B%%%8LF646R@*F1VCC#W6H)LBq_quVOs2kH9+b8=F(K!SJp z%p?I6_ktcp;x7c?p9GS01`y)D6Ro_Cx*FHn@xjdMWt^OJtA=1u$g~iC5X;g;uko`< zVIhJi9-$AzJL$29Q^70Rd8QM~c+OnnL9GQe2*QA9WxR~0hL?aRZ+w*@kYkIm={8<)fOBOOVHx?>fDiYe997VwF=;1pi1DrQ z%iS$ZWn>J5p)INdBuat1vo~ul3w>jBX5M>)zeR8@$kUkO&25qAf%%cw%}Ej^_#;`` zgBvpOSg(BuzL|0F1#FUcvRRk-k3*Y*`RgzrJPSO#lB0JmCiyg%e^C8?f8|HJu~&lqPdSAf zxeTu4!vQwmf3@zJynSl)m7@q5F(fIdQ$#qkEF*t=Uo-cv#4PRp607z)t+{&}7 zDfEwN;sfN{INK=aJm%J3FFU8!`2Qkdia@j=XB)TXydE2d_jhc5V{T3VKGRM%3ntef zc}plOdHc@i+YgHG$K13fu_6(;qI@O$xqBAvU=WSc!maq5AKxFp?`@}V$De0?8DuW8 zb5}Zg=!M-2i!VY$(nFVq=qV{F%_u!%YuWF{j@%W6C>+AHw8XBSn{WxwNB{-)ux zzOGwXbR_gNIASn&37#A(^@WsX~0y#1=A&BSrVhp10;yf5!Rvn!J4dXgcZ>NeLf3`w+%kBX3(W8iS$SqQ=%Um8gUu>FshrSndz=u z6dyj&yUO5mxiP^%W?K4;R}o5Qmt;Ko*2aCU&+iJJ;HE#XQA>l)hxcC%Bb8U!StHD#I_++>H<)1V(0$bT4S{ zK2~^-YlVzJ+S0*tvzPPd z@1KZdfh2In0DSh_#lk7<&mkOM@QXpEzAYHcskfNDz))?!?w8wAyemh|6`4IaBL+oiK)21jKut;_}$#>%_H$oL7aYKapN1oo4veJGi{{Y&2z# zygHKdPD$p%K>onaK!?eg$xzwS#MXM`gq|fh!Hi#4B=0Fhuxo2|F!)p1yUy@Ki^q1d|>-$Z~R?&FMYx$HyG-VAoxfmPo=MuU;o5xZdKa$W6j6#`zF!z zQMOTHu5GokzdoBQ?er~}FeO)~9=zG?ndxSx@VX*LsI179IH*9ZAg4g; zE<9go)hhBT1;Z_&o4itO%-3))$sM)wtK0S*De#qwc5MiIUOhZ zh%}ZdQlgxw{DB~;LtH7xBgz>ztV@F{yNlSPa05U4_djlKN1^0P;vHXMc?F&v}$kkNl~jh~BBDder{PCfl#wRoTG^QQx3_evKsUxQ#Y+ z&%}W_bo~^bo&eTl9uQUWnw5{ODLgP2YHGGn?ZI8$rgp6Eb8u*;@!8(V z`6f?2`O3kH%hy26{rm6kuA-ksVF-N~pT1|U|Gaj*k#c1{T;P{;{ou&l^x!YSqTQd0 z>B-AZM>BP9$*#wfse+zD^`><`b<3Uy`&+(gQ@u6EYleM|gU3b8#>^BV8qinR7n`b! zPFl^Q2X*^t8@rqJr_a{5asGy9A7?C@p-i2=lHO|Aku4hxPlr|BfKA*uy;G#k_x8pNE) zj1>D0@e!&!Q^p^78-0vljcq|v*(;|ElDc_i>{NzjpnU1ggY?IXAl$7Ms*F!se51zJV z8a=&FB}DRhYC9vB`K0Xu7TdgjoX( zai`l{WhCrs&%e_Z-^l(d&GLhlPEVsfu6-Q(;|*U@{(Cy#YhSC!`Q#LeS)<+&QC?*= zy_X;G_4Z|YMe^qwLeRA6Gf&rqRK0`g#)A$2K2(*0u|OM8Gx zL*|d~;IG>aEC>Be2hWHd&R$C9u~K@b6ndS}K8TdR4DNs&W3JXL(i%&`$3uxGNL=Xe zW@HgurG8xb)vRN(Sf@f^7dv#4@YJle!Bc4hOb}4oLUFOR23_?#oC}2;x4gOBI3VP( zds7DQ(OzRq6OZ~uwhVYr){+33;~tH9hdX!)1$R+5{(QgBFF3=7Xh`O4m#G~|#bsz; z0=ab2x6daHd_s_+7C>>q%g*Nv9Fv!Kz$8rHy-Ze>_peWY92scT0g4!TH417~+6P8_9GqC2V;vC4U|mvZXgZXod5 zp8w975;!b-u~6M7jC+fxhcJY#KE2qR$`i?i(P>7u#T)tCY&IP`N*rsLElPi+qFrsa zZQ$O!a*#=vb*S`bzErWFb~-ml{6d&UC8cF8F>A4VB?{6^>6t z-i!A#wgsERdCFvcXIMn@O{+l>D%o%b+CHbrN}Jx_!D@YyGuhWg_f&+B*DKWFS_g%? z^=Gyv%@d{mg6Uxg)4u(uC(B@DthS*|zu2+8qCa!LWe5L%K3sv5w+2?AOa#9?XCR!M z9Io5nsgvPsC!R=$>WLrpPWmJ79dT2e!8ku%4VL?p1&Yc1--+-j*XtMK^dvz$GiRqx zS%b&C?rR#yxJ@WCy!mkcW0TNt4g4oWn=T=GO^5SWlLcLKZFIYL4u3}=St3MCK@;}P zxcyv{8}gbIkt4NXv^=>@UCEAhUD$(7qvl^RJUJT?ee%nmn~euless`dq*x$|7> zn}d?Q>tU@?{Ey8daIGx+Do&vdYAN1-Y8|UQ8EC;%$ zW#e1vlfSYNWaxGEaI)b_GhR$xj(7DBqUlp$vmXyHD`XScerf57euxy_Y%nu|;@oj! z%+QmsYz^6Evk%k#Zb}kXSoL4u-Cgb%AOe9;wxp1DutxjYZ-MYpExie#vb63yH61;u z%vOor*PK$2cJGK_ZdQ09VbW@q=KWyZWp7Y6lw4}^6r>K>Y9Zz~X$>Z)*b}1T>^?4b zcU|ry?XlHf-s*>*@}wYZUjLT#8a&u7=!5+DMB7yJb8ojFOt#Dj(;$D*p?=K$sV!c2 ztn<@Vsz>iU3(}nwpbZX;$d9GHNT+7TK4lg~i*kfJ!oO}mQEsxJ3PkEsFHEbcSW<}?H3GSD>sMzs3kMGf9yy8LZ z8*P_&AJd`_W}wY2KBHZgoM@dre%C}xR77~5Q>9RU&C1E1e!)tuvLnK0>USLi=VND} zUun?qX6)7?He?|)YTH(KnW|I}BT}D|x|KS%Zzylp zE~h=2)Gb|BLOh=-s&s-!?JE>DL6wIaz_*f=<9M^}5qif~bsY`pFUeyaxIypx>BKVv zXz%y1Xhd{vX^5N|HhERQ93z0(orh>3>!5rLuXSHm9Qf3xk&eo{?&J7I8;ic#=$kLi zc$PNq>kgkWc>(+2aJV%GU(HRPM8TRJ%{s!vJWZJXAXf%HpjJtH9yGbPHrg}x$}G$x z9Oj_s)aXug-#I&szmWNzO`;rMO@_W+YAb7rZWVjoF`dQAFW3I6SS?+)U?nD>mCVZy zY$5~IPBU1Qz}8E2d@J8BkzF&IlxT^=?IZh{{0~ihce)<7p@^Cf8aG4cXgtSFh>_;e zimgRrT+Jst2<)e!cDp zfdMx+E%S-y#+Z@kl>7e4Zhx}jd;0izLm#$v*l9s=Su4ndq;u16S=a-+nioo*vX2`7 zYUbs+XOov|8OrGKisq%t90_G3m+qZfddHeOkNT4Zxb`$9u2GVL3+_D7q_`=#lwcEfA(( zkcG-ttvTJx0~6_y;c&oD(gSpLGY+kyJjW&~>(5*yI_1eu_gO95+uK?JQ(C)3S5#DV z6(I$~eoB`X^*u`XftWtt-AzAhXxa9du%M1^PE$;s5keK;!sgu0{;)s#!hzifvY!)g zNQW)7qea>r2jE{?FYkFgY|jd;b>4(Vb2W)$z!arB9xe83;GS8I2d|wbwX_d2sKF+&6XBU z_;_ow$%j8cPvX&6t3)|zhF|| z`W_@!;Iy3L8qW48-&jxNZgeZub{Eln>i!HGaz^n(WlPcIR!EyUb-XDyTWl4A0GW5G zN^pi{<$cqMU=m$xOXeQodqEY_N%3~b+q%A0L6@4E=n7?$Dm~~qdwW(Cx(?m2+m{(G z84ChoIpX-RsUOCQ)@y!;&tdx3DmG}3R-S=;eO_>A=woGpGa&|?iFI75hdUSR$wRxu zEMGq!QdE$8F4~9Yg`21=COq` z>g`Isy>=xT_snJ{eWk7GgMmx^&!dSTUF*A5Z>DV@0(; zt7)dtyrLgtzkn+K%^FacWnBN^G);5RH5$r%Td-WJJrntt#b*B%VI*Fqy!iPUv=%p;6}ZkXsbFY9v2{6m?2#V~VN zDZlMV{*!WjB-{ZNku$g;?YWtA`ZST)IV2iiWg`UJBeW);`o#{8o=vTe!w0YMO-^ln! ztR{ssi3+*Nvc%YYkqx_?r;2IF7sVJX=lV)?AV_>Wn0$-1uMAgO-I=lCa%ASnT=8nnCEV8Fx0Jy4NlC3WyMzH+(G=t>cU zf^KA+ZzqR=LM==lZY5Nj=liN-%w4-{`B;4tv(L{n1+D$%2Mw`3+dMnv@^jjAPtro| z-aJ6S+)J0grwfL-^4Vsd_C+<9+LRvi04Wz%q2O*$Np|Xes1BX!ke*Wu(<(Bqmf6#M zm+d-WLLquM8+6A$>q&wkJq6QtD@eahV(DSL?h;Z0e4nL;5+gx=^CX*}7G)J4Y&<~|I15!uF7gTHsK@6(Ls)THWHVMHlhxZ3Jvo}(CDc2@9HK;_ zropP3Gq8&&5~S1^A@RTtxvP~u^$*DlmH8!+S*?#yyu%t|N^!c2pyOc8{!9z06d+T# z8DS48r4Sje(s=1B?Q+NAtxVkM=P^$Qj(vYb6KH3JZ%E7~S<-;LHZ4|#t%td&1g?wI z3VV}(@14`SpXEbn(m1lh-~&$q4mQgnsLhwbNc8EA*lMYD8-FlK>WM1569u{xW{S~c0NeoQO7AA|%T zMt%S&P=rxvLoE(lhs2BHD0MXlS#u1{GsMlrzRQwn&MNoo>r-3{mv8E?oI(1B)2Wux zuz)>EkGZ0gH(|c~B^u{ZATR^O04bD;a^ti)CsUR(q)w9?H#Iwx%p*!)yU~BR31T#2 ziLf1!nbN}=W&~7+_KuQ4plUOF2T_vCUD}03*NOh5<&^_jlXH`{E=fDp4kME6N>__M zq^(so$NFPno`#88V_~FJj4qvbGDXA2bF#Fhen^`<4jJU7OEC<|R|z!SOng+R{bI1w z5X8s2m+d?NWVT^rPa8 zAAp6-<4;^=8sgKYk%tTy>1k4&sHw2&buHIn$?!c}ww) z;hD2>jT4_xqTx|Wfmibf@BMjx$`@LLcA>ZiUYGngc_8g0lqvwgr?+r&*-5q<;vwo7 z(U&a5KI1%&0>}oir`TGyq^`d`uF0r9vsL>eisrSx8rDfr~_qJ8rrp;76orMNWu+JhLHNiSEA&W0^cCs53|K`+i z&$B>gPUBFSW$S_L_@0l#pRhaDw)hr&@CnQ0tsMO9`K%dFlw!$bseram+@n(UqTLdD z9)b)dA9NK0-ph}Cl+xVlVZ-9UR{2G>l!ChsKs@nR(1&)ye0A}L$tx|n_8A}-YjZ(L zqAA{P1ZVXxQ}sM0hq7cDU-dx_&CNPg>DHtQ=lSX+nR7C2sRyXAD6aCgz1~Jdu_CSOV_`o)lrb*m4Cn|9=Vzn3NraQI*fXQP$H zCd9BS**?u)6nx;Ah_zTJs7T}&J0AP-C-r+n@{I*gFlMawG1nN_Q@&`|us_TCHkBgi zD!KJCGex1Lyk=V;(Q8ytcr4x07|tmC_I5_x;^M)V4S}#{`M1MlOm|h$QP~=6<417X z10IK_rw8>ju5x!!$x+Q($cXG4O%z_5c+2t4Cl(}rNBoMAdt6~}S6W8Ad|jr4qLG8m)ZR?8jz zTnEt6(Sg7y0+(x7Fh4h3)2p!CwXAedE;&jkt^AnIMkN=^lsAZ+k3uDeqaG~pK9w`v zG^g)c)OYN{b}icXe+-?bC?6rotfSi4L{m)8xQKCe!AGsaN{gV*3*|ryMX^>4UCC$t zl)q^OD)v!j)oF2_k_WM^O^n7z{8+;z{0XZ(FhbC6Wc`ghcHsqdyZrL1j!jPNhmGlT z1Q0*vRd1uan)OM`e=|X0kC&EiPqmvu?!(7c2o^wH6_NwecLc)%a&92!vz|nLcw+++ zvq?99A5q{OW{CD(_$;e?urVb?qau-1Nj?a)?G%Q}`GX%Q^3_E{P#)tZq0S^u0_X(L zsI{9Y(X88b(71PijIiAT%cv@eBy4Bog)94XF2OUqlBzg>S9{QqC1C$^yQ5cN#-8rA zN~RDI6~^`#`0S`3W74{?0kUKOh=9z93;Q{b=vB@fZlN74H}tO8sGY4@2NA=uEe&})yieEI%qq;%>HVlm*n}&qZE}JkZyF9 zEYB;z=@?xNb2^JrSGoezf~*zft+!s~x`vQzc{(M~73GvXfh~0incjR29x-a{Q$JX{ zAFpr9#ElWs&2!SIOLqsa(G_^5Q_M#9oU;p0Y;!i_Z3+>6wlQ2Ger}sKyh1st%3$1H z*~e=7O`~@<``2G~Ln?21KQNVP`vP$+c-Vn?=ks@i<@I$Jq;*eGhI$*V?0P z9&|d|d(bt*SMgx~-3^x311_Rs%@aQm+DR~-+DQ-JM$qnyya5*xwPyLmvJbzDVCbdi z&=sAXO;N1ylj{!0MKzSU8Z?OJC|FZ?k{Q*X)X4Iy)&Z$akmI+%Swp}6ptIwBmRrs+ zx!*b@@SfdF&7M|tf5t>RoeHVMwy%CUYjjBbSCsypk7^7GE%mfKmcfxqliQUB+3hx8 zvRx=bhIu7lOT2eFr zp#V9o)vdi8zeJDVssy_XAF(CGX;JV;Il<9F>87O)W~F++v>T%|SOl@1PTO#op8`M+ zWaq_KuE)^u{x}hpHq1Ai|E_i;q^4K$fUTXk#GzzZG#Wke{+PPoZl+R|p)q{pb=dXz zB6OuyW`b3JL{UML(Ml8N!bI3Xzo!_wcKF~Lgt-?3QSzTiTh)hjTEuuXx>illR0C;Y zLtQi?G3`F4DB+ZL(U%?;q{*DBA3AN#w#;dM`%Z|@x0Buq)q@@g!)0O2aSJh+xQ=HBv39qan0e&| z1GMCW*yZJhuF_plM!Z6__d}=g#vTym*XNr3uA^JP=}Q&Lf+${gyc#!+tox@z?Jgcs z@INV+{{Rq;bplN<1^5&>*j>icr;5J+pB00D48%eMCC}MM?Z4otq`5e{z`q`8LJR#oAH-*cIlB z31e3PmD3nC)*yqw!c0xm(XQKM$6sq zF$3|G-=!D!D0T-h)|kF)Z}~yQrHl9$Mp?JMQyaa?Q%dFba2PArQA{H08UK9NY%<;8 zy+0(+HBkzHe}&On*X6e<D(GL{MiJ81^ zqmIOOI$8K#7CMXF*@Z^B*JR zKiDO#HwcXUk8&8Dzv1gBy8;EbudXqM9@enaozg1`G=BR)iRR5mG((~Eyw)l-FG`bK zOz0X)x7*J6o=web{qo&x^!Pz*H4fAf$xB8MOswe9LiGNKx^m*p7Fi$^)lC_I30stv~0Wv-ck;r`eygeOGQh7}?~3p8C;?`PKZz&g1gZ{dJo;(+Y}; zcpyh#tI)tber+QIi!i9eRJ4SlXU8)RYa<0F2qsaVaX^()Z)T|O5V=4M5BwsEskI%> zRm9n>l@y+fo(h5`PtAS=-@u%o#dN=&mz^Fqc!FHrZyNU}?fBhRRF1JI7{onjwTG+O z9?+LaP&3do+yIQpVZE{y25;b}=FJ~fkBd3!SkbOLn$hWh|vI>7;JFwIMGksHsA%!mttc~cUh z9c$(;tCMp%FgK zEB1@*D$u-A3TkJ3-F_n%LB@gPvmId_E`n%|gv;~-XW)1}#kKEBpai4#u{BbTql5y^ zVdOFY{^8{B2ryH*skZ)W>G5VdR}*(RRV^t&fWegVQleJC7jInCHujk+ZC2MQP|0B< z6I!`_JQ(3`UN-=6nO^}~Z!8W7?-7VwOKII2JD@~iBq!A~C9ARr3Ea(AO?y)Wc`A#p z7HA(A7Q-iP=ayd6gW++plA2lStAq4T4TI+SgsVX0+24AobnRvTsJ4-4@O7XC^>uo~ zc9D3SC)@xwk94u)O z$L8fx6OSEsRWC!@C8{P$^r|BTh3A5ANs1|Ss4=Z4n1x>nYys3@!D|e8KNF{~6%C%o zTtONKkt(nd+Lj4cWg|}6jrbv0H@+Kke2cAar|xuu$~>`;=?Ya}$(e&T&vlVag=qlP zqoQM{JFYVdC5$z1|DfkL{ubaINIBnA86g=|VcA;_NcV#Jk46C!*}YemQ6FIPq~5^* zO>pk=@M|~q9Lf+Jg~G_zd-mp|UZ>W`7(hBfbB|i+a?W*I2>l?q$T~eI0FBsW@P(|$ z+|aX#)m)1xzHz`yo?LElPR`O?Q8B3x6PT_MbIVIXj1;J&k7v&O$L!;mYk7^4+a1hK zfS_zSt_;IWS!ai}K8;lzwho{ZQK}VW)NT=vJ#P)gYz=F<(Z7$lF)bCwqj6XDCeyn^ z&w_LB<&t&e5AL~1U_KZ&7HS8I6KfbuT>})pgjDG1QSNN2;~V?q6;p3jVIRomRjvkd zwPd=r*MP=tva4%XLODHGeDKVQu2%Wjj#t9Kc`7ABWanKN&C^!ipV&vc);m@|R-jLyFD)Z8riYr8xkyL@0(#WJ}y@ zcl}&(lQ|vk26AbzDv0ux2GX`i9tf9_YN5?nIeF(&8(^*AyjI}_s0bn6AzQpY=}~E2 zLq*y!m4mF0k3m;dN%0GC73kj%=X&WmZJtQBT>~&4A&MZXH4Z)11eJ~f(z2e%^}z?y zsLSdkgEN#RI3w>xN@1W>RPcTO124$m3tEQGRaUn=#7UaZT0w3x03*J0?u0U!f(Wq24pa4>)#-rJ zrgV6-94iH@j_b`CwG#bM;{4Rh_MG3#vwNJ3) z{4OJ}swW#3vQbmXu;>ffPMy6i*)^nkrg>NznRUSP@S_k*`+0$%qh!?BbQ{a}rSnj= zQOTjAtV|a1cB$zEED@G8-u(v%>30W%L_7C#_q7MhqcQ;vR;9+|`m8FxLS(Y5W90``pA9H?h$cCM)BZ1swHjp1Ql*GGD~UH4sI(C-chU8*g{| zjwxtn71IptZK`4-_x<1#3aq-HqLUdK5V|{UYy$W#s$z)My;Kq(EmjY8yYz5UPm2_p z6|88siFK!k7x6LPjRyeX(+0h<3$c&~)1Kc)Q% zCG}svq}%uwXEgYczW9G%lt1wR2GZS9!7EdO^6v9}Loxd*NaUR^RUs%f(z`E+AS%Ay z08bzzsccoq1 z2x2iAR{)e$0XSd`Q%FW3`qM;_50p|~HK%O;1k6xz1ww!^+PT4WlAhcA9j9S~SK_{( z$@=(Lew(3tM&^H_@Bghz+*ZjDZ}Kj$v|R2@wvr0H?ojsfsOo5US+B~Xf%nh|r4U$o z*-CYZ^3vsNWdI^O}U$zP6s-F>DyjkmZjEW+yW4QzbfjAV#3sG2Gs8WRiy3 zE+7ezO4Ob=w{N^8M^J7+6N|t;wtEgHtvCdB6M2pg$g@ymC3^YUz5fzyfV<%rk+Fs z2j2m(kXD?_CS$-$o^;Qegfdb&@-lx`ad%Ua3$#50P{q@P=Cjk|eIT-s?eA|%3Aq85 zs_JuK2jC`OFz-deDdYRqNBT7bO{ZA9<2f>@b@M3(CKcd>-COkl7Ih4HGEd!1vfCw+ z=%Uk|WfNK5mnw_|Jd;ijQ{9HRYq?1G{_>Xz{A6#-5zjC@E?Q3v|DBV>ZAeG-wa>^E z#aO(cLH&~DSQOwN=Bk<`sWQ5MCi0;FLJq208GXZXleyxl9}I64N2U|F0d|OnRPUw4 zx!Ag{%4PPVVpAT=pCoPK9&00Pg!qe*enigJZmZ0I;i!0ZY5?@GaH_`IToR)y5WlZI z;{*WR;}$Z1^2ZH*FY)Dxs73j($ulPpK3%Ip!;wD}L!3>kf}h5((pA+bdIt(_OIs%(Zc3QVl?5woyS=Kkh>HrWc^l8q#T*I%!8W-1BtG5g!lF_- z!%-yQUW_br>%_lwd7RPnmq`bsr#{Jq&2#W5BariL_L>0OEH{P%CpwViGw`Hj9&j@= zID#6Z4uO8uDu})A5Oi3u@<6?C4{)JfWg#ex!0H0P)gIue?%XMi%uvgvqvIAUjmq98 zotPCb{s+qRKh{&-jBfyLa`;vA+5=z*LhUC~J*REZ$m(h_2zCT2D9^3T?{~QMWIo-+ zz~@%+a^L(f#5q2go-PDPoRu4w!-g}QfrKQ;%QYIeU08>gp>iC)yu3pSYj^U%Qw4)^*`^@@d3LMLOUh=h=q)ws9Aab8)2!a6H^ z?bS2v*73W-?lvk$n_6SmUPgY~A*+;)Yr}xMlgjX@D3g}YCvD7*U4=9!Un6gSc;KVY za`|Q+Kd;8`bJOVU6i|&ArOy1>^G+5^uWL zbf*=3Vi4p;iHZ8|Uqf09*sgXx+GecSh_x_dBM8t*GO~=uuE918ETy{Nyz#1g70cS? zzn1F@z%_dSeUaj~EO<1|ziNBI{^G@j1Pg1{$e#v$#h<+uOChmSbo}-S4nWZ;2wUj( z;&aDS2^t<<{X(brRv)4$p{Vb3L`gL2q+pT5Rg`J{{2MdC6`I6|_de_aP&nnAFgpK> z_0I#91U`1>bSCXIHpycFs9Yi1cwD5}dzud27t8$!-{`(8H#V zyrwbogUyG=Irb57Q=O#?X+24D;xN2Yy#}wZHn~qN&W6KTtq)d*^QZwZ!>N7yy3UW? zFv@}-aN9=ZkUj8BU2I|AF-N*t-e^I4onJ)aYQxSHxWv#bU3&jdr>9sAGfKbCHP4gv zrjaaAy)#l}>|dUarX#rx7zIEixgmr-jitZPJx>u&=g}d350PApr-~3VXOTbB%dCY0 zfyP+JV3;wSZ&afoqr7R# zMcMBEnOVqSBLZfkr0~kvS_G;77NaUX1RyYc0nZ$67q9%{$@LpQc5MZaE^vn~$j5?Y zbi;o>2cw)Sc?Wpv|ET0-cm>AY4F3f6V7+I*QXaCAjolXe-gms6_#YtoKnfw%WBpeU z7CfL66rcebm8O@)z}STRtzfzM#|B9Y2|}HbcckorvR-`{pYOi~)U7ceptpBG!ZDiv zSYz{RzRiFC$T1yiDm;6(;b@<)(!rG*W`9iStF->V>w*L2M0>(zLd??Hx~xh7mK&~r z541O9DF$^w>xC|^=e|GP7H;VQeYibP`aG5rttJDBnGzi0Wt`~2{tf3A zU^p;dhRl-wNLP6PU=<`V^yPWnrR+asOwaL8tK}SMOUCsBUpm!*+dKi@2aWrfm$>wC z0$dvr_aDM>^qlVSF-Sa%${#XN-$DZo(^HJi($7Sb=VD=f2f$p?Fi;Ar%1W&I)8O6g zMdLsR&IKqzsP+08Tq)4GI?gf3L@+r3Gg5GL^^6!HHdis(^1LrbRr&*6Oyxr78=z4F zG@EkZ$$f4or2OOGfcD+Y-Q9hEqkeNT_{yzy-lh(k5Cin@vJa3VPKXLcGgh?4!NS=1v7`Q|#aGMdj*cyNv0k{M z`~N!0E$&LC(pOq2gXec{usO%#Ej56qH+fy`&^TGZDUA`3VeiBss_e9ne>0z1w}q0C zq1<2ZfsAlN%i7J@Sb*Ymfdg9Rq(!`eZk-nr}~`E2`3v=V4eEQDb6ZJ#3kH-r&%>T%Pd=`bq%0AP@v9fcx7nf=EB{YFX8 z|7fud1*N<@(i!~E$smJ{a8K$>lPzGvhwCmyUH$KH;a08~z|&MijyJ+(H_{9Q6Gt%=m^niOV}n(Iqmktk$m{3Rj`8!U zwy|@;=#EG}gtu_In*(MK+*1AdbZYMBef}x8;meLbM4Lc|&M1SXe`@-i<|_;^40`l% zR*wTSdQ`1m$Tf#+GJpFl0Q(9>5=^S^My<&gFcF4Vau$#_EC5(qQvx`4-~eJpdM~G1 z3BhG>$-r3qxsZ20F@=0wzivvGs?wplqKtDwlp2sc3TF@;q7Wak3*=i%|3L5#Ra#%F z2sE>wUW{i9|9803DW5?ntG$=*>Q2DKu$SJ$!x^9LXc(aJ_w4+S=lK6-?7%{*{G=Or zfB*}j=d;!D;wx_xuhava+2ozF zJh9>??3xKL823h|W8}5Q2sP#<3r!lJ4w9`UW+$s2H<%dL9+#%jUTjOvzh}=KNFs2qR+EEFDT=~n*e}4Fr-OTMpdOt zet_aoMh*ZOe&yJ1PonYdt3a_ZLNCBa7pi^iQ+A%ZD*$aRSuz5^8-;&)1^^t};nwHx zc_mB9bH<$r5!SvADd zLlNGrB?GDQI!4X*HNXj0fNiA$w4F8fd{7Y9cW(LRVOl#MbURO|hF)_aPll!eE{0O3 zci1g2)q|U8Z-)(Zll^d6`hgGSSD7t!nlim=$8)~qtq`5+7QmU2un?94?0^R_avX}l zN4@eHg}r$TIk5?f->w{{>N+tA=j(2Z}s2* z`$s1PT5_rsh?_FdP?T}S`--8T>B|1k4a;q#KrMN&zSABQ)4XMLu;AOE0^6jk=TtX> z)s3RuFlz>l545G{ssRl4);&oHI9bKoc*-WCA*+>!fkKU?)xeON2{X~wp2^WCTApHn z88%5>!qEV35)k^3*;W~LV_zF|>_c=2jG<73XE)`Oj-r6^ZW>yZ4t=25N}i1nZ+?8^ zcRqW|a#BzNWY^Fpn|tD1KB8NT@kWjaL*J5;@^1Fb((oLywbAb&DFBOp2UeYICgfj5 z@ygry5SdvMaWc?@y!jWUALDr(4lrV)Vq(n7ZF2wfJP>PrK)FTt$JGjxSb&(c6tx6s z^fnKR>9v@%T{54F4b2~6pn>~A-n&bHaeR3W?A{_vGSWbNT-i)aTs8fXz!$6bQuysy z2ij)hBf(qj%|e-l-SYVl9B(7j#(a^JxlLbUbt)|RUNr*wVqDpOCwjn&=yXJjfm|Qp z2TQbJhId|!8(V4pw?hk{kP7O7Wt!N>iLJqF@zA&hcRzz%%(lnOLmdI65Bh)ZLSJc# z*H#pg;5|l`w8nMg0=<6upF`(5HIqg!h>0kid>!eIFhChyNwJrQ-5g_m^Uw9=uf&(3 zMiA?B-!*r4lvs!TwMtnedFE!k6a8z^-hXNh|JK$5rzp_&mdZwPA6TRW-lu5>rF!(L z?B@$e*lM=nos@!mMzSddO#kNNH3H-AMxKzxC_<*O*ErQ@RUq2|naaasM{vJfMTkEG z+20Q1-^rA}&a16@Mk#;2BJAG+A^wG5YdH-Cvk$o5?(mxw|eQ{KKTf=!qX5%h(~O2hVYA^UBm56Lg+@gFvK5n>|yue zn%%CB&R=x`{~e|kPb0vl6taPfy7$*C+y!YFep(qcxobrHTj@=_i2z$%sKji|2XFB1 z=LvIogS(MW&ALcjmt$L`fnK>MKdMzQX`_(7gO+N-fL7(39G3LBX3VT?xf~#_y73;5 zxwHR!*be+6ESwwdVd9P{=|8SaK_wxp7X1Eo2LhKK1I=zP?w(*X@YxGtI)!T8aJkWk zlM)%Vpf4eW(Wt1N^870IuG1eq09fpA{(7vv6R**WXU+me&vZW&<&Z~tX}H8cr_`hhalW%SU8*@9| zXE9&B(5%R5x_O~8NmhmpH<~B?k9U;+BOQP5sOijpt>v47x1koXB}+&KYV?4T+{paKg@Xv3q%P;hCz2o2@C^KQboE&Kw_k$2h0E!38g_8 zf`D{)3`H885+g>ZbPm|4-+Lg&{oL>GudCiYwpW~Zo^zh(y!>`nOp3l4%fi3BO+7X^ z(Ox2x`M=V5@9^r1BzbkXJ6_W6;OEGYZ>c+$1>#TfzOH#5HF22UPDxKyPit0o$!p1z z&ATym<*)DY(J9<6mu9KF(^X&d3`hrzVQg!$&7G>f&UobCmjT#puEV7K51c3;z00a= zGm5_AKK{hHa&U$<;ZVU)FjIwZdoqBg)c%G@R+&Z7b7 zcnUuI`UMu6}vfBjy*p+Pd#x9!~JiYfdrEqd)#8MivetYKcXjo2(fijy6dm?%V-Kr}^ zIT<7;dH5~bm2Wcl(OIMwE$7%F$?TZ)g-%U@Oei6vIVF&HIWw9VoCV$0qB4L*#g2z&%!}dVkr?NmtgA z;Qqt?wkpL-dUeOnJ+;}5y3l@KOT4{kjCilBHASZLJuBeV+Tt=5QHc~sJAAQiTF z578>MnCX(zz)_Pgu4#yo@d83!M`I_)zgjp7FAPBiIu*TR%217{=c6e9Q8zq~Vc=vw zthv~*NbD`HEj;Fd!LmM}>$#h$h@XpVy&BKYQky9Gj%M<29CM22+ev5+mg2+Cc}qvb zz%cy?X|LR_ux*+KHk!f7(&=z3kFD*x$@XL^!(R7H9&^3!m;VM%(aP1{$<}#?9ODxX z3nOIX@s0PaXMibA%eJp%IkI`00dFP8M-9}PgK5dEmp*^D_(N|2MY<0vTsvPNyRjm` zv_XMf0urWJu6i-(&E%rO*e7!L1-|B@M! zOSxl7zAIVSwmr1bd2tBUzaff>8`mrDz;_hiD~w8yt+U0XWG6}7l&!xTtIVba5uRhn z&h`~cdw)WhOCI_FXBCHjxZXP3#xQ7|$8WWWE(-_uJ`{Ywp61Zk9!V4l@`YVDhT1) z+LBnF$|x*4rK3pBQ{XZ_%C-7VJ>Cs9QLc(#$(+Gk5j?I-==>ic6Y>@H52xEV$ufSh z4GsMpSpdxw%umwh<93Q&8o$#QiXCS$*A1NcvSJ85o>GhRkJTy8~ry6YdCSS z5z5KuuqpaCF8FCi<#Y~Pv1yN=FSfb*$yHBUpxRY3ZXH&Nwe;xK(h*QSEj~}L(l3AF z#B_-)ivA35y_8*kDkf_57kd0d5JI)$0~d|lke28E>pIC#cyvZT^LA#WI6CSUt5(5` z;V33J7D-v+M&QwE{OSF=Tn@WDofc>Y85%}`$#PYM8n(5FKWhfRRic~xr_n#agr@}V zq@2**x}HB0W1{w#t@!OzZ9#<+pC2&it+?%OiQpWwq{@n_(3nlp0j-&|(l3*D__kY4 z(cIr1fw7!vDfn8v=~Xq`t~^sI155djUo5;7zgw+DmJ^=Eh>-vF8^13Y@X7yx9IDel z)lp~OIxBP6;N>A#2YbOdyC(P3BRUf~jZQ}9rgZ!ph%xnZy_1&%BK%`9E@)k73W8dh}LHiNNKH~*<$?P{8M}qH4T|f zs#e^}|23V|N;#hX$t#Cj-=6{o{`e|e*No$(;)pa=E&+I7%Dp07WYJsXwL*<~IE&ot z6w8%D`$4g~IK;oSiJK!H^?hcw=HqUTVQH5A!~Yfy(zRI`0orXy%}HDUi?)hp^L&ql zEmaL@_LeE*3aWw=iYm9Styxc(S(1NJo*BVUr-YBfqb81|&cN<(B5)mDFaMS^xE`WI;e zza6de4;cRKp>|I9ctO$Dhla0>Qr(rqL+rVGQb|<%8d5d$Qvb$Sh?)&R*C7pExctcO zdxAWve+vin0-doj{XY^xKczr!g@N@;7~#xJ#0v+(oNsQ0vzr?oLG%KXB9P)Ru?5(M zXr^j!?x?%vrnw%2o%)p?knhr(4L%QM-6XawsoAI1`G;k?uNj|p=Ck(&MnPjA1iL%r=3_52@q{&?dzJ>Q8iQq)-izqdiyU+~=UsPWc4oMN9>1K0ReF3oO0S zqLyz-eRNo$;D-bFke_q#?#Q^Sp#?Kw(L6Z66zG?WNKBn9rnA?mL;)Z`Vx-4CsyLBqAX;+XzkcHYO}x z#(O`moe^%GBr04x4>ObU&DPQ@4V;8KsT`TolC9_cP^k}xSLnkZ1C45pDTZ7@DYE0s zO~rx4xJ*9iZgN_PGxNezVsz%1o*L}bJ#n{|0$KK%N0Ni?rPJnC9594wSJE8H**4=p zK-g~qveJ-4pBQ$z5QcUqA`E&LW}6DX;_#Nd!I(17>5r}Y_{R%uGoc=_g9}Rz&AI%U z+uc1uI%#k#4wTqRYKWa)&veo`1w1J|Am>D*4YY7}PMtN_YOQMHAWez%jUNQapEeN0 z>YKT$&3SC9I!8IsNp>AdD$;oXohvoBkg(nYDx5&skAy{{@m%@}0ieIaG7GWPE7V6x zqoiIlcb7U@c+$O z#m}*86JQpTI{0$WU-y0XW{WgFS1-pHxef4(^MOXxZqocgWd?v*aG5lu7NFc0c_9o8 zV!8T1CTLNjfZ2j3U(i5XUe5#u1;JW>_yiPYWLs7!fbq;#K07Ce1>QCpC>-ND8?mtv z?m1CtCyO7EBFu-_RcQoHCIU#VN7Vb zha)Sb$lI~}{{YKBwH5i`@W`OdmzCP0dapePGfoSni!^EYkIRHk-g#bF22|&Ifi_W7 zQiB9gaK-|(2~$#Q9Ad4yX+CsYpzkY#Dtln_712L&z*(#fwM; zTA&38f(6Q~lh{g+b-|zP@&LNUh1vk?r180PyW2BYKaJhzZiH98E8-i3_w+UZ|5+P# zUvjgONwNfwK;knmU3U&t2@{AFRK0?YA%aZ{q_bu0E#M88u}CRotRs z-?P0gRV%TA4~#a~=H^~p3>2*VS`6?s&x0e6xdZIsBA~ym#S18u?#)ytalnDj?P^hB zV}MojISc6aO0K>7*=8>H*q(JmI#L#NRKEfFvIqk&&maa;RC2SvnYwLzri_=^EyLP3 z8Lv!~3*dKng6S1k;j$VkD2!;UCI?Q;PCzf#2DjV;%}Ol#Gw=8>7_SnZlBfa2_@MQK+yl`Q5ka$|G3Ss)X*&-bmC?a4hrJAoBxZf4;-+3o#PjIX8S zwC}igO@VsYjV7Bbd0gtIf2kxL$8?VfgkVE`QZ*ms?S9T)-&5!O)#tnSNoJeD$kRbv^U)n6 zsUBcrvEQ7cEDJDs>pbU2*sppTP-3FKblLCD+s}3^K1yGk`AYfK`&|)S>pb;^)|Rlk zt{lmC9$S3`Psg+slZj2<#GE2%T#0tkln0> zCTbW4v;3zp93(3WIpUf1RSuIKIghn|mVf%QchBSB4783oxyGKjq=jH6trr3x>Bt0Tt4ka4P&*h4#%b-2j=RdDV9&AZVj^=|)`J$0Wq%#=f5cyp z20(Dxdx|TvDqoNP_VCE*lXt2cuHWqJ%wHYgygy--HoU?jwz8KI@myaL8%^!~APPlp z#C;6QUzrrg6?DUdzg6|DtxL(aYD6q1eE7eWK!r=tW6g#{AIr$f%F6IF0G!s8HaDr0 zsJ^2D3#cfL1Q9KTObH|Y0vP}?#$>a?ZDkgiIGaamb&eteVt#pib^sVOyQtZFxl7+E zYA#Gy8))!fIY*vz?;|tAAN!c#r15*hjqq|{6>P5{Mq0*XV>CDhvfQXaxjyq3PowH^ zRwlAu$xhEHDsA-za6sNSZ)t6W;A+6P@m=Bn3WAWcAQFEI;3s3ffYDezZxmRfDjhEt z2Lf{+fTnyX4xqij0B#)6x}LsA<)@xA@LZ}THE?Gr{OC-Ns05tw`>!M|Jty_}(H*q+ z?hdKH0Y_?c1@;p_qL3V$zW?C)wU_WhV)H9S9|z?>7xS?~F6qfNA~`o|G5%SFp`-4P zq9*J1rxV;m>*j@VM$8xPLl8hp0v?t8ydwa|=^S^Lo=x@tGORm1f}R`pIIHx`S%9gP zae_=WMy%DmCDMd6TRw3LKiks|K+AE#2ZLfb8+EmmyKAJ*ufffM^$dWYVN!W>io{{k z9ss~;%}&*IVCDr3fRQ;sTWAZwx-mF1v9@>(>}T|?zs&g#H|m^du$GDzDrw)LwJ-8&O7Z)daEhPfldbdfG@FarriptXbWM)k|HM9&g zWbmnKnO=(&-Z>=NX1aLglu9Qs_jYlbO~CE;^J^?-kfj$2_n|wZP(z-z%H8dS>^MS; zMI83@!+6wY-z4{`!-QD@Jj)g5**i>&gWKiaTKxc5FH4R3#?nG524Rc;z#)@L?Aj$7 zL9k9h2bI}mh`Cpg3=8<{)c!a?L1tvO=QvL%1$5)nwBdT07XBi1`C{k7 z_P~P1?y}KN^YqYAyylq4@DU?z2jbw2(JqllSNV>9!&kiGy(ZaKr_pw2*(@*u+`xgG zb!iL=I9fO`QL!RzL3W;_XVWpB6C#BV z&Y^nc-^@#5+*A#QG>gkL@Z$pBrE7!B;l&x-4LoZ?rOnt`vU%mPUv-4o%{!;(Eb8#;lFywuZa3@RgtoqwrcMz-At^Zkm z$OWI1s;3<4<{AB0#bOS7>gkRU$C%1@u>wegOToLjsX=_0)a9to(^_>RS+Ztt1#EQ` zrqtZ~KBQ()(*C#TRF_e04MTQJ@Kx)T*|ML4iX@{JfZLPf^ADP(E}oj~K-=w1*k#GC zzCWR{UQ#cU=29_w4t@?W-70vX-LK?YZg9w9?+FX9iKB<*rptgBd**$Das!u_%5~|9 zf%PgEJal(*HzN+v2~9ws%TMI@F9o_}sI6Tb0p<{8;uKykHoUpmi36tTO>?QaDaJA+ z-INuQK3r3BBKaHb=AYNLf0^Bx1-7*#<-w_vb zu$UaMv5JHHm#sz%8_Zv@nuo9;wL;N0;Il#PT+c%)X5b3z-?girZ?H-iz?oI`rWY?g?v%`^L4} z?8+bTBje)v|6vaDsXF|(0^5&<+&s2~acpGJxvC|*{D@?1d6kUAM_A$EQyZJlHpg!F zo5gao^Lw1;@od`@iEy`%Nx=QK&B=gm{rq(I<0yrv=pCd7i*k(>%~ckBg&t@?Bo0q6kW2x z^pHB*6BOzG=z_WQoL3CvLa^?j)ioyzeRJ}q$S1d9L zKG`(sk}9oDoH7>~lxl0Q0}J*X{^k0ApDFtdGPas`%H7yZ0QeNj+wlhO5UGX&hKo)9 ze5(Vxu&nEJgC=Un`ZDL6}?Rz13viA-?;I7&!a;4(cc5_0OcIRMm)q6Ej z#;37x#s57}!DAqbm>)M|hWtvBE&cr$lUi`$*>?ImCw64VdzVFLrqa^+j6||vuUn-N zHP^B7G0oVnl)gJ|#5JckYs9fR50#exIo{3dKKM({yz<8%p6ZrAbyS^wJ|l0_DTz51 zd?_az>`MP&MD@*eu9(;8skQ}keUXY=appb3`b2dLJ^iTg$0MTu#z(**9aS>ENbSSB zw(jr=a{it^zuA_Gi=yR}o*t@qq-pDeb=7&#nA!x)?iZ<*Z(Uhy^e^q2uOyXZIjl{Y zMy0HZ&+xV@dM{rdebMzVVg3Ou73|D=_&QNg=_(BqS&>>?Q3ayz5ew(<9#A-j8_azLED_Gs$6A5h!WW&?0?^qhxjjyL7q<~kH=mDMO^VVaAe~3$l z63W}?pc5YNWcSS9pZhk>2W#)9l9lvW=d@=kRMWT^JPjb@qK2E5Or@x`mIzneB9_%z zT-m(pvza5>Y9E!1h9Z~UI_lM05zFv#okCBYg)}&yUR4jp^!LBMmlI^l$(G~A9rlls zcj<%?7TVFBsj#OxMNj3%936@iyXGoizoT%8L!D`oHY-^b2uV>Rbl}2WUK0+@{d=7@ z7s;{d6qHvlpX_D>QU0SiTwPXI2fdF=z9y2zWXlsFJu_WCcgxs_GW1Dr8(Lr#V)brZ zL~N&|fY!UOs6@vSM(_*NOF|G6{s5to{Nxkv-rHT?GFBA%wiX88yTZ-Nwi2Cto7hOI zXuLpmgN5Ej%jd>dx(J6_^v80=o`TEi$J+$W28!(U^!1|5naBT@&uTAP;{cZ&qwCHE zCmDmtj?BFmXUuzf$M)#sT70Vr3%PZ;4|wgl9M>iO?M>kwN=-m5YfKszgW+KaT*S<{ z#NO|c^s<@}Tqoi75msrGg&z%4+SwRow|y<7m$L8WjMKmuP_M3ZdSxXVB{5TZi}axNQArWRVj94%SE`D+$p0D!XuuoIbw$y*PdJ*+Y~! zA2f@OG}%XZ+ws3R6b69wML39-y}th@C#^mINVuC4bd*EC4o=vdI7h534(Mu-JwT>8 z-eBRBGgkax&dAPel;~`(!*NY>gJBL|_O%u{cb>*gqi=4CAnLB$RM3_v>(F>TYGZUQ zKu>h{8~z^Z=5I@Ce1%GI7b^Ghg`gVA51l~17o%gXdwS_Cs3Zw$33Iqz)43;J)$UYd zJ!f4iGX=1^&rl_Zt&Rb4JDtIV@q5m>-y`*YePKornf31ZuH7j~r&~5yB9(Z*SzEhr z)-RaQ4JMxui7bivQIm|b32~cK(sz0X2%%Fzll>Gb3@dJ=cdj*W?6*iG-&T`K1nXA_mJyzpw!YdP?@Iex9?8AiFt`Z+7lfo9li z%9)_^QLJ6_jL-9Zxz47#L3YN)N^m(Y$!KF#Y`em5MOIZewsa%JwJ#aXWNU2^=7RWs ztn$G0&PpJ2CypjblX+WSa?0wJwKaNEu6*UWjLS%i)R1JTF9q?#aY*0doPq+Nif?rwkR&J|tBztu#7R zK2UWBDquNg`?T$~wQuT!o)WX>W1Z%`c4cPUx2^q^bo+{=zmzso9S#6b&<%ONq!_ty zmYjAdT~q> z5idd&pCmz@QZ@Qz5rRR#r1^IeVVjcmV)YHpZ;vBx^E7{fVpO0FDLoLtulKt0&=%w0 z^JVcSdOLs!96?IQmg-@oHR`xUTZY~)-PkPYNPJTrN@WiL7C6rHaPa2}=fHV)2jZj* zZJdrPE44~|_Wy!uPG2?&Xovn(^Gvp!Ko8q_ENGU* z``@?UgJj(V&9M6@Pmo+0PLy`8oM!ljlF} z@6?I`1ztUtl9e0E_|fRVhs$|`-2B!Bz6)4PTvn;A{vt3M4#&?tFa6U{1I}lb1DS!h zpXB<1R~b}$8>kDaIVqPiX57u4sRF_&v4TFs1L6tyc#V9P>iJfRYBSR@MItazl)XdH zs!-ob{wZ*bC1P6iw|qQ0Y5W!W5~mEg_|EM+x2@y?|IE~tt}~|kSg!p8>2?9RaRd=$ z`A>r`3h)|aU1hQ*S|k-G(WV?6=ui73NKcNv`v(5isrj}7w3Pu^i0LL|ZypdS8TJt08!nGe>pLD-+Z{HGD%hER7+=?{ipX}Kv zw~^GcYiI46SotAnEXl@#GLn0hS`Xynb;e;tG^dC zAsP_tS^V$^?r;w^=uzGXjF4H)l5t)Rb88ax+H)Ch;BS?8$n9C!+Szgxz!x&~G~VaH zp0tVc)4iAZTorPCT*_dn|Nm&6|a_5ZPI#^)A>T1@JkyVNlBzd!_lH3HA}k zp247P-|rZ%h@UXx^wyh5zIIEbPczx>9URYw5AgICoC$IHD7|X`v23EVJ$wK#Z#ZJH za-6K0ua&?1M3l5vxXf^-_2&+;msb8x!j~7QRW8QPbWGBH=~Dy5uN32mQ89hEcJ0@N z*+9s#r(iR$a&Lat&uMKloY-hx&2n94JEnD@8r9~MWBoGDGf~B_=K^Jp&qVxA7iynC zHUtKr#TSO|Ipl775cAi9LJl%9NJ;Brkv@2n3u(YgW1{E83p@M5^vhifb@GgN72mV7 zUK)%{%3Pnomn$zUw+ETEbNX{N@A=Z~BeqK@eI-L(1K*`^l1Zh0635;xk8%Bd=3bzd z!Mb<(!e{|LrzRjIu}CT$*Msuuh5lSu{H5S^Cnj-U^ty!3pQ?c2J-Au|r&`}ZMpP$7ubzdQHY$P)m;UdmAc0yk$aimzuBBqd=)w>ozI zdG_ai1qRXPCCWH0@A*UXhj|5>lhF_9K3R`6xmZD<WpqK-5@>5Sm3QlcY?gxJkX5KIz0cZq4r;8s|HV$^Rc0WG&rC>QD zP>YCg*AG4OH=a6LEso8FO&OjBmj0{t6$f6=CXj4lD>bqOIZ?XmVi+e}$+LUE4hH>c z;?+5-dr^|tQ(L;mQEhz>ruFAgBv5&tYB580=!F0IePaW_S4_WI=$A+dQ#a>`keM$L z{BEK(bUr-vSn>YH>eh#7g>Q1@NXrM8n1$ZdJKNjVAlTP7FVHd9l1nT>EPvaeA<$=< zzP}oy^HT$Xq>nL6aA`H;xmNw`=2mz1sg=)T7lVGgF81KkPf`j|Zg=&2ZC5PjJHmk~ zNLY3h_Gq?KUlH5BmoSbEfX!bHAKq|N*bgm?iC}4Rdtz4u3kjG>T)M*@*gryFEb-cI zHr3i1luOeCEMv-s#-R+p55s`Gu>+GtbvIZULYA(5v0u?CIJMf1KaI%C5t8X@xJ^Ne0~m5Tl9;PKukOemm;zi`83TI+rGJuy?pHJyG}TT=b6H)XH?UR#lX$ zHc7+d{pHr$?3J2{g2OOZdY1r_k&Ta)dz>$fZFeL(K!) zYhT{p1X}Pf@T`cj=T?XDZ+R3G7$df?XaAZXuR) zeL1^Kb^OBkr_4%PQBuSL%f0N>o=W8SgM}5MiQSRv4@Sawib{Y5dm=`Cv$E~V&18x3 zZ{#2#oyi-i0{lx!Xf=ZgU?1jm4amdN*}49{>iqnfVLLsUG2FC1dcQ{ z8JX|;VaLMfWUSoRHb^fnP)Vv|>B%5g3d3R*Ez zP|)o8T&$8%SX`seUs^CX(t@6ua6yPQu| z?GXONIF$x4N+uIT zX|=el0wJRHUcU9G{Ip-Jen&&+;x41_9Te^4`dJ9EH8vNL&4WoBZ*~`%v=~4k5@+ey z0_=6zO#}3j;e;53s*VAb@*BC(rD*;)=N4vpio!liZGgU)Vd~e(aTtQCMU=HaL5F9=^4(it?DDCT<*SF7H@!C6H zMnAa$Bh!w7B*v0ko>2_HVwOF_j*~ME$xD+b2x96SBdjChqr|ex2eSm z_tg&;EBAN}3Y%@_H?=5QWACmdD$gv7&+s=d#VCSf6&$$-{ zES07qB&39_WT#;()ax6ek7d>tH7A#>tu?i%7muudQZ_RtN6&tj`r#+!17p<_mooNX zI1+zSJ4H_X3H!idf8CF(eEEh?iuMEcaz*0|Jk^3BHqwnE!dA~yPhW=}KQ|4x*FW$G zfa%{<@+q}-GV&z|IlSb+gsS9ww=RZm%8pl^=ipT#hV)w-w zvsNZIOC|+fk;;}X&B+Xj9gCcf9&`EtwQ~mJ#N-gWPN|77HKWIDIVZpy-V#G>qtH58 zecSr|P_mgO(&a7d!gpTLW_gP#m#gc?mUDh2GBb`VhaEzHQr+|IFbp}KPhYYZN$gVI5aQb8Qt7G;+b-uI29~(l ziPj)ybS@cjiM*Z_vTdW};J1wuOP8su-t{r}1nJW$_hr#*sTgUUHP{yS$_K~oETYl| zJn?6{uvBs9xyiDQ6(8HHqg3;sG+q{0yDjLi-N$PjvjgWf+{S!%r3Mf%1i+7rWEC+isLh_rHl4)8&1jCVBsj%)?a&|*?){9`v6M-B4xZu zl>@mq|NNo%rKPW%MZ0J0W+=6uM*%K^sDpJa8P3q{84Y6kF zM7}DXES_DdI^(nZOFs)<4rFo?%^)x5T^JC~RC6_0m=TK7F5b6Br10?!^4@y^52lia zWf7woUDO=Lmev^dq)}F1e`GXBuWpY1umWkLLlZSr$HC|K6a~o4PEb80cl6imODGg+ ztrdx3;QwHE^EvA!Yk}h%!i|U*tNQYD&xBS*YdY67Q80PBC=B@ z29{=99X-9!J++KvK~`A%&g-b_1g~2&X%S;A-5BbV>>5QLLo?NW2T##tEAO%pR3puj z-;+13Hp{QFT9Z9gObXaaWs2D7XRJ&l`H7q}6aza9|S{kAa4a98Sn&sj}qs0%N5a++Tn6z z5-n&YB~PY0l!X31s^wMivG_qQ;K-8kD|Ri4AXBvN=0E(2cZ^_W*e=Y_tx!{v%SkDA z%qfxcpETy~4|`cT()J{f@K*5)5;uovmFz~1uzWKHo~z+|4ibrA4R&cMtLwebZR6{< zE;brHtmB5gd>prDQAqMA4XW78hXl(!xz`i)H~>sMXG^ex3tFFnIy zU;R|)AS$W0rIU-wY`#J1GlbUPsz(Q1Bh&749Sz8A(5|>CN%OpUL#Ny7KxVsn$p?mK z1Zf==dJ>sbJtJ@}s6{EF?VrMEE!qImEL;tcTK2NVMr`EP8UIYN0?V)7JayJhmfCx1 zGppb4igBTkaLNsKWWWW8!_p{wg|1RK2_~{4g{X1{=CS`U=9|sz{R3%h!eH$R`Kyb- zur{egBW1=WBdYKn#3|N?TPk<%9(`tJ8AI5YMA8v%gX^LX$u9<;Jv&})uFEtSRU=9S#Q^_fTLR7Q!%t{LlVQr+FY`?^8~s%TcMt!nsqcXhNf^^M$ZR^caZiikHn zp=SGmXSMUmp{Oj-kh9?xkDrw=?X%?c{&yc;0bUiboGap@n4if$a`AB~z}-F0WcI%G zHd5vAryAv72syZVkhY=E>J}@d_l1riVTGx$iaaZY1i#r7_`bo~IoWW<+y<1+P2Ywt zk}|Iil40LZ5CE{AhTs^J09k;vdeIqIg#u5zOM$n-lcp!^Z?}*)*#LEB<3xQe&+7c*GH6oe~$4j8HfC9_%f{%(0gru zUBs(rF_?LT$ToK|AdOz23E<$(|5Rq~O%8G0xN&Ke0M<7n+!e{?g9 zSHFh*lZ0eC;!!f3#;D!)UEshPbx__QH)-(_f?D;S4?U`0ABiZo-G@_h112-9Z`K_l z;;+~Z+@|c>qP?2-r8Pc=41+<0BV0my(4h^RF-@Ks3e6?k7e`<0MY+b0Koxq&wKInQ zD1N|cjfZhb%nf~)QYOOCPc9uFZsb2(#~|O}_>T6}GadbB;9m+&%5W%-KNYHDs(sA~ zW`B(%rH+JD#wsH(eb7XURDV`l*kxq zbZ@@Qk@ecV&MV~jZKR}3PUBY+C^|TqB6Fpklfzxo# zD6MzZ`Wnp!erDk37I6oi=!d_+qhO(nH)Mw-}=+-$}vKSmpt8 zN|!L{0jXyAm01L7cAB;2itkz5j{rkw{r=!)<)DYI$Lm;^I^fmy@6*7$oEfCy^zCip zr{mSQ$!uR$%^mOu1UZqs-*CX;Q!);#=s!y==lo)#Ov+Kflo}`{6BrrAuLTLC_Ce^r zv9>(PXzhe5eSC`3p2{?O$t-RR$WYa^U$V^pc?~tR4dr$vXCkt>9Qpg&?wPhtbjCSy zg*XC4z(LKb%s%av7`IVT#4e@%?s$C%fm~Af{=%m@G((=)%ZwPfa=RMKzWyG)KGcKp za^8NoU1+pf7T4b=|iwCgZJ(Auu@DNWnK;|{91ei!!^A15C zEj&f{hzmRO3!1=FYy(Vi;5qrzoiJOcm}%MdQZ@W5t%?M9LbH+A%{Y}Cf*8r6Ab|>% zp<20 zwZy>LCClgHPq%gSI_oCF!7mk#C}b`FJVc7=gezCGqs7O|`wHaC%%j1PxSMr)+*<*X zldBjI0lDLJ1+z zR9g=m4r>X};C1#0oWS!1X81I%$~0oQ9U*)gV5u2xd+oGCx7()yc2D{|F#J*zU^=P7 zfoIY*z42P!8uhfhJ`~dN+-meKB0I4Qo)*U@fx+u(hRbgw?sJYTRoSHn$$E|R7^MQU z(R7MKh``Iknuwu8kdOQ&LrtKv8r8DG7IubC?8Li&ri;6?RN9v!J?>q5BBnnLupt50mnW&T*-qqnL8L{F3s z5Apq}7Z1*xmIM7HM&4GDE+=8KqfD5OIQi5ipr<5zvp$k#HJ(D_Cx4eR=#bQYY_@;O z5+P(wwlHctTHfLLvInea^G|iGHe9DccaPp+O{8Ie_B&Y$Xu(yVfPNuym&8+y5d>RH zhL>ssaQob2zm!CQbvdVG#@cNoTsS*A;q18V!`ucgi$>G--C-)7QEqVL2?ZcwAw2OV z`(ClxzI&6+ez2ydPD-7dhJgww#|bB__a zEDj6qUFFj*l4cSnL6|AoS(6h$liCPF7ixVwaA6}5OAZhew>1$DAgBp>okbB8LAn2? z9%7IXJE*hWl)a_c9|76Sg`}T80x+-Dw4kfkq4z5np9^FRLf_rmAYikm=TkYF@ z-QIGL*%G_P=u`Rj2oz-YsJqkYs7_ zv5E64&qi9pl%T`iq{8X-HB8xouj^I=y&0|T2S4Xr3xihGVJcpGop<{3Rpc&c)=|BR zr;LW-(X?&O^(lht`lz64dy`+d~}+=iIX8HPsmg79m8(;6PZ3fY<>R@&OmcRabHU zBkUGroK^ASF7@jpkgeaa1i%2ZJV;LZ1=+MA>mow=kO|}|kNdLjU}sy4J`mVh zZt@9^qy(c=J9&-h^idX5{gzBYhx$i}n#>#J5c|X&WJ}TG`bSP-QX;!af%`-zGtWVw z8AGEv0^YieQRLaYL*yFWg`5I#R<$tLHLha#nB7OUfpQYjnww=Mvc3cLRbDc?wGW@= zlG(7?ks@2DSvC5ha$$F8Av>(W0U+X2V4L8;gMAL3zQv(#@g-dvgTMnM+A<@wK_9(s zsx69NL;&Jfhr*fh{9o}qT+l5)Qhp-GGT)G^@Yppj`n z`@Qhr{A3F>$tv1o=O1?$u2(J~I`iZ388wUSGWsa=Iv(m>ZpE=fT*=n^_{w@`V)!qz zayJJ^QsqR%ZO$jc^b?1aaqKT?KWU`WFxFPzcE+k6IHG;{`jl2%9x15kn?KE%0jSOa zq*Vspo%i>b&hPtx(&_$ENXx0V$a8vYF(_5Pq=>xtTN}8k(yvYJIlyfK4sj)Hr^*Z) z6;veH>NUXZF)>(0sQc&?Rm8yMH1R0lO|>{>&Kjx9(8&9=LmU~PVULKckmmK@>>HP#oJn+Xn)3PM7oBcu*4q>Oi9%2+f0E`A%xqh zV-a*)!Iwul3@#$YK;xBniMZu6#ZWL&8aFxKgMU2ls}kgZJtvuMm*2#mnemivsXM!- z?$U9yGN9wWI8fGW%Qdsw_nv_)d5eX&h}|unk>Dfocl;nVjUF4#&Uc5bimsOLI*=Ce#i@?)r*Er zPJFsAb+URt|Gh6mbKuB_;USo@re~#(iw%T=sUFvbIE(<(_a?Li&1AO^)b;0nSp}2d zp3lR}&m}1O8p{6+-qEyaN{WoFtd|JYZCBFwuY`Rq$Uc#D=%2(^BL6%J==_@kooA| zUBjXYAXOvXRH+GRPku(TIii>8v=k3 zVOy4^R#1r1RP}k=n1r#W_)Jo$`O{sJ{zw{FvDqENkl_N5ch(xUCb)bPwDvx-7D~2| zd+(FLD=!rSE^ef!JI8HTVd89URa?Jyy$#9hR{K&t+}6md)ePWi7O9YhyM*1|@)eqc zuZ&wF(tYH}bCd`{?u7rx)^*2I{r!KWP%9@J|tyB5*>5l{Y2w`mkMfz_m7He!xW#P zA*^jo+lh0!o`ps`&#W6nf~kz3IdS6BNI@g{30C}y{W8sUFMZxM`Gna){7PoD^h6Fjd9Aa6lZ(Mxqo!sh{n>28ORxXa!0Wa>e3F}F`Lsg}NDu6H?8 z-z>p$nd}qfKrPo_?fIRk!i1gb4RtP!Chy6%LAAzbEpDA6Ont;x3>W~Iw?p-HkEAqa%G+{)X9rrRxUELV0x3lZ~e}da~0v zByJZOt2Ps_auaEC1uocDeMRSHzhNsRb>J|jCVu<`i&ekndjBABs!epdT0NPzn`T&Z z7VzI;RM|Azq>o@(PcoiFR(hK!c@SA&3gLEl z_kovthXsU7mJGW7ZHm!nM#38X+f%F;z&|O|4R10Xtc2j(M zt@=CKWMlM@uxHhEg5z`3on*Zd>vSi;$?t_`dz?9Ug8^s9GG1G?9a0Ddqw+q#2}q+-K4E%16@%|5hOmp zMhICTe&V~2g2^a(Se|@?kj*kh1_OIlr&HGKIoK$$UbG9jXywdX-!Ac1m-0{MMAK_O z`$)K4u;=VwM-P6^n~vIjygs~oK9A0p_n^(&>|-Mb#?ZCdu`k={$LxEb{#D%AJC51B zKGsjN?zARNO!3grgb!-hCE+hrk~=+@-8Aim;5aqggKl_Fsd)|eKVA+xndN8|gB$31;BqS&vJ%A{2X_g1ZNIiZ#TrX+E8#i272~Ulu~GK;q-<6|Aov-#UNhT=2bN zD3!vr6?qxU39<%=Y;7#Q6!D@nxgg8!u3@5ES^~;CwttdF_QqC<}a2^ z+}`~!gygT$_?9Y*X4Kp$i#^7Rh5MC$6jb$|B7ru7GRO8fU%O`rf9v2&qN`kcPO^-9 z%71>%ayaBKjGxJ6%4~;G^oJpORaZ8kkA zD8|42JzAI?DljB_-N#asPRXO66CVEBn zqDTmR>dk@rygwti%_Z%vi4`OtIJer@BVtj_2N-JMwE`y-v~*%SJGI3;-%GWK@b{_c z7blMWoc;VQ@(!`h_uCF@&w0P0-8mNKbotjdPKsAWS1U2Baxf>gcE93Q?JU2a#6~)$ zXYpDKhadTRP3X|a_pM%hH8a>63!l zMm7bLUh8Jl3#lf>Ju_$+=YkKst3@<#x zKaJej%!-hgcXGsxH+&jN!LQs^k5+*;Mc5_7Hfw|Q-`QYnsvfmI=9&9c!%aAkeTW2L zTG9lr*=)^IrJleV0l)>{B^<|^Vaizop#9v!KluRrFy`!+1>{9sQOKbC+1m4|)I_{PC8LbUV5THK}*ArAds>aHqlP$AK1W z8mQ#FdB^|H@k6W7n_Njyjl6F{S%gtsD(d4<8oyAD(di62$|YMH-Ag-UC<%cf2C`7Y#de9)W>lf)d}A)oCJY&sLLKde zi5QW`=QT$EYHjT5m=rZ@EWsAQ(qEgOHuaxISR!AzH~H4h=j}Af0RCrTaDVSAgB+??R)B42+)RogMl+9Zf^W92M zms`qcQr_UqntmCz@{azx_Urt2-0>dYI5ID^2zQAUH2YV^f}<&}+PIo#8!n0K6uPlh43iZ}V5ozaL` zFDiUn6kX)&Fq9rtuxV8gDlHrR&hqW!t{_zxBW&)ew`Fw%zw2b_bi+n9Uw(bC>R567 z^nH#{<|V#d`0V4@W8S94qAq6yV)c1w+0@F5>(ong(!-?Bcl!H(?%WK366J;ZZ90*6 zs2`3mL5Ii;fvaYfu2+RBC|knx$6m{6-(Gf~Ae~N@iN5$&e_E5OHp>FDSx>xqtGGR~ zX#POxmC|yR`5wreo$8Gop`|Z1Ziqs$3r-SbQVjQ8X~tQ?Ev~Xe=Z~wp6bxFdXeH8W{!^r*R2-#X+;Y+FH z<{p2idcXGtbe}MFFU1F4Og}PSt|ntiW;|dkWS0>ND;ru$)9BT)zbl%kp14o&PzZ$(LKODe#EnJohPgT z8*!AZ74TI%qP)%+ytq0HhJ`%jT#3Nh?(;}~uZS4D(~{n&B)Zlg2+CqMQT&}NlP}x1 zM$hQC@vFW``*EU1X7ugwjZ>ZOj;>#EcU}_olp-*3g3};;o#T>moa2+L+if%l(huWD z<{pWdPyPwrLHG(GPhyN|q!&EHIHj|wP7|Mt4zF11F`3ITt$X+k9Zv(iYS8}^3<5*s zQuA6RDX#s27?()q`#cGc*c!FZMWwL&Kz`N0d6&7Q%vkg*!{e;@$<)gN<6Hm8YR^ zuqk0uLUOPV_WSwgNhhHcTT5I&uI+;tc5Ma8g+6t<=8`q_)y6Fiwi7B}}EQyR`89&c-jR1k~} zUy`pr1)4Nqcelj^VlDPC&P=9;eid8^C%=kUU_Tyd6ntn^-vzE17(x!?<=?MiM5w|?_ z07Tf+bGhYVX@+fphuP@~=8~;(8n#oIwmDwTy{Gh?{hu-qn0Ue{0 zT&^`-oFfl5+?4HZzTNTxW~dz@T@jPBa18-lhrxZWPrR3d|JnMMxF9n zQRbie!SD5h{CSH~=r95Tw`)VCAW7`0R958J=#wioDh3SgFKdMH-6$5aiNGE^ugZfM z3<>XPk|FPILfflGJ87S?{2jwURlS`HL4Rgjvikrr!E;w8A+3^$;ZI<(VyclVN^^c+ zV!kTWmBzUEg*{q|$fiR^c19t77HRdS3Z0~yBO~?gNG3DnS-CL`tC1Vf%~!;25` zKS-ix{)`heV6FSxigxn{=BKg|_|YiQT#??sNy+fS>z!e(8)y8xwW~jc9AmdZ4&mIfdlONLp;l zRgTsd8k}^b<^NAEpGB_Q?l1!r)b3Hkz4n&kRPeeqLd%r~7cuC9K9d z;^Rgt?wKA493cphZGlF|kL|S8p6!3{ZKAONiG+*poeSZW)LI0=A5Sw#VsDHe3`Z!r zTGyL|HC;;-%53!1ta!MLt1Z{%T}sUF3gT;@>Uu!4Ujtc-_2wp-U+!?8+ciC()91at z9cHeokV`V}VwXK*MLlQV%`3-ZB%$~ffY!4b6!(giq|2Rudu7;V<>&WaKUUVmJ-20< zPj73E$RXP5fWQtrFS)$}$6+7J&A$tYdp`l^dQ&yD#pf?|*7!(3+t8NAY}0oqdt*l2 zqtx?OdhD^ktf^f2)(w`$O3hBWN+sG}&dMs^yBk5Gu zC7mU6K0{eC*4&`V(9&=z!b&mytos&E(>f^p@%NBJvW)dl&619eomb7vdtL{}GJqP8 zSATp(bSvikN4@owsuga}x3n37%yNumhTQO(#_m7t@V!EUtn9POc^pYB^!PrP1%82>DbTHuapH zMTWY%@|%s?mgKe7mp*VaG&zbm^Bx9fzRq^$=7hf@qkj_$w}+3Vl>7D>V4s^ga3ZOT z#%uB;Z=F2MUau@GX=j`s1Mb$4E#1|E&nKnq$qg8b{Gf|Ms*8cBjf_=)n|3S{ zH(jwV@(la>!6+%y{S({$jK;_q<0f*>CGXP8jFdW9VxoMcZrI;)!?d;fy^T%h$~al3KOzu^%-&!fG< zP$ct{rZ4ubfw=TFKawYlj?CnRF_f;Q2IK@tM3A*#mAu)eHEwD0^R?7E_&XWf%JPi`&{81NIMg4+R ztQoh$;9ril_Uh!ZkQq!tmgEz&s9$B?8#2!f{`Y|R^g9nI|sT~?0@enu=|MG0S>i>C=Foijh=`0-)e zkk;(9X2NXFi{xO}iMt>Fb0Z6b9>d1i4eKqF467E&nu^d4?#^nx_Nj=<3ixNC!=CU= z58TUBg2wl$)n}|q)G>k7W<^mSZ~aw*Vj?|GV)0!B00j3ACUJ*is7Zb6)IYq{q{Q~z zetwmMRe*lGPProI5WSBR`LLlN^3*gAz}&2!p+$wrFrudjx_GwqLB}F=CB6$>n};3Tnqw}Cp7xxvsQRPHSEEBJ4#+;K3043Oj7 z8zQ|Gga-ki9?DFyKC3*h)v1vD(l`S5Xyir--~)sUT#L=Ry%LD}dp$&YPdwHert-63 zVYJ8ek6Q)bI{cU{!z%8u&!y9J(ZO*%%$+6>-g_XpH_EEZRj=aliI1X_Gn!*YDlAqN z@SMmz7G1IMl2v>5q-Li3w7F?U5OM+)32Nurr~m#|P(+*T(9^-(8WQO=@2d)Mq;E9U zt4!Ls!Cc)-{!E5>f)Qq5-6_x~`V~HB(YcmvZybRQs1Ds#``9anf_&;Jh02E0@~9;5 z-`_oVnUqOtb54*y*Y4@)Y2-0$6{XMFELRcPV4!1qB<81c1WR6`=wstNDzWiHOwE1# zvtP}(&e+KT-HFv_9lH4DPoYiHbcpD_NbV)M!5gN*eZMKCL_-q}C0QQHU`bt(%XP)G zoqg?i%e90ZrfoWiV%VzU=H3kcPBEK7)71vnyv4ij>i^dc$$A&##6zOOp8xFvfPsMk zp{j>yl|*&zE+mt+?tKRf-or}P60ZR+v%r9c9IJll%WF(%QIZsN1S&PU)(>_HW< zTn7KB*_~`q{U;y(OODf?n#%6`1prmj`?I_xU(ei+`)9q`dE|6SVA^cYC$lh*xc4B* z+k9}?@8}7rem1j;VSR(2_1C$l&(Cy!Gc^-S?(e@-qiPsu8?QpzB%(C`tS$L9K+(d4 zY&EIuf<$!pv}<jcN3&(PnulsCD} zcFxUCCrFX}COo?5{>N%J?x^AmR>vd0N*E>x`;}#gTnuFJ=9Dt6KG8$wmGMt>Jj?ND zc--N13|#J?Rdz$kl%Jt7d1@Oy=X&xJX(y(hOpCEz@|bZkR-x!$xp)(25ZKQk2RUx; zK;x_hn*cvRiK|gu;|q6B2^Ad?b;Ye#{oy(R35@Z_uudCDp;=~y)!SfucGDpC=%rkl z`-TA%EACwf1C8;JmBe8k?At}|_L|r`f5`YZO%{g@rSoE>*{EJl>H^NKpQdVs6iAmjGDX@;+uZIiBEUhW!S zSLUsD=~nbU=2fC}W@5EIjbfmba3lv8w}_o48g zntjvW>WBLu6RZ*CX&A(m!ox!q`$5<`u4P|_$w9UI~!^l+$*^qC@f?GGQ+gE`*p}=*^boKS88xh#v z{+!-VyN&rhW6XF(V={P?Tr?hvvh%Xj=%sxfhc^lGsSH$|;{j-OiA<5|zZq5!o z(D3X_r^A$!AO~{klfuzw$a|rjCwNe0z)~QYkv5Ennu<1>KvS)ji)0~DnUU20aWSk3 z>j%opzt`(UR@RqSdv(e7uvLZ`L|B#Q_!cw8i0iV|WJyL}(p()6;wvIO?h5jr@#y{h zX2XY|>Aklm(Be7=Ox2lH2^2$xqOi=ke*>V2puXc8Rzm{-_ z@Z{}k<+av=b$=4p*UQk$Clb|p$)WgqSO9W@_pF}pny~9#8R$CbXW%|kkJ$orH{3N~ zpYd=`SkMP)Q!@SN^s|@!1!ah7ZYwloy(YBJnYuYtH#HKqdfNT#h4~JHOsCrBQ*2BV z7H(tQ%KM}o;)|3Zo$T5|PsTfZ)hk=1_)O zQtV@IwRS2@ilEPLp!;rMkCXLoIn30ir=$u$8(4FZ`#@Ccfk|IkY-TvRfKIm7$FPRu z>}T`5oE9Lh!_t-=H%$l0YEWyv09{b*yJ7iAJ3(uqZ~9vS_pWvLOi(S#G}ER&!tpi< z42?eIdfVM4_1!jhU2^&OBoAaue%jV}GNSn{i#q@fYc68 z2N5b+z?CIEA85RFZ*>_)SngQZ`nqsi)uoDIN2If)1#Z(`%5Lk{j{M->KgQOv?m4vn zT6;3K)x&&s^h9M*NfSV--q|Wc69Mm?FP^q|=or|^4`cy(V402`bypw;>nhe%TScLi z@jqEf1|3NK%U-(`>j3qP4kX6WsP-vG)#N1xEn(g{q8?B?34EXOEpM4gZsJkMNxWF+r{@}+>n2;o3eb6Z_dVFd`wv?ZFTh;i$DkUAc>B|z z)vU5CT4!45u&o{|=0Jww>va*q$s+Z+YJp#wA?}gx)w#bLyDmMnEZsp=`$c)xS$f33Uk?xUc$ZL^bj`WtV%Kf< ztzn9EB8+v_(lY`#2^3&wfZ8d1V1g+A+VK?p*{wn+WTe+B>*MxJ*7gURr^TH3H>#)N z7FD+!0K6XHD|Lk2z6Z{@o?X7r@3*$SPS_sm*ipF!gRW5-63-ThsLXaCLO(O3U%9Gp zmHb(M5EJFtuLBfOXFM0&jXg7{YwP#503_L0vnzcrInC*=`p|m3k>Ob+Lg+m?s!)VY zlnzmLc;@kpx#v~=$~CJ>?Q-0}f=qv9;whX&^H5so+Q?PHaIt0e#j@D$6!&|$->+)d zoB#|eZ~)`*`H~yzs4x=};$_{I{mAE3a$ur=jbx-`U^{OXRAx{;d|7A zk@*p~Tmrmw>o&+6t)%W-sh_cSaLB6VQk@oRqIOiJy#$@z>23S`I)g6t{X~PbC@Ql^ zG5M(zfs1?LCAUhOEZ(WEjx|jv;5+Z2-Wi}iU){=VC_dk)<#cL)dDeVBQ{cd^_TzTc zoDaYj53DvICUg^kZE8;?xZ8aEvQ9+9W7!E~h}%S|(GSMhWaAHB^HW}#*ypH?=Pwx6 zWVS|Vby{7&I(*=kS$f`qz^W3`+!{3#G`et^&gJPzG>oHO06Jdmwo%P)LTR=O!u)RoW(3TzeEIc)6#sSiR=3W)nKh&3xUL`b&WLqq z9m3kscHay@Gb&0W5WVdQ+W0KOYUPI=xcc^Y81_a7Y`cz7@VB@@}s*Bjg=ccNeM+A`C7|7 zpQ8H3#34jklHV`*wAZiN^Gs37>_p;|ln=!8qlVa34r{0da$B^&d5o~eJD#c@WhF80 zIG(_N`~Y%Wt1b0Pv(e7Qf(|23dE98sh*vO!L01FVd$WyR#n7GQ0>#qAiz#0>*o{Rw-uM` zahGyA;D%IEg4}2Y5x}oXa0f#|j1idVR=T%A>j1~Q60|9bB28JKbwMBafWoGsKFlKs zIGr8k^aWK9Y&$Ab257BoLFnxu?no3W_0z6x;Sj7q-{zR}{< z24t4)v4npK0-g^!Q&}qJ9kbGb$KKA^zAtQbsQ2zE$f+}g(TFe1Dgx*5Kz`ab2|C$U zcwn2Fhoryz!43lP_p1U)I?b^F`_po!nhzCH&9cLUR4JO5)>mQ=7J?8ohK4kOJ>A+m zFD-D3l2i|w%VC*wOI2dx(Z$j$-d0jkBQkJa#;B1@AidO>X}=J9QDv6cZ32`%mw6kv zfvP(PS>-E>%1+dZma*#{0k;opvsbE&a;V$<ls&Q>c(keXfHx%CKqv>P*b^aVl_HKBSH1(@~zQ)IhZg6H?TR25>VMHMBoa$QcuZ@ zP^rE@&eHHyl!HPhUu&#=}0*EL^nOXgo;g8oO#)dWMX ztCm2_fs*sA@0ppx+fhf%;SEU=M@+Gesljj2KT8}ih0&)Y&`M;6Pl{rnNgg|UDzIsp zR8hii*#O0J`jIn7oE6M(M7qPgl#v4WC0$pdHZqr}(ydQrJrhw0!G8DJDw0~RgO=qL zD363*x*dV*HJH*h9lF^0huhGp2jgMu*3emnlpHn@14kAAfvArgxBX5S2FPmBC1!4f8P zNJM< za84MzhMonB7exMoD#d*VYElwUqNWv2kwZZFc#6Gkoy`zNG$yQ=ow5#*X}t$#F7M6sB%D(2+XoXa)rL z5&PE1tggAVQfkEZ1?>L4ynN;Fp%wqo@<@D$66rErw9mx7n4j3Q<_eB4gBh<+jtENJ z_ES7%)@{rogc^|-31QzFnuHc!$~eMRX<8SewIjw)yg0E$n+G%A;6FBRKbo18J%Gjh zk!jC84fmRNK&8(=^1u88e>LQaJ466%`uK&<#JL))C7h_q^2Mqnv)lG1Al!1gOZ&tQ zm~>;fzvNxn4I&S+v@!L0L(?s95nIn${n;fMz!pu1=_s*KJJ{+Pfcb)8?0|tfP-+*t zEJqdYv5u%81MHx2X_phCDbwf7h9(MM^Its ze$2Kb>&YQd&=y>~P^}S+6%?9Rl z7GsBriGtZmBEgg$r;AU!kNFhq$VZa&&BgnTpM3B?#S!9jIYpDO>TO1zoRwuH?P;zK z$$=~=zO2ZWs4=e=X!N#uu$ufp2CRI5+{K1wY3>@c9xrvlW|I_r+c9)ttpG`~&g?|Z z04k{iB3PT(;6;d2j{D1HFsL=_cPv)COX!YYtjoLC?l;A0MZ>tAv{INRlPsh@&v$&g z*3HKDVaDIjne9d9Ij##N)0RcqQ9tk-buc3odA+5#X-sZ;#j22I3ZN9zqmIsFBbJEA zIqm?CUsE_Ei^f1gX+)!&;!i5QfTixH}kA?nVTJw>ePZOzYG`LIIL@?5kHeDMk z;R5yZEf}?3BQVCy`UWrcm8Mb~UehT&vl*=Oq*Q6A<%7!G^MO?;nAe>tt$(-?iIi_v z#G^)s)I1GR*g8cF8xcS1d=3HBt^`nRQ_%TR@n4rq;$toQ8ZgluXCWgKkI`BY8rr81 z!%+ZTWS68^o~={KnNST&^;vZ|RCMN*}`bg?zLq$7PI?L-fq1c&MVL z)!*}u^={?s)TZ`lc*PVT$T4Tx841|IvqX?J9=drTu|nsld})TI#I1P3tO)f&u|_d~ z8YE1LAA-;RdwM}ll7X3!s$%$&_0L7VS7Cg`9d`61kpncS`?Z!+!N*ga#-Ht6U{AbS zy*Zm_kSNEFa46s-`N=YdX?w!*d1wl1GstBbG77Yogv z9qnALETc*(_!mD7s(hX2Dfs8;UjCNk`4g+>QDz4OC7s^v<`^fZk62o?t;*(UXz=X@ z{FZjiYF0ahQRfgsc)43M-$iRFXLhTi5 zfDA<9&l9DM4tDMQE=+G1>!=V#O>=NNgoa?FXVE2*M~3V3DVKVG6MJ{vIg*@4m%KE) zrzLRq{a>|a;>V0Y5>`tKl&2HiQtX8{8hyaa=G+D0!N6_^GGZHa!IO8`nU8_c^!?KWZn%Q46npPzbnq;jnfu{i;}I|e zEW^RdBB|2)!WO^Tz$Zp3o7F>IYEe@*jCfs&FO?y<6kAx!U(y7`{LqAwY!&M-|K>;n z=cB)2w!8U=uPk)tE1;VK7B(^>UM=@|%gx?vJoN5hs7Dv8H-rp~@5*0G9omB>`fyUd z31i@@GMfp+H1cMO4w=>KUnffi(~BO=5x6GPmib?J5(ldgH3CqNTjEcGo>oNa`ieNr z>6Vn>d7INQ!zhnV@zkvrQ&l%cyxL-5a)PTFw}eOOJ{fW^OETdhtA7LNGa5hn|MM4F zGEb^(Qs=LirhY^z*I|er?ku)oC`t&K3}6>o;zvRAC8_AA`CV3#-3GhKYjPK3L3gYP zwgH!sf9V;}+uTai0X;wFDA{e*raP{OgjH*jR{TLa*rnk^%XMi^)Xz-;DcM+*uT^r3 z8?lvqIX&TBq@ZbNF1z4i5BT|30Zev5K$5Y}c7bHj9_VG?3xhPQS4Xo{>R#Qy3gzJM ztY^`ah7rGhlC#EoB%Me`Po_+kNIpMy2Khl3>wnAy8}MXbNCe3ICq;MZ)QQ$Tx5(oJ7hf|PvKcwlKiB|jICrY@>;sB)Zqb?KKRx5?WTe?*s3`O=DW>XwKt z08iyv$yaJL*uH+@W#}0;GSq-S-51S+N7@y4v0=KBqKdAYPrzBRv2gsRzzpmfz z3`oe|*9;GqhrVo7{$b7kF^`bna7Zw!=HiPN@sD1%#(m5a+82}!L)(+PpiFhtWq^Lf zn1u};g59}zQ2!wVxqY1RbU};!^RYU<9Jnj=J3*_jx!jY;y>l2^{(3p<>PV>O_JuxB z)d+ZgxyERUa1e~R>5?Dq#T0fFD#+|f9qqL_vMtG)7iggaRDFMmO7AB)SG{YjRfdRT z3Ta@+n-5tn=SRxV7(#`RHzDWl%|2V}g1Iu8$uge!U9g%H^nuW3N>beYik3=JzAJ18 zpEm2C*DxZ~#|#1V{iOk+Mmf0js8K)_`*M|@YhR&Ob#&eiiE4(Rj11|8L4B>PM3M3&V@>o{dvp0YaK!8n;H%sAid%}b+B zJrT==&0`x8w(QSXGW9R5&uN-Jj>Att0M=4li=p2fHRM|D07%}F}>mf`{FJeF+x(U(7J zU;BPKl`8wz^NGhVdE%?8CPmgZmc~~7cpJSv&IHT+7FG)b)x;QFkXg498hV_Z$TWj1 z=pDKaU?NQhaC$kfh;5bEl3K0HOgyF2Wd^S>w=0`pw+m$|$}9x7p;GnygE9elT{dB= zPbNu&MR;2#y5u|`P4{C(LEvB}c-KV)67uLqbs{vUhJq&F!K%stJN=%r%uIXDdQ z+H}%4hKLM~s>n%uarI4VTE;SOH_++bt*AE9naum1RuJJl46%!4tqBD(QeZO6WKoPe zpjyu7atna_qKe(AM=}o2Dn`u>O5u%OU&N0B%gH%eD4?1bbeHG`kmuE8(iI3PxQmZ* z;f5Csdo61npW(&@mNRD9OSaS;XtBwAI2)q40x8?2>~x(jyZxTv?n*$Iz)1=v$83t8 zAu=JWU+F_Rl~P(D2U2X+5A{#91XdqO+SB+W>U5$fuQ49?Bk_5{_QEyCH?1;F3aFwL zC{>5@w5y#AY8yH~8R%K`&6sh+{-trhEBAULb4;B%OH$IYdVv7I5IHI)Nka`%UsI@~7y6pB$p*ctd@- zk%O0)`G%}{+`P;>+s~it6H{qx<>fYxzluLNbquyDbQ+m{97%KSDD4`R41Iq2tF}$5 z)`Y_sewY9Yt=oa>Rwfo)4!eCDdu>{7~Meq-Id*FM*rl?5|gDMDv6eL@45}FBUZy5ao7>KNl2E zvR)I@hLRZQpd2su|EPcvE3>{6o3+x)XRE*YfFVVXpq10LWe7E9H_Oyq=j}7dE35_P zpvj3wjnGJpl=G;VMfdC}sqLL3;N^bCk%%P|rM(KuAQovRGQ70XH?MofTKYLV%KW%a z{nNiUbG>Y0-#Uby$dbE-o>U5=S|RDmD?0YmxiXC{>&*SSqi31V35snl@dtbpx>q^g z>cH8W`Mg@-sM+GF1|`H_^e*8*+09exSc?g9FBm2N@~JXNYi6p$IqTBU<&au=IIFdE3<~_c&Q1r_;m8|v%HvQ>Od(!>XpNob@x9av`t0~}#r=`X|35*~YpyOBG1Q_1$ zPKlog6~@b*bMf%-NKgyQ*LZYOqCmh)eIVs8KFu5(X#(wuSThVi?^}V_Q^PS(#OQmN z7p4^>-d;dIpY*jrG{7qErJPvsV$uruU;4G8o4vgc`X7?JgkCmzy}qSy{i&tpKgD9O z6F}^nnuD{b`kv@zYO$3iLFspQ-}urNb-#W4*5RIPZwm9Hdg;4{`%nJ(${R+UV|kFR zKF-CdiT(0>TUPeO8Cp@)rW^%g1|pAu|1yMzOm#Ff1?zixPZa0y$rb0o$v(dmu(yEs zXg`05edar(e%b|e%RLdOPYQ5uHsvNT%z!KGAX`^P-?jVe4F!3@K0LA z<`q#_tUdjP{CM<)dnuMlu2`=(I@ez~`$(lsf@jxp8o|;5x}c1GAdJrvJp^@ka?{c^ z!LfEY<>=Vihk>9Rl5=cWx<%-K>A>ZJfB2ik*K}QxByoU9*=%aMcu!VF*w*wq%gh%r zu)bjXTfE+p;|D_47IW_IzEGdnTxbXSR<7pwpuk1jdfJ#D4PtsT3#=E&@+t8yYcIGe z6y=KP@xMmJ*y?Vg>KgbHIdJdeBf}Om%9?650fTJkYqCSh?S)!B%US$xwJ8wFJeJ#{ zc%JTp=s^2_{p3L~3oZ56F1mg{b_Yj$<+^uh1u=nv7nhx4PY_e<(Vqx@viUQFP8h74 z(RlVw*9X{j@!XStssya`0o_7*xT6AVieqWwNwD4@(P^;vjAvlJE>aY0cj=HKrN-9~ zhAnK}H}q50%z2k|*~UY%)i#kYjAVHWo_TCrXcBito5_8T><+A{fS0>wJgbuEB~;6NotvP zwZGL)Qt>8ba%Nj&ZmsfAC(-$I)A$(R9PmQbOYONEiiH9d0wM6;(b=1bT~E9~^+T-P zA&(g=Z~qh9C|#~HP@EVwF2iy3$M+z{z2p3&Hk|7P#26jU3Ja6AQAu#M{*QVfP6lEqf@S@K)O^@{o%>)Jm z4Mp!5^cC4h2&G%<8Ce60`6MzjwOCoK|Gx62z|qbY%}zsXU!x+nEPwp708s9cx6Etg zs2Eqn>L^>;=y#_5^MHYr!tT^-E7R?7=?`F4Qg_4XJ(&I*snI9U< zi3!+#pn0MR*O_Lq#S6byNystZBxSX}{I6KU4kSXYqO)+nlga6pO*Y>#usuW=oXmQ>XAv)F9<)r;YOv3xM& zT;~3R`FNgJdqL`Yi2%f$v|7#EYy5v{6=Aln59lVaY|_2fJEFt$XX=L1QTGIr|71E=6es&NwaAsN{!-M2nSlblX=?pVy6fy-vc97H#*5JM)O;vXk67FB zfiu(}sW0dgDZn-%E=?uSMHkevi`)4!6ao)yhC0_je$}IG47&VI;8tDD7XiFyjk-;v zjRS*@c=PX}aOb{V^`@O-vsFD9mVFE;)L@QG6jk|N1*r? z;e)HOw&rrmx=4~b!}h3q%tlv#Z|In@SnZBzYcqmRC_9fY>z%WrnwvhDLL zYRms*8u=I#k{BcpXI8vC^=WKqaM}DNh1e&OFxD;6#Y0w+$9JL%^c!-gC1c(GE_+-3 z+SifXPxJEHp4%$&jVU1dq@!2LtXmzXlI_qLuPVMmpR&bk?^QUu<;Vg^RY3sX=RtXc zIskFA+TrRYvEfgdjvo&z;(k~2f`I*A4#10+8b>XkHdf5CQBQExrpJqfJn7Dg@37@> z<<1w#<2k$C6!Wb^8mLKuVG#OcrYHMaFVZ?EAdF5v&IOxW!X;CBVK5O{pzP7bROgg8 zdTR%grt6Ou9g^S?@9G2726R%Qw+qVPb`#KKg!-UkJK*g5c%DIvRbO$f3QAn|=teIl zopRWmh}Jn_m478!7X1@sZnyX7XhFX3LT-(iQ&EC=Ux1l7Fc4igh-vO+^kL#P0QQH&cF^kA>dy8aedQ}z>Nth5Q7d`CJ zs;-GWP(Px&?CK3)BEH8_lWt-F2#uu~5wKnBgZXu^?WIrO7Jz{3T5-wF)NOzwFxD%_ zL}q2wIPK^>ey?@xUD+8pu8G3_m`MSH?~5@@{S$)DX`pTD{Q5+p((n$#jo1zRsK;XM zQp0DYJ}{$Q_(6uS46D2t8ouM3h1tkje4Z}l&&@}VZ7gU+O(1=6gbl|*9zvU_F8k}1*+kBMGI2awW}nVX~g-J>ApWBeZZFo#Jc zxa-2WsW^0fC73UpWIyD}yy6{;$)?<<`)UVc!)zmh4}E05;_mA7`D`7E}2cQBdHHkEKfV|iA-nZdMb!k3!lNYe^L)YwS=CxJ4hQ+@a9OO>_@Jai%- zb=WrEj7{!aG5++d^4J6JFjJm71F>f9rgx69S>?&K#7a@jc^~x_lKMF}(8_Y3@y{aS zh&qUE;>wk1AuQ++yYtQc0B7{U}m$S3<$zH!gv4->;ACLxXH}F`9x^riGzE{m1jfvfuWBHEAO~F9A7O`AQmYc zB^^1)uc_53dgdt;uvvsVo;5li@{Uhg``@?`8|DM!=f%@9fudno-}W?W+tLw?8UKg{ zKe{wyj*68IBMn`zB;Xw@valbrB-a<^Cqq5P&vsk}s)i+OGfA&KeiQ?sqh|E#T47~5 z(+a)6#1a5>rnSRBlU$n!;`%UaFeTa1<^w@bBh!+ZUE zb#-+whV$uqbu)F0`*p-8V?%rM(^u*0-E;4u(|dmI4=qsic6aslD(<3jMHm+jwEUWU zzU_jVboEE6^?@T?r;rF&xMI74eE=~kJ7Uv+T!ebp!3YQ zE=S}~zVkL+6SP`v>n$-GNFqJnx38^HVoRhaW%}h{kYbU*$Dry^y^91b{orMBFmal@ zzb&N_J`L{RjY+4xw!IO^vM_QNFXoP+b|WDEyEFUxp-;aR0J6=LR?L=uo!#64lLC|z z;`RVgPtK|syIk>>sc`B07y$n~Se?9H*6kczzM6j^r768q9c0_wkM|XS*@q?Tz5?Y@kfL}g)Gm$P^P^e+Asd~ls=;q97_CfoP^Q8_y z7<|Y;ZZK(~eahm{9I|3!w+r=yO8Bf%RCUXom*y``m*oLHo; zoLk=&zy<2NZfoT`m`4i*i2hUbJdl*N7Pxj!s7m^fpzx2rE$1O20!FAyY~eebIpwu? z8xg5e$(aWdIjXKTQd<;QL|rK`ggl`klGG2Ey*1;2Db~V5X=r_Hq zgM7d;aJ%ScqO2!=3bAdmT8zRKuR2lUV;sX+7==oHCY6Lzk`UvKK>Asw-;CgImq`y@ zD7g6k^HW31X=|1cgtpW)91I<1k-d4Rd_)#@_taNUzouN*sGWd;`7&u{YZ>xhvEPzl z1FF`K>3{}16znQE^)3I$)|JOY*>+*eRHGC_St4TzDZ4_#*rGxzE%r4e21B-(VeDm@ zHpUjEP}#R3`<9Wh6j^7i6SD8?pzj$iZ@u69-}5W!e(vYo=bZbT>s(jQ***Zc061Fe z_u;UMy2Hc1U_Jofd{;2%>V2!pv3?md4heE2eHYkDXp*6sxh=@M)99ZHOp8-%JOb6o zUulDSZzGz3Dcwz;0C@opg0`s4N?y4bzY?PiDKB$2(%F}+(CiO} zG66iLcAZRFj6Att!~zctE{Y9a+iq^#(h2sT|Mu;T+0ivQ6j^Sv=CG`-kczUhFM!e; ze?RI1*~ij-x*NbjO~ow%Smoj`GK^F-qvOifIIJoz>rjVK%+ax$bt)St_V^Ap+j*|& zoxk}+V&loA>D>KWE0n75lodyR8``gyZ_RLLk(3GP8b#22oRTacY^0iY;9~+Ols9RCLeG-@^l|?vuA~VvHvAN4K&_Lm~1_&!ge@v z`uC{Aywbw_AvI7xO}pm%ch|i7revDLxuaKA>B`jo9Yoh28Pq-igmbr25A10_&BAdG zttc_nTCi;0TVAjj`TRq6u|#*i>D;*NCe}^ZeVqHwft)8@IONEsVnf@w5WlON(%x** zQme}%%ZM;pas7SXrxRG^hf>PE)%TRge-JFE@&tcCV!WHJC!2R6%eN{QqgV2A8oakD zi@;I2|B*xuzn$h=AI!f78`$I(QGeed z3-3WHBl{8U^+j}`Y7)oh!e)0>rP&Bp@PT!sHfP~;&TS{uQbPCl7>%K@Fi0NI%-?G3 zo^5gRDA@m)Kg8yN;Xv$YAcVW}tDwYNavWry8jZJqA3I)jC75x8rERJAT?}emPa?yS z*l>zioa)ZWp0;E%sjU!LVI#jlC|L=i{*!aM{tmZ9_I`bSMrOt3IY;dkJ&mwJVQtV- z&GX!!mu!H{Z*&?(a$sO`Pd*lKY|UZw87y_lQL59&E9ty+$KZ8}R6VK~2&ool=r}_^ z${%zR`E;sdp*S_`=BxLDpTuZqA1sL$305o%XScu!T8Y4~?TJvC6-kI5>DoQfmFuc{ z0G_(kd@5tz;Tgg6(^W9dd-=S3vs?r7A(OA5+jsgu0W{>5E7GN+$>FN8_{HjGlBbuK zLECcZrx_awA48=C9&H++zH)09vn=H>I0Zzy$wQtc>PmrTF?2%DdzarFK=h{rIOEdk z#n64MF^d%=4^Q1y!H8f| zgEz3F)I4x}g9v~RC!;*et!LK$C@#{W`ONPU_hvT9^P~~x6?P0M$8B0J&7D+MB-{Fz z8lu|EToXpQfEq189kwC}Eqi+=MG=HfBm*bzrLtq+(>U7DH-ieh6$7BR!7^mlz#(Z} zR3=*nb@b}Cyjj(@nc4OZpFVMFZhyUX`kf}bL*M-+KPcTv5R_!3&7OmdM(@;rfNKjJ z%6P0Y2r5apbRjkfyX6Q~6tELG#;gJ0N4?C>Ee-arTyh<z{g@d5oRJ9hStVa5#DfezquxOo6#gu6d8%z7; zzF_%BsR0C8KB40?tjX&dN+)WEzyT_Y`S+wkq%Wo`6*3|CW`yVV0lBGBIi97`*I)HY z(E>A48EKPo<7}`y(c{IFX_h%TAB4y#aamsI!he zP`ZdC7E%lNmRPk~#K*G8HQvciKuLX8or8o!D2_b2oYVub&pbn{X#PI#l6UE*T4~mY znfyw+Do4)_X94g@63X_P{IcCEW|~$wT-$2LyXVP0v%@N0E*Nj>Bt~eQB}O$uT(=9hk0KP) zpA?^ldlQ!PHf3_FA`(19apkW^)m4l)4JgiK$8*JjmV>uv>ovcrn4(q>lfZRr?f?}K z!_VkaeFfyzMaV=!wnopoypdhL!L)X3N5%XhP)$zv1Mif~Eip=t?zC%tzw)UYa{E0C zN0Lgtw+S|vVfeECr~lLE6u@#!VkOD zTtClL*~J&Wn$!8AqyBbVqrAe178hd^89glgb!K7Klq+FzkFP0*1Gir!7+(|qyfBl}Iq&T31Io@A#4;T|s9Gz+}`8H?kP|{}E zgh*{qUU(s$y^NhGyVz*dk41hEyn`x`}AOXMQyFMQH=x6l% zbiBGiCkRynls3M15RwhzS5}XWapu8A1`Su*JwblW-_Cte^0{d&`#@`}%`L2e&ROj< zT+rnVZT4gDacxIikgkzQ_A&iR)N8-eDb^F`y%zAPU)BjyM@A%VOpXoz(brc=2TdXZ z6n))|@`HbU$)6o(4hD3k^7X=lPT0V{@ODBuQRV!0PqwbtxcaqPfPL7uL`)AiSNc4r zn3Fc@dDvW!mhTA-gX&DRpis#$&RkJbgNL*f`Rved828;Qx_S)6ygs^9t@L&Z7UMWj zyK?CSb`iK2g_|2q*m2DD_38Us&|)^!CYo@69)3jXm!PJ#~AM9&d0qQe*er4 zj9q&^$y3;00CtJ8TH<-^lAxXR*1X#m(bMgfy%bySLvGMJDm!dwPibmJ{n3CG1;~J# z=*^_P-u%)b{OQR6pSa90AJrZGF*MBg*_{wpSeqa`GX%Hrx#AQ(#kY`HpDbN^w;R_q z>@_T)%*LZ0&N|d6dcmMC<&A~0R=^7N@EKrew0mqlU?lNmM3=+oF=GU*32@@uSxcGq zN{4FBt(CmgSWQ^li5L93)2wt<9AJ?}(^%bZRn5MbDn^!egKqPg$(Yiayvj4l=_8#- zy#8xGK@xw&s>Ps%(BjI4OEs@%UR_Xr(X0(@M0v4C?`7q~4f#;n6}h#U zPnJF0)IK*;3WT12mZP?Meto?JCw@vEgxD1g0yG?F-N3#4>k0X9#FWN4LwmRQ*X#P1 zG5g{S+1YLNPrD4a(LUTsD_Qf%X-U-b?uxpDaTyuGoaDVfM)oDc8SG9XaH+%j-mzKb2cg$PCBL|>PWSflH-~|1 zn0(bgQL}6nx?7MTqliX{6j&|EC(W@0Bbi|?vObjplN?e0xb6Bb!YfU6J~a5TCLI!T zBKZD7o9j|J@s_b@u@9ZMR&AsOl>_jiHpkyQ5T-)aGP!isrUP}We}xCYY@(Ql==;;? zqyS?5=NISb%5aGiYY&=dv8ldg(o&%V8w(a4& z)ciqFuVIu^y6xG5aI;Lh?G~s+*+SrpL#0p4Lp!d&QS2_dbmd&f=GDHcFP$0Y_`g}q zvh)6HgT9Bq278%4#oRzF)x5hmzR{iK89}V!wcbCqAKD{4s%2a&3U7Q;(d-oj=BGsY za*!PLOQ)n$d}E&1^MkMF=89h|rj{CAIx`F2r%C&mW|Q@8>ySzXl=Ji5?tWgf!qze3#Yw)0Vox-GZa z`1&F*@LN^mF!*H&QP~M$Yr$twZ4OJ#`h#HAC&J?1WK7vbF49eEI%mKz-RD|K(h^uE zOrwy45|B%-dUf!!=h}J4!SZW4v-2RYv5b;wvo~E=mZ6blH@5~kAci9g{Q2lcY9sM8{PebPSkdp1uLBihFQfet3}kW zUboW)N-mB3B#~0(-Mt`AVWraDuR4vOef5JGAU;z%gN# zMlMV>UAYPJS`Ia&%>!Q#k%ZK6A3N5`|D-UrSF6^PL3^DBlUkJuqsZ7_lprrb{J^qZ zfI2TA1lb*RZ65FFXJVdqx8#u4H}=owOGf>6gC#q*V|{PU!U=IZUVA@9JOzMD)!xku zF?Q%6H^$2wi9EvFqsAXr_9cwlC$SP-MhGRN+H!x;)d1_v%>j8~uh5j__zeCT!hU{( zF5X9&j}DGT;loW#@r*jn2ZRrL#=MN6PN2xD9k88HFKjO!rdJ@ySxHoFE#AL*yxzY# z>Prz?)hkoqBZwB!iwWPd*;H^Z1)K%&IZaLp+||@LhW@l}JTOo@HjQdFkSiSS5N&KZi3 zP9;k%$jzj=m;TGspvA%SJg;bS4?KzarOf=>oPdNb2ekOu_xUB}Qr@OUmYNMOq7ZMy z1#$1Lw-_81ID`v3!A#G?!{cSkuHc7)qcDjF^1%FVfpi4C&>SxyexBr|zi3V$a8+<2oTFQxK!FYo& zF8>Le9P|<4bUh!?YA&(~dDqNXGDnuujgd6&F+oE$&ANvHr1Lru;QAdMSA;gO9%h$UfTPJK3+ zVs0#8YrATY&HA7&*)G36M?aMK%g6j&W9(7dh8NFiUGDm{tp2{?{Nw&}WzhgA_PT*` zyTDL&Yr>ImCJKl0Y>xG%m&P0Dqr-a&=&Y6?W%oX2#Ob}beGWFh^(9Q6pIL{)(~Qcw z>iCo{ZX?U=-`z?Hx&)rQ~U|#)E$Ty~QX;VY_YZ1$Kj^d*4*AR0xohIjBls9PA zhid!Eh!*UCU3|wl--wK8`;w$g**68OKK^UJZvsI64zz?$hN{dz^56fspAO+_uM4(h z^>5L=iOgT_*)u~lEUv`(`1+cd-k(S4ZQs0YBKaI&aVb|? zhEv?O(f&QGw`SMUfEKf7=lzzdDv)%6xk=Q^w?|vcnrB1L#q@5+D?)B+iYlKOAhBC; zeIBgr7cS{);2%4+2U)N6t(!C+QU-9^;=gi+l&mA3+6_jvIDw7-;tm83QwD=%>jT6Qw?p_jzkMUsgGIeBrT-NPh`PrQxQdtm5&N@4x@fPBbm-`>AM7cir( z=I?1?M?=C^z+wMJ-%@k+3pB&K0Kf2WLNfG+_>QZT`+oyzpJS%u9~mNVN(-rh`zjuO z#8Z5Ggh^w`M~W|*j#UVubw99hCqOEj7oVD?34T9gY~73l=j!pV6I?6RO*}WhN#IX+ z=jMrz6T9pT1nS4)t0vDBW|hArU-~=m_GrmoGLMaRi(Tn)i=nwy{MYzS$0pPbe+@L<)RA00AGHy4lw-k-4tgNm+#s?w0Z5}#t2>zW9vqj z_|#ywhMTL0bczK#KUQQI@GH9hT6_fBUxp^iQljVIpNIMbnz}>v4_J?|A=F~Pnk5~@ z2Ixh|8;cDCXx={lguGaA<4rhIK#(C_jjeAGV@8+d&IX8|rdLC;@PcUUy{OT3ZT9cV z92ym=L(WaxV_pwFF+D!5Rw%-`dft|j(1-Fz&eB_t9MW>E_k|Bv&SCWx_IAS^W`iLM z(Q8d(O78s%q`z;lz>3ne{5@ywBU^Pad1C)Se}BEq1&dJd{KE58?>sO!Kfmgg_F_xk z(9K-mWi61}xrXC(hQX>q{27k=50CY2rONY9)Z1?g(AYH#@vB9_MaRG7JH@eo@6oYc z4cD$pyCd&c(!jeM{!N17dubc(mz}I`l2dkDf4~!ms2DeDIV($OdEaA`yq1+oDvRf@ zwDo#8K2SNMP5EP5n(84fxImK&TxOq;2-05EzeYiKGaZX25Qp9_k85bs)rSj-WHPyr zAFK)v<8|rAdwrAhIKpbV$MF&jC>(Tv9YXkMrsmCyH9= zvW%M3o$SHG2@bmT$oEiY@n96C5wHA5Az?R4ReONBC7E+4ENVUCo&HWfeG_$h7}NCBH^QFe0!&vQ-U|lUV{F~&AZ?kIHk~KABUE1B(_Uy)SSish z25|`3B%eMQ@V0rH9jmCDCx+M2xRDA&04m`J{bWq+CMQC7_{0hB@3N|tZuC3kNp3Bt z!%V|-AeMR`GiAaZ>V3KOAFB(+TvlOYZ4&m?M`G22B70E6;PzTn#*4a=t*j+t_qmq5G|qFPSSDTbV@z zzeZCcw)q#M>Cfm~-O8$RC-8jOV?)|d=JeOflNu%dR~{Iq&O2-hGrwf#Pqfhu5+syD zGDM9lH-MQV8Y%h;Miy&qKL&C@%v|5+a_gWh0TC>i)uPKi!O<63!U({CH^}RKf^|E? z3izE%Rq3B#tLoW=(ai8=K7Qq>y6L6*;mdE6t$W$e%4zdKccjZ5#v2TJh`VKh-=|oG zj>zxtKd0+_Uw?(l+fdquoYk+`S;;Xk6_N1iCA@O!jqGdbo$4BfY%e z>XxLhG-cRUs+k1WkVFl!R1L+JQG z8eVD%r^$TGFWXLvSwS@=OQQUuOFhm?q>z9sL}gE!rFpg+%2o846|7!2?1R+AEK+nF}~p*7HH;Nv#FZ$G?rrfw=xk5F8j_-h%z z(x^}DXbeC#*O`hnGunj>e-c% z@+efwVCPxOIeEWwvXWE?9N?%Z?j(|CJ&H31@l+61sRDBIp8HsH+{2mVwes;BKzpBS z3b7BU4O4&nO0ov0=SvciG$z|gWymYt;UW#~6}g`=TXWN)*!5qbdUwgWb|4Oy5$J0J z5H!pnR~-yvRVSwlt#DJaUT;lr=q z)31G^%5YR)E2HjJ29;f=Xnu~}Z-G4nTe+ON!cxxU(^GeHS`ce;4G;^(WDHgqK3TKZ z^D$Zp^;-|ky0R0C@+CE?P+?b%yMO0l2M3RSn9NA5gh z<2hk#Cu-**V+uegbOPGV{gZnnyi`{r>=a+|5Gnv>ylVg*%eHYUP41DRABe%fllBW1 ze&Bhg>wBN&hN&@^46TY?co-Y9$!rK*w{a#9UHw*TcUBKSZZ91X-K(n2m3Mh0ez=}V zH^KV3+es@JC>NP*j$C=XBohD&DhqqDDLY*(&e(jm>!y3J^A-hl;_-5MWjku`4$)*J zS8DpyDx|wJn*s`kbf7CX+mh+-kC!o0GgV-c&wE|!=$=QRJ@Y^6vZN|AvjsZgBmgf~@_Bs*g z4vP40qVj4=uN{`xs)oAALB+s7b=QCJf`?1o>5_ZWr5|qw4j_@rmSV2gg+(nR1Nd zbiu#VrA)0M+hoImHEQp+Vxh$uwba+vbdsumyYf*FmA?r{eH-)xYR=hMkPz#J;aIUQ z%{su-_HL63c&PwTDp~k+W?HM*{y8co%FOulq&z{!l>q>Js&ETtvy>ff$<4c?U=5n)$oGU@`CFJi(ekHB`N?eOiVfD7}%&mI=0jEyI7Su9iyge zM3$?lOzcB1upE8H0{NN{Kg@QuE_GUjC#UeNT2+buG+s#SiiRI!et~(yULJ#0O1==D zJ4HDvl3M`r0Iyg1#jvX;k6qPbky(5xzlwjz3}>oOxRsBIl^C~d)iYWA4<-mxmUSxC zaiU}9p*04_(cw>&%Zp~HnXnOgx&kQ-up8x%yxS| zb$xJIckp`E6pRWgbON;CZ-Qy84bJjvZ!8`nJgM}oTzc*>Y_?cWMJJde*F60^R|43} zUnF%>A*^`t zR^n(wBDFYn=0g3DsVLvC-}~KCw`~6-mj>=&CGj-1!XTLf+one40zekyTpR(%i7DZw@H}t~ahdfuUgIBE+His9aeu<#*4X5P zp5NyCBZ3C2SeW(_zFhR6Z_OU|FP06=UQi)r?_GrHgFz~8W5tV6n|?{yVPL^CY(zXq?)%MdBB@8*UHotQPNbalWIn3a)tFz=cyjFx|hl@}PU$ zC_uXm3Rq`GxI|?M3S;+*$~fp&TtPlIXOJ$_>MALh{@~xi{KvXq2|S8RXAY43R{x?? zS}k7pTtH(GI?wfih3&rE7P)pFgDVYkm3fDxSuHvw&E~1V%NtwW&F3^?SGnius#=F?nGY8JJ7fE?5daGU2kES$ zVy0LCXT85cgU_ZcxG%a}w9$hpWL~%(MZ8%Q zZ83s4J7yQ0VE=U&hWWP#u!XDxsnF&mSJeVnhPFgSVaG!2dF%9ut5@ax0-R(39Qn-T zswALT?Dk^aS+uOyF+O*m&k+M6EtbKx^D&Hq;F@RK-bC_jGtPrfOBo~e1>+nWHXM&SXIix zyjT0HUpO2(Ylg7aMq2=s07kgMwe8#A|81rgBoGJTDvkUs^7hp4gy4?~Zgn4s`6jPZ z4Zo`jxDmjrdTz>od7KI(jGm@scP|!e`CBtW3_yPB+;+(27}XB&wu`5|ECC$fbBlv? z?MpY~Q<#ud7I=>U$;!_S&gUGr&#yj6#F|ljMr}G&WdkU6I52$b10ij=x4VELNeUME zIr`&S{nA8Xs)X%2uH@_7yU3XT>(5bi{qvsFo5^~M{D=FZzQ(jTEl4CZG_>e^Y_wni zM0nYr!^i65t!aP|4&ijOTHKB}S+E_W1k7L^8p#ozB3O_QAGCT2-jNsgyf}|nwuBA4 z4&e2?I=7E}z6OM7nQ-aVh{zL%*b_?H%^Td2qIt|2N}Y(Ujp9=~ZOpo~*u6T6f}aEo7rY+JsyGkofinXlUv zB`?XJX;WuY8Mea@@P8qr#DY-^=gfFEOOY&%QvF)TM$fW;SH!xxncbJk!pIt@{a?I9~X zCy|&ECc=(hQVF+!W8jUg;{EWYsA!e?V5I4k>$d%V`h;Pdr99KcsOYydyOhTN)V+ae z`hCa>P*epO_gAm+_u_}{(t-EJK4uCZI-w0(6WbiAgB37a*{i$ZvNMd|rt$VM2$F53X zOMSUF_@j3<${7EtwY|MTI-*L<>3JzhldeqD-;JJ#9abogGQD|E9yfL0oLlS z9PG3WM-S6PIVAV|0f+hHdz_?$+i*0h}JE1}SEcW6CQLA3vc6}m61M=2WtwsA!Ewg!&o{)nBW~b0pTxQUjf)}+ zS5xX~VaTfa2xzd_uX*afHl4*lgO^`X`MTZBYChZj1eO{gn2rbeo3^YLy=l|GY&ZB! zW=raj{Fy-TLuBNT!<`yvxrgddH?Ga?kQ+}2UR>v_8KLE5)O&WyM00}#oe~TXwP2givkTvxJ9p?XrM{_yCip~ z$pF}@4S@P^8+27XKNL#HYnI}MO_bRJRZO>N!|yjI%t1EOOsz`ODWl$H7FreDOo`*7 zD;f5})rhi>D{>TeEHc8lA+HKZdLHb`mefIp5{YBp`0ZDP00Rvkb}DJv&^~-*>frX@ zUCwOPA%Av{|4+z&?0OfUKR9d@od;oGsd#o`Mq~PkU_jilL+DSHVvvZiQvooN?-g@9 zn3DrnRPwd??iB+Y{V(ij^B_OsRNs`fJj{<7MJ|Ndt=U00m>@ngqC+Zg39ttCG_LZ;S z7x*<#xO)sG)#%Cz=jeyPo)NnL@-{!*w*K*IMpFajOxv3(8h3%hR`u*AOU*I_VY8XQ z)Pr1_dFKnF2NRgLKCl!&P%XS}1VS<1XSubXM)s}sJEGK)$#-EVQ)?0r4(Sd*zY=TM|wQJ2B;!u)ca zulwk4bT2jfDbWsKm~bmE%GZ|n{0uMuI2G!Pc|4;b>l*TN@O3RLEY$YmOI2z@Kaa80 zH1o@zmx3TwOFlHC3^d>y^Ta(Fd`(00{>XY%^OurIgX4HT+nGSsiBSBP*yRt$vKd!n z@CVhZfb!sTrdCi!FlUFKxFM2=P6dZ^7EP%6c>H%*2Jo$0(_p%3)_T`LT3UEn~pDWZom3#CyV zl_k{HH`<2bYiW_=4&P#QH+kI+c{$djaHFiT`(^|4HtwGcJr@AK)1Rdul3nw6|96F) z>fO;wK*er|8N?5Yuz2eX3QNfqg+xdm9EQMq(^N^C3H+Ptxlfp4h>qPS-IqRdMnSfU zS_&_~5>hHfKW9|W-R;ZOPzmA?PpM2xR!kjU|9Fk4FYj^ZyFqK!lUFE)AewPVj6;V| zh_%xAvh-}GVnSgJoa#~>!&S1;#CKI$115t@<XZBeSihKB{i&RC*H@|H z*cs>U4GXp!sCWQO*~kjA@>ybQ`3!$mHc_&3(c)HX&#i})3w7ew@*!uc&OF5;noRZE0XbURTCK8%xZdvECT(F-`d$zT^`eN?~8Ts4(b&&Zfm zUWnU*BSe6Uik}--yKF`G)!*-eb^^=OTgY*X{Sv$U!+L;6i-UCFmsI8|BA?x9vh|H* zz?NE1?(x&#k`HkKy@Wb}gu^JYMa;BEcYMJn*4=YogytW-$!X~?EU=7fblu;)(a7`I znb`a&VbR5hVVa({)jhi)As%e*dJJfiI)^)q3H}hJL!-c7k8%|!8*Pde`Cg8MiJzfL zux4WlargU1#yb0(&Bp|PsjKt=0DkR#M92R&9$?kHAcIS6KxhI#XO81bnWB&{1>H{o zOlXMdF`JF<`PX|qgC2l&T!w)^;6gxUgzmn(-xVusX+E;Keubd?(b3N|%s-?DlUN1h z$Sc>XkdmA!OkI3vQ-SKDW5i(Q=SvD6oHau!ie(LbyN;#DlWug$d}45)xVK%Tb}|`5 zWRsJ~68oNyn0jU$qtrGSbF)jU^aqk6M6epU=8si2tNkx*7xfTx(9`bCGKgmnpJ#v^ zRx7MLf5o@(jyR-Q)HbXg1Wkb;2{o&#X^@m}zY0cCI!y|KP0;h4`!|p1sl(;8fdg zrUK1HorLO|M}fCGG;OB-3e3MPDMZNs#T`b#a-yuc#R~RB*?;0jLuE&R(}`Q;+aCko zz6_{0l~Im&rF@@;tqN>AZ5AtOHdPWF{idt(xV-qe&+`%%UQ?rH?jhP)JyS5l+k~lW zqTS2uWLYv>a6!Mqwky}K+%cQKdi!+RvhZb35-5gq&ET0`cSmoa^_(?3oI54rJa_1w zI%)T3-ZAzv8==zsPB(_nTq()=V?O1NWXHmkF04BjDFJErG7LgB*M`XRS>FkJo*R}N zA`MxOu2t@P=0fxrAS1*2`Bwvp;uzX?xq9x@YNnUwK?$!TKT8q^7Zwo6@}e1hUJ)k% z#mTj{QINI80sk9E2u=4V6D}|gToh~4 zr_w#Zi6de8}p`KS2HrPgcPb1?LVp`NIy=|`=kHZrRO-}|YM{kR4Ps|yZ z4k!WHr~6!w`0wBOXRrfwD1B8VuwdI>Gh035wLlpCeQGy9d??58>_!vNXH;ibJJA#w zD^aT)Gb|SD*Md7;C;31{+DWGES=}6@GV3zTaGro&5eCd{=!no41LZ@92=vuTds^gyU-{jHYXtuzuW3h#=m z5?Kx=@Ba){>o&x%9Nx6#qMCBV1Ddr!@3s?DxFQj5$B_3?a$@dh$nnF^3Y^M*LIFbf z9{Ff@q^?TgQ{gt}wEc_~5H^ZG1#z$3=-}bJ$MhcHKYq9!1YnMe ztQ|!uq}?h}&tA0qj!y2Sb8gVN+Q>J#hFvFTq;jay)El%&s>s&vZ2?T^NxB#u7~p97 zMv2!W{(y!{D$(tyNiS}5{4ERnM=%4RqTF4tmdea zmxt=%VA?pPNRQ0B0G}Da7;h!2{Iz4QU2|G4cr~XKV>>!-(3t-v>@wb|$2sg0P!{MR zI#71MA|UQYBF48Vm=>pqfTtD8TbB&Fk%fu?!d<0xMYdQhN;!+Qs#SJr^4ej!;8z6M z-E9QMmAyQahsGXi{a@AN$D+|9EEX|mG*Dq#v5^wcr|npP?qI*RLapfHX&7_f!IVlc zWu%p5J$2epoM?B}s@@q)3o&dt8mfN!R_nZ$m!lALWI`{Y(RmP-sRzfwR&Uu-xX25# zIP&1?u76ai0(r4HDA&20S|`;l>J&t1bH#k@ZhksbIvY4Tvlq=hHHxUpt|H`n1ZG$K z%qpj$v`PC8)65Q{it&GK?a!O6MLURj@%(%!3|cYfzMl^YwJlJTP^$)T7p2@m#z@XS zO|HDSP9~`#DywP@)T*@bZH{c_HSq}k`BxRX-%r}isGF@iN2MxbUS%7MajP&B5a~bf z7}Kn1MVW?UDRa7(Ln)GQqi5-2O2mVo)8BU1>_nx`WURYo0wobcEjLO~dc&vRa5+gQ zIz=eYGzI2{uLyU1{_g_F?iqKc^F40epe)L-pm7rpZI#xCihH{dIJf(7s$rqvlLd57 zs+j6{xn(l>(yU`(4dgp|Ahdc^+tpjPbEmc+yvmCI0Fq~usk0{L6_o^8y5;%-#_Vn>yY>!sd-`Cn-o34YQsjUdxc7cp8@Z}s{Ki?B-+>p~@Tz_Shr6$hsk!$xE#!}!r zR5K5oE61ei^r3)Rf%|M~8tAwqk`q&O16P~l5xJsRv`Uai+_3YJyd&;5A?i)4 zxxuNKvg%1;2cFmDL|$1#=qJoztc44w1O zQh>taZjBx8r~`vXvKob{aP!jIAtFS$$9ro4sn2%2WaBw|K$(w^H95qE`3d$Eh)x zMyRn;&=5lxELXTfYmg`ZFv*luN~bN$&>_OUh#P}BBlgH*923p#9p zu;KLtOF)U6l+Br}Sb&o#h7+FNGi6vl|Kdx+GrYxQ!X=Y}ilC0wv?_q6;&B@F*h!v7 z2QFGE-W%Ozc={us`1zR21zP`hvz5dS3y=-EZrCLU9Zzd_XlMn}YfgPxZ8m=Ge9M{s6^ZH*2Od^tYEt)k?hfrBN2Jg3Xn$o9Iu`D~KYU1HL141snyzXXp3yffgSr1qu;;Q&t>&irQRVntCAunV zi|z)ayA(ZN3>j#wxkh*H~w%tQaT-;hB zNt;h37ajBla_DLokUYB-A~a(o{>=G1xQK+cQzU7>+Kke|5V%)y6Rv(#DQiMVZz5*n z0bbPgA3BZuK%%HkYoX^NMjeRw>5BjPS~?F@VumA2n2JT_f%5geAXL~8^S17d`A`+6 z8fZ$XXfbixq1Ly@^2!XrX1B{trek846! zde4ddu(~vz7b5RoNz{*ei||o-XvP@Q7UVRRsbZ(}vPxe%l*N=LUV=|3({yx&Y>(d_8Q%CfF9iT@lP!I?G$qz_f z*hDy^T2=hA6RU~XD~2|w3e54t@v{LachY# z`Ha**SlN#_wR?sP9;j$=dUpX{TAPn2SU4*}&x90x)Y2C4XxwN;47qiwIQ8lUfSKlU zlC&}ckzh}4W}rRj-uvOwXFWx`pP1d>D)`zt4RRyXFIsPUoHM1oUPM_j04E^R-;um62oA_vVO)#N%0FUE{o4>gv6cR zA4KfduFKST+jz^`lTE+j^Vbr;CUlPLc8GX(NXz!4!em7OZV-)U=+3yZ2{ z?m+YM%D^x3yuUx65rgE3hzk!epHts3!EjTm)=l9(Of}|{ci^N(e$KXsu}qdwtEy}B z#dm1#HO_vj>`2$^_IP9XE#LVq6B{J@a|}{a9WfgqNZI#ThyosdN4-@5;{`*Nhlh`C zpQ>9YNKSkt>Ig(UFy6QIxcU&4$8|v>e_}JmSETq}fo{{MpF2N&0IjIq(2qqC+kgqx zl37&5=FBRp(DT1sfj#Pu-NtWfy6{lE*3P`O(EGSOmY>a}e(q6jz3mXlNZ+1yYnH2A9ED zyM&Q)>AB3pjWh8l(F-hhSesiO8lOor(+XnuEI;n}M^vniWst4-#aE=E&b%Rd=c~4fNuSK}(j_dTs6!RF znU!?@3Ol>>x_7INNr!HdG^#4B&-kC4Ci*SFl{@PX<+1($`O{rnp8>5)@^Gm_huf=K zn8v>JAKro&C_xs6HGA;WZZ^JL_6=_uLwlIeN5zTL`7f&s`PZ9HiZ`P^3K0 zzWBIp7khzij;3d7%6nt228#Pb_w(5BnoMc9&=`zz67x1o?~>7eneISbAx&5$Bqc?%GYy;j`&Cm(m^^t zk5Mnn8?2h`WA8^jKJGHU{Bf!{LaJ4HhC<)A8vVaZ0S-qlh#9|7qZClt4l8tV>Q54& zJ7u^2svjADtq~hXVGfO6nEKMTZx{d+W$U+QJJXL-E8Sk97O(Z{5bo4z+5ACB4&0|p zp%v&REJMHdv>22`K7Ts%wE7c>zvPr@wg>}Io+yzjQFiEctb2iOn9aG` zRM^B%U6r57YPA!y1*wOV#?#uo@4!Fy^6PKSScU|YQUsNoWj;Y*PFwmQ{fFDJ!}JEOnXs68Q-B}o6O(5|L6iIEHf4f;?${B(z+UY zr^DL!jX?eV)kvB(%C#V@%ZO`oy%?E)3~184I6&vE`S3;dJ%MV6F%dFhOV^Kmrr>Wd zg9ShY*@}3?{&;(;Qt3ZXNdVdKx&>6qJfZ9Wba_JkeWFBdhWO?Ygn1AW{FH<7;f(d5 zK0AM2*bdmCR3JA?Fu*X1gOC@#9Rd36RQYCQ;k^<5R+pcS3rP{7DOJ=$c9xr^)9^9j_HNP0X-B_f zbwwptS;KfSlp9;R#c?sq?z4zjA(rI(K%InPO@6DVJ-MXB<|s-hu1i}$?E2rCS2dVn zeqe^tUw|Yfd-3nT0^Tn70EojYtb_@ovQ?5t;oT2#zZY!PSUnzA_izLufwm#1Y5LOD zW1rb&Wws#cz2dYQcZdf3@^~2nz;eKoA|DlLb8x)onJs1N+I)$qz|zGR+H@r6Jb<&u zX{j>kM6u3X{;`-MW6&iCFml}D0`j&oT2!IVWc>S7St!w!&jD?(`_rUL|G&yLP<()L zJso6qcs8&k$D8@nIQW$d^YUZTA zD_;0d+6r9F$-G01{^TAeoZ9~_PVy)ncn?4Ki>roF#fR5Kgi7vu*3hcz+irxGL-+T*__kzUkC|& zJ{5iJKiY>DLTK;-0DDajV$cebEbv#-Q~4GetRJM~drwRB*2C?^8dPS<9jH}3FrG1F z-xUG3o(Y^WimH~40guH8NUl<+q%qK+kfk9BB9K9y}q_OUUE z+E^#EgmwbM+1Vng@cqj%x&ptr*8a%NBr!B|QTrG#d`-4ml1})K9&>dG1Mda0QdNuK zWvSYcZM+2o`FPx0rXjD2cf`uxexzG*Hu78xpi$A2MIyC1f-;u z5DBRvrIALYMMNYANonauNokOfM!LI^M!FlMn}Hercf9f6`{`Y4mTN?WGw<1FKYKrW zpZ#EwP~+CdeAcC_Dk8tA>Q_F^hQK%9(rNQQWTRQfl?NMlKD^ugxY;7GOt!QZT;*>3 zSR?^4Lt)y!$4=}rb}!=cMKYRA0^${aOT&udyHbnRU*Z+O2T+EX6c-FA{i=Pd_1oh4;}Yv|#(Bqopp?Hq;8$=RV?YIh{kiatRfSi#cRP{US# z%G51WLg%8j^quDp8dw>6#Lhdvd8|6C#k5{%u;Yhp+*Vl{^gC(=s>@k(if+~8v}3K3 zWu$!L2R=|!s_@yeF4Yc6nzM_5L%CRo7$B>j4#|O3{91J?zVCJ8rl?c*_n&mzzoTS8 z;aN2w4{ET~v%mi!K|eIadS5Js92O6PECsURa9`4DzQ}s0>qDo;bIieg$@O`X$do0K z^)jb2VR;ofI4k^G!xU}hdHY4HgKkj_k45KW|J4a8Z~$5H-g3-a@M20S{6TFtth*Bt z&RImX#ZdT3-|y1wLuQzlXoHH4V$)i37)33!B{sYm=hw%chPmI6k7DzB>({j3MIx1m zSMr;*?xyras&*I(2u0F0z=v$KiJC&CDgWcbH*C{ysw9FMr;8Dv;CMDX?V+XoWE@`F zV19qVEL!UBFHZx>{3K$k!aR66o@|k}lWR%n{?2BLO#=;jvBzu77Nyt0=Rh1>W zMSW+E#4Y+4fH~-*b+C>by!a4DII*{vCcxq9fo)l%%wmUwI_SK6mvfC@Pgr6{UboE>ElvnLe=Wff6AAjyK6L4Nw;Pm%r^4T(IF>G z;SGE&IqANk}j&esKeyF#m+=hwP%hb#&{mUsYfEWZ@qj~zbA8~CTu z?n@ZP<6PU}NYd{LpHJh4KhT4BsZU~`n`q&94G@s{$+r4j-meNh+{V9M^g4%R;`9Q> zjA54Y>N0qcQa{tF=s4EQ1krGowu<|d=6zjNqTs|UeyF13If>4%4@(rJhp&t>K`h(* zC;$4l5qk|01~_H1Y)cS*;BO;%*7pXCM{rELYi^Eq$@(5Sy=K+zW+es!Plx5)eELTR zj^YAv)1bw+!H8c8IR{jyUod88RcQNiW*L^q@cuC$&z`3pPpw#SiK0eEPd7;}{F-KB1L_^e<_PJSiLoPxBa&-Gi{qA6f zc%iJ5y%3CnB{6YI6bG&L(RJVXYZid+J&qW2%XvTZ=1<(FTPE*wu9t#23fX~qrz(N3 z==;45fX;en>-f$Z>#Uf@$6l&&q>yn~NHyPb0LSvud>uo~KfLl^Z<|C23@v0)7escQ zFYo^&6awBF(t+`0_7Cr*N$R!{WqQ!flACbTOTNZ}(H%>~c~bQh8_*w5`%c0a5+>nG zx7&x`0kgPJ3%`Cc!u6VE^)tW0y=U~;TAPuqm#2oq+bQM~qF9#$IW+y(0hE{Q;`#lWtu<$cR5mN?k!LU%@+P(pn4oC*N8Vz3G41CxrYN1@B^TGG?I^3ns_>kFcVlDW;@4K0dWo8(AQI$ajk;G%x(K z;=*mf_{KzMJT84-BLEG9^JV_{5~F#bBi^wG{&fdX)c)}(9n$6`p`Ts-ii@76^pI{6KNBa)pRD+?%R;V6yoWQWI(D^zIM(S3nt)g} z=lu7yj=wAz1(?A5*w7}_i{zIkux(P*IrM}IqEm(kbfNI{k*C)5Km;6abCuatlrnYs zfcPslz~~`t@1N+HH)fVT1q?eR!|98`T+Ey>YMTSb240Kh8(WU9dauJJYpnAgr(wBP z=k^mXUT9xHcZgf!TV0CUhh=Ds^t|>o79=Y z=0YP;o#20)1fZNrQx>p~#YY!X6&M_09KgFHPZdB#x3;WDF^#>~Bgo&W;^U5w;E@th zo8H@ow21-cTP|$s0uWM866gMneRupW1tiE@0I-N2Sl&IWX5kX7Z8tiy#M_TUZKDU zXD^f+Izi{a#dHIftcfqGd?MkZlK7daxE|zyu+3`S6C9$2IM#OeZ&Tkh$zG>g>3+U* zst5F8+B-Cw(>#2!0_MkkJKLEuiHO@|H-mPQLUt=~)+u^H^q_C$k>wAxPe0K*-&QSa0}djHBxweAP9I}#Ihuj_@N4e z_sNuRH{UHUNgr3Jvv$}Z$7bZ-1{Rg!SkbWU{oVIsHbl+TrB`3Wd60c1g-6w6Kpzd! zInVNI!A%T=2z>MD_*p2|v8g6SRdPS!6x992&oR>|o1{z24T@y9ze;jq7vqO_S^|@U z=i!yEzsVv2hhRo%h<)uT#(wkDFHJ1Pk5To@oCa9YT=6ZVpesYjne#IkFdIT8T}gL% zx#1lK(~G%p`?1T8cM4Kw~D=s;UvO(A9d+`0;9LnqjHp z4s3@X7V~=Hq_LyC)AMx&{95!xD^qPDo*0xy6P0NV&hp@Offiw~hvd6+LaVpERSlPh z7ym-DzoW=Y=><En#3 z`aGBb=45z~hG~akhM`Hi10x#*zj2lKmfKiGrPpf%{Mfscbs`gbABpo$H`8x^M;}++ zwAUOXk3Lv_MXD{ehaESv4bfYE6V4<$=2FY}N;yCVK_27fQrn67DJ4E09T zFC(B%eNpzr@Xgx>&=IO#_$t&2Fe1C{TSrpxkcITe64ZNnz-7Wm!I(Bzbo&y_ghTU% zl%13}@L?t~knrI4&bEfNE}rD&#=4H!GtY{1t^@J1;Qi1~`yK9g>Bl6F{m+Yj96gi= zx1x|>&`ySr5~5Ngd?x(^!xPF@bQ~aR)Vy$GKk=SlM&bRrY(s4DZOZ&RtycdNAOl(j zbaPFiHbq2kfK4>N@uS82D#fz2EF8rdb6nM0_WJD2;xp?Yc)&Dq>j8cmbqfT{9}lOE zFn0(u(_NaHdB~2)`IsIKbkHQu8}sUR@_pI~cLDL91!py4M>1U)f+&1EZC<3DS$^3= zP6TI+`=Qzv<~DBrA7(%hC*^5v7#2T}Zv1a~=3)(yYnSzN{TFms%p(CKz?d))whuW* zna%Xo$_QYd#`U@_vE&Fpyn4HF?+XTK9e?e->k~DFK6^o{BbcJ=ClbhR!zYZXtcTOo z+>1tU3#{kRdt9V;3ELP%rVDCQ@WClS!*_))F8+<%&0)serJ!MSAI}7K$2#^KKgXf{ zVg>{g)}WAD7L=Ji z^n<^|k^XRP8_6hp1Ob9J^lQNrpEp0E=S|*5ox3Q`48S!KzkeWm2DcHs40w)@pwVnL zVa)sHR9#PEufyFLQ+jFNYnL=^JIh|QSp91Kd6JNckUKEyfPcL7n>F=W7sjA%LH0?a ze(B80^gT+8xAqJAR}4WIV7?cP^(fqjN>9EFF>VKc;c$L3 z7hP|DH=W4hajO9}DAm$SYP*Gt4Cv|v(JAtQZ56Z1&Y_R}mO4`GT59b=>?Lf^RVUE% zfKj*?FbZd|$DB#Ebvzc5D(wR(H(O&zuVtdMdduICj=-=Q$%WKl z)Mhu(D}#Sx1t%IPbK&zQhZHUT5epW1r@j<$vr1ah&1YF_lrIck_J)mr0Pm6(9JpaU z-v2mkBmn*l7+eXBp9I@J2%`W}$3FIvj}7LX{TAZ2(I3rGmKt@}@yEH?V{O2$fzxA; z+v?Z$_lrrccZRDLf!3v0B;xli=yTit1JNRyvcP06b~l?8S^hm4>)kzNED<`aKpiZJ zc*Na=mqTm>qD}Zvbm~j`?ErwZuf13H41c8m<@!~~Akz!Tl0%!faU<0_|@elmS!w*2YX9cFie_z(F=1uqEJ3nr8 z!Wr{@4h{T6{4mIih$@Nl-ggL((>bt3OuEo~k?zGpTxl`YOG1zw5vLCdd%g;9Xb=s} zd*)pN;(Kq@iNJ&(m_ex8k62-ZQ49+$Yqwd`yXcy!q7{ko@uU`9*Cj`FuOW^f);}wy zB!Aldw41#9X}!KMKDV%bceil2@LhJUHqc);)uQDS?G48KaVWNuA6&#tJfZqOLV$^7Pf zi}Xn=Ye6EnrkTZ3omub#gM5sS^U+%S5$9%4tVRlC-4w)V_`BEs{5n2CDf-GnBziNTvPp| zd|5l`2rYO)SZM`pXH+x3qH^fuha$~R<4A+R05zn2Gilj|Hve9w#bn7BG9lgDeHrgj z&-mp$n3J{dGI}0T`H6_-frj_SAFzf$tmZA3vLS`d)lBkx5OSja?&Z}CG#DOYd3ry7 z?1vfW`PZw-hae*TE=PxLZ&tT$FXP>gVH}WlJ1)_MYCA5AXNLRi>rdvBU@!IVaEt%GB2%mYY!Md2yZtUfUZ6z}JSJ=|1((EL^4~uK!gVlJ+&7xa* z@MifuxZ)CPvW6b|5|}L+P4#BRe2xyY_YX{o#{7byoXB>tLeuYGhZvknyTm9*T@wTU z_=Z$MaNNDeFE5RWdE!!|pBH@gxSXS?hyknMRrr!0z$svCfH!zk^{$Ug= zd=Vns0&j9lkA|0JT1Q)|dju!hXS+w`7*8hNI2q9x>vjeR7Pz%OWVW|x@{Xofd{m1v zo>lGZa)@S3X%xLDfqsn?=Q@=HAjPn+@WN$ba&ofjA#IPR{@1ToTmAF@SlmB8NsVk1 z8;ZkGUG>=i8xto!I8G?KMS4q#_!M^W6zs!d=@WIN+Fq}o#-Ty%y`nQ>WVSi2%;28*dASbT-aH{y|k`vitjOgBx02KI-CGGj4xm#9?f z6~xHF?GP5l8b zI4h7hkyu7XY)7QdsU2bB%-B(y=Vqqpdx&56+M~G@0%-4)5{gt~eu)7o@uVt)&`Igx z2-muJko%0cD$LM?nMnD5q=YCu52j0HYI*WK=Xyn%%s^gwO?(JzwXouhDba-`Ll)J^ zbtNnBCCtzg|Jxf+)teT;@Q^zuE(@9Ru^l_J)i)>T;(j#0_?-sNF2RT+$gbdnYA;pJkG;) zMyY!>4T1|0d>jhyB_j^0ZC(#?Uu8yX#;aY)FMmlK9oTiiS8ilOt1r9m^J57kcIt4v zU0XIq?bdpAaa2Dwf6gAeDwmbgQ7pwNH_3eCurA%BNUiG{xS($Q=$I-W5nyEUEch=c z@wd15<4uztUYt=x*Nxs0XvwVHUpwS=K-i@QT69xDsIJI_kJvnK`f4J8xlyl&MPG72 zouT|NscmM9oiMdX9AVl%mB|Q2`84=7MVA_ z2JS&c#}$f(A2CviXiXq9s;6wjPgs6@dD{OYTMbfOdJr6?p+UK8syRKH>#lj47i4km zIvTzNQOA)`OA?m3W?ZGX1T-)6P%0ajhTT{mo8{neRVu5~rGx!|b%}J#90;E#snygUSN?sz)owH<9yDe@l}(A0fbzzq4RBpWWkL^; zf)tfE*W%;(to3Kx1^yFv{4kt~BK$;a$^JGJSCE2}Y??P&PKp0aP^4ckM<~6cyX+~C z7Xo}pB$M$amNh}<6OcW>wf};tK!SQws_iRuvRq=uW5sfA)LS6L82pXCz{mtru)&lG zVfidfU|yrQWo=@5sc}-I?NXCdee#*W(8WO0;4?=$^_$Yy<-Ad4Gah9Z3OELQQG1+1 zZ1$Gv8CsP#JEeyVe&)gEAsVO!hR^w&*RB7uiGSn!2|+Q)u--MP_D`cyAVOsfQC$r? zCUbnJ&|2y~xg7X@xufho3gIVJGTB-6!W0OUi-a&v60}wM9lcix4yE@offE}na=lpd z=VxZm@YAT+PkEVU(9L?e$k&~YP<3N!6oN`LgJDpoj)d3ft0GNEnvzD(>GAme^)5TBY=&DFOl8uCMHhgEikV3F!r6vh!oi?7dQKjh)MBXL z^5FK@H5BD&LRw`~oa5|0p82Dl+>KU?j~nbZhhIDOo0*6vWB#X;cOW#p>caBe5s@xT z`uXge!1!IP=7VMVDPfsiLmXjCFWB_$&O5n{}gAB=s?gxW!2ntP6Z-Pmvznfd&@f7zf;{3;jSG!M{{DzK@s6}}1W@iPtY zE(&$1FR4-umd)Gl#KE}MUXgz&W>sy^O_{Ex|V zF)5tR&s+?=V7Ij;_=o?ajL-gN!V&G|ARB2rnX%{h;-nc2$Jv zN>$Ci_Pg{8(@WK!mZwHJNi}(ekw@HKvNSZSUkdY2JA2+RBcRrRA>g4F5*CtU510TZ zR||ALcwXL$9Q%xebzomPdy()z$neKsMPo5I9n`uBZgwVKm9v*m+w*u$VXi1fy@Ho? zspGn*Q;>5ui@U-YIC=^vLU)NrV`RpWnS45Z)?$0d zr-5DLuvPCRg~3T^hL#6QLb1hlJSi`Z>+BQ4*Fh-oi4v3E^oK<7=LxY5X)3d()W zMB`6Q$9RKaqp2mK#z!-|Fw6|^rJRJ1Bg><+90~_7v!T;lS`M};4qqNP@4CCoF|h|F zl#kd{e|^Bn`2c+E+7tsIPivY%+!AYw+2Wc*g}11F3VUW95Pgjb3_woQ5*^Blr^_R3 z4c=t0r%xCBFmQoMMiG?xa`f=2p2}aU+p0j~%L8NM5ex#OV1-mnkgC(xFaV&kLygZA zQsqEG^PIsX53rD7p}OB{!$MN7l^Cky=5XeaGI}AD2;}_g`G3JsJX2HEH*7-Dy8#w= z8{{9M6249r6(CEhNhsGpHm1kRk$giY7mud$FbE&9$8^$byYg(&+hX-G zv;L&bs`)f{i!t>FwP4<(bjLh*BRPOGE-X6YLxqiuib{5xH5RRl%PA1{1aSz48lRnG z*z}jBr)ZwFt_j&|54(SCP-XlPo=)B4#(W`53(`5L`PO95hf0X?)x&sNO=0?y2gXu# zx}H~I2GV^?Hiw>Zuh&J4utm-%4#@G^1X3|%^zQ$Ebo*~N-$xKi{$#?HITKDdpi;IO5WN$xiBE|_>{|zCz+aCg?P%F6%&sbp ztds}ZL|rPMM}|%Vzn)VYyfL29MVVJ=AAO65$WLm`%hL(huZN=4iRRc85cVD96$gNgFBHon53L~z zY7`URV|E%987=cI6AMS#E4z| z#L?GmUP2}jLXbJkcc1hRIwxP9v~ER~c8*%c6^mVDIJ55ZKMtwgj`EM|Rcbw)31SlD z2)iOe2kcJo9*qekDh_JSAahd0+-zi*xD$RLy)rL#b2LdKFk^+OHrNuHq}T+0Lcj1G z@nm^@luP7+bF*iH!B5k`9PLI}zDH%O|uPSvrmi21!r)hGa!o)Q%ja>$_qQah}DMI`{ z60jk1=BXJV**snqcnk0Jiw(sdsdlW+tx#oIrBNF)h-2?5PSM3d(4PI5&+H)#IO4+I z4iN~rz%6n%&Ka)vea)g9_DA}``2?iB8otdnFqm&Jw5*cSXxz4x{uaY#jzKj_9zIz}5s$##c-w@^N89fR{T?_~I{TQ0H{MviG#b^?GPgbI_7LMU?%*GgaL#>+Nyd+w;# z3utrK>GhmrrX~)YHpKxV-{DXb(GIJGDlHc8Xbi48J{j^s4>GcA`^7tdN0a}2V;9^M z!emlo-=eF74D;h&fmFLTOj@(F$PnHwM@wyz2s0d1Me$6Bt*ms)YykT?>b@B0Hyo;w zukhR+vY)q`G`no4`7z#`QnS|H2g_0Hk?yRhl@@ioKTlorN=;{;Atz|;-JOvuyLYz*J@Nt?YX)=%gPSK8O_eov?N>35lt^tt) zSKala1NQz)lAsncB9v-s6^v2A7qga*h~K--K@HG1ZRhLE!ers`X(a3eL1qwX{PxrE zqZxIc0RwmMmO#^-SWiZ~VwdV1vB~|qN6zkMpC)Y>-{0T47ctkWII29JV$l;NXl~jH zSnRm~RRPKRC@(O1PP&zco}oHRMV1SLPXP_4bSK5P%b zD;YIvl&|vD4*ZYuVnmaK#nF$CIf4 zPkieoG`u~4<=M-9@Bw+Q%07zb%7~%aC)V)bC6j7C(r*+R<~%GVVv1lTF_~TYcoLZ| z`{v9RS1u*1x1;<8mE?Urx`xvXs; zJ{oA)TbPR|u$mIX`PW>PKHKbSf5&BL99=L)hR0rXfO4N%d1DrlPta(d7G<&{uur5^ zmkCXWU?=Kl$n#YvA=_LTxF((!TAt`aIR~@ra_k z$}0`w1WH(oEpk)hRAhs6jp{M%dIFWR-;t{7#TRP{+0Es(MU4p9eq5&zV}AYV6kF6u zD>Z`izJ48le&yUdHlN8>;G9r;spE^ojo*;kp9O32=Ih?7Ar#aoi_qaMh|qCRX)HPD zUCX;t6SR*sAdIAqG6p`}7k?CZ8EmVBf!UHn=OSl*5c-Dgz7G0@@%pDn;bxAl=nO=2 zKg#VncGYR$-XT9oZVhtZbUPeFJq3++Vhy(AuDaSfdCxYocVw3HUNxEbhy~bFPTU>F;@pAa6=!E6uU`y=bV;RHxKDNw$xIvvJ-9Ps@CGomspvDjQt}* zGp?x>to)^Yzk#`3sDtBip$@b0WQoa;O*rt7u5;3?4u4P5rWSx-O+7ky-m5?JLyCw( zQ#Sx9d$(t9C*POBbLHdu6l2fL3&<*VDyIE-#&x+5>#h-(tr}W-#9H;np8h-3G6Rz1 z0B5#beAF-4>ETdnJ7?{2IbA%WB-k!C-^*Tm-;YaF@^1c`bOXW7B6na^uRmL16R=7g@q7zoNJ1-I~W@Y5BINUbn7Xd+czg#k4LNm25g>@a5MaAT+ zA&Z*Ltgkzc^KOp2SYB^2cV0h0Wx-PxQ~dU72O!?2Ig}bXGh_806)v5XbZ6&hc(>;y zxDGGt?^@KUqtYlwhH5>!AIjwT-Ql_Emf6n}EDIiP~5Q2w}Wn7&eWv-7$`jqEI*4Z1`xSS_>bL&3P-|%n)%Wee=Gd z_kXhMKmPjq2I^Yx^Kvv!IV z!8^sRx4CwJA6A+OqPZyUnSZ>V$eoQ6Kmh?WH1ahZgf8Y8z~AUb<4Q?wy??maaJfrC zjA~uGQxS*Ah*SzXsXiW-H`W^fe!xDU+bpssHA@!oYI@E%T$tDOv@|7@?TSk#vnGuw z5JVH?DJY!L)b(=nVm*brU+C!DvoUjn%PE_!oNT`b6%%wqKz|D^;IYivh*3(tt*~#) zZP{_NeCRVEB+hBvdLQZmG+*uzb8zhC8@$VR?IP^HyMD3S9vFBF!Cb%nzFRYHyF2S? zxSC@Hj*m%e@L2P~A3;nmQ`1{nznRrtQ_@iByUx(h*&YHCo=07m_R;BQs1__}=@b59 z%|5s!lM1_82O24(;Wbd9Kyer6Y^BP*Raed-%b#{qgE4sarW^v>thfbFC;CvbjAvO? z`@<9zDeCDG0eyzO2R)`{2J?RRP-a}N04V7GjA+yo;fI(Zvp4^u3jfWgbP*Wdw|hl- zgN9YDg4JQpdEIy9DMVOlipp+KY|-<^?Er6{w#)NQ$X$35AoXE0uIGrRi6*q)bfa|| zm@^x91&m*HO~f+>ZFkX!hxB;oZ+Bw3FvGb9+pye`Ba&Ns4G@PmLsFztzK^4JMXnZc z2NV-kNc_H^DhckCH3hw6-uy6tFHOnih%I#Wb+XXO8;2JIDL~w+@3O;FDL|8lmEP3p zIf7K3-XD%_cqFtb`xL9}?d47-uHy_;TK!dq)r!QlvS)@>?RJU0I))GW^>gX3xOb}z zE_$>&DoW*O^HAn&T5%Rxo;u4tbDXs|PHx*|DzhGJV}?f19l+YVZ~a)$=A1MWe0Q;W zk-fS7XVt^25O2W^W#}?sJKg=AJDc9z*On4zTMI6)ER20b2iWqBGA#0=!E@ZXYAz|? zx=Zr8`VpJMK3Ha_4`HKh8ix!qz~&i5+S_OYfzXJeD4Wv|AKxr`48uTesMaN$eBXg$ zsynEz5eYwc5QAx!?-MrEn7toz$;aFe-22Fkex#IRodLtF0{?#7Ui+aO$r)>-D&G(; zG289qo2M+S-L3gD(ZQv2K4&+g2(=fA`Z3W$3Jiz8ONef*{7uILe)9P}6@tR5YH_Af z_QH`#2V>rOJ!x?ra_!6y+xQ&#kZ+^xyQYq>1g`KSD|qk9U9$`6T%UJbU43$H%s;sJ z?e?3|2Na1OsswTd_;A`tIx`W~I#7|ooC4G*B zY?14W0d!4}f^RoBV;f?355ZoXw7XlocQ<7Vk6+5Ji6)isgx%fo&MPBt#2&+54a}Bo74u zExmW`#9D0yZj*@Ecl4v7FlzvipeKzcy)nKQN9+C^u8X%8ku%EkDW>n3{&*s&{O)C@ zgysXG%k9$TipKo=Uocyyts8|4b;)iWhX0uymqmFd>r4Q&OdKP#K9x~VW9Ap#2ZS5g zsB`|Ma_*|fKNRR;>gsUi6lOPX84f9k<#cc;WFB1=@BWg~@>XrO_2O+HwC3~@0dGhv z2N_wshlp3`ZSd7k8B7m+1l&_SrVXw&8t2`bK=MS?&z@xke&X$mqf@YiaDeV^PoaH^bjxD~5wD;v@(+DD>e z@l{+1q4Dl~(choSafa*mT7GH2fQnccW)ErI0veRK{uW1_jj!FI;l63O@M*vei6}0a ziHft5UiM)7^bjOm#3W=Hr`}4llBTj(cLJb;s`9$seQG3`1PFNVoX$GV+YHMx`QBo$ zGGu4z@g3KFt7vgc?gKKWduCoxzLc>D%x|wE+sn->#PX-K?b5v?_-7KEI-ctgvG^%t zT@~V3R0SVCy>+AwxbzJ`RFeU6b!-~DTah0hKXR5fkxdMLInI;w%`qy3=!VI1%6k8i z+THKUHtc{g>d}bKKVTq(c~;zVWAjLUY4K$5lVwpU@Q4`)fySfUnQ^iXmCZy zeI=hmZa%oYnG_Sc2((leLdT{0Ng)EPFVuJ@faeJJn`eQ^9 zL-kGvOB?=uG1VAy*7=2!iD%sTN;hvE_I|2G zz~Q`Ec2Y%4?_)#AXTehVH=`MfO-nRiHoeBaVN z{)&vFHfcDm&`jp`GO)WYlm8=o>LEB^SVZ?X?%%Gb(JQ(FK4f7rY%5QiihoHL2!iX= zpIlp3&hk#(=N-pFYicN`!V6fUj#Tr$7wQPx%$8n-R18TC72XnudmxTdq*uj=P(G9? z^3A)xHFPm1!ushe2f?T;{9RdqP6X)*8wG@8zQj2yG$kewNp%87!qAk#8st{u9J@y5 zUE5V_uQe^AQ186n&k9Qm_$VUn*X^h}B7<*3lu(BJOj{VKO$E{AH8vhY>xZV&A(vBI zg*Vn;0_0T_b?#>4L~h+CA{ZeJ$H(&84X<%RtM31oSNwzU<-7}0WJ5X3-Xmu`j~m^} zB}z(FSf+BAFj@ox1MIU|M*~XHE%E7`>#PK;J4~<2rwpgE0N4^z8e`4BWob*LXgG!O zy4Xo|+f2~IstD;4?0iuoKj*xs4susLZEf7_9x1dL3369B&gx)mj1u&&TMy|8F<1Aq zzN_#qM900}qMbv_rIXjIo3^RmYZ%+Fp>aoie(^hE$~{IG0+8xWz};_3PN8uX&`XUs z9U{nd<_6juY{5QdTo?<&wE&t_^JC!(pyGZ*I;ApuePU<3CboRZdshd|QH88(j{)0L zTfq~xYn$ffDVdyh<2F=3QK->q&^PFYz-f`d(f) zHP3mR*a6P+N7+N0iiur+w46#7brg@sLX++zqo)jtdT2@-&O?xoN{4}sqi=Jb>LL?!}W z-SkZ>uWH_daN2n??yxvFO44}T_wy$*OL(5u0Z1kN8G&okl%*-SI>~`13FBsAED=xr zk0aF~=6Uc@jtB*E`Rzf6ibt&3zZlRY#zAvpNYquFZrL$YMBy3}8f2R_ssR?eJ!nSP zwvtRLf`*4u`6A=jLrxma0^nVPDe# zq8G|nK#^j$x?fj~aDxsN76)`6Nwqp`JkKo3Y4~Ux^FW6LXTY5Zb@t5uK+YD)A^)%8 zrY3;v#^v}163qX}^A8e$6i(A3ykw&C32rUDsi#d3Qp-Z{W~bchgTHf2Sjs(g!m zNV8XY?5lY*u{ii!A2MC}pO;q*K2Y@URSL*6{>T0eihvZBPE-%+zo{mQI`BYPV2jqk zfBur(^aaajG{BvGAXVnW$y0c)b)Vg8*I2_;e+f%wv#VK`=6VJafafqzaZp1c7?*|11NbQk6g(hyqP5VCDO* zyYH8z68fp;Nu7h-^D$Pf!+Dcf#@oR`_+*-ac&nnY0d-kU;oVh6L>ZQ)^d@fW+ngQo z6s7t89!=)*pR6@Nf@~CRpW*5k&?i zGldCGwaNMg{*&uD9;0BnC`9~Bx-HR@L>-o4?6>aHEYoI4J26xT9UEDwg)M$c`>vVb z>YnPv5A>bj>MF<|1W0{M6{qsIy|q& ztqh5hWGsiv7#hn^Js~FkB9b`$X?HdFu z*W+hG7XQ_+5`dPM={!JK2^qN4f`rU}p2K;3pb@OZ;FbIfY%DLJ%Q5Ao$}LT8L+0-- z`~NqbC*X9@!l)W2%0G78YmCqkq}1a&q0P|9$h7es*Srf$t~*>-no*pK8D?~}X!tX& zX2*{J(YifT5kplD^48`X`1I($a7@Db2n{W+=B2~|u1PpM26p*WO0-IiO^2>D;1Z31 zDG9Z&htgkfe9pv{nVuc!Ay$hKj%$5?f=XuO&w zQvam*kHb9Z2FkHD*osLvP>ZyhWqMqNrn(<#o1%3@@$DNIU|Lz;DD!l2Dp4~|`dkwq z?*N^TIyH7?k5q{=t~Kf^@cnM&Jxx3B%=P%`)Z%ynCe&CH;2$UaN5V^An5)e*MA4fx zq2G0)Iqg2~1&(wD+t3crF9(Se*#SF{WK`@N)x6W(9zwE=xZK0dE3DbV`t%ZAr=*kZ z-j}XrKME_11d9^ct89_$_og&ozHmN!i)05wd8b}i-^0EOz3oS?etj6*NSS;PV!DzH z5XbxeMh8pnM)fCSI)y^iaZUFK24Ct`Se~0^$~8VJ8ddzX3FSDeTFN}x)bbZI@U5icTE_}#PfSiaH_9bRCVjrcaoMMAuigK%O}0iHrCku)t%?X$rQiIJ z!q4_+qHf&93{qt}`>t0Vr29KR^Ch-PAtCyjY;Xcd5L(9@PRrl1M0LUI_uQnvOmY44T-nf#DhD8-5~VH`gwo;jh)&ptzJ(o|LeMXybfJGf1LnMhIXk$$}6 z)^4;&D1GJjBgJIuMWB)D9_YAPbtN*16($Qc`Zy)DTOm^|8RV8zVo>q)uA`n_1JVa5 zP@ob$QiidWsuJymv@)B?Yb)~(=(1pBxvP@3aj->&(scu^kDm{ncxO8Quy$NRWe?dD zSr0i#xKZsw9isfU6<|rCTdmYR1=!Ef&DQ@7-GBV2S=uHiF$NtJt?w(;@D1t34hn1R z1j`ETNAZK|2*~B+0BG$m`lc0Gap3RzdeA}P@3LFXG~+OB$rWO0U)TWKBCA*q6ZA8< zS|WvVz8X>F3)M~wI2-qXj~n&zjH{0gApmVNLie-Ud0(VN5>%=l8HVdRuSP@ht^lPo z+mNHv%0^*}<3TKU*g8{kX=&*qkm>~Lh^G>Di9+)BEF0mcTsS8$Xaa=pUY~bLLl6vd zcPupV??LV*kP@QA>N#jn0F=t|OB6ObfcS>sOk0#HA|<}?Zv%{3b+g-650>|+%TD?D z?XuV^Bzr6rh6WV*0poo`qujYg&!b*yubm|)fIUO6EjaKLde{;KJLxr^C>ApzwAy4u zWg(QQep(TAHG{71pLTcs9>QZ8)oB39;OGNJ5*5 zKir$roUXr{MWowP)(QRT|0%&69_24s-2m9Wc=9H&`{m3RrrVJ{Ohd0CPh8I z{roeE6XIgspL;-vd%Je8VX?f(&659QRBb(KwaKCX@w!hidkyn;#k3XI?H=nZK{#=d zlawLFjAPw06_Q27uqdqsSYv-6e-@r`J?dKoE^UWhq*XaC`jXzfX{3+j>Y~p}vTv-m zAn7McB>8yD;9MO0mCI}|N25^PmPQel%l21k{_Wz{7H!ico>(GZp{=W#brC*R8r1=y zzyM!F7GVJO87XxvSAcpX)jutX^#w>VHY72P?GNZLI?evjC%p?8m^4EhlI5fX7(`l* zqDUnH7G0ZWG^nVVKb1qkj#xJ?TGJk2CyRheqE>z{QmBi6;RqOParTux%KI;ZIFA^G z9_Jd#SBTbPaLRIqwnzp>#lCD6vzl~vhqoLqY3*bm22VxJJ1h!BiBmhw9hB+k*{28= z(nnnlf4@9NC$sp2`Vg(fOm%7|X(S`!tf9dMKX_J56K|BvQ}*fX{lhF4sW|q1M=D_3 z7$Z^K*Z>Xsh8!ox*FfO>+yeMRrk3(|dq$1gLH~-3a`4_Me_p`*nPA{d>;nusYayuX zjpy?R;qTHm{8TA!j9%3qzD>E*b()@@BLUj>vRbVH4n3sD8A#eSz6Cdv5-lhf8ZWG# zPFv$PUafL0zj+7v_BS&kvBZZF4=QIKuJt+80BrV4U8%iDfVI`FWyH)gslwDizvy&X23xzWayTk6>`}F;V*a%pTw3c6+I z=ee$%|BOhLg;6yR$4MH6jHlU!1}5nU+KD=pp5PxYU1+S4cU(S!vA@@G_ugjTr}E7n z0b(G9yQr{Eb#hKU^5UGa;mUWwj=cW;j-Y!1vX0v&xpOSpi3G$O$^piiBA8fK7NIUJ zm`rVAFBlb7fZOh9nRTDTMFbKZ=bTk?Zu>sgB)EsuBIlW5@*SdyRks?WlWu!=5#X6L zXWqv;Re2z5UjSwM;0kfp*vXc9FM#5SLd-g5k2{5M8AI`~MM*a!RPco+tiCJRbBDGZiVqu##SxIc2c)Jw3kgGZ3o%ZN!&eg}ENC;Aw@@#mTS9rJ`@sMWTToDtV9*vrizVqAgSqK~#o@k_8 zhC>*cZ{Y}Vh5pEjPTeAobzH);n>g3X^Y4BXv;sB3G0*q2K(7?}b-fZ<9!j#)QXmwq z0a@+y`hE>4@RIt5yKNh*02&S4{Q=Ya>C9cc_1I;_1?;@js~6vVh7PoIPa3h zT@~BXu%V^D?#gwyxCA^@ryr1~wMU~s0EJs0q^>Je`FZ!VPkTs;ahhZxfsccgu_PK= zR7XZMW!(q|O<$D|a|L-MmUfDqeueffOABwQEOMt@D5wj)6g8c;tkh6}o=G2byN>v>d6Y)t}dZflS z74x(XKwj+vAUyejB5U*rdRC985XZ}_9o0^4$K>BdxRhR>c$u$TA37vusFe#@&+oXc~r$145^BvWRzLIFHcTAAMVT;jyj01O6cT+GQSYQwNEuq?8}*~J!M<+ zU|A>}k1FLBiHWlet@56^KuppJwZRYF5&%xLFf)2Qk6A)gj@+7$vRM`r0&TfmIXJUI zmiTo#eq5&I)KvY>>vMVyVQ@C=lj6EdAZ&E?0p8BwnVtuROQ!IJ7OM7r2{3p?XL$HvLyk|>vecU%|cpyP6McqS&vW&)T@vK8a zG{~#&vO837#Yu?AAqUuaONORO`XJAcgRmp*wo>19tmTz=2G(rcsjbivtmarZJQgJt% zlXIZXxi~!?&2`gW6{%+quZ-DwkYrw^IL@`s%@ZLH^j>}LFx#7i_5t2~>z?OFG~=5S zmRsnT<>Z;zKv&yfDx@qV7G;mDwZU|wz9d9=Zmh-*2sZ^V_i;W$(nmP*Lox)T=QREdxY7Kw-93*7?vs{%1>UoN9?HQdpt|dQC9l3xg39E~O|C_M(kvte5ADE5iDx1`5t}8he#G%9k`1s8L~le?WZqWxOp{-FV55 z5Fl>8S&9t(etDGvdG}^CGfKnye1lr9)ZXPd&)a3K$4(#qkFBc=i)vlllpu8a(@)y}xs=YktfRLDqWf zdG31qvV$s=Pyok5UgqTD(7g%+7sC%3Bo;h1HeZxSt z)*x|NZ)Nd^K#gmAPsu;aQ3vD8V`J{wYVf^t@(LxYXq=rn*#yMT#l_p>0KFH=E&0h3 zl+O~|#K10E2`1?R0I@+rteK;b{dYAfT|Wq3H@6&-lmphJoRG^L>E~>0IB$6xX~sWW z{klF zQQ+r1e8L2X0^LZWb5kNwp0m{_1nDjGt4(imm8KCIhhkH6WA*QuxMML;<26Wb*85}D zL|F?IOv*%2O`h{kIdMB9$*B zKDWNZvVJH&|NM8ot_IIn<76SXOZO{Vxdc5Tl8sxKdwt+yf}8p9TDhc(r}X&~|Jzxx z#oTgo8bao@g||zLUVf=haE8S|PSu$Zy#ndHM2#^8kKQbq);6$$p(?&zb0r|x_Rx?$ z6&+VK6$h;|A+{Xq|Gq!``QEAE1I8={ZcQy;fc+_5gcay%S`=rKY~FO=f)4;fH)5QL>qA|EA-dY@im73)t5 zE3M4}Y#{(3H~0)oG$o!Gso02ZNMxL2$)5y^@>bj3d@H0kvlQbH{2dqY(qfnc_@FB= z{bsrMlU0dVp6l)DJ5zPdpr2(!sI5G}@{-8=RARyHtP6Ar-aD`TT&2d}b4Iae+lgl) z_@AQf?;FJ%2_IX#VhpW2a_fJ=&q?&DpG)3YC_&u#xo)cm<04Hplf0*4ByIXtThr{+ zfSWTz(>0#lGw#%gm`q3Nix>4aH7Tov+w>o`idHhlzyOoKn8-j5XyA75(KcE0XZ|

LsT8b@ytOgCd725y(D8PKMg#g1O3d%zXcB#eSI#_qbvlz)neltsiiRYt7i4^UDn z{4zrU-koyi_^dC;;bTsl!(Lv>OprXyz&hK6FcC?t1{PeWZ7v2b)7aDoyR;g8MJHCc zKD5iJ=wnHnDMB#5A{d3KBHpf?9Tyi@^OPPzb)3wh_><>f-OJyOA~fFaibmUu|An@T zcin`?i}xgmzDw&Sc}*ko^G1Gf8BTxjgy*=2V?Bqzr$cF1;((JBM7B02qnU zA3!r?2)k*QvhCGqz73~g?szdrvbskuCEBcZKXb=b5`RNGiVZvMZjc4cJ-q-3_*5GD z-QUldO%U|%f0JZ}+?Y+p>}Ovf0B|B$Q}MVHPwc>V#oUegYD0P=UL4kfP2>-VEY$-U zCgbThqVQLFeY(a({T}z-zHz6D{%wW1ihSoG>SUg7Ov^a*#bo;dU-3(P1rUYQRtE>I1E-V1IQ9=>EvZbRGIiE|J^Ccca~N#BvR`Xi1A|B$w350-xsp zma~rK6A4#QY1R+v%}C)(wi#HtyMURXxIzjR5NZ(<%BY7DBl#n#xZ_zOlaQ+j-&u=Q zS4%eMvM!wjGP&{R$x{k-x%XBPYz`e%)7MB@9%IY--PURt{n=5<6}2XS&MR%=W~vAd z`057;T4)^$-f8RO73NZe|2{JkktjTtcE6rv88{=^2QEbs{=)r8`>7ph9aEk)L zQY3IOIS<=BaMS%hTa=ReH_FG8fDlD$sk51D{xHnNAJCf6+tHS_Hgf~LkxSrTaIbkH z2F=^K#|bD{ly?5g_qEa9z8gdA90=QsR#q)T=QPHBLA~7KO-$IwKD&zen|612KgY1a zR#{I*-!zo(-o!*Wl}*2~Msg@n^d@P7J{d6x|2foB);>3*C#U<(1IVXVI`1CiC2uW@ zJh~sCt^+-eOi=O_i^9Q^Lc7TZ=3mc$(N^s$0@@SqTAU35c&JKM)a1S$A#F$8YgaM-z+?6HRr zh=OzoBrx~Buonek?*KwWHEL>lTii97<{q|uf9tAAruV(#oq+h51aIq1;0Lat@>&fr zgWJ$LV{zI7+1Iq~XMnJg8m)({CA&I+g6^q&Y4d`0GY6t2z**i)Bq|@N%*`3KQ9*n2s6eWPSKG zsGEe8CFQsxyTg5mNke@G&>TMsM{Ap_ZuA$teGMr&1H_J`XgZ|b{_G66%ARz(eDZz5 zjo)bb#PXci$*d6oBPR!&1>%SJiK~%#Et^kQM8|oF@!Wi}xymHU6|#*zl!0rbV6`zN z(^h7)lzsL-92k>5h}%7;%_y!#RBPniu^kRvbd@r$4!XQkDXc4_Xo|JWAWrwuV!E>6 zQk&m@WzRI6%FNU3Cu2ejoLD>;BNNorTU%XE507J^h&+1%Ss z_XaNzO-SCyL`YH0ZN`rBr3kU zXkhykO;u^;i!AtHFG(WN!u}lq%K)#|DS0EIi|KSI^O}|Ka;=#$`c#@HA}2ErZQBM{ z$dYk=k5A)`drphMa5~QP9|KSbQ5TJiF2{+2t-D&o#5f>}-#F>ug7B=p5aV(m3SI)) z#^f(Z;1$rj0(#d#C9f?t-{6=CG>yTVit$Ul!zhsEei2}{@EN9F1oUvltpNK#4hY}( z+Z$9mr_EBlTG1k1S6%=#C*-+fRdZ-qcD`31m|)u^r0aRuffi2>nh)OOAGo-JstbHaR796VYe1%Cbd`WwQJdy zo1IKmHSsR^0AM|y&j28s2F7pwn_@t9LJ^1IrNkC*{%fketvCf_1m&1q_jFIJGm4~L zuMI|KHbhA|zg5(3=4vFTXK>IM38Ctv&=Xw=JUJa1+d>66Q zq{+G{xwiFYh_^h3#LPim1s0|$c|)Emd^^e)%YHp$w3K73U_F(#JQ8ZqYz%Y(MA<+M zXAZ*L1IttXAXV;T3A5>(3+*?CEK2vs;S)VMQ>&)>)THpmiKi zljVv2$$!WPd^w55DfdBFN1$|CGQ<^tyV3!;UHB&=&jswofAD$-8O+ZuYCpBQSCY*# zE|s@^(9kjoomCx5kX^L}tRiiIKJi}T>)^Tjb+0AKFtoX?0Y;Rwfn*D9gy`b05Y&Ae z^TNhiRSKH&V#}>y@~2*76GP z!e#A%7iYt+wQcaB@=_a7CWz^t79-I)ux-Rce?6YP)=@8!Hw)@lTpSOnH-FqQ9^(!5$I~5Y?%O-Y8zVnE%+4-)C?W&R+Xa~rI(4Kk^4cw z4oHeY>FL%jr%z1A)O`{Nx-e>p1N5D??d}nH?Dx_fyE}rh=x+h{2_FRJzDwYR(@0Zbh$-ZK&UqW9@v?I`alO<& z{1SQ|T_55-RMTD_D?W>J>b&+K$#w!RQnzO>-IpTfTZ*f(J)U=!z9!TUw=hsOeGZ?? zgpfXHhICnIak)e}IqCxw7eTV1zm{-Y?ac_61%Z!oDEY6XN;1vEFVU;6qxTF&<3Hae z7;sA7F9>UK%$-#m+Gno^fLq*U=z`BRDbq!iJv zY|lyL#o+CCK*^4$y+GiV3p65!Ltg`IIlWv=T}ULuHvp}q8Apt*i+|}$e;&VUeeg2f zm21t#51Kb;MaJqNXKPc*pMWXG-A32fekjU)M?xg0Y$JZnYsmE7rx(s)qc)k7vv0J& z-szAUE)k>>N$dwIQBA`7sx;|Jprq-lE45_z*3VAjxC{^j+(y&tRn{U|ssm$In7_X` zC)NOXAooYpU_C`D_R)vpYbJ?yt@wvn3ol<9h5r2X{CI+a+6z9U@{KT4^tYS=a@ZA| zn$g)z?8F29X_uz05MI%IKD)ov-6mC&L&;$%vY2vpvF`f-<6$_O`KY5jWezVW0Gwxs zSf+>uBG=e6Hh)zoH%S492V<*Uh+u2k+ ztS6Ee$klHODK3fYF`CDzu6oA%$oLj|EZw8JJ-m)WVO`X_S?5uNGh~;!I1t5{^D7~h#tZBKfR9QH%hk_F2`DK7^iW5$n2OmORKsqCU9P%t0- zHWTCJRPK}wV4*B?eQ@?;JaSec3R&*KKJT|7@WUkb$vSG^+nWPh669EIAgRv*!-n8t>L7TdSc6D|C^Mhi3wN}N)W6Geb!MW|0r1M{&M zb4;PV3X_SPfkvgo=Wy;j6TCNl?H(=%rqNFCH7(QZ2VrAbG5(HjH4hg9RQflWQ#|c2 zm>I}-aUWQ5$M6e31#xJ0%R<^$mrQyMqw7tyWjJqfisJ7|GLStm75{9*ZB>(65`@(;{mDM!O(i?jZqRh~0v@Fr@8U1s}i^34HsQL!b9{4 z+y`8Y^&o3L%ln&==bJsa-i>Csmd%;K^zdVRPpxJ z0nyc6^83@^E}EF>zGt=qDvlxUUM{69GtlJYiqp`5Ip``;Ayy4gQAv7n4VO^S1*iY` zdis+H!mUi}uE0UhY=jVR;+gC`jcL9x9ck>@%lh*Dp|D9sa1_?$Lw=oppl2X+QJ16y zuP3>L)wh2?IcYj+^dV;W;q}p13M{(5RW;y?Afj@m9#Vb28yx34#4cXU*{ z?O#{=n~6ZV-6XT6a_lWTVXWvLI6Qbr+2+;sH~={TEcYEIX!S23Rzu`L5%&j4fSTQU z)Z&2G=5V7;E;0<+`+Ed38bnZsIEqxD$<}?#GDZJPc*utW3yhE$6>o@ngIr4{`-px*}C>o!pexYV<Ef`&wn4xKzO^_B5)kwz68QHS9h~|PUz-;RQU32H`q^5Mn_=}?Y{)X4KeL}kL6DrF+uh__h*rCM z58KlT!YJia zOy0^D98ut4@nhuE&IGL7pGxnHV0TCkrz3h4K94=I=pmRP4b^$9uMQTZQ(sY+q^;x6 z>&cfFlU&M!RB1kLX}sAsk^!-gP_+S|R*#OvU1O}id#P9JZYm>un!GV401BXp1;6uj zM8akVUK3Pr9?z$*-&AS{OqE#tjD4@%gb$frlCv35ksrf$Gg6BpUu}EfB4we7-}xl- zZJxYG&kLKFxRO!m>=M4Lwr2@-LVK&@0|uK@BT|6&HtQ3{z_Dmsxxqf>tpTJ07DHz# zW&YapkIrFg6m5X?*cka=`s`RftPqAD9D&5t%P*{od$CNoTz}rinqJr|?V<%sD?2lo zzIYnBhxUNhe4R$OFl2`lg^`X|$l@z1&W3~5gT!@F<}4!*-Kzdkn=}rlPAn5Za%NTo zm`&G}^9VPY-Zcn1>sIPi$5m0z&@XtIW3NpS8g*#{xP)771z0rg(DxD+ zXBgKnISQz6wd8qkc-#x@7=ZN{foqbSr&z-(y#RM{XyU0pq(r5a>)L z9BoPB>^w&I5=BY&o(BhhZ9HcBIc=@{>p@9oi=Q$?`qoUbr2U=yxqVvjX(fF$K5=8D zkey+weuUAbanD%Ph>iF@QW5;B>>R;85q|K4>t_;e`b7YX4L?z4TR+<|tHRrHE9&9g z{>c2A+yueAYFIl_$E(K=G)-9&1>gDTxMeP$veVe$>>6;IYvt*mM$i&U(|N&fS=_9a zC&EO2Eg(fO=nd7VcQDX};Ho)=JdxCx$(v4fl`VGVF-oCZRUlQUT0#7@5H%Ix1oaHC z>mbTzZLo9=;N!WJE8~0jYn_23V);=^d1~kYHd5f#fO@T|8A?d8Z6$h^y&=e(pD64} zgc5yY=ax<3Ufal+hxyj{z$_Q!8!|#50zq*}(FNA@zx-&cFOTe-k3!ZbGqs>}V(h$g zKy713&$d%KRT>Hmj%$j9MGQzbo?`AjnKd(={#mX$aP-JoKe5bIgC?caj)3GzgVKTi zPn_V(SbZlq6f9kyNtPRK5b{G!V8>`cr5Q&3{+~w~@Hanhd1t^M4l#``z+#8Oy4`q)LSslzPIkvE1%|Aegx98tUJMX0*7ed&VNtBUs#;h* zC&(1;w4H}VSGU_wDmXhk^RqFqg%`|P3XCSvs?JE%egmYVgd!XzNW&^>_>YOs)1C4G zLKZIj-_>lgUpW*{Hl#tqE}Gg~gIP3c%dRsL*toNlg#*cwbZ97s8nZ4VO$s~KZz=nb ziBRh{Risx)Mpxs^9SkpC-VfAEy&;GqNsu8QZ5 z71bv)j;3hy8G(z>gz2EUQPr=!fOvRA#>dPENgy|iY7G)#X)yai2=vw)a`av{+h<8; z>u0Ri^H>1{CTTJ~J~dse@D~@rgcc9ER0j16*INCv4+na-0E}gJbT3!~KpN|jC+IV{ zOK$>~{Dx>%o#bIC*|fHbEui}hSr#gosP3e)YzJBYhz)#4J#Q`BR>OWY;>k|jPw`YF z>U(mb8{?$#-I!0mSYjk6PcKbWp28r26>Uj?V%7e4>-!Px8$bCBB9B;Z-vCWX(40my za4o4-t7~X8j?tKk0R`a?$PR~`uEuG!hC^_yfEi*Rxx%o42BejjV#?0bo`!LOSBvt@*9;5iE|TA9m)l{-w!^*ny2$6-0< z)=6wePy&FbM}3YUuO@8fQ>vV-x<*XJ!C3AA8MnyoBZ6lpCYRYR-ep~GKE}koLvdh) zVLIZ1fa_pTh|crmHebMZ_FWPKSRuFKFe!1rQPd1MB?e3bI&wfnzInv%v|Z1JXw&O# z1`HusBsoCTlEL#XAcTYH3$!>|e#bn)FcK4hvQfjDG*Wb}9-lOB76^O)cbu~|Q;Wd8 zd5=aDW!GXyzl+`#9j3ih`?KTvr7!GD(ZAflwoPYiB|mW=wA;2f9AuGUP_}uSaLqJO zO`>U5Cmf2M??3GAd;5{5;h$p=a^c2?DRfV|rvNt}C5)FGB6grhN&||ME9@>b;DEvtzN}@h z6mRad3(eKawr9%K2Xh_m7l3?uS%h+62oiu~(e1eO-2B#?&L?vT|tvP~Ubk+@mEM;Prqn5ew;X768zY%eqSXAaih6IHJfUEu% z;Qz5-UkwFf!{k#@IDAHFpYK^wDx|?hY`kylwOKH?IC03#!nVtj#JVh`o0owWXqe42 zRune;E8G)J_wF*a;CVpPT!;~q6Q%@+C&UJaJr&{I_N=JVLj4#VCYJNk;d#NT(@~+^ zV9J4LeJR=7jjiREZim}>)knbI#Uhm&w*C#739nWAWKeovC>ozn&c`xg9rS)wT{ATQ z>1)dHlZ3J~v+TV-_#~?fzx-uJM_)au=|F#fCzxd`iwvuGtn2W9aFD4Sm`zHJH|(@O zkhSAA{8{Ux_p-l~sdMcV=yWX^UFRf~MUbjaKwovH8j}D->jo3%p4Tg9a-AP1@jy6u zMeaPfxx{Mf5jePA=BNX$Im?OmVG7Rgt8}q(58ndp@d{rpCG+*cte;LeUwJw3+YneC z1dci>BhT}jk@>3ZH=b#A$bKPSmwnpX?Z1xAl;1#qWu^6PIq}_&DKI?rn6u65AQw1I z5rox?b0OxAW<#2@Uy$wy4vCgUW)#-@p)^U~lLw94C9OOso%BP9yloHV3;o5(^RMUk zGQ~%UK8Zi}Dmr$*f5O)GzmlQ8O8eMO0rtL>lPrPN)p_Xt6^+}9uGY8kgGb&TyE;bD3=Ms0alJhyU97&CQf0-AyPfwR$ecT(>I(|#Es0a_^%rcHd4 zwZmEJpr-&7l(4;rOwrHoiT%zK^qZ1SVV2OXJ=$qF5CskQq&&e?LALQ#cGz-we)M*< zam6cs={VcHm-##OpdhJpAV6IOv<|3UxJ5Dd2I?f2akdhAru+cihxhtCCsu<12f$2F zC9}HYy{liSD>l=scN@^0`>Dlv*Hj!J&Kq%Ek~g$-*3+#nR`*P^!FJbHmA{kF*Ipg( zpoB$=Fih(M!sh5(o~Ps6G9@|@jjEi$X$#rBk&h!$vGit-ywSluUxA^1;j9kM{{Vfw zp6;L`{lLjf-AGRRrCY61@p6ce5($e^>Sqt|*YBxFo}H)Qs3On&q%?f2L^_Q-o%97a zb50l{599AAgGMm@4#L{%!>Vp;csc5UpM*VfpEJrHXfc{iAL+PunEmtlwwSrw@G}3< zJBCkhnt#13LQ}!HthlU*OIF=_-NmH_Qqjv|N>nqrt)edTs?e3#3Kr1nVXH5+b9J)2!R7f*c z?67UGFWax3e8dnDPwDx`TWTZ+H-qN*Z4yX!(Q zmB4pr!BW`u?bvLQQ01v0li&Pe8f2u+Yw^7jVu>Xpw zUS?`J+1{}14zDNUyta+m--J^o{wDlz{Dt{S|Ipz}Ma+MQYk5Y@xrDou%RVh8G-=(< zkGkBz!*IXWI6*7bSe(n0tFx3EVR@*j=8$4hIH1dBWkXe(GA6K1jaG^dZ`W?Te^b1KQqzL613K^z*uN%>yUS<+SWqqhKnz57PWRE<&JBToF z*=%gO2i^arGV4_RWAnmDqXU)ekZ6|ObFa-mq7$t>w8nyCEBwmw^5k!vGtL3!bEk&{ zJ}-iNR;RgkJUgut-B-nJx28<;De`4Y4jqviXoPG3HITOO`@(qtd?a5PhL)370+8q6 zwJ%p3C>>f$aYapfmB1@0W|m(ca~1WJ{oNl8#3wMNo-E`|wPX}UUjB-(w&x*kS~NXq zd!loxEpkU`7PXXl)`=y1ad27O4U?J7DtdK9d{)yOSDSNl{{X zpT&4v+-T`fe4^>NgwK7~$6Q?8d5p^=F~%lRFr)JS)unvWJ~G0rC?4NwrQv;l$YCA< zbS2daHnu|V6l2~Bc;hsa6O`Y?k1#b2XI{e8AXN&feK_%T`i17F2kOdZ^D(=Dz`_-p z2X-#d6~94qQ39~Xn&WKXoksXwZukI}a69#4!dQX5>0YJF3VAQ4eUl4TY(Mn7RESvP zrDDu30y3(Dx*LTVy~ge=?;$+pkMhbd`>(5>fA2zB-&*9=uY}{JTx}dq@Hdd-sd!I@ z>7~btbAaFieK9w1of7oP|J#IFWPQ4e-)x32SIHt3BF64{GeuO&{%CR-7Hq45etuEg zExq@`8Q}OVWj_a(0cA0PZ&uMz9_bwmW9I@%uh;5c?cCJonJTCtwak~2`UX+icfYMm zPw8J?>xic$#eq0hAUSxST^0A$L%T?b?n(WQ%3B44E{_M^6D@n=^SK6*3bpp@lYXr- zyex_#3n^oF2i=OwU+Ru4#0pO@H2k9$1mu~g#_`n{HocRvE$$|u9gBC!MZr9ekqf4~ zwFv;W_ID_K@Ew-C9cq!tJLrq|gdQ1FZQO#?Ve$ORVacvsGg)9?x>a36w>0b*kl~Hn zu}Q87z`Ii~4M}fAm07c-S-no)x@~eI7ZVUwsHAtOzMprbbWUG;6nO&(-8UZ2f~b!} z&*EAth!^Ihy9xcw^MURU?64my!4e~Ecz15`%>de1vZr+DpKJSbw!$Rp7pnn^)!AKl zY?jOGZNO4T>xJ3_OrGLGpQ~Q2@gw?#4ZkImwxi(-gVL&hk0<&|stT5;TwevkSPJod zHeCL%M2a-G)sEh^6x#9K{;}*ZSedDv)qhb`7K|;A?~xQ190!J$Ns@oU#Hk8Lr`{Pe zDV&NaHh)VnH*m({@knG_Giif`l^-UU`}G2j?VBUBL|c$fMKbK*(-S!ftp~4mbLu#^ zG|RkXj;_Dexx*qFe`)9B7Ad8bFj}AS?+aLaY^q{Zk z*)U))yKmb~`@26Z3(PgpuZLX(bl1zrXZ2duiE8oby@#&eneKdquzIREwZXZ=~rIX9{B{sW#X71^r7pivHaPH!+)xGvR?t2AZFujFgdW=41&)RhglPHLv=9 z7n`ECG+bbf0H>?K9ZRT*S3#_IR=iJsiGgNf{Wq14|v<5tr2YcgFWElD4lGx?n zfdXyW_{ z)3XumAH~c!nICHy(F`qzz$af%xtPtS2V|QP^ssYrT5u4NydF?clONuGV}sCc49nHg zGg7FWUj5IWE1rOVQ1q-VTt9VMS9`qgPwhL41IL`-dt2wMPQ3A~CGtd{=QG42;Y*E? z!DIbmK}o2lTUgScVnf^;Fh9IrBuZH3yfr|pk^W+n2#EY?BVKqbJ8v4$m;GAU%llW6 z#ANpC=hR&Qr$e7vUI*Kc;pzzt+S?rIN1mv?k%z9<8b2hv%poWspCHr_(}k?#J;4PGD`LKAPK$7jN6#?o|q<8+jGA(5dhzprs> z22SuuGa0kYqZaQ|_9kp;?F=C&aUnxGRp4Cx!^@phjl9+gbu7F`E^vo^nRdPY0ixnR zZ=&T@F~$b;kL?ne!pd}*cTz_*KRVDx`7rJMViw_!0g9CD&0)b@@2EYtBiZeQy5Q&+ zqSuA<%DsiNz%n|W^D^0Bru(p^s>$%CZ!$Uxd}il8{NuAuyqGub`o_Fwx4f!3rz8BE zqNin3yDimv9@J&`PFZ+=RYD4Lz>uwvMnFV_o zvf`9kbFX{rzPw5WwJ76}VyJ8GI8wjUb#knQN~CtLk8;0(uH)Kd1Q}2XaP_4glvxDL zo65vhWzx7Ss-+msZfyrDq(m1sOtp3){(Um#07C8axUF)UC(@UF&VcyK-)T&n86rrS zFwNzC)?0#>CjzpDLfiZ3?5E}C@wI^zx@FXsX7eqy40i&sPHVn)0lo&J-9%QktI>ag zb^of?r=(JMv42=EM`@<)(5K;Qf7S`CFY*i?c~*=9_8D5(wrTd}F264)$-=T5apa{S zUhZ%Tvw6LzdflGF-G#F(Gu@vTQt=So#${r-n{Ies$kg8hNKphz0+o5 ztG23BS&G?oVab;F?65{shqtB|T1+~Vw^8HjRz9NhmZDtW@;ONJyrl4ZkzZStX!zvY9bqz3@ED%>A2l_P7a1z|FL6RUl8HNfLH&F44Ynt)vEC8H>En zNjN2KD%B}VVp&#@S(1~+b#@+*4nS=KJi6U)o-Thh-N&Ij1?crioF`3VazszjiK6I` z)~~_QFt?hVjqya5lK^nU)jW?nNe!bgP%N2F;xSSCV$Ish-Jar zG#r2RtYEP-WB!3DUS#fvVm?%BnFX?Ampsc8+41oAQ#YKND`v?}+iT0i=D>VegiB$a zKOQ#qx9QyUZg3g$~U%*0-UEN#G^{DUUn?u85eWs?gl+ zKcynK8_s(;7T;dhkDx%T1vKq`pF`{Jbx0XHi2}`>7S&Y)d1Zlse_Lb*2(AEs5|2Db z%M4qj_kc)jaiM)cQfcYht20Ne4`eL_0-|wHLETo~A|Us0x+dYr8xZ3L-nFBsQ-OFf zQEj)cWS9JacvEml;_X%?kR}Jn5k=a)i_v)si^FnUG&x#=nyEB@KP8#CS8?JK&1nQa zco}43=K*$}t#||!tgofg%m!n5d;gG0h9c4wK}>p8*U-qdjw1=GIx)?pVA8*~cC;Kl z^pYBR;SZ_E>1MmPmYE)wd&SqNHUfqunw8_!I45W&DaZy?D4w`+c_>r2LQk|;BNV#8 z+A>FM303kJ6$3eKvsI8F7qPjMinec6w>qWn*(PFnS8;T5=+*J*r~K~VX7+oTIg&0>L9xqRP>UNps%vHUF060ps@ZKyXz+n?J%&s&E>ymOm15r=?<{FU7 zAoLg!J=z-WwT<8Biv24u=)EgN22lf89ln)c9Cpi7zT$xA6z#YI{YQR#seBeSGpK&qZ-S^{@;h?ppH*fj zA<}1Lg>qC%)B8)(O>b#R?s(h$(0iNvRgy+f6ELT^>pbLs9^s76_w+Y_AN-XC&~kv& zd1k}ySvdXe={fDs(#KBAF|;vG3gn0*)-C@vPhaGr1M&&yLE zP(zFyTxh7C>$f##<>zS>R&LE9N{&ZxllVanPszKw!wzBFkCZ7WQ5H>n9J9JYxtx~9 zl&C|j(d%n`j(x?cHC;vxlF5W!*ShDKt+?FKVvK*B}BhKh!cn?;ubr zW##Kf6_&u*3*ZyLEx$E6^UTkmNFcQ1Tbk|~dI=c53J-Mp zb1R`~GhwU5W}+t7hiA?V2%=%|a#Tza)4L*%d!#M;PS?VoNK(M1)v4z{;6c zS6KqQUyuEC2j?L^D6TDSXzFSQ_Y{^t^S}HNUrijP7Nb= zw$gbW>2#CvcfR-(9hGiFy^1|R0SQY1U$SjW3x1s6l2{A>9WDgde}xDteSCPzNR|>& zIXn(e7>2sSxDD2OXQ%}`-%{ehd~ZI$8PmGBw%Gvh{Jy04z=h;k4b^@oZgHN>e28HHaeZ>%*R!E7ZQh zmFRv4xq2Y_p$lKq&n}U)xNN~T#TZ;Ep%SJSvMS?KvI1CSb5c2RWXslA|$A+ zU8!+n)e)h@h?Z5WKfr=Ti8m3lNYhT;Jz}>u$G6p8DH2?(tMEPJwjSe>T7^R|*cix* zYu=3AIsB{MLC+4~(R+i5v0)a;6C>TF&_HtuwP}}oblNtm^Hpa8xeicKz46wpRl+1& zeq5jG`q|yQd1T>y9_?{S;HcM%iLq|GE1 z=yV_4Lnjr3WSIv*7j2+7XeGPB72f15kax2+IRJ4+0GgL<&hU;MbSV1UtLbLcbiUd1 zZSB#BWOzoMB1H8x{OUA(JB6ePkd0f?gy8KvSJ%@-+5VLz{I>$%^JYQcMtDpD1q#ez zp_Z{vR_cf?h3q~1EG?ru@);5p(aAa^oo0D`FR&06xS%_eh=%-*6~&3M)}_KypiOo! zudmR(*mq&=F(=$OBTBg026!erEfxeart#4M)3Eg5h_DbMvaWM#!r?e(V}hJwoH0r~ zWI7N2=io&JRB7Udrd#_YI#2HJAK)JOEvL!R+rO@c-hGI8`Rgbqr7=x&Y#%vr)w5+h z;qvna`3T6se{mkj2hlzUZq!VeSz$t$ZCJ41?nl??WFSn>v(*^7C`rhD4^IS9H&#>! zn`!r9iWORzybU|8KrJpBApjdk>E%8@>|%-Q@*N7=jCKU&KbTp-9YsgzZ%GN%spzev zi`Nm{fMOuXMaNrK%Srn`v3h^~$stNUR;|E1hjZyI;uYa-jNq=oF7J`HLRTJ@_Vc6qCg3 ziT+!MUHW^*B80*p4^yab_nL2HpYkc5$$;Eo86$b<}PJ1x$ zKHOCNF`e#27tU%Xyjow~a` zxNZL~zZ4&(fKo8+c$(lyx_?-_9U?UQHzLJw9-n(1G>B!S$nE?rZeX-+{PJ{aA&%GV z!()~8HGyU~ltT*z?$rOEuRi!Y(HLA5XI)PSFWetiOluHmCWKxF7;kgTXxpu&f zhOXHRsM-bV?I}LcNst&!oKiHL=jMvsv@r$FR`N_mV|{;2LP|TB08|t%?}XUn3Ed}& zwpqcp)HRtSC0~|1T7mOF2`t+|LwW!H7?R_}iqP$(r7@2Fd|BCj_|f9amAU1I@a6wM zZic%!T7HK_Zz)iHtDlzmf4gn|>g`(Y3Jv!4yo*x6i% z-Sl>x123k2#=oS4q|QDyF@f}h)^;KidfhgA8^TqLgk#KacY!Mv!jC)Q8%j`$_1ELB z(sdWVfbU5E^Nr(NwYo^#>kaB(MKQNG*6+g;OLg8Who$@ff3>#Ekt}?Ai0kcHX`IWI zy(F`e`G`xB=id0oCCU3( zz8y!rL4E!9JNA3i6Hy`&ItImEEf5I z<=(b>X8McMNl5iTwPqNNYxU|BHfxbJwnO76_U6ZKkyVA%^#zr>5_V<(DJy+n`_xWi zM$jnSGEPJNr{MoL%3N|AtKspW()lCAupzlqxQcY+L=%$z5t8c5_>8f#a~$v^=Tbgq zZNte#R99}C3uz&cOa(Gcfqrj!!vLHbW=DgNNzqU)A(_U^!_2yK6Q7R{X}Ug>FmI^^ zy`Vqx3c&=}piz_{ zDX4hWVTl!)#m@2klSA37K+g@QQRYV7=R8Ffim=EL%HFxL=56HKNEa)zY4^+?^4IUU z=EJl^F!{4g;R4ykzE(X+2*V4vdDgM1gsR!IKQq4bVM#?yiM#WmJXFjqa)_vuxbeqP zh(ng{Ruf6kIIKv!#3hNg;$bH%M=r`{b7jc^tZr1X{hdDYX<@5AryP&dl z0SWVkgovu2ILd^37}u91nOgI?4i;EG-kQN>kh?EChb;*Y@%cd-bw9p!(9=iCo@%=C zK(>@|O#2Co@EhclQdF< zL8i2cIfpFRETIi*ypE_N=E3z#j?pApqA5nFn2pHjXPNpIvkIubj%=_+O>Tef=IWHh z^zOgcIP%^gImoqJAe-D1{&Py* zkmc|bg&105^4{RZ5hb9eSH(JvmxpE%l#@r(RxpRkwdo042%qrcI!BOh4x@X?1BJXN?mE1>UdV?((FJn^< z^^MN(m%~k2Lj}*&8Yz_CI+iwABO*c8as@=*YDqm@C<+@`enTwt7TesP7iH*z(5n(l7Bg|)Y_gbf&-5JOK`2ZWip z$$V#O^ocdoBvMox`4XX&!Zu%o9lR#gMeGrSw14IzaQCw7=dN*HSEEk3hD?#Q5`Lr( zQrq$T&t@1cZ`jQ>2K7?!_dD)->;RUHn8^&=dL`v;g$=29x)-$Wq4^xDPoNsu)EaY% z{cYGnr(Y>C@j|FE7PQ5xZ<;|ooA@HN>E<36PF6ehb$@a7>QJ@FH%rK8&38*Wmv%sY zb+lZ=^p$p6_C+m>H2bLlYj}eNjn`5}g5_-sJ;Yvt>uma26DJM6Ebdg7~3ve|?81}0TAyp9_+ zs@&5fax{!*%ewjbdXdat-F)F&GyF;R4yHL53Q*{A)hB`66xc|o?qzHkC_DXvadTrL zPMpNoZ|25W#*QmVECVKPyC93=sMKDj9<>@`0P*J#(3PO?Pf5F?x^pgjvwhtK9w89W zb+Dfn_G3yxXZ#}V>*eH=6pV2XPfGIQtIXrDLz;9D!6o;_k!$k^@8&g_ab583d~Asy z=b#Yyz)4{WZ;Q1H3BFVqj$-cFQ0fi0I2~UTBnk`+M7)^iE`w6}dy7-`(sL(|H_2N> zOnttWdYOAO_h`KOH>X!&$5-KBBq&DHcKb+goyqFo&%)XD)>}6JI>hj>z)flEu4?sF z3fVK(y&FCABhDQztMmL`L1KA@(M(^74I3R>aNb@e#jpvYbI}!jFq6)&U-nnxHa=B1?!9ttAY`2$p_AU+-8gW9CQyoPz6JV(a&eZm!%K z>!~Gl74qk^PO?DxK#wZSLJncsA0dm3=*(Q?`9HG00xHV&`x=m|pp>A}B`q;D3<86q zAmE6Eg4B?bLw9pg5a}GGl@jS@hE%$xkq~Kyl8#{*z8CA~{eNrCVvWlM@B7qw&e><5 zeO$!(Z!hVtUhzLtC(UdWliXx%*vd5tKOCg%i#_@Mo9Dm(IlW1JjPcYZ_P|bEHip(a!hmyik+o?K-tiB^mEB=(`=OdU zQY&}=oVMqxa`Y!L<#t;Mq{CCZ@z-=Ixl-YKP97dcZ3vvCAYO}5@qG5RBD;FaG*lGh z*=*9MZ>pfNgGIVYq32qEJ3@v8x7niTZkIhDfE<4&3d1|TYbZ)I^=w}J{R3-(QGJ?A zT`VOP+&kT^`w`ffl@1W?`~((msgZDfO#sg3S!-{VydbbT^@MS9os=Y6LhGWBR-LJU z03u;XW!_kJ@Gq~@@D@SO85~dYy3d$3ftP~+OA3HoFLk4B+mMBgkM>mGVdXEExk-=6 z!uBTEtkzwKGqVu`X)nTITK0?F>IagtpiZWH?U$V_)g&;T)ceHEPe6Fl;8Ri9hu8+q zdo#a*rr&oMj&Sd#j82y{gpROmX<*7vK*ng}tr#ubSSC;?BSIihu2^3j5?iPBN}y;! z1-rmM5Gs|t@P+-1b3bk=#M5%}!Q>_C_N&JnXurNjs2-oj;)Aw|mHzDIIpeh9^q?WE zP}r~b>%h5hIwEw(F>RT{xQyr7TuFhIm8UZ7p|jd<8s}oWlSS5Ao$WEjfayU=|K7&| zJj5*s74NGiW1vvc>KKOC|LlSrc5+SLJMt(x4T>Wzb@&>KVJ_l}Ysy?RMg0($zvk%t zfki|mn^CNcF2ah>8MBHn>@4m|+I@XeJF)s!2Ag@~QiN>B`)EJ=Pb~H;`1ri`xkchki zRYswyu|_yfI~{ZT{#4Ow&q$`P?oWHkm)zu1+EzaJfq6x!0oD+v}#C^ONC* zY!$Z(9e(=u9dMa(b2HkfgGQ z)yqPzQ#p%vNvfK0F!gPr-y8M76kO2K=`0Agg^s~P2e951Bbc^o*(5h&yYE5KO*H?DcBt%In zaX~a5=(>bQX-8IE;5LE{Gh{CS59gca`aJ;w+r)y^V}P?>eXRIuTX(Yf;?iRptkF?> z=>BgK8!w^Ovp zF40kWXIL3$LLkDDVeJ=7?HP7~L72%V#%A-L$tPOpN$4M_HPPSSKz$ofE+Ko1+$Dt4 z)+sOD##qAWPe~)(#Qw2jK+}K!g4C$a z_LTsq6#8vdn0a4H9yz8dg(grA!x|_z`s5r``1t{Pgc>N9Cup2+E<+uH@gN314=QgR z@1puRys`1&v1i_b%LbltS%}s5-U|%Wr`gNBp1bTBYx(_^t*2wA+~e0E`!csxF@O3$ z@CD+*vOgnm?mU&8T$zxDgZ5C~+f?J12$GUj`bzuJo^QmhHez-WZLN+@9;m_2A@hwF z4XOXTS>q@=XbD%Zc+*_$+YVjpNv$x@WPW4PMH3};0Sq{%d{2t^ zW~<+W%A?_Es*L06XE6a}ca>A53u9fB#gL`Sj2TRCOqkdnUOGE$J{01W-6>hZt>Sld9z!fC%30%g9#?MZg;s2GzJQf=rmHZE zJlc3qzw}$W*^nUgCQG0k?iq1A>LG-hFR%$LyJktXG5=Zhl1OeS?6N6iTN-dG{)_5K z6AKm}QsXUq*iL=8e$}v4dP#-C)-*z#?PJVdO5f{8fwhCNcw^1t3W3;M3(;#mFTKP& zSO2V903ImDbT2sF{7@&wK)iD$(&NRC2awz`T;D&YfNRo*tC)!t_89k-6CRiggce4b4x202;9eq8<`N{%W(|5^}(x^ zzjV@m6E0Ucn;3BZkhCnf=CUB-o_K!)!?0~;_b~G9R|-L3KaCXF#h(3D+A4iLUST>@ zk^Jua7w%xeY#-kbJxLq-wMPwmfgonp+$Y+nVZD}6Kzi2$%$W%(Rzuj2PALGGvJkOF zA>=uuNnG#lWoom%L#0^T8EiIYF!NWO0u)mvWL1B`sK{i(em-`4c7PEFmGlRtU8JL#_8%BC(jD?4X$@$`3mzViZyT7GChIKLzYG zEDhvCuL=9S-g)qm>HlnPH6lR>keOIGJ29F0nx6b zIat+s`zdm=BBaihv&5Q!#*TEBrcM$?@=v?~?Jl?NtAm=bZDRY)J(rI$6Etpgg`OeFtuWy7lM*W{}_LlKWNmObGV3c@VrrP;doNA#NHTCa-WNFsHm~(%<^i z^8lod!j>c#w5g0qxC5Jzk;T$%!v^~S%lRY(`|)xIX1}NK_eun@HPxN?|I=Kh0B*!3 zo=xXJ(Mpj)}4Eq}9;$i5=_p zi5zcQf}1WSYOH=qOhc=E$NhZ7_G!xDoO@3`T3q9vZ{n^kDu4E`edf<$Jy-QdDqt;P zpR66FAs(GCyI!n z!#_vGp~Km(h(yV0yi8I;7Sb&2V!6xtxvKPYLfd%W2?rf9rJ}Y2n6ZA`j+fbLGRrR3 zZ4#x@cQm{)ndQ6VOiSFASW<|ss$dH|Qhv#?2Q@-iy~&70X+ zWUg_WKlcy1$M*(r*B{yck0^C$S!c1+)u9j#Nz8Rb*LjH26=K#|?8CWp?q3xo`yX&V zv!127WU9y2+{8g)I5o#SEBWZ$i+Iy;KT|G@c^QY*Ca|s^_!~0yBVs6#+hX_hdsO2E zu&eFi1QUGquItFhx;n!MR+n_p#6951Usi&5nrR-8&zA0FftPiF>-O%QE8Y#&zzNgi1>PcVSY0Sj00zq>vlo8ohw zfg{gGUuCRE?m>Cbo`#kNQaKb2WFz6MeZaN^+GmI97BtS$k;u!15tvbQr0q~LtS!_$ z&9M`{+#>k6EkI=@bkgEKz_?^1MK*}a#8D1c)OWJ z;6bwFA?@o<3LA_IrgVyTG+MGka!LQkBLP26!NiA@$n082TL?A= z>R@dukt_RA9~jz(u?8<4aTl}MjP=_*f%YF*Fg1Q#A?H}aXq-8GI}Ba-aF*AvLri!U z`Ql@GgWw%)P|T>SZ2LO-lzj};r59XZ2!~thpMK^1E^cP0@m<^lF}?&pnRKqd{q@Ig zGHfK;l16*x@I@0@;}u?bXR|ZZ9wTjhwOj2g>yTzcSB2gs{L^v=Ea1yJV9dp^D&WHk z5&b~ngbn~qCeG^rZKHlVG9`9|eLgux?ezv-#>Va1dOvQ%Fao@P#|4!QAtb z=ois1WU9#hFovkS=!}M&uk_|~Fk2P+K<3n+c#o}#ruPaJ?WvGK@xV68?EUD5gJtus z1M_E2oShpg&cvK0HUwe3`X=4tXp_b19KAkmU0hWKeAm{ovZ6AVzc9a0BNv3+NKR2) z1_;13>3H+XW8=cUyg~VQc1JYT<>?SX!Gmf@jzu7wCet-7(Js~nadqg}j+dbwEp5{u z#3bzIRU$FXvhJdnTN-n`G)^o2bN2>3Fr>U7$2oD*(D9n<;}7IFku9J>Auw!B_+p$< zMi`MsNKyKQ<`fsni3j#H3FmM@6CsLdOZ^_~NOw?+5`~G+F_Dh=$Q57BEj}Z*f7%t45h~#XPDUh??$3{q1NX~i7=1Yu{!M|9(fYLs z;`IBeC%hIl&u>X!e)~Cnl>DhLy%<|reK|YQZ+lu{*XiUP&M@^m(9phsU;K8={>F6h zTjGih^NQ?&2ykvc67FO!;d&w|fj$nIUCHIoOP12#2=OXg=P#3O6MVd8I@+P8c+4Kp z`KFC3!V-#mPwiqXjazw_hJ8@U+_vu42Kt9GL4)ZN6sM;%{N!1}AE)g6i=a6E(%V=X zyohNb#Lp5PAOj;T%xlR*UXT)C7-mBC!Fnh!tH@_0K=_@Ih7A{&eO4VZrX5IMZMEkEEh(MKuf`|DSH0!Y)A_3D3Egn5 z`@N=3b;N>hl3Sr^9_j4WKmCGlQx_@d`LmPx2Lve^!!BGewx5y~T(%|HvJ#;z;JTq@ z7v+# zLKPPxp)K1ee&AZMbEmq@o{EdgzH)ciV(TvLJWptasKDseT}o!gXF}w-_nBCs zM;QIZ_Bte1S<54DN?e8?L~5sj6Zf!epW4kW7~C4(P@SAFIe*88_x-m^P9*4Zty0&3 zs@orw2vTChZ;y??_1#rf2oS#3A{pgQOMI)&NaNrZ{nQ?;5gF0V{rKX)HYxZ01S03p zy%9qTg0LsTMRO-uaOB|Ja0mc$*81*-0zxpQ^O@>%a@ zOCFF)zH=}<{a4JJ=ke*L@V#fs>4?vL9a0#?XV+?nXN1b_`FVTF76}!F_kvLX`p+;q zlptPdpfQ;=XX3Rjf#xxor&M<+C23VEF6>DJjcEABQ@ey{pyonCQI@E^q`CNr^}#RV zGNU64-#=vLr%ohIY=@@C$jQdpmXXkw3VK6mOCiJo3}d zw-P2=7}9(kSR7x7E;ZcC4GPp>Tz5hpwE_huef|H*3p4qi2`MmeOcnnMLldlf zE7b?$evZ?XTbaD|SrJf|_b2{JE}dF2;)N~0vS<$tNePw1KMRtXzI)L_dwFMy!)G%# zPd{BmC_t6@!bc0fnl3>@dLU$M`H!rh*T8 z;lcqzL=R>BWJG!9ImD|t!^ppSonV{-_dV92n!Ersh zi-w{TLJD#yjuS7}`pDXjN|WuTp~&-rqP73{q36d|jt74!1t;@fU1#){Cl`QEyeD)m zF2L}XnP^Q*crm`(=>3m)Obp;$s2DseJBIW;F&bcY)o>REJJwxQ@t19(&5dA&`0m|p zIeYdcIL+n%M0da&CMd|AxOt-uBI@yIFO*UH`-OOALcQK#f+4s8)wTd!E+p0GZ`c4} zrPawNwctGl<)b(kvs5HD?kRm0B{%Bg?VQZcEOT_Rp&T3T7mKe{f~EXmmmQ?~i=)Q3 zEFEd;J0%5|x!^O5{rl=b6giT}{?Wjfo@n#r`U0*$052JXiS#h-^!6d!_*y|)Ku&YA zKh*GbYbGvh_y1qiA&3oU$OB>U))IW%fW&)m~La%t1A}V!x0-T=+aBoR}zwBK2LWva7T6 zpi<;9tBM*1dr5VHo52M64xdMFe5liA_=R+DD`%7#vd;ym#SX;cY9es{0_+Sd9Lp$IngPDZBkQgmQD*k$4*1vbpM0_M0YALR0B+O)n+SHM9u~2yfuz=4r0s*b^D6W}REPoos_vpf_`Z z0cc|1gnRRhb3e(C4&~NtO^Iy4KqnAe6O1N=LB3LTYK;dXo4Egfxx=9TmEv9*4$N`& zG7V%M4S``75Dnic4%J%X8p^qN0_BvxtG$l;%V8>rl{@WceMoV-nBq zcCUVD1mf{z|iyU-S!OJNY*LYMIxM!KOtb{GXGiPdTUiG#p-%GvLSF-*k;o zaIR!cl|?qt3@=7m89INhFFU@iGJYUVL7M*lfZ=`AgQ7qSl8)z8^p;ZIshWymhOxf< zpOfwM>U)cVQIL$_$f_&B- zMF={3R__@SJ4mnm`p%TQM!ijyWivnBMat^pN6Y1TXZmR9F#|*ZpZ3HdwcN7LC*sc3 z?CfmKbyYp7 znH)OG5nO*~lz|h`5?%`RT~*U|(Mf8eAsidGR$z8AbF|xWlvD6~ao8Cj z0m}9~Y3cJ33xE?jhnK?9aLWqIu`+3yqjE)J(FZ`3!7{n{S>MRh_*^H_gz4_*G( zJt0S^-CcIrc9ro2F~IWpBvH*g#CJ&()4o`7Ixz-4GxC@@xT7vWb{^&a{4@;{CN?u} zwS$avWUm+fD5pOkqks5K_{~L&4(`YFkJVPM{EPIKZ_Nf>8`^rjZ=*xD3#PsFIumzW zqMA5YviE|DDL@*XcqrK>c6_pG+tqyRD(~OBVX@U-Uqc?TmK^Q}Bm)moW7EO5wUNMd zim+vv!!^oLgULJKuYkuQH&*u*eKzE&WYcV(TqkPPcf34RCE%(Lx2tDK`GH}0-OJdp z4#1$lReJM3&W-nB#?8jENsThC%de)#RdNxt{l|oE6AyoX_3X2khP(H8(Z}C_j^S;B z2~Cx=$*0*07w&|`j7QxhgY2eOHnwE=?ZvaX08nno%m}?hj7^IXA-8;!Xn)ElE4I_m zb(OuUlaqrcv_8%zvd7CUmelaswAUAb)oOKM+=*=v#tgY2L8oqz--vG^A!^A*A8oxj z=X*|1D5~5~8CsqacmM8mlJNPkNL+kcBw_|c&_h37`|uF-rPzf0{>6X}u~RCJ!%IGM zugI9z@8<*stgHMdFG!;DtqD{V^v~+xpADG9`q-GILm`<={7c1g8|O%pgdZdKoZ zH5OFZGsKiNCTo8#(CC8<>tf3vMrN#wZbFgy-5nO+1N~QaYcSmjhVd5h&!bam?ERNd zW6Kt%4%pDae}!#)FY%lN&Iq^E`w}MqkiJW<#YQ6HQ*m{1SZo|?W6U^L~+@~K4v6bF+64)zL(%MD}qb&st9EPG+ zisBva!j77^>^3(xHlxM<%+H@MIX6F7=Bz8Hf0!@LbqffSCzPE%>Ri+o`6(JL;1YUQn`dXEcA9q(G1oG(k4e8{q;G0K><9`9>J3& z+Q=ZGy`j?lwP)58rpYk0+NB90+2t`>rJ-veB`Mf(>_B?s4wvwML-aqv14Y!pBw#|- z_SeTs+?r({iIPo?$o8qpyM7yBvO#fu0g{P+-qUx6^j;FLivvIL8|4gfyoI^h)-k;)K(P7QnOYfCM=_afBLADDDK13)D&90s*B+BQL29w9h2Ey}s_IO`HMB;$^s) zujlIFtLObEU4RDFm|lHa{Q6-dpGD=zIM;9we0mL=CI<=yO2ng!Lv<=7LygW<5xz&E0YfR!BB3G)oP5d|nPSm4 zYVBQq`u?JdUjhD{Uy$`~O@0{bV4fDeVMl2M>%^+?pSAh>%aW>`SIo9`zo ziN+fpwC@@*W})tFb7_TP1g*TQU+f5+*rFyI`9pXI-WHE`i&z-P*2+4PIC+}HA(f_m zc0~7^&ta%tuBLk5Cg%EANvKXc_h$MJ={R(H5$yGO#fZy(Nw7?TZwMBs`1|0wBG0Tt z|4rr)*v{81kBe%UjTB_t$LIu?NJsZ^bJ_no{6m}&_Bjeud*2ZwyFKpjEJ}o zfki68S+UHjfa)u3f#wn3%M%t`=Or{Qf%H=uzF*uid+5w`x~B$zys?0N%BF9MZFZI* zGxry9YAi9({7U`~8=AzlQ-@CzTT{Wk)8hY5Ibk!&+I-I$-jD_`q}EmGmkamXc%1*UM3i8HdO-2vFwTO4`@)?Ajv-D@YOi+q(-Qzh9U_j{<%&eed2 zVaF$nFx;Zl#$?0Bq*Z77dGC&W6$Q?iia_FxuVGOIGC`n8I#g9qi#5OcF(>_gNR~t90c`uZ2=*W26E`1&N773m7ajiv> zY4d*c?J-I3&pfgHPd4V~EXz|yMeAZJV`@K-=G)Z5&#Ps7+pqq`R-(p}oP6h}N_JMkK zlqX~>usaH}d*4X-;FE;kt^{*-I@oE+LKwTi1wmpy2g_I#6)wbn@4|oYh?4)!Yx~y+=l>y}PS50i|i`)%1(L zt-J2;Py5ffYkXw~^x-heVV=L8v`|Eau&P&%R%xHB{IH8af zwj6R&zFMAsYIFMXQz)oGyLGrNOqfpkb;a*`Kww{bQhxSZ%Yn>)tW(&f6`xN8S;h+3 zM(h{ZK3&~I16t=&#n2uzA)+D96s$voyFtzyMSFoS7V!hjvz00B4J+VCy+fP3Ybt&| z-6c4HLDKtu6$MlbGlqeXM>JaR%6mkm}G$4L1+PO z-2MDaqYmdGxcL!sY!mq~?E9M44A8AdM`rIWd7%Mj5HSi;ItKdg0#iH!I~{?IKzOid zRX-Fab>=vA_w_87KTzxI_0+sStuXxP2sy*)1{{fMPoTZLR&_|f$qf3D+lMC}SABpM z!}XYNmmu1?GC(`%f=%z`+hZWE{eP1{l)}jQmmDC6zhBdWphwJ!8GZsJAqd=CAxJnT?@FxST=g}LvU=uz@KcyesFeAHLuMg*8C zfrdC?#U7O9X)VLG`Ikc356oYZumU$g$4#B{JX^7Gj{PKOf5a|7xa`2=>;+DBw&E-i z=j4bwd=DaObCSI8<)+5vqj~1EUm7tc%i$~3(r@$^0aTvi2&SbPab`YAZFM#SlPT5G zmGwbz!Bo;ad--i0LRZgQ!VL6dgdOTSp|6h)Xa<`|R00u_*T-#@#4VYozjuif&vl>LVp$6V3KNbNNQ z6BP3tAVU8(BfyuFhyMGh(RHw&B^^oT=4Ye$`u^oEpLDK(sjqBaMlF-6xu>vbZ4$3>`f#Dgpa0MF~(B)Q$nD=P{K(7>gc3^;UQJo#O&8~oHCn|V& zBeoAbUfXn3r5DFMn8humClKJ2J3tk}TflCUal=Vf4T8g&@CE}gVqPxAd8TMc` z6ZR^|JyMNylGCGh%1BQ(qkD-JO4IB88 zulf~*(6`MznQBo>rSu=&?!P@*k}Gw@SUMbRE~VqW@xb6uiI;Lw+Y)TJA{ zrkbwelHAqN>cw9-oZks3u}o7KLUZfeD-KGuF-NcJ+S>iyw7jd1>N%fWw}Ab!G%Hx;0u z0Ge%wj?xuVCsGq}b}DZ&agHxhDU&Z?8(*_uMEmw3)?Qe*{ebk;3BnM0vo0UXEfW!p zVfW$fIa0}0oxc-1Kq5g=ZmpI(8M_xzk(d@re{1@pYjG&*B&j1&SiM8XQQvqj z8!4}BJh+~wn&Fde-Y-~dJqT~D&Hb@8iM2lX>6EP}`d2GnWugxM@YCEC?K2~W_dd6= z>yz*@0aH?7G=NmQ5W^|lGTXD1)Y8LJGJ*CKsMApv8k7}yr~$FoL~0xC<(joutY`tQ ztl&TRz2tefQ$=zdoUY{8S9JfFa%G||tc)Sh2%v@#@tbk>BeWD}&46YBb~>u}lY0e! z-51S=K&%qRgI1x zwq*Q;7Ia)S+{oOI#B$8CI=6ZDEn(1-=l?VAMljJI{UR6}18{re1>UHvz3$Go(=g~! z`9f2d-4@XDB~|2A%pJppx2F$j^_0tsL(-jVPdPF?SP2I`W~Sy24*8JF=f?AaztES9 z{wMBB)8@bN2_O-Nk5Aj|LVIx);#&Du@ad38=e6+DafYd?M_rmoD+&5&n*_@tl1}Oi zL!`>+{`-T^`{I2k#*%%Uz#_K6z-lECk-)c_%CmdzqxPU=kPD;EBS;YNG$)oOl>DWQ zXqa$Y_(txdAI5lGf?y&xI_qplWod3=&=HgE;M2X4&pa96*Bjweof12%dn?!=c5yY$ zOV1V}QuSsHst^ExI7SC_Ct5mLXZ6_%j>%ew&*@MWKvDXIUB z3V4#u4TlAi3za<7KsU`;QlcHh_1vAPIOG+@=?@a4?R9!Sh!n@>u-1Qpk|0cS8Gm%_ zFSmP<(0wWF*gLfCrL+VpBE^^PhXFn(+cnkDl22T19+*i8GSEd0+L%5zIWKt%puK7+ z8yp*ktEzzxIWF4iJk@O|@_bM|2_l&AQt|37e|GXBkHHBZUuDu~v;;=hLo45SMZ4(7 zIu}Z zBA!ygg8(*P_;qy0id_V~tBqg|gjG*VIT2Kn`(CC)(^J+VB1y7xqcP{=zXRHjaZ_NHh@JUp zpkPfOmxFDItX#^r`F$hP!~iW|>G8;vhRy!YFs-Nflgw)hCEL*q>bs0Q896W0<^kA3 zO*R<5)`6#|e5DgTLDI>ltJUld*%87=UyR7a#SmLmE!jThX+IM#JH#vhkjhwkSq|ss zN}o7y^~Ax1fl|kn#W_sHM-*qtH3HNnF}sM_oSGbWOv;ko;1jI6Br+A4fM%-CPH$qi z6#JC%HT^f^P?>yCe^+=)GstlZU7m_o0X^_kR6pen$H%4U0$Q;r)9 zAj6t5JdpPNJwCwzC&E>a%T#1cF~6acpJow=wrLJJ<;XfY>67m}+7PBnI^6xJW5xDJ zA>akVl;**ig~%m^2*)0DB{5H4xZt4a%5w2ZNa8Q7QugZSZgxzMh%SqFtLR|zO^{@Q#zSAm}9c!NaaIE+e#p9O&oAkcNaD*;HM#V9M4HRk2zZV7yBcy zxbaY~%@PSB67Dw!#Z*4H6+&;#lmmXp?p{g$p+@uFd8O6+K)=hm=7}}sWUA$qQw8epp&i&3M4Onh?hDm#bJ}>WLTVQWB$ElkDe6c&2=I)@P z&+PZ3?fL6It=v(Pf0Y^qu?n{7-2s)cf2oaym5Gx1pvi{jT@df6llRnpdnHV2M2l3x zCVDDsvbvk8#66MzUn>O<=0u-&gOHYq2uHa=Oylk`)av;@&o4f zHIYTZU2p_khl=znb$0?xeV?N0XTn(<6d-xYY?G!ruhGtvN-WL68ydAEIi6|i7lWxV z&qF13*<+EQrsn+KZRubMMLqKSjf0b24oSZ*On}|997l?m`|i8v-=qU80^x$#XvUcE z0ExXe)fNS~C5)3NR z64{;zC!@xtW8N~yY|%ux;BQ1E>kVJ`lT8d)_}|I3B-$z%y>n$tB?qTj%v0d)a;KAd z$=c-hlGOIAwtGho%8{0@<>!ob7-u1z03M1yl{<814%-;>$-!JRpfz^;8MqtXB}mFw z7N=clb$2!aSoKRaRIW}HAHFnNPk>%r2__*hTTsxhQ?wqxd?n}&O`FVcMShZmgF#|D zYse7t<|DN>or>x*rRO7 zmMYO;j6v2#JH&o{ECo;klU83#9Zl{%K!kdhepgnVcidPRDDj`~`8p#a5_>YYEOL4+G^2S?F%RUO}hq`Hq?AkQkplPJ(C?{25@a)@u z*h5RPnG?TQ`Prb4KbQfY>o*{1Kel$Ivh`QZdn55y%Ijc@mKsh7%976M4GkA#yfs*r zAVq}1vE%stfne0O;a95WVW?!dR?6Uzbjzd*)vKumE(BIhyft$uZuADs{7-?kHVb~Q zwPf<*;&fsYC?GLR;ux@BZxazNRwBXa8{a9+vdkiAWw$OEKCBa{4UzL^iJKCQ(|A{{k~%29xaYXR7czOzJXX2_*%BWI z1%vBzjx;3CPNVcFGzjPaX}2;9>mP*TnFNG2jVFHV_O($TucR_0Nr+}hw(`CM>T5OX#MkZQ z2&=8w1`x0hNA0$XBVE<=LO6+0ti<>K_^HD%K2Qjm_D* zP|1n3eDh@44Pm>K@ldn4fnrylM@C9>tv508X|Za>o#{)wBcaDhJ1s}=BOy8WA(pGT zKs9fU<;~Y=sl#knvWc$bK=RNNUsJ#{-pZCEib1!S?&kad_LLNyCo?lc|oD|=YMV~~| z@8tlEz@|b4)p18-V!kBE8 zscVmxpY{jQwDrr3#66`{O@C(~uzuqDTI9PFGy@o51u0%F_}Pyp3QwU;RF$|~Eh;Q$ z=Y<>8Lz$4#2hy=kr@^R0SWFv=Xh65#uV5{pUh{TUp z+|L!SkN*(+yv>qPN}U&Iwb-AXYnkW!uzzvpPBoBzO_U45bD{=gFL#4NBCvy>ub%Hx@<^4~&BZxq$lWi=Av?Db+)LG;PI_O05n|l3C0{m3HC79#2#H!0PBO z-aIKH&;S}07USDuzegTlb?>QdIS|6x5(zFbY?-4ktlQcne>e1#RP;CJdbFdrE_{}A zMZ)u~#~p$2e{?@+P!@??#|=znMP^x~MwXfxNjxvS$VMB%>hfy3fF2Ut=HS<10afrr zLE3C@x|r}GaOQ=5nIw|YPO`ZuGRtbcg%7f2N~S-q3NX+qkZ$=M^R%A#FnTBVI2%N} z)=}B(aPEy$8bBw$r3XcNnK(*Z7pbZ-49Al1eCXtA{d z&_BrwOX+D@AeI7sY44fw}svuJf~M8H{+|+9awY8uvnw-Uvk&HSvw6 z@&2jxH%v}AgJX1k7(w!;$l|$q0~CV25AH!KTkMk|Ie9KEtw*xH@kT(tMUWIt;Tea z823FhNf&o_ccz=@2Oqq7DEk9cPN5FSBWPDf3J*iFvhZG@{$#)A!-0lDsYt=7rwQv6>$rl!YTI0&zK$3!)x8=VTN=9hc*7%BSeN zSPe{CA_Tv8HcHB(5}RkE5{d%L0G>J^H|@TCJ0bK?Ij^xedR80mhCKIBf%?Pji(bOx zA(&d*{58M_r%9JXPx0g`t~qW)9ac%bn1xAtlv=~Veww;Vx8`NlP+D<0`=mqW@nF|8 z&@wp9sQz z6)>s!mlO!_{>fJj18&K4Wi@QI&$%ET?JI4TZR+Xwz{G;%8+i*H_ZDja6FrhFLle3k zUsr^+M0HV7nTDi$XQMb4UC{4SIVi=^7ZZzaFQkMM%w`l5rb)Sc*xb}l) zxoI%8r?Nf3>@w(#R=QO$N`8DE%8e?q_#{v*eO9Rdv}b@5e>0@Z1qL{*4-KsddQ7zG zBNhi8q6CZJBnmHz| zH}{{C0Z_hkrE-wJu((MkAJWWyZ>fbtJn7hN|dLdkSV{VOp@$!z$owbys`p|jdK!qG8*w}G!}|_esgS^2>ZC2-U*BCv8}&6IWBlLa8o)R=-wQwe=1$j;GTOw(LG(p z>mZ3l#+&bTg(@hKEnW5n)Pq^BQAL*Qr_)fF1xUMN0tb>7J= z@@l)nB!Jkg2JRT;t;xZz4DSmnoH}ufIF231X8FL;mPtEGAN=wRbdg=j%JF9B%15>Y zo;d^EM=2WKbh$@w_r{=O_2(h)d9LE(_)C*b#&*xuuvt#OB=UN_DFMtksfUIUgxW9% z=C!B+C4q-YKtu}2csTaw?SaR#hvH;?*YxG-@xzfF;*CqZajnREh`lmK2ia;DJ&2^fqS{=fA_MONS21>_Z(u-H&n6wK66>_ECceRi-^S{P zZx89O_xjVRM(`>=|K_u;ZX)=r`?L3^-#{z*E}IQ=WI}K%PJ9x%y)+mmatuTB(NK+O z7+I)o1K#^qw^Sai+8aQ5>|P|8OO%7GCB zm8U}>mJbmrJ@Fu!LHoQ%0s^a}ZyFMc?^Wk{$D5qbSRZ2zQ0~$$XgI&x))Vg8jMZhR zs?KZQm;F@7HBgC&RgUx+&@jmQ6w{K26@R0WFM>HYCEo~XL@pi^eA9sKogG+BX`C&9 zm(kWNw?RhDuUNaCo|$pICC^o#21MJOR~DVWLHb$Kj>Pu*RvXW5 zb(#QvOcYm>JKbSApxs&5H?}a2y!zS6z{FCj0pOx#s=8Z7SSM$$F~=vnbIU4A@zgl3YdKw3pZgth$1*KCg`)us%wK%p}w<020{cptDqMNq>i z5T5M37;~*_9vJ|buBXV^&ON294tXR%&e+%5FJqhRnr|0VT98vi6L~H}(IiEb74(Qk zy0ockG7ayFEbWPsE8N!xw&tkv*@-iL?}5XrKHXFAN5(9{D!be%UGMWnX$g$&x#qRA zM_$5CbN3o%QB9We8N4a0$`ESRNn@m*9ZizsDI*h;t8IhY^S*&G_Xk`6CwL&-(*Cs( zb}&x{PzB+%=O;krhOF-(a#YB9sOOFA{t_`rxG(3lnWvN5$B(HKg5z9#0OM9*I)Y%n$&bd?j`HBITMOTC|lX_M&v>*nN)X|!@YtshwnE; zdr4Dcx>!&uOMOQ-b4wq@xF2WQ3@UOV6dhu2H&r_nHj)2VkYkru-o&L8qh~JhgClae`QH|zA43!E88FwtxNgv zRC^||U~eqoR^z(MSmdcr?>w{VWY>Fu3WHcJ{8Eh=Rmt~Y8arUyhNpXTd)r13oT zh0|k=N{>9K@@Dr5!yIRLTmpA+8rMWFu_T$@~DvDmf;@JESq(D-4LDkvGP<_9+-_b z|H4TE;_E14wh__&i`QA^SE3k6;>*X{8FfKn+-Uib{UNuD6HJR{y8Ny%39+LmsoKoy zK}#IwXR&tSmQ9a?KCCowQrGW_v6Y15$7tlbJ(;STJouf86q_WVRG#m**66^RwO`_Z zx05bgs9Qp_Un1Nrkzn(m)g+cGF2GK0Hh;{h1~i)4aF9~a;Bd*1`;L-@N7z6^Mukj| z2zP;EmS9?^A}r#3L@mTkQ*nHaLhgR76<731k;CsZLcWUbOF!6yXl`nErpVsSQ^Uto z>ci6MoVXS0>1#%_#pgi(F_r6@d7^zm|e4QeP?Z&y#9W*P2o6XsT-rNs&l$u zSPJ4?KLVkq5rA#_9D;y;F34Ib`5@cKz=pW?l#@}Gii)AWI9sG!3V;Dt)bw4$^W`DwlB%H{zakRGnl+}u!Gx~m-`}g5-dgiZ;LoyZ<1HvqFr<_e0}57cQN=0$ zzGSWmia!4HYfCLz`$4I1FWH1+a(~}0euiW1gJwcc3 zfTw*=Z|sD2iLo=mjpdKJMQLK?71>zos#ZpYtCF;Qq8p}%LYMxelVJQHfZyIU3lfV zM<3sw`>Vwqxu#3|H#n0YhY`~-7uxY!Cc;V?whMzX!vz+Q{?lB#olh|$H^FYBPhcK^ z#Ao0)&rsdpk&wrNr@0UWloGmM8u0I)=bt+d%rDDgQy90cWdtY+PY*u^2#M0y41oIK z<%Vo3m}bwItvI#FAOp35+&!)c1{{ZSr}Gz^uzd;DC?mKQVVqE)pS<}Yz z06c0^Fj5)9c?dP_y0D-m52->{m5EP1)rl5p|{H(`Npnihn&YR; z7gPwv000_D&x-|Rqix#bpB)Sx72{KTD%tU_2yTZ65sNr(bH1iPz8ls-FjvqfkAy=dOwf3h&ws;_XkMhmJx6M<{$JTe=%Ptau z)_-thHf-3I7svZE1u=j0w~HUF$eyMDsSikSP+e}x1MpC%!0+7yit{XY*eR27TF59X zS8zfg(lVoUZ>(`X8?D2`4HMqH`~NX)TUk1#guMA8tkqcx?%I0CgrH8e_syikqP96s zEd#bsQzXQK%3MrVUJ(&WwrjGhM!FeH*AQ42JUShv6jtTXwt&m|m~#82y;w{G#%bQOW?g0h3qMW5!`*F;H1w?u9MmG(+=~!lwnqM5=MCbd`T6}2c#=Oz@~`z|!?ncY zwx#8FKV!R#0}(hxtRjilZH!;Yb@uHy_YA)kTvyYQehj%A2e~+?uh&sOlUL4cmSEFG zEVGRr80k5|^Ve6wp2fudzGTCmk&vJY!yX3lp8_|>PvVYL&$m@cbn!f~eX5J~*$L&5VxuZ%e6Jizu&J1=JdFO59=`SDsk^S%zu8e{CggE4Hi z?}MM>l25R2HrVs4*LU8B+pUkqRR*`BYeR2r`D0ncYARd!gUTko@QapR{qy7&Jz3w` zZvdsuH$vIv^|kdmVW1hBRT0lKW0a~5Y#baSvfZ+}>3d^}XLtOPL61zl=MJ8^S30eU z97g>R;CjJ$hV`m1&R2d%-QZO&yttLd98HrQ4S$ zu^YwdyDYQo*SX znk9~3(e!4*NXF0-%lhrl5uisM#NR`6mw+djj2BUcu)ZxXcPx%}dC_}1l?~dCb4_!G z9`&@Gt!9^?OXAcs3s6e}h%r)4!pden_l(hta(S*n=69urN4i-2Bu~j11tvvj5?* zA8#W{k7q>hCiVR^vVU`PHdWna{vH_{YZk^g&6|(c^;zGSxmR3;BHZ1=>T1l=O33@pz!h5Q z7;ew@^#c}*tb6zl@4ccmPyfb$`^z5ZW%^Oj<-`;>DJki8Oc$PfXJyb~>0wG(IuZTs zS6MB$bbt~d6FlEbONsw@XHq3(ztZ6o1&zEI49BSe$xDd#NYJfOpA+fRHt0xEdcB+N zn&4zV8YPkMRP|TC?$2^=!X2}Yue79n>GI)RA9T}E%*@sXz4#3bof1zJjjq{UW(kGm z*Mc&JB)M>5(fQ&Ptj$eVACQ)i*uXXFgm_u_hGMKMkDB!hWkui7laz|Oz)u!MZgNOG z44~P+L&SPKh7Wtx8Q2hw%iuKHriCoYTXnBdVU6jxJeb2$^Xtf#JAgDH$@o95ru%o^ z!vY=Q-wf+4+WRIUpWGA`TM_WWMuO4%Q?tb}~j6qg|*raZG*OGK*-h^tUD1@uWd1hB(+k zw)JT)jDl(H^)OBVuq;6EP5{+r0fsjw7bMZ`K&*QBK=yf>alyYM2xr2L&xn2F2#DTC ze&#c0@&m=6Qr|jsM2`nZo*hSiI&Y!Dqu!^aK~fg^uTy~WN%_p{21dG*v)8MarTGskEWK2vp2qB>s5eYayfAK=X$n5ximk_>omAwr7Z|G+m~@nT1^QVpB4N}D(R6F$I4nG!-C_r6FXW#E z9?xTQMaRATQ0*(M?&S`Y1oQ0Toz?PpX z9`(6iS~o@=LrqcdMODk~&XINnSNO6Di!&%JuzX5wES@Nxihd5`1ew}E56XotH^ijIZ!dD0lTpq^0KSLUBlNY_7C{QN(F;=)VXrIp_6k7 zg`mpfs|Km+%#`4%AaZ3&hWzj#cl)IZ-=>bkJImCIhn}>exN8w?K`_alMCHAWLf=CL znLRb^1H@AYoQB+Cc6zW7VBgm1o9cr1)E5~%Bg6nG9Bgz>3z3bZjBh(JyD<+ zGkXsieo$y>40ToW{@&2c^=~Y}*(XNPNTKWK-L!DbrR~5oRW?U#~Ymekl3~cY3=tNZ_RFMT8iz z5&_6IyFz=ay;8I)jc)<1%3IG~*BV6yE0J66X}`HJ-kYsY3}cJ0&b;jp_nO zetnX<9B*y-1|c#Mg4=i)D=kv|rOx|BAwOq~eYev+mVc8a@e?`3<4kUW<*_!Y+oYP% z{%KRZffXUr0jk~gRWIS6b4#_WnT7b2M4o&%M4}@g8D3rYeo^vsA()d2g!k{wb+|Z9 zUX}zEBDMi_eL$7JPxjs7w6#Wu(T|AL96(*Xrhz+`GuZLxouCO;@n@d{`-f{=l(bB= zXEiXE<0Hv$4-u@Lp1z&$Ts8)m_DjKlEOuuqAz#WGorilYGJT*unt6wFWaLn{#IU3g z%#BtEv27(j)9ORTEJ)qt`>p(CBVxpK_bN>>YT*{}ptSJL;ERU&l@KEJMM>J z>>`SklCSyC!@k+R`i{k64hK4+n|5vJF6)k=^1;U_#+%=J-nA=PR~Q(7ABFTLG^ zLrJ@y(L`}h$ZHaR_aL_)WuXh#B>dJ*en{e0pp}B~#M(cN{_22I9tUUO*JXWl++Nr6 z0QMfDbARWzxR3VN`Gb z-k67_r1hbdzW{MUk+VXNx=*&zv(F4Ek#?d0TO0&&XVXUTUtl(9==)RS61PQwR@8C_ z^*2mSlv}n^Lq0pHgmw$R$#jofk&{$^vRm# z_$S%|C_4?uJwu5Y$6aqIQ`%4wP_L0nZgZQ*5=OCBN+Yn-FBj{)x}t3>w?&5#QBpC0 zZy0Uj*R?A~`qLMl5t8#OY&_`GP68Q?KNM8wp#3yx@&(`&TF`Q_*I(K?zs`fs7q`g7 zuQQ8Jx8+6YxRF^-7LcitB z4JPAEX}sQ06f-g0cWO7p3KuHkK^SO(PWzJusZ1h)j@GS`(k`^p2kPDduWQFIy~{<{gIdzcc_-z^O>eY)?EwX$0%5 zwgpfHCM@aCr=K8tcU*$cx3r;W<1lN0TzO@6Sd-jd)&lU)+cc>F=BQ+8W5;3nMCH|1 zCEfb(d9;kKG}g&!!#%#s?icX9ZT^^&ElMdJI(yZoy-YR&D;*XFpB3h|$*F9+1MF)+ ziu>T3?G?ANE|G6I3|(}w3G<%c&`f>7M!M1wl*lk68a%#K5R}MB)&IfmH9F2}7wsSU zN5L8^bNgdN_8x}z>!UlCa!fbgXiLs^hH5^F5$WZ{-`%_Z5RqD)DVl1@aQKY{rVy;_ zX8;>V4dlSQe!*7+ZUW-~{aRZ*Mifw$G`q0mf*>DRGMFok z?!a$$yFIm(9}|1h_TLIkjwV!+W#leK&uo)nmGcro;Y# ztDd2;$B4pV@=4J?H{enD=@##72jHvnB`8uM<0s&zQ8G7=5>eb~`BBrqbs^f^2P*NO zN75bI2*QHu4ljQO2OmCM!aS^(xm@iuKRHJHzCcX0?|N2l6{`onZpUF!H3kkuZ^<7^ zv*Q57@p#K0PV+fHK(2?%0_5a~QY?+LxIf>~q$2R=agfU)mZ>>GdO1w`aC7F*ehNz! z4W1tZtd`UH$d2)Qn#>@YVMrQ$mAyLtreQB3e4OIa%H!?0?qv=gI%z*93e)UbzV=5Z znhyteQs|ErhMQD`74df>gR4-x*S*u8P|6>M$>mMjYB$!pTtW`1e~OdJOJuTR zmfKc8cB?2CGD%sT8zm8Pg@){=G0ei-hx8s;A0+2xBltx{ty)+0?q6;;Js3;9Si^FJ z0A0l6te7U3yT^h;OO#)PE6kPf4=$FeC~te-M6aSwh>h;KGb9NG*G8~(P}@cuPisE`$L80 zD{DO%RoSL6^6uErC?$~j6*@7uc+@a!w~L7l4miY;!$knaon3K+ubQncUxEKOvP=!> zixtw!Ab{2QW$1SZfmGJnF+g#Y08q=-(b;wC8zD_gC(%nqyTHMEsqPEcE|ERUl`S!~ zkY2qTTST{?(mQUt+{O3PHu%VPoA+p8mAtX2m+tD#2Aexg#}|}C;oB)1XyLI z|D*Q&^cVFm~&`5u%u|gPnDSuH5X7rgOS_#P9fMFikeCHs(i8@czAi z_sMu@>FsmAUCSz6IC%M|me?DK7SszqI)wX_kC+ z#<8r-54`7?nk{+yv)?oRWXt*`dqH2c4<}qjj}&kjgYVy8eM5xzAty)4HI!2H{wHuY zQKM$cRSbusm(hER^2e$O zRd4m<0klqrFQP+gArMG;-fF?6a_pF#C?4h3yUwjU4LxW;(IF4#X8%;jQ9V01Z)*n- zImjh<>a;obhviARhs2POJlgjHSxsXE33p>9wu)8(bqQVNw|L7~C(x|>@5NCmmRc8B zA&WYqN4FqsIJ-^Am>%_@z4*bMx5i0n}J~s5=cRleM4Cn8Qu!53q3+f z;=hKZ=+aQ+FN(ba%Q}AB2r3YdyNG+a(jEJOaS?`3o`{2%@p{!IJV>pTFjC$`Nxr}z zG}%!13EE)64~o`L5O%&OrcNvQCcNl7|E2CsrltS@^j#M-w zF!X8Xgth#k3XeKZD0=0iqvASjm-g5k7GH|+*tHGotyZrlUC>(m%}0$=!&(Qk4+p;< z{l)wWvS_Gk(6Gq<>*a*~U;!viB>1|T^{+(z4*b^MXj4eJz(#qsFKvs-$S^BP8IYIZ zg<-wF`Td-oQC~UTOWSiSmSuPB)l~o&@~_fBPEzIlFday`E}3wYGvkZSh$^k@ka$={ zDFYuwdukR)9?!dvg|29~be>F92EDARrI-9sZrm!;&X-FoqubNhad|Yy+GiUXE4kZb z0oZ-3n78T0uP`T#a$LNnzcBlNS{p}KCD8R12KUw445l8OTK2HZwvu24ZB*eO*hu$C znW?zr;D39$x{lfI??h3}{y`o_`#E=)(^pAV^@ykV5fmds`(ZB}NhR~fOfvK9oc(tBa2WN$j|Wr$Yu7PMrSQc*TCr`hY`IaS2@AvJaUY7iG0W$`V=| zx**FFgCD?a(5)(-t3!=ztAo<2tDDn_X*x8ImM#~T*f+@cuo@wwH2sWfz-`K7_3FeW zxQ|s27-MnsP4zwbynU#Kd*==|q(?=_UaTll?b83o(2udYYv#y5A?5DBMk?hh$jfp@ zNr$sVtyTeGXA*l`X)o%vWeXF`2i^Mtw^H_e37O-Q2hkC+vR5aoSXlR!u)omXdYtY; zwTVAqG8u7Q`>0pusQS&r+qs%_PJFmF-6yCkelR)X-74P2YN*V-U-{ zp9c^QL}BL7dJB6MQQIYC=Fi-{kk@W1wQ$P z3*9udZ$DqBaB3mWFV$k1Dj;g_6s{6tN&NtCGcp0!7q*H}-fw%}lw2xZ7k!k2UnG)C zYgg-oFE&K&lDhE_efQP=*9{(`D9ng7!XcZGgr^I?1Q-ig+VUIK4!rcZL|swp_DZW5 zqCV+fI=47KOk+bf<^NLZN(IbmOVvYUUESO2oIkbNwv%r+cC(*9mqWh4fA*1D3;^$=t;_>W><0fnFRaUBxCHD2b9Hz< zO?}rBdt88W=*_}qRh~8aiIapJ-RM5uD)4%|M|*Od53SOVDL?0MqrD}Wtb)ZZ&92v|xAoSGagO*(StTu%8iDno7 z5M+(|)kVr1BewefpxNa9-G({m)*7p{t|mQa@7B2otk3{324XA4$iD9sLmcBnJ8k@t zY-9N?sT%xk0C`G$jFA6svn{+w?HwSe6Zn+S3%mH(=}OogM$dhnqEfiG; zqN*^gnDM!c0dqR>^a)z!%7`uyL>p5_PGWr7y^BNF05C%EaI4>kVfq-CEx{tUi>^0F`a}r^VYB2ZgfS(}_QUI>q4$&En;* zH$|-PfaU0v;VGQ+6bdLZP+$RP9Qe_Ps08k}K@TLa0s-A8Hf(gizwx#jmu7PWSH2qa zveNJ6Ep&gyMmaydrObIUP&f1d!AmEL_I)PR`kv>UJc)ST34m}$iLT3`KDKM(JmZgv z6um*Jng~u4ej=7J!*70CQj1i`m(|s^K&dI(>_O z$_9N~z9;50o>-{xHZeX#Qnd+s_=v?!DKh(WG9B zhaKE?8V7X7xH6g&K>BQAHDmOW4{Hn`vGVY)h|Or=hoJ!@g#*youkG9KXoLiDxr0=ic{wh8`5kQ9cmR(*)=sg7-s)6aJYwW~SRC8_ zFR2Qy%Drb)V?C~kiWkGT5&C+cW!Uiryq8aJN3EE@f``1wAb8!HPdK0~HzmIA+A;W= zQdT@xI0+l8)>%j$h(%bcUf`!nR5sf*Q>b!SW-6?fbO=c&70(9{mm=wtpIalew*nHZ z2CP+|!=smMd~dqmR#2dY5j*Nn?`gQHgO&?;!S(@b*URUrIjIzrCpH#K-JaGs{?L2% ze7EtRkR|tpFiYn+i$I!<9VZAlNgSD12ZgN0EM7mb&5pR3aV);W174R{W#Ep(*mH+A z^%EchNWQ}TXne)C{i7D4w*Os)r*`YaGu92%V z{#jx#yWi*yIKC5LBSytsHm@Gvi{8U8?G zoZzj6i4a?p!nJ3!)UtyrZ5L8zYw%!ebB_2u_SkZIYmKu0+sb|db`bcl)yNBenHJ07 z@#jv$N>s{G;B4Z<(%Wp;!Py@bc~^?m*THWv()R5*+Em!?Bg-%|F82qO*PmE<&h9i0 zM9CP_#mbE5D^wKC7b@K+G72f>RyPSCkYB%7uJ+(S%%hxI<36w`-{@v@3Kc zm$$yBJm1xy<9i2;qWk!xWgG&Y!e5ugec=K@`WM_FjA{<#FPfDYW$hiwp*ZNu3f&>g z3h%OcYJyamUJ7WLe-j)29;Qb1tBdBDguSfC?=hcGu@9-$S-7V*|Ht6WQpcbE9E&}AA&Kp>I+{mm*$=(v-L9JoECikJjNN$sTF_yCgMWSH1A>_*r#g+k^}>AG-eu0I z5v1(@8xY{r4>;{NRg=V1PL4us*Yr`l^NFZh4jlq%sL=-~UG#nl>v{C0)LKkyeykg< zRt^=tY6Ar5)YR^u@7YRspg7vO?-T1YE-#2k7*eht%{o*L+8;H!i9SPxQO-X0zLkO` zCHAYmX7-T8ZKOc$%+210+hy`1cUEyGKrl64h4ocgYq`8jx zc9);SpMRm~Eo7Fu2r&2+2d3BB?NIdySD%pv?l$E;+}+6i7G8l}6uFmg3Jl${;!7np zuou$qcethF)giy!cBU?APn1O9bJlyU!|UcV{$Zd>hz4A$I`o8{=!Jv$mwKhk(;I?- zhO*4vWnb@CU5CZjmpQ!k1N5iMoT#*}^m~P2JTXu%qNNc~TMR*&Z_tEKp&Os_w8C$; zvyc9~08 z>r9XGWwSw||Cdj-`NKTLkQS^TwA01Tx(k<6&pwI(u%ChoA zvg$|r&|8=Jhadev17NIk5Z}Ytt$e_O@>sh zC(Cs!IM5M=$lZM2K>OLjSOlq{t-{Q&qsA|nT`RSYk3RZUzt;yIoHjyx5fx6ZFFm%qAy z*ujk1V)jG4Pf}*0(i9{8%bIZw+rGM+O;h&Xa#!u|P2GHFH=yBf_kr%Vs-hf%_xp*I zU{r8Rl2|e!7YlCYVHM^oLXwnbJ=U#OYgHmSM>#RQK867uYx&q&l`bhZHa<`3`p}9Z znid_G0Cr72MGs5q+T*n|cZ&5jvFFP~r#VTvACf<{OM%4|4YM{{y>Kr4C&2+IQH~Hd zvrD796_A@7-51|y>rjT<`piVfdLEQFE=pXkWFig)2-^4HoKe^&-R4-+1r^ja6Qr3M z&j8`Q)%P)0f-S1@b~QV-!(o7pcke@QRfj|6RyQO}2>nuOrFFN(0gkZ!V9$71OpEGL zF1<4H+syAFk^E8RTZ^-0c2(rJNpGHP?4@6lq<*uu7zbku4fJ|KLkF~7$>$tv2T_Py z`!b>GS{}aMS27i^GdcDAtU*?!I=9dr8 zcSL&BFT2QEpe;BZhck5#q0+n_R)AtRMCm!w9}~bxCq<}aUHe~4={w>9atwbD<^kU! zpf}Hp$^(7Y?@tr{T@>_dXt`~Fh?7KYIb_s|!Zg6`Oowk1t~zxqgpIS}7vIYDq?)c3 zJ}gwT3&#=tD$swD7SY=?YV+Egy1$N$Uy7E{yC!@5?jfgy>cWt?rkx4o}`YEFtijYdf z!m$M@C}l6+q#kV42*wv9|9K=uynKzee}9XXs}y0*GHi=SEl9vpm|c>ANNv)^L=m?|2)AD?Q=~SE>du?5T;&0CMY(Vqd2K+45qKEb`!| z7)o18W54R6b?K6){Unuu9_8NV)1}OChgohmQfs-*RBkihJK0g}RdfE8R9VaywGN(h zREw$^HJ-$jX3*z}Zd(=OrQB}^1628YgTPj!XlQ!Kq=d%vPt2-t_U{GJL$+I=CpHYC ze`1^ykLW7Y3FeElhPbv?<$c_*bP>sB-tVF4$=p`@ab-#`KGpsP%r9dw@H{ z8J(>4V@&S@QxFyLRQvt2)ux<75R8K6f8BX3U}wb5q$*ALZTItV%5!wl3x#FRl;UND?41k23R++UPGszFy0ACya~?YGx&b3UO~bohiv3t6zrR z{E~3F+B|}oL@aU`yBoO}AO?D=qXTQFQ1|{Lv)%S@@_ji{Yq48p*ri4lbgF+@a}Y*huhlI- zpoTh>MJ1FqLEd4AWI?Tu^VXjTgoH}%;nxMo)VFGx}CRuO}Ej!ns8%&Eg*fO z8*LAhGLDaLQgRS9#iMOB`WlOT+;g8kcQYoof&kp*{VLZ%S71@wLzPlM{%PM*A=IXUn#g1!Q{Fk(v42Q7Mh8K(Sa{Ctkq)7~y-UG^J?r z<-S9nYc3}9mt9{Wawz>p!BC);4)rRZwY^S+wJ5&FkGT>MdRa{A5|nRW$~%_SFHOjK z-QHMR>nN>Zo?&fFT9$*PfAFxf+U#ba8_oYf{GS*7!6J-6l;x~8h4bGL&S&7Y5WqIB-u2Q=Cm4bVmOi5w)$Y1aoh%uz|UGN)2SZNtQ zORW@gNHJB9;Tte_O2=L2ZO7+yNBGe@)nDV8|C{|s{;{*%*|QWc%Df|m6MrbTzwTQ8 zGp_LFYf#}y^c!i@lwQqE#f)?szgMqw6;dk!a?OB?;<`ibAdfqSa3lS7NuiPAbW>(l zeRxaUg*@L@?xc23_X5gC0AEJlnQpR`PAgI&q~g)4vK z<|;~jADHtq++U)m0aF%t+;~;ZB5&Hcz3aI%ALsckQN+|^GAERrtmT`2gNC*!L?kJn z+rdAR=Qx(56#MkH^4J}MX#n}EVles;=HYAfvUQFaZByhf(n8o3TmNso-r&VWWogyq zCJwwjY&J{4{6ci`=`J?8S=Q8jqHtluw{V+Fk<-RcF-0S6K#Aqx6f$=@-5EE`%hUT6 zeq5le4*uC-Wm-&v`Oww}Rvqr5PRXlcz+m;W17`9jwpbqaox#ciYQoozR|gJ)D{ppi1s z{^*64D4bjn{*AmC5w%Fcolpz~Lg6a|rP1n*p61RsRpg!E#!<#nI30xvxh#icYEuep z!(WXR>|G7zCUmV_dR+nxs??7WrKV0l7uKm5lSCg+m+ zJU7j&AKE(>bpJwV@>!&$IyCtKa;^miZ<72E^%Wapv;6TEy|VmK6>gC?H@-@n-pLdR zh#YP%RZ$D+DYt`PSl`iva&KFfhA^ z*T&aXVNlOujSOS>5W;4g^-(nToY>PO`0}uClx?<`CtT}RdS0-a174I|h(pefgXS$9 zr+x)7y4@~2Jgq(Xm1jx*DxdK_>^`w(1`j#3zdzZ~6%&XZJIoPsGLtYn=mu6n`SyS7{|4U^4vf~S{7(ZMNSRzTxSTjVb9cRiMP;i{q^W)%owTTd zhEwMne%*9MPo;pYdB1ZV4qLOT)aprjP~*d->+vWSRh+VwmiGLdWNDN;zsbMHlp3e&TLs%@4;{K z@g{3Qt#qX9!HYxH=;lWDu2aytN9K0AZpwvq4cF}*EqzdZ^)|R$%+z#wv81VDZtaZx zZAe9~h`+X1@bKlkCIHmEbaQI(9<8CW6egGy;}k!rX0`o_`TpQMM^=3)%oMFl;R!N` zhzV{RD*0COtymBt>olvE1W&0`SNp&I7*1qnF<-vDdk_&#I1nE~2g}KmHqeV+sck{e zs82cJ^{dK7eKPP#9)Fy2(LeA#uu@TOpS;hnIXwQuIc>r18L!VA!D4j_TEH!_288FI z!M&=MPp^f^1@fdeGVwC>-lrbH?KATCC)FY>d^&h71*y(o zDsD#~pR7Me1#QJ*nk8dncdUuowD%ax%H0?yml6%Bzh6_32E>MR*WRCvB=1{}dd5niWy%A-*{ea43F@71G7{v%SZ>biVbtf+J}3 zS6kb-23DbvI1YzM-@r@}&SB*C_D4F66W&8$gR9W0cS7`)aXN*@2HWqVCSQ~jaX1m6 z&_PLzi{jC0Fl=--p1rQzdM)RE5vHV{1heXa#U@~vWrdvC_j6bg3UNX{x_W6AK>s_* zO@K5$Sclf$B7jEmHeIuY>8?wfKFwvS?PrMwP~8(W*Re1p;hKA~*aUaqDH-=R(1$g{ zizi!79rG0P-8DoFPjf2BCL&H$Xs7s|Pl0(}K3>7dd@Ag}gij^r1+gC__EpTT_?-a( z$12!BXYt-baAXg+{yI;Ti=gVxi`)DEQ87*d{B%YIiAZ1eJHlC#ef4%Dri9GlZz0Z! z6kRNyx{2A~_l zzqAs{m<1jZEiR09P!^V-!IH@z)of8`ZT=xyODQq0Tsf+4lLGf4%|dCOTvz+6ZDMf0 z>EyMHWhUFw+I@ zl@A?S%o7EirkhO@uMG2(ljYGnM?gP>OC(gxSs+=i_#Ge92WBv505$ahGAEyy-L2bo z@OZCEK-PL~<=xCYASivEeaJ5QiNjq3|0eN6E1qg7*VM*%dT_c+>VC|Atg=daG$Cw7I-yK1j1HOy<^cl$ z&5ii;6I#yoq&!pWUHxwspE(o-U<%&)7kTFV?P!_de_%)GDcN%7{P*KGCMRvS>fk*G*OZ2D*l0%8{e3~B)a8)vR!e>ZPqWF13O#!-4qS}Hb4UHFYg}x< z^UZci5h+M=g;V688S(Lwcn*0^&}@wn^>0_S@(i9I6r24dnV!&el?HCgOMvQTp*j4q z2QeM_rJO$=JjWq<^wOPYO-!YuMCb#OjxoOQNGZ~0@V|=hi|kY zX_INhfl(dUTYCpA&VqwGqySqm>Z=^AZc^E^wj8fRl1*7m6*vyWn%5P=p}n7+Q>zmC zu$yGscZWR{Q5%^hRiQSkGK`FkOe z#aK5x>8Q;#zEAag<60Y#grWvaoS>mDoQmbj~YkU%-?ZBj@YHmN0z55^hQ7_33e8Ktu9Jg zA9lF4Cy7?tN-RCgqm>Pl3QRcDU)$k}O*Y3tVQblk`DN|YP$3=#{iGl^^Ohj=j7hY4 zUG{)6Ir-Q<*(g~Q==v#Elt2St=*IBI2AvKowKJDxt7>A}Nl8=5uY&!x^_AZ4GnZS% z^shvJb9t)9HFA}oS^Z%%N6DEno381JV<@~Y{%i(KO0_H)TY3C$JEMW zL6V-roX#!%)&lk7x*+-ECEb^ODL68Ga{i!Y-nx8 zq`+q@NfKW3i!yx922e(m)UP-z_PnjK)zr@`t|Vsf)WXnt7&&329)}e1=xXNe=@3&} zVHqlTh}VLeV-MWZ6276V_4KnL`rWF{Ve*8a?{>zY?V<*qcjMDmJeSNGA`vAt&nveI zcKjtCX4iOesDX`A6=EagwF7B&V^R!L<*0CnpwDG+jb1hVXHk>(}b;L_Zu|ER#Wpm+1f2kg?siJgg z1P0!6WC$W2%YMdr{8(-kM_BKE`~#dc^Gz)P``%CvQKCu0y-Py@dyu4}esIIDYQgcJ z0vzBlq>?5+_qtO=U}Ke*R8dqx(jr`QOXh1S;@2pP3>hTnHkgjLppELO`Y-t_2BaM*Pyd16o%kpfx#QpZ1|!d?YR6^A+bE1ZGz&qtgDzo&rKEa=^Tq&K zR4xkNXE|fMVVaiWUl|wRnel~(WGXWEGC|o5cpaJ8Rd<0IFw4bDILU#3X?cTklAli> zr?s5P#b^c3?ZfQ%?|Xh}C1v2YOPVDse)as^X@%e9tiNWN{^^Zr0`(FL=R|QUQ)l}l zj@~ur3z0Fhq80*m)73$7aj$rT7&McwbHyOfT{~pzupYa9fksi2l64SC9MsJKJ{?!0 zeMy@Nv7~T+!G6ks{mo_>KF0Fyj0+eSw@iJaX|(c44omEr0_i3-s_DH>#|rTM{V`j{ ze%_3$DmQ!hDxskWqYdaNZU1d`s;uV= zI7pfe#($$qux=rVkh{Sf#5 zdDCLvGR49>x1E`T6qw<$R@{>V53v*PIBxTs;)!zW0Q zV;fat{iFuxlr1(DStlcXeSYG`f9f+X$@rodYF`>)7Lm@h-6IR*8m~@AFPE z|2~|*{;+drxo3N}-XVUy=N&dWRsFbLi#ph;LltP*N0`Lf0Wk}&STeY;tS*-+9~csp ziJY(2fU>yp*xtC3B(t9=o#vFc81MeT&zokomN~6wc~b{0LNaFgyXjtzp5>r;G)KcX zdKYx`IPBig!`y()FzYYA2<>j$Hx-U1$BAIm1ch&wmFD(SC0#t zF^LoM3`nY`9g<`}QUYH+I~D*_DMP6$^M)g=q_0OlCU(ZoMdZ%$QNBwPo)zM^wUA=U zLDYy(kkf8Ui?jJku`r-^CUlw8U`0|xsbZ9ankF)9M7m`XdK&L0A=br6d?~L=T!EgU zOK$t$Ld03Y_)P>{r6cL#SKD;F;yhe1WK!hmgK#b)7lE?An(0-7#?hNd_6=FQ%CC%c>i0gcz$J4#imp z%2XI&><-#_dRIdde+b-tKr5_5<+;DbyeFoA6eo_*PAYJgOn2_U7xU6%4_zKge1e*L zIwkV~$MU((Mi+PgtZIOgNC|B(&_DlwWSwkuGVF7KV_J zp*yYuO83wsol+uQf($Z4r<8QVNH_R9gZFyx=lffWF*u|ASrg$3nVpbE>-UofXx7~B$(qk~PtV$Z5GO9=v7hNu*~b>n+%62yT+v}o zsa>EZ(-ufYzRFbjLt}n`jdtWeql@f;@-nJhXNuRpk2q&>C0Rj^WpH)qG__=;@THZT zvzuz1t3p@Xt9d^m1o~t?5B1|Z;D~q@Z9YOj>v}Zy8Ye{B&-%mlI?IU(0NVc}6l3z4 zbK+^d&DEj8m~7{BaTT!TZ}sd%^KtU?XC#(@7s}{|;k4*>ewF;nb|^1WG%WsmKm-7MtmL zeHePlc|0AeVhTry(gliHp#_Q+5lUUIyoS=g=$v|yqE4eYJAKcw(A?CQgXW@{$l)?O zh5Z>X>LFjuW5sNP=JH#;4ZgZ3U`(8(-w}s&dBHGr%bO*$fJhSGceSjCRqQzZz-f45 z*XEm_`Vy_{FpIVsP7`CZ`{>(jL1f=wfv08bQ`%N0|93%Lm%)+}ULV$bof?}mi}owR zcd~k3FoqE;MPl-8m3&#MQDRDZur#^cE^!VxdBU13vOr#&vABdYWhI962uXT6xCCXR z5*No&TQwynsUkI;PcuU4112aqMJ?-PMu`NXPe@JABMEkJ5h5;I3Y-o;CF>Jex5h=# zQ=wCoU2hz2&zVEj(bcESPlCi(t;aQ2^+!Zt3dbC{Y_Nz}wxW^#0lAAb1%k z;(x6f=W*28Z!B5%w;p{`{c8_@`N?Nn-6kpJ|B`urulnafSe#)aMY=I;*Cog9U(S{= zjCQrkYg$+T!gOi7eB7KL$HN@{F}B0OQE~3C%+s3K2Z1eXho~1%#n^;}py#vF$O3kV zI~Bh#PD*Bx0uP6Yqd+j{MIZr<7~Eo0eGw;LdJ#SdF+`muAA{DO_qUme!V@!Iqv$MK zP%anDzC6qWYB5@cjCYd+W&5EzF}PPh;_3V_-{X(IW98 zxglk(05{11SS$9HK|HeGC!-<##)UxENE&dd?e>%!I(#dPV1eJf!{hW$|X zlbSVfZ>kV)!A**tU8M6luorToKeCSNhFy?)Twtcjo~WW-q`T-BM{lJTeq#~XNTwHy7cMR;uNc^Tzmdw|JK#QB#^Fd3m`@@gH>P9W31j(R zsmK_g4o`x?ElI+4C`QmETlv>oHtTkAt^y8(C>ApEf;#@h%8{DC+bBMljg8YHiMK3t zxf@k*cf{%On8`nV8h^)lE&yd;DAf8WAD<%9o_X^d7M?|qF?iH2ZF6GDNw8T z5e$_N1A9;(=cph~101K`_{kKEBi8>OOy2a(BDPW_gN^+C8$AB$x84y+N^Z1fd=~c2 z>uzsaj<6Ga=1b)|DY7F^al%zVI5+!SH2ddh*YU#Xq{)0rD||`*b}4ES+r7KL_-)hK z#_NIN$7+}B_ddREIJ}andqC~jTtNd==Ug8z**lkbhY$yUL*FG3=-uGTopdAdQv zrSetmTDD*ddu2*Grx=9|8{4~FU~1^~Uk?`JW4E}_vlNq#*0HPrm-1cFIB)0HqCRC8 z;dtagyzWqOeSN|WS6HABJD^T!O|Y4;Tzb#UY8}0qo>rK@bN*-fQ{rMTva+69&ho$0 zs*A$BFD`m)gs^CylJ472csRALvbX4~%qC0a24Hc*R<~nvBAjP2CDY%L%pbzwS5AeA z=vD#}E=vP9{0+KmR9U{4nn;${%Szq`8b$a{14K+~U{{o7|BH@KO)m3*JE)x9OB{`Y zy~on!g%9rItJhk+Kqx4Rn!-ACXLG^z35YmA9yz$id~!c%_s9?-iD;{ifg?JPVw|so z+JQW*IfLa*o4aL-1(LmE3sk#>rTRw>NgjpUHw{OY6KkyHP8dEqYEaLT?|J3C{#$Ya z?qZxD*!Gq(of{Y|wSU^pZ#1Ny{YA=2=>hn1EOC0}cHi@M2N0h%1s2BsO~@sb{a{QN zx~9+r;a$1=Hz9t-1f10Tb`uF~J4~dcs#ndHo;ED66tDv={hD{Q+Iq~k^wjvz{mEw% zNM^O=5TA)H_Y5m@TSN7^*eka!5W2^OuqgKz0RmlL7ZRJxZ%!vg|3z~^NP4zO^24>Y zI?v!V!^Tl@O_-#r2U92|fl&n7>I;OQQ`}Vgy!4~RI@pCcZ1lZ!g`CH^iq|ROb&gGA zU)*m_W`#K70=>4rg)TXim9>z~Mq|?bI3Zr}pQAc~5gV<*x?|(F_BUV0|NgDsH>vYC zPq>tl_ol-`?U1i{ir$y!uD~%b;P+RjvfqiI@TN)CsCjZ0=?8EZuIIts*yOD5+6>wHpT7~1r@z)$P)#S2id$}P$YW?MwfPhgN@A-!rZlR> zZ?ki?cAewu2TZa;I8$#<0AcG3_mK_DG_ZH1`h8xY6pXJPbhuNc3igZ$V|}JSW^`7U zP8CraHqv6;`NblcGM12wnv;d)rcY2ZRGEcpCz$E#6QCv!xw(HS-TxF67nfho{`-d7 z{9*H%X~28g`k5}REXO}#j*SIdG(~Q|0j}s|X2ojPf$5x_EVV)^5^nz^m$PwU8&;bx zo#h+5Ft!vMHHk4}$=;4MzzH&c{*_66{b8tJvdK+jb`I_3R!%&!lA+JO_vSu59sj&V zFjN6ig7KV1*KA}~-CI~|Y-xzzxrTR~KA}#@=_n4Lhx*G)>pNAgP}>pLO*xWs6ZzO)KU!$xI>L|9vyBu-%pjP;D;(N4V=bT6 zihM*;YP%{jx*BR5y6E2Bsb^^g3Au4Ku2%gA!HG9Z@qa$E2FdhH_D>=%k4Qm|(LvN``YoM7`3@FAvKR=i&9*Z7ny-v(* zG9T@75+J8s)O60X(3m$Z@YC^Q(>(bn2vlrJi^zT7B(&Et(k!*KoEJHzGzS#&`OyGk zX@hD3p5ud5zY8X4yG=l>Vw;pjthmZj5`N`;*E>h>a+jzf+yWMOauRbw6O16RxXkL> za!$yg?d#+fEx!reNdCJk<^MJ;7!%`Lma~qIi1BF0W`{Xg-_Tuv(`{6jlTK+%+x?z= z3B(b~k!&{#ak5YzT6uhf+p&fh+uiml{_~Aj$3!uM^x2KeT9opX{H^ytq#h$l&+2yu z?Zf9De2(P~-6@l6b(F7ABh3n@Q^myWQ@t)y_J^D`+@Rs$% zgMIE;>lO63{d@0lvGD*P$&TX@TIl7l+iK$HQ0UPRj75m!SJ9TzuBt7~2zfSZ%)nyU zlB90UCj=~RRP%d<@x0gwQNK-iLVN8|%3$NBcGK6r&Zq zwy?V6bU`Xx`1nTU-?hX)XA9NzxW@WuA$JvPNr?KlF zCkT>=Q-ik%Auusu=i$vIlx^i-Wp&6`HYLF{WX1Ki6{#NC^SP5})ilIEEMi!Xp(>Eh z1)cFf=ih(7=~Om`>_5=|>whvmHX`hX^<D z8VgMQ{fzGKW$vpW{g}i$UuE$=(XCOVfxcY`EQ{vfs+rcCIv#efI1{Cv?%CO&rme^9~IKAyo)K4&$Z8%?B_rj1F7bg`F|GZpEu6X%Z{Bk zpzY?@{(tjFV;l(XtU^=yAw#Phf4+5omEBmmlTzh4CJS^@!II)4eenFTgcVM9J>gw` zQj8)POcH8fdv`VR)`K|$dA|yLR{G`2@lx4^)>3)U2NwDg-|C{CGyaBK0s2>kPrF|7 z_M~Qqs=D9yJTl{)E2Xm&eON(nrA~3Mt?KQX@;`6<_mUv+$Li_P!n)TI_3Chxk+YuOSSwgeORr{XkXgJ+LY2zKla^8GU zIZeFhi^TkEoC?AU328)h%nEiL{H^9zbvys(K)Kjz6uA8!yhQu|q(gr;Td581?Df{l z<8DdVI`6f&bK8|ec7R-Jdn@uK4G+t4+ota7Qf(iHmlq3V_H~yPN$!ZpwOpZ`YWo<^ zlUm*fB_`Yq{jztL(Q31W(OG4f?9g9|&hI50;Vp-Tdhy<40XS!!$WI(H`rtyJiOqC4hqLa zIg&!rnEvqQBHXO$AhV5xErVc#Pw|sZJ_vK>6RWN}9TR;yVE=qt%eLc>>t+>IpN}59 z@bVP2bWqGiw_l(u{=BV=Z{E;0zwmlZiq!*LGqVOiJ+!wlX51C{@l6q6XAh21ew=@} z!0D^&wW46el6Eq&L>}QWm>JyTv;WBy+jr)=5mUGLuhZGLB+5)mCtrwsO2309@nA5? zTYUVU%3Yn%+#lA(H93bNO@|?~0i0eYTctswHjYWqaCH^Y&($LhXiX6?^O_3A+Xi<) zGPfE1+A|uFze`fqyo=OsEI-@~5>GKK{VF&i{IlK>db-t4f$VGOl8-+&`#;HBm9o{d6%~b6j;%S2-}iHP6MLBTor2!ZW{Ob$rp~#ItY;c&RB(Ro}mf(>3Nn@Ds!?$lmS*Njd~2J8SfT5R-n)lni>c-$Ff zUG^>qL>%!tc?Kw1+U2OUWub@is}bis2W(cg>+rST6IlLL==`M~G&{~+yZ(c1yL-H} zw=3Kk#`Wy6=qoKue9UAT=E{@^PLc2Y2}GK z0Y2=~>iKAeXa~5TKgd!|O7ajj=&P@;BY02NN%WirfaQlTCw$*=Ktt zz6QOQXB*G=N+>WmmEgJ`d&O;%V5ak6V?E8GEAZcB^~yYf1Fv0(B#|gDv%EBYD+asM zhVk=e4ZeH%aNUw*B{SOeRp@{2XMSJQd|<#fT;CEIsF?BBs8j?mDRg(PCE0F?Czf$R zH2e{H(}Zjql%mg`W6LIdK?KG_d&`&N^03fwKR$?sVA4Ln^Z@8>W-D3_FApooEEY6;NU4XGtzB>evm7LpH2 z`*ZOOUB*u^XeN>6CqyhaL^hkhMg>*A2BX2P1jI?Vx!4*-PDXn;_NRP*Wvw3+S%|VS zpKJi>rL7RSrDp-g;<RiNF#!hEJm zE+5Ktqg}Oi=o0%Gz)zX*R{NTaJ@LR;A7@LCd|2hKbq-5P4Lf;)h>| zf~)Em*z>8XRq5uR#2TFcYmG$*y!IGK4z8Qru#>yJ4d6<_WLW+-|0jy}vf^yDQTMnh z-EJrfEx&kEB7iJF*wsu~shVEW`4N^axn>lzv2a*ULPP#$`%1815wp`CPQsjox6W9r z#~YU`_gv4RKjNA9+DaMRf1>~?d9V@A%qMc&xtn5l5C?h7jBQO~l|!~X{<`>_pfmD*Kf}2U#9zsYi^?l>@$wrmR%4porzpisI@!O7R zV3;P#bxXvmpLMnYMT%{~l%fSwi`IZ7s)`(Ag&!!!T9c2SNY*G}&CtYbhCG0HN)8YR4m*9?q{Wf-?{bpPRR zfZLyYPB75g-_Nj^Z&KUvnRGJiHECZbRuzI6&v^dDVDB3El139#N*Wi^^*e|tw&!YC z!S43x7JF)InWx?NU6}P7Je-f6xPxg`%ZLPi=Pk{+c5;56_pK|(hKO9XO^?|19zA}$ zCBkxIRZo+ODg!B%ElICe%iZ$3ZPjG(lE{)!+tjbKDcF8=&&124Q+D`|g_v5`+JV$)y4H&z3~eJzCc~0z zz2T+zy1`j3*mNr`Tsu)lY&)t=jxkk3N(!VH=))ju31c~7^ew#X*wP% z32r#};py1md)n@sLLSWotCdB~ii9X+0p^$Qi~3oz64{~tK=6m~doKq0ft)Uj7jkAi zA`yGk2+W)07OBo!V}x8}4s%2h(;t8cL5|t=1B|{i#J|c`VOaRU4e_66kNMu10cUGr zL=i8b(<2e|c#1Y{o#r3Y;SXHrZQ(-FK$w1X2VUL0^7LH)-GaMbuzR&Ln|Xfl90l32 zAk92XZvniC4FcK6zq1wpWT0N;Qk~JA6s-qk2pv$;A$>B3i@ZZIiLeAN?*Yu0No~hm zbgCDvimse9_0*u~2R(uR`HKQ&{q`cr7-~egcH2r5x;R`hNUY#r$0fgN10K-IyRIYJ zmAzc+=cqzJRC5F$&n4Qcxrg(`H!O|I6M|d^4;GJ9?lp zJHl&{;Xj9wzYBiRF0{M>T(!-n{`9gaoANoC%s606_`rM67vaOZk*sR+^dA-g&LN;^ zi7>ZQxqLTSfn46kCUTST&!Aoe2r3KcLYy}F6)LCaFDy+C8dL)V9>n4wz2&^-b8%h+ z)8s}QKvNv2h{e>*m9%9^iB0N>tb%-?gx=#zzD%4Dk9wMHc>?U6ozglEIWYEpaiE~ zbZoh&LQ2VPbTT#){g>JH|9=Q6ejvjUk;x#4M^WZb2z=z7J_snd(UC)XKa1eojyXE# zRSN_)i{HwTWYW$c))jJ#=1h_~Li{lO-2*M#16_H}qRdjm2M~#|-vK$~sozN=LYQuj z{os4!ZS<9zavGn%HwX~MZbU@glP*N*3+@owaAnC1kUj+Dl%BDlob6sWiDK6UY1sivjwC{%gmdib(I;$~y4*GY$-;P`YJ$zNlsY*HNscM(aQomdr z7=cgyJgqhMTa|RC2X*1 z%!eq8Guy2`q6UlEKm(tng*F#po_7zM0nwCF7vX<}I$>5f>_;9ZCgQn|BpWDDA z6^+YpBQ7@Xu7C5**fYvUju3IuUrCMHyEAUpgn#~@R>qqn+{g*%R$A3~t+xfU41_eA zDyii;T{I{y_{ID_aO!-J!!cmRDo>o~_j+TjwIF`ONe`l^&f$&*32&}%^ES`@*EGVH zx1YYDxahV1^UE>cl!ESD>FAGYs}FyK)Q31S*t4Nr0-_(#+`=Q2V}?p#N!90t$~trk{Am3PC98i4|lpF{JaW?N>(CcmtDBQo|ccoyl|yMqTWjUrxt+ISh2ulR*1fRJ3M zbbzuupCN4QsVDSdX0jBASQqLq;t*%Gzf_Z~PPmJS^cjojvs9i=d}kmf`Mf-=l{z;z z?*?+iHZU>^`O;J4a6oVb_Jilgi_sEJnbwUzTMMY=H;u!YNG7j+vVlvC9vhCEKo$lD z$~PN>KKG$d^I+WXtZIX4v{h^ddM8DUem4C&Z^Q5dXEBtP)f>T9OK@KwXbV20Q-8x; zgm|OiN>)rThEhc`bbvXC+b|^e4YV4Y`29`(Z-HBwxExQFfL zzb_X9kBu?Ay*@oR)#b%13j4~v=k=iS;xq&&@1oq|>{61>OuIm7&8BPi|Ew*{@`}aH zY7UCM1#ijlO(b`S-+p#A)!g8zyyI9|iDDpIgUhN{@qjGzkF63lwrh`G6u#G&4H6HO z5^!!iggUDStC4@|`CG#QjRkY5=enE$O=2u33w6gc7tQDmX-l+7)4k`pUbbTVNrBpz1F=nT{<*~a z;~}Q1#1vQZbvPNau2prDAHSO?+eU9w^)z23Cm_~WRByIi55Z)f_#^oIP0=-qYieCPP|Gb(T|p)XM9tl~asK_S+6XX*`Re zT)p}U&Nl5+fU$IiVqzu6wGGl#ZBin)=1XcG!}NLZb9Hk0K$Z|b<(4ME4a})uH7BFhZnQ1o8MtgkXhs{BX`g7LywWpq%EM3apjqUp;SgwuVk3r4wp&FR? z)l;*QbJI(-uu_(Y?JwxXzt6zm|CZ9?>f{-hJ&C^Z`e;G=mQiJ187<{R*ex8k!Hjyu z16)ki%zPeaW{^Zn`N1n_;W+XoHIGk$n;PxQh}f{;x3P@pUtfPOy`ODL93myk7Gigy z+F@ZM7@3dcR$j+*7Z4VNLf8;Xm0`8|e})D8R@R$mZ@ycL>r)~EJxeleZBBinEIoPE z49JAaQxT)YCbm%ir45GEE;HQov#DPVf-8v017(q|uQ(9C*ntXk_t=bM+D^*Lt|UIW z_dnxGydW5u6P=W}OX-R4ub;ojRJ>Q!Lx_Af-?U+9qqA}<9%^b2M9y{xV$>!uxejSy z{|`VTcwP_C$|}OwGfvw%yd%Iu3nubjmXw=`c)Qep%@@eUv0BONHmLd$880Tqc3vTteta5)5&iM4J3FPa)lIJy_<(#xHMI>iMHaaun5nnk$z)xy+u7Jr|j~-6IxzKwq!8@lyDQ@x+n;f@YNK+T8~{X@f-$5JE76G5<7)ozaz8;L3>j0s z3HgGlRg|35o!;Fisw3_vAHJ&UXWDg3KzK2|x4qaWlfX779CJ8ZO-9!ez7ZXp6O zY;Yf=zgZoKoqsr0`L(Me2MAFFGhbe;06BCd3+r%Eph;-YKNTAzxzR0%t@favu54j& z#33+%|ITk%Cuf349=D<2m!6LJo{h{VPo0_{PjIi%0>a!c^-{-aOfCEM9TgtrA|H2i zQBJ8!|L8c1{rpYAeEpN^^L>2#KIzu$Sc&O%D?!J!y2DkfCc&tZ_aO5TVLNb5WEiHJ zbieyj-$@{|9@1q-)0n8O@6^9?@41sUlQjtR&NNRt80@)# zXwYY|r7um-GkIu7UQq${*FHOTh-7q8kkjfDHC z9S=-=sCi+|`hHCey=nnz8g+a82fc;4K>$#k%UUwSa*``M@>P(YbquMP$N!+|*BRb5 zM=!Kiuw{=o+EKvuJ=MO?@c^7bN)TuC^Nj^D(&2EC?QgTXJsw(H@m2NoADJ@-5q7TY zAA-uM)g?c$2fsY z`%t8BmZU#g*|kLxgoWq*>-ZjdfVP^O2}?%sjOzWuTAUmrA#)OW6Xnhe{xQ z*v#k5MOFq*{nkd1J|Ql{HUCQflJ{>e1vo54H_n@E!`g-j?`k3{pqkEWqp@xmf7vpt zKSq{X*{NF(44P4I043p!`D7G$0|MT+4*%vY>7}?YCd#D~(N|$)ua!r&oF084*JgV# zcL0ZT5h`^!Y0ksCc*uYn6Px+Ol*zq!GGB-w5-yj&mYz(Jzz@G(6v;1C1UD)M`g0xM z7uF3|^@yEhRhA;Yp?xlN2}T#)ObL@M@FLkkX&Xd_ku;)PXdtG%c7$c>u!@T%oNP68_aj1&K8Gf521XQ5X{RpYO` znw@JXS#hbZ2N&Mwosx~QCz~kt1*TK}5J^AZjB*Kjh=f?xm!}lhjSMIeaQ)knlh~@y z*IO*TQnERU2@QlDP!XYaZ3>8L-LdI<4EqN3G3hE+l^czJHE8^|{+&>iu;y2l7GMT( z8gw5Q@cZJr>nPaqw7K6?V-TQRX$A~4KLcY|TFDsbbaAwAlIxe1sBvDxjIjl&4125i^( z=q-0cxsvf%DU;qOilQszx4FVo>>eXY4B5VB(^|t==vf zPOm&*9zk>LEV&n+YR=^I{Mt>Es1HDM$bL7O>H{J%AYcm<%`~7qK*kB(hvS zR>YFWj#cqoFkZzl#*yt7b{7RI_9&t1<@<-JVPMz+^clE>KXtMdq|o&;eL_%25V#iU z1Cukz4emK>bb-wftQ1Eu9mw^m1p9g?soOVtS~Q`%uZ)=e9j{SVYzcbz~%lGdoRc}hcvWWfyzu~L?D9mMr`;s*_ZTEs43$Ge3= z8!W5u_FczGs@tOFTo7xSXV9;tAar?nS!$$C^=O5({}{U9pOu>N2ZZndNvis z&T@hab}giai*{x19JRT)38tl`98%(RBVI-XOI8~DF?F09ju$YZ3n?<`VY6dk8s9wT zPfJ>ePvI-7`B6>xcYO<78^h77Mb`(Ft^^cRXwK|B7?JkW8JK_cW!*)`oTKq@wnFdz z3bIp#6iZtt^g?{Z)K~t8y}~{5&`*OX`my`2v6?g7N{MxBG$#CK49#U&qm# ziE0M4ZI+A7%`$eWTi_@sVwUS9$@Fn2JLmV?S4cUU5P@jDl{+V(*N!gOt23K!0a0?; zc+k3WR}8zmZm3Y35k-lx;3{Pzbmk6Jy>jiHPPZrdSF<24`1&eA?-?Zic=l8BM09Dm z-D%$~#Yh z5{rJk{EtIH=OvEwPn7e3e>saSFwbsQ^--T&9!*~nciSlraGydu!0PD4iu!iHa82p0 zNJ>szzIclMKDR9A%!LYax9BzEl20|Enlqa2(x2O)t&Gh_eo^bz5!-lzW4dHoWW^#% zh$YBu6*u7JOp66Bj70)Ul#QjglV=?Ozu?g)>gRJ3t(`RF>b)xd$cZbcazuJLp8Gjq zn0DPCvtf0929Y2Qcz)T;Rl?}erJ7&7$oVRvOm_4S7@m&j|A8^*h8)E1wtAO9;_8~5Rt4(F(l z!}j8wyTuYK3?(ylzhbCj7(v1xj9~mrR(oSwTE{%!=m1fESirTd*+y(<3OZGs>}2D! z6J5XO;UU~6k)l=}#}qq@N2aIsy4B}L#FV+!xo&c)0z1}a*3{R4?NQC`z3?tK%{!lL zD*34iWsAw5C+qZH=T6pnKwcV@d_7RYz6iec%4m)C7zX_P%kT z^{wZDOx%F>jlY}T{;nMzh;_DFRvoD94~8!!{AqKX7rQk=Sq2oGa%OutMMxF5RNgEA znXklO7KyQ`YgAomU(>Rq%TW@CWa^j?5MVQ}ye~ZcR?hW`9)wng`yV`+57e0th#6cp z)|0dJbNPTPtJyidXWOn}Z&;7%k_+^wsh7+azJlI!uKsR6!%ue>M~4U9GnejXP23qT z%<7lh52Ho;sO`3fl^ekq1=aQkihcAWO8@CX2KSKmP00wgpIJxdwuOL*b`fBk%Son% zxO~a)CBaP-s)WRgo#*BpJ)PRONjc>1*$t)JcNDk&s%6*&O{(uyY`oM%@P&OIN1^< zWP{~;$Wf-S%H1=}YrmgD5*ng~wOjL#x9bu^^V`fYxhq6?9$RsBTT3RIR%BwvU+rq~ zC;q}(5|`d4$A59Hcdyu5?ATF2>04`tIAZ66;yun0cU3Bylp!)htea_i+oKpHQMJXO z>biA-7!30`()>u@Jik0oU1l|qv%B&`m$V!);-P*G&3(*UAVyM@iv5v52bM8lW%vjs zggTS!;fp?cM7(F;_)08b{+?DX(-Ksp+Ppe#uG?p~?RSdz;TLW-JhXh`&CB{zNHo{) z#NOY8xVPTGy3bZFFK>M!^~AJR70PYkG}0B5+bYhR>B+l+V81!BhOZ7_)QMCSJV!uV?(fCIFCV8oQaDlozo zb7r|ydN>`Sb|nzj3XvzP=VI`#$O67HffX*zE%!sE{IfbHTtN$=T*`x}J{6sckbaAU zR?LKmx`6bnOqi7X*%SSP$jTFTh{Of~_`w*&6m)9!ddWLaj~snF>f!M1v{#5N)hfe_u|1p!OI1IW!>zi;WHH z8ZgpyZJ-v8yZZGcXX0x|DoRB8TtE7CG#C2kMXtrgiDO6)CKUYfO(;C`3(CqNz2;6E zuEp7LH3U6DGnau$%z9;TUc?&r(Y|PDKl;qoXV&C2$){_ssk>T7w*uNmO&`~MUXET; z7wHEc`^OXoq$e2}Q)cwSpFy$u{OSj%D$w#`mN>Wa=!*gKB{iabjCTgv=EXp14QWnK zdD3ddxD%v5s^S=8$PBRA}FAsgR@(Q zA1Czvk)sJ3&fisD=Hv^KmOv>9$0sjR>k3;)sUc44;h_^XYzig3F^IVA2$O2tMHSkn zlKu=40$EPRpZ|DyB}k zSIWIXXa~4-jS74bK^13c#X4eWkhw%T`IDZHCemnyH2mv@u|Yx)ciVzBtnC`j5~D#{ zPKc14o``-iXvS^2`n4u<@foN|*k7}pRV?g>r0EUdVNY^8{|q@_&SCH4bk!8xfRrVeIUIBriE5n}* zE69C);?>`YCGeTtGX9HiLxHz8D}afPb6}g|m8J=|w@P2}j-_dVtIoEIF3Y z%!})!q!Lrb7JV_UMMnP<$37F6p`koRU|!kufKY&~N0OUI;_=Cibp zc6gXBFhvSDp`J&Nz6kp2rcqWx2={?R$o9bB?RAcC=4mi$fLP2{2MLnGWk0NgDky)` z>u?2lyK66@_HP?!X%-k1{jpT`%DBR;Ge%9P9J5cuU?ViG++Da$Um81}#8dSelHeB0RAAocu zs&FaC57{{5wKH5ptPZN-VbuW!=DNfUkG2nsytIKIw zLXtkeG8QXa#@6B#`2AP#zn&QSG$jwJ=>9(MoN%DwTm_u+0&;f{L#pQbBJOKhw>4#` zwTB{fdMd1vI39wF0-xi$eytgfk^l5Q7>nU1q;zlNgvmY73JHPB3T&_jB0xp1fv!GX%gG9#p^5}oh8;%GF+joNvx zOOx^Re*4Yjo|Ac*MRYk2Rb>M1wvctoi1I3Wwy`=v=1T()P!KH@WiYL|A|eMp(c$Gb z7*Nsd8E2EqEa&yRFpFiy2_NvRw!qI~1`jif<`{bNgq%Yo&>E`yk2u5_Xkh#Z8CNz* zw20Wxo-7|EJ1Qn9|GW|Tc*FLA5(&hqEz&cW4680G&l4eT^RuG)wWcDd?_Xt1AS=I<~X3^}@J zdJ%A<#N|Y6RnO+K5Qz!OcE~brf5?O8&s8l_>DVt5AOCFpw^_S6gQWN6bs~{+ZnoMT zx3({Ng;y9v{dfe1P#>2W41L;I72PS$5^8-vk@O0Y^knn6YOE$VXu&`=6aFdOImX%A z#!Ua-5;et?Ssi9n9S(M}uFGueOVH8()R@ANc=daQm3n!F6dYg6liYu+_t>vRQ)6H9 zs3gQO{RU$Ja4KWt@L;g{5RNYP*2ZBm6iMrd+in-%?Qt z%Fs`5q3i|sYSf#5`d-cSYU^MZSBqy;sxiI04)Q2UYJW27{r@pF96zDuKrE*d=G)7? zV6DpPCl2zol(t=j{LB)zSvEN4eO!bsF(?ap4@ocql} z!?Aq{VWDU*#S)S!jb#0B{N49hb->Q{!($C}y`Y$4v1_Eq`1M`2F54t;$1gbQ@8KKC zg-g;9I9@rYq=kqxpYEmG+O;~6^tuVId{V#7C;HGztjis)m#umTFl1EMP)q!We7|lp z#%XzLC&q)qeh4tLR*xC7i%mPqx!U47%6H#l03nLf#?hV`X#b1E6Esy$S}(tus3{6A zysP{;@igfRr$Un4WKwIvM8RoV>*`e&UOZ~+N<(a zBo_}zCw=J(XHBd7T@=22P6T`lwafY#q8;%Ksb+WPOXUw~>n`;(Oa-k{HHna!s*L7( z?KtghD~v`^oe6f!(yX~t5sr+5F|gPPs>rwcbxFdJT8>vbD1-Dy!nS9$t7j9Y=HkPA zR==Hr&kZz&cT4rf*UdED6iEh{MKQVN2X~y>suk)4h1nghBh!1pO1^v@ds5vpVnS8< zj`EMk?y{NI^3kPkT^a%HyqViyPeIv%skx=-nkS~N`GvTdO)~xou4*_Qy)ksEUa}h1 zE$?~&QZ(9$zm@)jX$Q2#y5jK5+s7|jGHI$>Dwb8N!qan6LW5gG4Lx>E_?;nGa|(LM zx1muycR1;to7i?(ZwPf97Svth0^|{Kr=6yRf$Q(&Cfp z<3N=U>-Fk796OHO5ip~bK$Uh;z8|0pe*QZ&reW39gHF>fd=MU~MO4F8fEhtfZ@&Y-@Ljq#~Eq zX(r=Ar@MQ*yVi(hRAAegC4Pht`?qjh)3EhVi9<}%~ z-au;`DtbA9u6=Mb4qKIH?o)#oi`(E0Eu?VayvS#sX?EroL-?kY|5bsaV`IuG;Lc4I zwheXlUM?Ij)xmWtnSX)&nGP~UnQo1@3lG%LH8fnVJfcUxb3b<1LIsSzgv!v1c)Rs*6|3eIV3ciuUu z;fWfs#{O|j-JHJtD&X$d3{k=3%|9V!LXlQXcGaJYCvn=)zd27m6>E& zhysi{&q)ldCpCzNw(jOjA#Dn|+s}nhbR^u2J9~ZzXLq!c88N7TArqH}!(bsamCRUtxXMP)I7s+MssT(M3d|k_Z7CbS$|g7>V!V^cTJgT zt3hfRyi(whQ~2#PBuK@YGo0M2P&VRo4dKL@@20mh)o~ihM_H{BGpS#GCPYMESxML~ zsD0T?Tf*gv&33(YA z>h62LZn=)~N-IUP`mgMF&*npX`lox~*Z0oJ8~a_LI=4& z?hn25W^v%|$QV6#*w&WV&B?DazD}?!E?WF5 z!Yz!`T_+J<9!rg3IyG;C>1UkmucUMrhyT(x@(}N0ytP81#Tg)dwDOzD%l6pcO*|pH zq*IvhAXYEh-3&dPqV8^qgc4?Pu(GM`(K_{9)zGPIv;S=ZnFx$6&vGsGJlBPufy*9Zm(4TPk`uimY zyCc+mb!pDnyu;s_sBs={HB2A_>pL$N!_7IxRNgtHeLc7-lFL-2a@C|>hzngI-T(FJ z;74DK2ISn=z5AXMj-(o{Kb|w$C(0-Gu3!9axA|dz^OI(WOY|ruJ_D2Hf=NO^ANR1+y|3G6^*y9E+0pS znCaTu$h~^R21C|a4p!vr%RnFSgkF2oN2q;75te;&0k??q8&0R>57ry0>l^@B7}DID zdN!Nb$0zaed*%zsxDHTVRqdr?3D{n;ZU99EL}Jx^G6$}zT-W1_TzcU@_drUV`FyBj zjd${4v(7;M(>1Ugph*#=6lY%nPh1vhK}wd#SWY-kHTnM%rpyu$bTyWNiKyo4f| z*~QB=i`4WD{SQEI2|mt?OU_}2GRh|@uQH4{OucZ9j=$W?0XMa!B6g?bHfic3HmcH9 z2&ittY~)4tb#WSX5gSh$Ac!a~11@PE?(AU%T59^sQAAXzU4Yyo^jZW@ZCoRp&2xQS zkAR~VFV{0!ZCv<3<(BCeSIk;4j zNo?FL^94swrsiq8!ZrUlr}rfP76|pOT85dTQ@(P!I?|BS^gpXAUW(&5-YJzEcU(kvH2V{NCgZF;uR@69&>Qpygi9haj_Z~;x#<2_N% zEk<;=fq#k1f8|!Mx1>HJ^wAUx8NAJ8K28!0(Ix&$3_-vUp334n>BQ}; zOWk5qt0J-X^8bWkl^j1y59eO4#4lCc0?axn+_)h!;2~#$_LiTPGWsju&ZRLn@>eYtIL@ z7&qll<|~70Wh^yQP<~~pf@);4$t`NFL%Yl=pj9AuX2~qm`7#r?vkupaGn~rAnK&$w zj@2VF-JQzFzZJXg)_UWO2H4!Cwn&@iFVem+vyP^FZ+wcB zU7JDaeD}fUI@7e!J%^U{A`4Tga6Wo3y$Ai=3a71RP44+pzM-i?XNyHKg4?%7L=u#G z-wg?X(du^n7`SqYfH|)G+znZU$`+>;0pSVkUgl2tyIAUuf>GxUYLhL9%VU29MQ~!P zcbV(MBVOzssBc;%SX$qS8W-Oe%DOKKaaKqZe|kYK#7P@jFFc=TFy*TLDuZpVZg$1_ zZ~v|n`-Q3r1sUVzq5}b>aCuV(^V88VleB&p<9l2DG`* zq7`g(vdu--ad%eD>k??rM~z5zNL4@(Imu{e%w#VMB8{?pIE^X1mh&mn4w<-_*1OAy^<4=}bYp(8IH)pcC3ODxl!nUBc!&Yd#2pVBZU> zJ(UcJRoOA*<9+T@&6+k=F3h#*>R4kZZfdBNHBe;y4;rU}ECQi8eOZ55t#S0#IgK{nQ0 zO03O%bEVf6tP{@=1%hxYM&-=9n#wb#Wa0E<#++lL+U+bp$$XG#J9RYJeJzCDjzh^F zP2{_M#V%Cgl?=X=eI!V|Fq^*TJ!=@&qs~I}bLeiiLjlgbjLQl$?@-Pl2>pkf^fp^llAE9gtcbB~18%*0t5;;miB+ z$n2_R?>*s4dk0amFMx{{@q%-glKZgH2-C?Q;rrDq;tLB$D0|Fufs1*oggZLd5+yG^ zrKzv9Ys2|OR%+AUK-R@Np;_#3yn&o40KeO4MaR28CAzwjvO=g&eQpC$cUokF5^lKG zObP*2;h)PaFxV1RX-)j7JrE_N`dU>R>orfV!~Cg~zk61_F{|x-{HM#{*M1?4$!3$P z_-j^fV_y|C8_P}Cp&QU{=K!r#w=sdlqVk&`xeV;sY;c^=FYQOh&eysx)oni{4qP48 zs5~33sAef!f_0IGf-p#R+_kp%kyIw4#w4LqEs1iA`IFoG;4h6c=D3C2Tr>?udA`rf zSxUkBd$Vd$3OuI1+Qiui=hK_SY^QjydKE!u@iw^@eQu{}|rI*C8a;6Y1v{;GAX*SD#DlKwzWHl#=5M$g@R zgOx@`0sjrfXw&3qbAt@S%T_$PahqLGPejlSkMD(urg5+*MNkpL7RYP8bhBQh7_aC? ztA1C?mU229FkfDtEs7Pw@b?)Umi=&H|GV*Aem!&gZwVcG>-!_d4qb|*mvNqUx2sD7 zW2F?AmSe2VnwDY}=A!IWh{7eaq&S@v?OfPtNYQlBP&!5_fWXccUf zzQOy{Ww?5e`j69)#|U%a8@qP7d{VQd&b5SEy4xXAw4R%i(wtO>QQispr}zvuSc*Pv z6@>YoOJ<^9RlgN3Gb>OS33?eX$DC`XoGh$dG})nK0$rH(hpIe{g9tajgJdSlG;M_M)VhLd=5mB=>pOWBXD~Y+`k{8uxok}Nw#svG-)F!y zBy`)m85rS81lRea1@ON;B6}sBL7DQ-sue3$0DSrFZZI60&zENVatazQbVF27-p-6Q z(mgqZ{nJzTiYj~XaRG_8Npw}~=ON#)3Js!X!YGi?j{avm_i-ILhuR_^(b>+MbN7UZM^Ydj&X~Z34!nfYdro8AEbDy-}EvQ8Lbda zkaZe25q{O0>FTMIl8Bugnh+;6MQf z!BPfKp9D+v^HrC=oc-XXXWGBzTYA7NWrO*ETzA` zd-l!q2ufMwyaF`?smN?#VphLD!dFBGsqnd~AhGrYsyw$R*Eq>NTYA=n#!+yA!Oq)s z8=7*^awZQ$4@o+)kzjdgSJXF8P*xmx(4N4PC94Tc>m(I=4_HH44Ub=ZRgKE6t+>rGdby{>(F^cZZ3bcC1=YUkkBUSEeB^YqMnA<7BTS9``0@u;AA|*_h+HNk`W+8}tRTnzZ zUo2XBgFPT2f%u%4#yTX~>eE!p(sfH=gp+nWf%O=NZV2XT#FC2INzbRY;S{?3y%)n?!}9!pS3w zw#@epIXrJWZWO&cSDYiG&llSxCz-lR;Ae-0xDIoQOuBe1wUbg=b?(-RE-_6sFw1Ak z733#N*K|_ARCm+)Vh}f>!obqh$f(gZGvB67IF#fLWE6=_ZH75V->IAQ&OT)Dqs35c ztU;V3T0C;Cr*tc&3aTuU_TyR6nSA-)ZSJxrkXTGy#P>{v?I)6MDX;&Wcjyu!%@d@I z%MlU3S)`7h>;2i(Chq1?rC7s6?bdxLc?;;o+cGAs%oCz2nc(pakjo*a$^3Pm#ysi= zYuh>l+i(v84I+l=UpAaGM?_}@BvO}o#5cDw>F8BA0O66Y$o;Mn#(|a-mLhC2*?dHf*)q!wX`8l zo{yRVzlZgZ%!YS{Z#bml=R<3AYOfFYs3DH1S&DB&r}`^QesjW!G~0yMT8oa$?Zz+=BIC#wp=jHF7lx_B6JJ(zL#z;j)9ilR5v+m{r6-l z)_)dMCku8;`u+>4epA%5%stk|jajv+i11|NxSv)jhYho@$_g7XV!4DYO}*08&wWqG zVD7bJFZWi-`mtRJWMNXp3$iLrS$5|YCGmaed_#$!NwT) z&eK3w{mm$KNV$zbV`Ac{1=Z=ZJE;8CK1rIIoqB;R)nNT%S1oI%sA9MREQLRS%C1Bc z#~L|!77!-hD2!ApuAk(byupsq5c-W2y~@DB_TB7$R{UA`eM9ra>NVNs&Oa(36BkVVZ;XYJiKnSLc1-s>TS83z*fqav zlCR4ei80&EST!Rk(__eD(nobg>v3I!q%+c8{WwcEon}jU3c3Ie$4nS4MF`p<8mZ5) z@X$vlrF0%)H~rdY4uq_(W?b&I6qqeFIS-xhqYYT^VB^S(T4&83v|X5mk!QV#QHa6G zCH6*(>4kWd!z)F?C@io!cuNN!M})C;laTRG?TyG2MlVNS(7MhjgL8m97z663u`gpv z&`8_4(Zc$iF0mO>(%^n)r!_@J~h-F2Lw@>G=jz1B``-5b+Cp}L#c{Kj`(7%SFZ64X&-OB*h@E|7ND5hz9* zkGs0OYO_+FApgwm@$|vmWRolSnN6`>2M6G<4N%bx$1Bmd8_V|y%3!aHdL+`%-+X!Ig{G0vYr=&^N=1*=wjEy!eNa+iigYS5h5dZwPuX7XFiAOdb|UQVV4;ZAl_8-g+`jJS9>Q6l^i&RWtdxj8}%-J|?-6J;nsIrQFeS?^AdAQf_4(B+hvX>=bkyhGFJX{eT z0^%=ef4dwT%zjVKTpop`ywBV{5878^>WugXC{Er~E!VAH-!QB(ds532$46gYWcR(w zd`!zy`sgui_r0;bLf49{ncc>%K$_5cC&O;fv17WqU?grK^UXGB+A{}S&360i$OqkN%-?%N-bE!DUpGdF>Z6^B2VJR zR)#ETh4hp%E@4RMeUSb(x=VMJn*jZaT-+XEVwV^>He^g!B!HkTz4|b9 z!kbfp5-#qU`9&?11Xi#mH)F(SNSM1!wy-5MRIeWIXC@gA4MfwX9OS9_!vwSf#9!D< zRVv#aJ^l$|Fg9x{DNydCjaG|v=a%-{%FC8B3(A<(+%!{827SU`UU7}Jk#;8(u|3^2 zN_XkCT=Hcad4ieoQe*59bkK_#!0tDQO=y-yAIBZn4H$QPnJJD$JT{j>we;d&&NU^k z7}@aZ0DcBugvMgrfsKpezhlaV>yF#DS34W5TC-e7`+7|6_p0HlC!(gxg{)-`3AKsZ zd`ZRCY$7andenb}U+DKq2-B9wRMt;=sE1uUy}b{p4T^mI5X)$?=zdrrvXJd6Rpsvk z`silv#_&zmn7J`YFs0@+kRc_rG$7PMU!SO~%ysZrdBgp^%joQFlW)4fpo#49A;t6f z*Aar#ib0yd3W2#dW7ae(s5cn9r-WQIda>Q$27JY`=Ed){tt%5I&UBGFfQGvXt=oU(s`p8A>!q4m)1LEtpZ~Ko-CcYAyHVqgBsEpz=w+E6g5mPO zJ!4a6671_X#9r^Eic-n^cf!DW+PcnzvmW$Pi=Dq?1sXIeHFTzYqVZSI;iHinwE7ThssTyk+c@)9mN7TMPKYm3$^`w8YFI3K}}D zHam~Z)G?L^16i>6AV9sEjLVb6OjA7IQ0qi6MQy~aV%-mhs2>?}1PrDcp z_3w3>U>b@IELTCb$PTp@gIgDM(aUR%>B?N60)fp+`PgVB)8@`Az44uwdL6|yg7(J< zB$Z$&Ncu5i>TG&ROe6MTRQ7Ofm1UVCqxw4s5JXijhv`9ePr3=aE`rwWEu<-kdCTUrrUGQkGV+)5_x;?*d zPHx|`vKZ0X=lAao$qEnDEZ^lX?3tmj_j*AJ?WhHU?r=^N#EFgcLjK zai8ndYJ0Ymhg6TJb(-Ba$@(5;J}Y$uLY>0od?nIwwnh0vQ+W5us$jBOCp)@`~J z)mv|c2e3cjKFfrB6}bknGD5ULRF;^lln<{v8dhr{Wn?>I3=VkBjlRx)a7%VkvuD`t zlcbEgS#c`vpqbM9Zs}xNs&NMU!}TvN=`Q-K$2pFL_oM#w6)x+4*LN6G%&?~S4&L24 z+bXmHXS+R@9(*SsVB;l22iCk2T1+vIW}QFk6-p|4DRTsZ#uS~BTj7OG2$~pw?N$TU7QO~`cU z(n(a-Pp#z-{|G*}HK{9uRNV>KUeY3czp@{@5kir=PO!pC56>g(kE~aw<-L@k{y#V4 zOb2B~RUHnxqqgHlWgZ)`FZ-XUkxscvmAphxteiR2F{JPQ?4@$BZTT>^$DaihVA*&( zv%KtfZCJ`VtJuQ7(*+0a>&w;uk`2(Ct`=8rbjpI7bB1P%;XJsv306YyaM`9Y-itGCGT| z2W?B=0wGQ7n)h^`ds*%ogSdj8h4Z-uQrml#dLN6X-KLDji;$MdM%rgEj>v>Pcx=n}rwpIVr{$f5{B(MiYsvbG(0jr`Coeb(o?|Zgk(i z%P85W>Yx>PEZ_I?dHg6u=Y1Z&z3Ok*gnR+I`j{t9DWJO*m?A zjX0@ZNASey8~hP{Wp_*TSIW#&hyJg0Wai0E?_;in+}hzC%|@&z*((O3$nssZ`Ld=g zP1BRivj5(0L5T%A0XxK^sK!<9fMZxsX>pc5+KkRF3EY%-(D7ws>BrY0lAC(NY?BoR zI@I24xtIA&ON0SOGEQR(%%mW1$Z$PYZ0YU&F{vc*37#&^0p`grIuN62EiWRRtnK^N z{+)(z_C4sd2+Su^g1p?o9k%pdhJf|?Q5RW}`}bl_5T+pqq6FLW>j-fERbrqEz)lWa zt8teE!8OxBTHp2DpsrYN@?I=#c*Lu^UFuUQVD<0~PcBetW-Ob{Ju=izj4h6rN-`F5 z$bwuP-|aLE%u`J@9r*c^8Ja(CMO9HA41^{R-?D+pyp z|C59tIW;)u@#53;I$ zrXMv*DLR-ff>U0sdhQZ5-N?kSQ_H`8Nw8y#Gj!H#zXaiOjz2+OU%4m9T1cdI-JO|3 z=Uv)u(YQ^h_dLX*Ntpj~Sime%+Np{_M{n$3>!2QEOg4<0zwiK2e#JOAEGOqc53OJ+ zvBIjVfaeEf1l%B*BEt;Cuqe)`?<1murVL$_Yk;eWY~gswBM$RBh}ex=GI<_${&(N{ zdr^_YoOPtB-}WBq?BR{ z+GKN?qA#mU=yOZl5z#HSN?&5cnSw9qp0M>u8a;p#-UvBp45NzShNL*%r8-CXN1pm_ z-D%&>$WHM*FBf^GHl63<^84R$=2I|{gd1}pLLZUY+i4kU4M&UZ=o)&=xvgXF5qLDo zIDxaexWE>9YiQNmqxE>u4{e^u18Jd`_I>Yw0Az8$6$WP7?eSl`Qd^B%@w z34X~icH_Yd>0WN*8=v`P%khQRtVD)D0jiIk5>e2n#Z)$_XM|NV6!prrok_ue&VaSN z@|h%*ez*?bCiEoIVlXKoo!?LJ=lFS5FQ#w4Pg>;Zz>u9oa^R4_P&-;t2j?xH({2&` zr_e`VyD2ih*A|DK~&>ob?k+yyQ5pb~@fJcJK+rqlU>BpSqX?B?-c|Lwv z{&9$H9<0|p%MnQ0!zMYWsn+>=;l($^c=O*O#UUOumdQ(Js9|Hai2c^N@~21O%$Bm0 zINkbq8Dz1}`NsG45!$1@*mlbiL=Y?6qrt}i{laI?xoNIJ;B%jF>co4kAdK+{#$oN| zz|%$wPPLEsi6GXls$=(}0t$vGLK2Tl*$Ev!aOUsd-x2A{R03q+!k>jWJ$v_YBbbo) z#SfE1R;-ulp4sb(@cK=`2TIjP2|cLHF_Nawf^xe^<-{RzZjZwKSsvmW2(|RiY|qFC zBA|3uHIX0n0Bx}n_!N4RE=!U~r9c~2R);67g;Gl|P3wwO^6ia`6*a=(mT{gH+i@+| z-d?06i~I4n7=P96gW&jq0iGUkYUSA2W`-G(Ee>s$&#rs_Z=rW$7s&)}wp+przOTJr z9Q2FvUA`XyP++-7;M`1m9zpl*07Mv<3!?@eM1XiF?m5r=Go4@VlJiH0p+arBo@+96 zX-gg;G3J>)#|-omQnK5}u}I%T!+XINdWF-TuSAa#c|DBf>kfr%w=xe%5ax6=CFl4$g7;Bt5vt_{ZIPKULQVM`O zK}damw0mewn41$YN(*j+1G9BsWWL^Js!05(dDkpHC4d;EXIz-eeg;IEm{rf)MePK^ z@n^~!IUL85)#XSB!l2g2I}Y1cB{XR_79V_uPaAoyB=(?#c-ue?KVw2{mXv^!3u?;2K-4^wDE*Mr=Ab)#CV zLU}{GFctkZj_%vOAMJi9!835~)wEOwWoVCt$Gt&BL;ai7KixB2XGoL;Bpp$ZCsAAd zVIQqGVuigB#ZDodfyLYT_*-z94Bvuf<$%TAc}DzxFNx%wfmHH0_6MK7S>~(vjM$&_ z*JE6KqO9PR=YWWJ_}_avrMXM_+)sP{VO$+rL1l!Q;Un7dzImlhlhP=ElJna-`lkHP zSBuQ|rex~p)TqL7S^JTdWaF&;@g|ioXTXz9t{eK?D$IZGr;aT?$A@Nll9Q&HJav|X z{+t0a=%Kt~cTcz!E`f3~=LLD{~=8J;jaB^!p!-89lrV@&Xd+ z7wGy0Mw`CpJ#MVo8D>Yn*B~`d-xqqHsc2O;W%%Yt5v%{YpumFUS-{?~??YjP;wZ*> zPd`IuJD3d~a7j*dRh;rQZ+;o4T(PONHDFmKH(cr50XZv5;hUlsKN=_a{&WXcPSRkX>cn*|l|p{Q4AqGTR+U$#T+ zz_FMA>`1}l(cZx=4up=wwa4g*m}^Afq>RF(L-VRzC9OM{RHU}1$Q6zhZ;~nu=66Fk zl*A#Tv_3`pZ6UkLd(F8#%lm4fYt*+HNfICApYRmfmPv$HSU^c={DG-N98qYDWO>(8e8wp~`uT=5r zlCtW?1$oAgLlRcRBX+E7Gu`R_oA;6eng`$08Xf=-gV%Mq&3|f!%T>nIsZ1t|7p&#C z5~Ncnz6Ee>r)rN0pO+ZhLz97p0XM=wWR=hIR}Aj{n?TyQjn7%M7YettT6d>-GpNMN zdp9;Qm&dVupiEIiB4ei4H{)_U2DE@{Ucz=~L*sNV3EUsPan+zfR^fr_6g*i%qjMHN z4qloJ;m!shQlwj4N7L|S{8|BVAlXCw`?I-Gh?wE zmq#Ms>ebciN@4~*mzC&M6`fPwSFKG1(8QCxfYIL1C1X(7&Kw6EGtKQ^yL++k69el?x z4J(TCmXUob=f$RE2_V^YxXXk)dpO?BQj%Mm^MWUX$GsvHoU2TWqa#cb%f%Jq`*Lg? zMVX-n2N2a`{TTA%6S*hCFNKZw-KEby@|Pd@2@QDh zEwTRbas@9z$C5*CvgP*+Z)Cm9-yC6C$qL&Qh1NuDHcuWHMr%Ab9&^qRL5l7;!xnnm z=E$_|S4}Vn+S39=vRzdaoB^?`bU#>$mKG-bRhQFc)u}m~GNI~S(;}xk-v3zy)AE7m zRvlMH@<0D?0xlUq0*Gbas%&b)*flk#~ylFA9$uUvM#NnANC1^ zf+}yDY?*mI_|B1V#f$iEl1sSz+ICt;BP71_COF63#uX~9N*&T$T9yKu=P>-2CA?E*4|1>7$z_uBy|#mJ6-^HA;6PG4Nrb8|vcQfy-mh)As?#gjWR- zS?A^1pZwyc*Y@cNr9`f=)(RB=^+IwV)Dm1QM{rh7Z07vUG%3sPYWmW6sB66uHAG= z?wPt7;ipNfu(X_1j%U6-AzCJ&ZF*!5h*8Zv31W;pql6HBs9*Ee-t_jZrDlf@J2&%g z8@m$6bX2X%f5RbCF|sJEzXx3lCB7iPGZ&s5=`>p5*-uV3XSs_9T*Ci8h0c3K@~2-6 z8QgX;oo^8Yo&sybzjYA!O?52BA)~H@Zd|~72Ilp!!n^ZDX&8sW-hFzRugoK9G47jY zpSp>MCL)fIxND0j>I#jF< z#7M&&Q=tR3-<-|oteM0_2s=+uzmt?~7a(+f?do(I1 zd@IC&bKMfbeM2*2gD&X>Oxp1bec%xlNn&?Kv=+#NK7{?%lKAdcCC$x#|ATlY8p&uo##!#h18Ep zcXPG;7$m|l2aZv0Vrbw!k1O{=*}HP~h!oe~q?l^c;~DW&qjKByPDfm;ljHJr24nse!5d)SSbS8!jna(b)m3PGEPv;8)SWJWI8cJTf#1( z%*|Ir!9WhASE%ZxKsh7WZ`}7Akoixp-R|}3A}+*BS}xySneT@GXA$v8Vb?h^m1gbc zFo*U>O~~YkNA2D7Dg#~R3C2#D`&&HSG77ykaW&-U!M%uT=Wmg#&-IfeVNoPlA*&ZD z4TFN*&PI_B+6Uz{5LGsYJ5PWPZ-zS4jXTdzc?=JA0iL3%O z$$QB;nAbumXdJe^TzhBa_fl>KZ@1g-DI@48=m=SGk6b0&NQ}vM&KJUK4Zsj3j2Gdd1 zBbLy?J*&bLIdXfuBAC&gsrBQw<@N>mjuB6oT2e5~>Y{H+FpzNp^E33qu_u}wC`_3l{|nrH(>K$V|e*3&E4ju zeXqRiVNUf!CpXyJb!UJuJh&Miv7XmzPdnyB^A}N~7K7nZ;gPygJigD{k$TNGV0i+rwI{UOha8{d6sLW4JneZkW_Iv?hCM zhS~^|jl>w;E9TYgQ2+aaeyk$4HI{@?ZuWTv52Xsr?>4_BbV}-+8KuCC90U+MMp#A= z-}fk?`4nV~Amuy0;mb<|{QHB?uw-u4)u&Vkt<#$A4n_D?JinPR?HM+6fk#CN2Bu<)zdL72?FsYd(Sdy1h z+(&tmyo9MZ3GG*BzPNnYdeEH#Te>T`Yx+z4qtikQNM0ic7Ep9&__*+9_nBMwO(6!D zNUji;;%+v??>dJ;q68irsp2cTW&NmN8&L-PeeW; zem#(qTROhyL3}8MXc#^QB@GnUYCefWbw!O>*AVk~Hu031aop9bu6Ug27y4s^GZzGlPQ*G3u-iR-$=P&ELJ`!ex13YWbAXSi89U8nJGJybwHCUHDH{)t}39 z;Eg@2Ck!Aodspj1ypkhxj2_*QY0E;A}{db>sSY1(W(gg_e4=lA{03XQMOs zQqogD3aGB`3<>(;-9)tTMq`~(5|;Z2Mm&)|RB0(1ZN81RP5;?bI@@mbRrxsoZabJd zM{DMRQe3xp8K&INfDHa2Zd$NK=ZW&UR^;IG(5h8ZbiqB-ZfdGfawzWy7EaS;7$0~Z zIj;jJISR_-pulnnkHd>+5N|=bj#jy{2LCZ*NFO?jKOXC5ve+L_U8B3RkUpfEo9o!M z$I}X3yITb&bl=&9G+Z2jGBTMskcJSJA7V!q)%!LfP zi8`Rff2DUaV3V7(U@B7Ze4?n1^2#~})WgVtd3;b8OcohoQr{>?7O&ZxbJTtNX7I`! z?nG;qH8}T$J8rRQ(>tanXY-EnDd!l=^t4VURm+`4Q3elgTGGRR#OZ$Kon8D56B-ccVI);*;hzqp@EW;oD zXIDjEr*=%waZO^dz><;0jjYAxQ=FBhTJHkSj`FTr!xh@@W#nra`+leHzA$x>zBO$_ zy_Scjr6V2ZC2dY!X0(G&apCC&XW|Y!#nB2~ANI<&lbEXIv0ZoW)rkL$HSqUjDOWkW zw`;KciI2(iy<}fClc`r92i5t&iY4kehp&}fj;`S8U4?+wSs{t##Jp^hdo6vFcC!i6 z)Hdn8w)^Y35|C#Q5lJ{V48cRpa=-KF9e8CZh$5U{QMpjLxHaSYS@`|f6Q5DYhf1D-%Tq=2j zKHdIFqih_L@y+hC;H$32-gm^Ln8WDro{jzi4HY32u?5?uIa|_m=5h{Ybb5)=4)a`- zsoUO)wR3=;n0cCYQ`t#*14qk5uoA=SC)$mPL?{+Sd^qo;+wWqzK&LR8{r9mrW94Oi z@F=Mvp>Z4Q^Em4m64v)!iwo{F%tLwn25mAW{brV{GB%>PQ&e>f8ih9+zHm z3HTv_b+uH!8$$6vnw_Uu{$<~>A0^HXSGo!zaQkRSn#&0DMOa3iA_CJ~ji4*r); za3>q(?)H*UL?wa_f{ietg>>c7uC^^$uD+S{$fWb4{eQdCPfea**S+HSJi6fYPuGSA zc#DpZagX+@70xWZ!a{mHn`fFDc!RizSbVNOzcU%qFyuqUFEug@(vZC}nx2KQOD6OB zxe}JCT2nBO#;N*C6eJlt;UydIs(R=dp2Q7lOKt5?%G7)w9K^J2uGVPAeWLSQR}wVi~M0q3h2OqliP2`)Rh zH-gzw{)a{!#a-#R8s-)3n0BYqLoqst1RGXf0)*M;WiEb`{aj0(_6RKrCPlFyPr0&w zYn~aIX+}KCqAts3N7^}qKW0GF7_zS9PxIQ?+-zE4K=?pp*cR7O(n=y~HE4)%=*ItT zu3UC82XP%^fCO!@S@;Y15r?jSx%t_2$85f2d({o+b@?%PI2q%*v}xKD zteW)d8V?^nxh=G4#Getc&)#TcTlS5=0}R}*vGzD$diyao0CPZ#W!{u` zpbX?_)F&y&W4h(Eg@cLoHj~3O3yz_&5y46Fa*~$i9YmVgTSkQRW(^X15GU(~;B%af zX&Vmt2B$21QTJ$@B#hP9lb;QpxsH6Tk{Xr%b)EeA_;slzM=Q(i%#genjQU^ z*A$cprQG5P%mfhz+NKF&%H5x?ZohM%3}+wTs)A+FVT#HLcQ|)vq_9iz^a{+Q_J0lI zTmd@X=k4AKGnAIny^2(_E*k zS&f@HZNJ^lC~v4QMNhH1L7mceZkuG7qghZ2F@!Syu;3QM+@IEu|0n{cx5=UpL1@;V<-5y>|&e0%Y*1&1x3#Yv56 z1tBZ@Bt|?2Pe<6l*_sURtwzoWmU$02jLYJJ$9jd~AUsg;N-yUsUc-R=Ej7i*qGUfk z{N`CVW4Z;Ib3{C0oQ+RW{NaIiY*??(DgdE?mqV$ z7j7vbgve|Mt-!*Pr*h8bmF2dm5M=*B@#7(^C?ADMmQ1Si^>+sU1X})k0S0EX(;eB1 z`Qsw84o3zy9V0g_OK-szp0m?B*6!YMpxz1xcZ_@G3m3n5g`2AYhZYUmKyz(t^fIviRyXLn z4TkMWPp>4ll2lG5d*U=!^z@Q86?FA}=S~00H3-&W5efHFhB%u(Yw&WM^Zs}8C*%_1 zwDo62V-Z?htj4Avlr9W=>+*Y*Kibwmanp6r##rUgMsPjIh*7_*wi;`pM;iu)@IDc` z30G2EXB=!%54Hw|h2V}dwJf{U+!2rHT|VNQ$GO(2;u*Y zOj+u?9|LFjfrhFst>wpXNdm%R;my*^vFoIPgHhX#TNOJR>$!w}SWN43yBUsMxII|3 zKLeLy!FZQx=HkHkcRfgppH~M=U#|2wuYr=KBXs?TjZ&VThN8J~en8Bel;6f6SBmL_jV%8I9#gFi$Y3N zbwY?$RAJF!Hskb}BqFe0FRmFUAS2*(g09i+eo(E8G=ffqd$lFzUs32k-3N}_L6+Xa z@DLSamqCG^5VGef!0p@qYj!_WQn2|GVBK&DIP`>*_DEeIq>93@Kt8+u!z}RU?Cno} zFiRaPTta%$RFnug@ugeV!QqkKO@XIq`psT`6NKtlQ!b#O1Q?uc=yb5r?&shXI#;&s zL0>nL%p+(I>6JbN0MQ8N5TFt7t4@=6q6kgTsFI>$9t?{=@MAIuz%+%?OL{Jv^&|ZI z40U7Hck~GvDVRoZ{Q_V@5_D76kGSwNIv>8Lhs@Xz4$!u$NjFOXD6&-;5&0loPU>Xz z!x*p~{#jc&2<$rb=!C+4XSvic;3-N9NeqFwvc~@<>h^N@&Zy$L@hcMTg1R^*Z%xwdVos&vv*rO5 za`%Wgc+Bpp`j2F1VdiQ}2ciiPGExa)oI-xxsyoH6A}}DK+-W8vMF82Robtsg@^OC# zRNMe~C{Jv8Pjij3#jNNMl%bSAMvcA>Kf#}B%XK3SFiQ-R)Uig&J?)3kldYFh258xjl4ZbmEIJkdz6)C_$&Y3O z5Fa`QO9`mVmG@EH^CW=Cb5Z^MW7~2ozz>HH`ywA)@`0cA)<9?tE_pZ(sQke&T5l_8 zHx&-R7;-Z+ehMZKEfGFmj*mcd4c9AzSw!=s<&WD|egJO7 zs6)yC1(OVD+&|M)2sEz--cXCwZ)zitfC7$ri1$Ic z%T=kfH_3%DUNHD_uF4kZLdI#_a`1SOn-Bf|MbCjtpL-K zX#W8Y#65$T7i3DFea5aUN{sRwiqOkU4mQ}n)M@{>66Fc9Ci+h6C@O|BZ@5#~0L}&b9`Ih507X8rnI5`2IW0nXVeUQ4YTM6E7|fJESBEeh)hMYi795TKCXh{0uYu zeSXM&H^#k*FM)6Fl`TA{O;s1oe>){1>K*+EkyMxv_N;u+bY z8=33_dK2k_w}1Z<6d1gsA%55XS-wkE^fSs07MJ9Zo;u52OH;&hsiw+P8Jc)Xr12-= zTHoWnx}tL9j!&J6ci&5lHgStu`+51FbT^Gm=SYBtsaRC~L-UwjCZPk}er_o~s&oCr zyf1V6!W>m|xyrZuR-Z@tWa21KuGIeTDE{X#J5dVsYqNwO*`OSgj2_1RfJj|9p*dp( zFMh5YCM=`rf@FYFl&vZ3(2IyC^X+=mxOcup)*+w0t<~z#v>3X)Y+Q28G&LmlBSUiK zGzB7F)C5Qz+Tv&?xb>yXYon}6We3pIc9F*qqIa0sj2u zyM)%qGB&RwE>!RWT5z`bZqjlnrK;dbzKP0*k0<))(eueGygA2W6@9|m^UgOQm;KyX z>}ifN=76=R6m-Kf`n{QADhDin0hUmhI#u1Yz|zRzmt>$}g9`~1c%tF+w(T7u%(qYo z>t25q-<~sR7R+@0ZA|D{9ahBcdnisk1Pc~HnvXd}69r*f6@`owwRQQj*PWTFdZjH2 z;wIcFULy-hzm3-3PrdEEpO9ZA4|?qWpIaPwk5XuYERANQj{$041IUAZuzDKMh7NCJ z`Jb%!EQZlNr}o<)?J)-c8ndeQUy6zzwl93a#-kJ-Wc%V4*GJRt(7Ksesf;G@H&f@Q zA*=3Q!3$HLe$^LL7yHT!%2flUk{udOCuLW(t6)Mgdju$0AEbL5uWy(J7J%YDqqgU? z(xctxq~NC!AR#P)7tD7T?8u_+2Qe4ygO_4$G*~VC(!(7T!k8msghKEy^Fx;dOW`3B zo;ywdd>9y4z^$-9_=t8!gCw*1_uWe|8Q?h#_?&P$*nnb2Pu!(>0Z$|um9Ov^UcjF zBagXy&++v61d2lhyA%QGl5P+Y$)O|%5CLgvkQk)9L~tj?nkQ>~kW*jZ=Yr ziKMdmX$8#B(R7!XNy!SS-XG?VU_~NVfEL8o1&5|d7xqbcw~XrN4m0f(xW6RfUcazU zt%W4l3Y%vvk%0`?Q7R>FNwy*Lc7xLr~h$ zvvJ?VquI2jn)98=A46JWjK;2_04OY0&TIlz7(*F#h|D+8K zR#?*;D`bekn>@%BpAuCr(dLd*y~3b_8mtLNh`{4o1Nzd0_Fe6E1xB`!#Kj;D} z-FIU|lv*^mVnFAX@f((mO+P6q=CBU#k<73ZRXut9eMCby)5GKZ1Zh803fRmLK#=ax z%jtSQpEvag@of#ukfnc9?956*a{H95m3fc$1dKap?Yo}EEy#2+ga961&}EmO_$glq zs4Tb=QHP~TXG}gnS!%?fN}BnYqeRwLv(4+{O5flb5EE$gGfk`pqeX!$xidCoJN_;4 z(OxsSz47YnSi2dW~2#E?2n0 zwCgJd6VdIs{=!|GRk07QmG!PG&-qN-BHm$9p}T4WnOTT?($BqVn`br2V%DJd9E>-c zwPCIh-03|mfP1f~Y@ak;$<&lGtcl+f7}zfQkw%^^e6Uu9^wuI9d!ub;o$8T5tfaM5 zOeO^{gWc+d{C$INUz{tZTHh5%bj9+ZZd+3^-$dJ}PayG!yd1y#tIvGn%MyQWSI&wu z#aVn8V}7mvJTJ;0_NN3;-mxHn(*66_O;R_AXiiAD5KwgPv`xNK`z4q${>S4hyQywm z)M))v?zag8<7Ne%{^?syr_VN{nvf>JTZzUhC7llegU_V_JyJ7$Nv=Yw>fsc91KZil zb8e8VBJnwv(h9WZ!Z!+$;FjV2%Ci6rF!v|9WzL4W^92N{$jlbOU$^Zh((#Q7BwOHX zDXmqS36bfQE?|2*17mw_q&l?Ma5;yFjXbNRY5NK=-=B45w#}GNi>v1T>_6?i+e-nY zqfYL2ohBe!599kh!Qh-9(`vl*|0dJlzR@3l?pgzPesRkAJe9G zP{n=cM<fdZz7qlAujXHD7Wwtco1@sln}?R!X?gXId#2WkMK4G6(o$@F zu$~3kcDMD2Fh>sVl6JMdSy@TsZ9{l`^0E=E0}y-4%B2B9SXAQ>JJ@55^iD!}xW(`` zXB$K6As5xjDFAA#U^SaV2I_Kz-~QV5pa=vkJFJTUi)K+p2smiD&s0nMW?1PTSGUIH zW2Sy{u;q|F?fkE6z6E2i@r_JHH4%%?g>ly|j9kWJS#i$R7K=9uwN)Vt z9zhjKo#j_zvOA63nD2xwS!qiPn4G9inV;FFv2Z0BGG?exSo9vMOaz{%4-S=YWcDIe z_8tI+0PT8%{__AMvtCqt()W*40yOFE%9Z+@r8Z)Ve313t*GfVDR?^M0s*}S|SE0$B zaI9`BACKhDtHZIy>ywjog=Xb#@0*9QZ+}=)50b*oRH=v@gw6^N#iduq zZ@)A)0g%C*E#V;cB;TZP)tn%RSXIu;)z5^O?AP3sJW_O1#ECkUIp4eOAi(Ty0oHv` z7k-m6c6&@xd2hGQL}10fwgIa@9~#moMOJaQ z3vlIi_3Rq&w(QjI8cY-=-MydyZ1xoQ_X5N`qTVw>-G?85U?t-=wcA0-KrI+`s_iXHe6v`qKT59ZSPSPjfZC2OyaP#geVPXs8d>V}K znX#v;v+C43gRT7V6smxtHSr25@9bEiZrlphsqDm{r30C?5ke4PG=WVD zRC^(G5_<-I952;5T-ZNJKkMm(Nzt-vzYpg-OblswSDh^~^ObFOIn>$Xa9CmAGLQU6 z-*RBH3+HOwMH|aVwY-rqTCv+meq`D#(*89QZL~mJ4Ov04DvDhpOu|KuweE+9Ws7WE zv}B82t!G3NH94G^17?Z~NdM7&U>HO&JP%>$I3U?FAWfz3v7rW)b&o24boXZ^{HU-6 z2{kTI?7B#ocCpnGw2?c=g8>(CjaZ3hq_J?ut9Yn6;BefgHgy7^Rtre8OTfhP1e-Ox zu-&m(!PLq0mqts`tvSrU&L<-XmL0Ui*NcEEV=LgLtEp<#tesN6@J^H~V=~Km43k;v z63vjt#p~Ievi0NKYm1otgSHrbYoe?5_fxGFM5~ez8+Hs}C|1>YKPlJUs?7puhBC;E zWjZ_MK4UMOq8yZ|22rs%brG);^XfaRyXq)qY&BQEAdYoF>&mt^8!w6ZNNv%uQzoBx9pcEoJZtj|MmN6|({>E`*UW?X`AhcY%jGXRKz-+vvTl zuCC1#=QCGNbx4@lqWB^u6Q8RxVc~hRvw8v!-#;2`zafq_9MqlfO2D|xEy7eW@iuX| z1ZgTOvyo4GYXl3_1&nn}wn$LW{cere(|M%*tO+o=^lZm2K!^Zoe0hHm$l7t)Y}}`& z2$AllLy2g`0Nl6$)W>W^H5+-3&}v{(?J*L?EH!Bf%CL%OzMKj_KG=tETieRHqcSAf zy=c_MGQ2q-BLWg$-F+mm#eaRe7>o>>7Fxgsy4)5#s%@TGu??W}a<<&aK?*IrUQzm! z;??9229d*!22FseZ6$)R0Z4<>ssQwpDv<-9?*bKaXHLGqRuxHHQLuek1M6s2RlEGk zEL{h7eZrAq6=bNk95NQ%%;`-nhJgNO1GsR8g@@vu9^0Kruw^uLdBwRzM zZg`kx`L2lfD$Nalr^wNJD^Yb1@xmUAr?`ddP%34O(?nhco2Xmw3A8SL; zEWzPFx7;Yh9bO}7VY^b6Soh5b-ivfxWdGp8rCQ0RWL=IOX__3NYw{P1UF=CjRk@e3 z>(mSG!Btr(iqgb>1)}2`?K7gtzgqSH+`JXesLtA@31?Bn>O}e|8yDgA(wi?RBUfi@frbT%j!};3 zR(n>F;La4biu_>f1@ly9KGLOOk}x^jV} zSqopz<0`F#>JWnqy&vEXli$T70K0DxWIxzK$)E30CjboN$PDb!;JSYP_?DT>_0C*y z%1KCsAWdjz1gAjh|JJy#fnib+=F^T}Knci;u^{oD_dc@i1ULyf5OujGk*(zByr${bMXE6-X4AtE-{rUNn3Fy5RtjVQh~Vx$1B|=I-#x z*dv*EVnG4JUTfb12$4;cTVUO7lj@}D{E!Yx1@F?|iHs)bsG915J{rBoIvKFHRqenQ z-g0u>!NMesjh~uc;pGx9E0u+_^q?C2ct6rT+5fV|Dka&WS3piD&o#k3#r16?3sV>G z@L~D!eA{I$|3ZxG!4~#^yEOdwiGkhRl&gk8lBK} zKI$A#P+>&!On9~N)w)8b=V8%S^M12EWvZ3tTmJgybotR=!*4^3GPn3%Rfku4Zz!ay zcA6@EFFbrqUyfS~xjG;An)H1!C%lueL2L=L6kT>16M>tVwi*BTV=zJsmiBh{65a1w zb*j#WwTbMMqWkb6zKx#%Q|GLnK_|!|SunCeI5c&0YiABvUxRJ|aHP(~etEV?q01uK ztd{WkKsXx^qlT<8zP|1sIygTBT~76h06GFDiPjY|PzuQ9B9)=XwxuBpa;!1_Y2AK! z#SPG&k%1mBM>XA!OPhv^@E(~WU4CtkY4K%0t!Z3!)nCd@*3T9CAx}d!R~W^YzDFM# z8@xyonXFu#TK2Zz@{*s@J52eHDZzo(3JCGF6`8`cSQ=hTRsd>bp4fxWWkX=^qjte! zK(ExgK#}M#Q{q2wMgK#z=}j3_|3vL-tT6?9J{`fT$J13h1Gen6$SrF6g}AJoS&&e- zr%J@L4<{Bju*|lo+UDn$egwq`rPU70czvt_8dCgMOd|92y>HdflJ11uu}X*-%;4Wh zdd_K+v6j78#(B^0n`3~PSQKw6=4Z2z3bUNxN>rouVnE{|X)v2ZLN{}oA=lyCv)KPH z4T59Y?RJnIpAPS+NK?e@WR5J}!QoGJS-^*C0fxg&>(>OC!r1QjV6M9W{_3@ld$Wgo z7kVwpv?rtFrdkSM`yVUsALPzD^oeLnT0e2JEfNV*cygm53QCy*lrn<(>edx*5nvC% zyME>=36ymPcxy0MFMoQ~K_|-t(fKYo>f+j+n&DogYkdpZ%%1L#4uPD4-9q00 zI0@IO?Hc-k-lxJn++w{YvTh=MN0l;Mw|?KCTPz>%8_zk=2XK)g+4=Z@jqw@~C#@B4 zRPsk{MU}X~FMPcrsfO<-es2K~J4nlr5kde>6+?^KSIza3XNufpFL6EBB;t4jOcEFK z*-3^6^jzlsaux3h6qmlVj2=@kaiLg2p0J1@ne!0ixx=1ypa>+ub!oU01RN<@d!Dus zGkT{oROv)J2Tr_RGQx@e1+r1(S*zyu--F_aaFi1^y(#IyS|8my4P(1v6Ulmwh+0kn zI}Jg(VCc^Ns`gK2p_RsSQgJ=qjQCy}>nGP(mdBBF~^QvDj?i+HUf<@J` zVR~3tV@8`#4WKQ-(2>CNy}Ut$&&EtHfBMW=wiL%!3j24<*WD z)N%JYgCusxZ=dTo^711g1cItnW}nj)5qaz$Z1E*spn8v7gU+Q;@A7b7HQf6l8OsJx zSot<$NJ(iQmhgrDY9ju^X0n?hOze6;oCv4@x(8Q3j^HeTK>{$O-eF6sj?H9H*woa?akHi^j-ED$dImwSw#?Th3Hz~WzOI^ixZ$DUxlc7bq^R` zZ>lUFxPGI2c;1e$$p1}z^9YcjiTxV!bX|o#g9;HSS`RcqWUQ%gxS(g<3!8>FZjr-B zNJntb2Z_#2dqT5kih@`vM+N3|eo}D#I^dYkuHmm{L!RffRA;YLm6z}vV0XXZ|H-KP z8urN0iy-%(4rv!zpLv^nPDJx?QO^{EiR|%J>wLq0W7%vc2TRmsFolNbVq%XQ0h;ULT!uB7 zhM~H!J6shr>6iz*JLiGau;1ghykmj?0SAE$p+?->U3+JFGhStIl*v~4tCprC3$APc z?9D@wADm6AzGtu5*E{8>?t8BVKQoQWU`c!f+!6Dj+t%#jy-d;QDU?vODS)~Y%gYDU z+}Rlx9ef#}+O4t-^$v5l35392!<`3k+8%mLog@#2b4ie!iq8sDv7<1eUm{AC@=6km z{VOt}T7arOHu?tB9HlJcqY1ff$D+^>V^7@*Pq7TF|H(5;qRBkQT?&uA#@ubjan!GT zJ1U^r(v^o^wvfX_HQ^m4j|F_0?=j`?3I-v(PVAGzjS5rU=c zQ4aK6GpdJJtAS3QPG<)j;tJ-gYLJ1$@UJgaKOTq36!b>qU6Pwpcx8l%skTvdh_b|T zs-OX&LH5XX*|91}1A2cn5%ugzH^0Atl;@Zpt$?Cy?|*&3+rmVjn!U5`oY8_|TepCv zMxE{C64!maX;5yHfN;4EC=kl~d}h>%usyZ1dZM)XFt=YBM7SV3v{^}B*JrnaJqt+L zlhVUpA%M7tJ90lscqrmZ|cmXwl-=tDa!wBU_etGxHbmZ7`$&v z^lizo_M#Ai%2#Y+*M3s<+f{?3YXuY6FZPfs3>Y|INEa+@vIz@I@UG1 z7+;p021RQ)q`DkMG_gjdvD~FR^R}y3bpJ8;6 z-JCuJ*^W;k#b$t0nvbc2=@&o`nf|D>01&;&?ueaK_c1ZclykYt%^IMXjt+21l-65S6TJ{$;#a5G(dUalxTN_S~v@Y)4fVKxi1@kAT6>ks`LJU!KwSRd;WYGO7{=7=YUj8DYu$Y^I&@mk3-Q>ach5pk!y*kf*Pv^1v46?=Xp@bizVMkB)gQd4?> zz~S${?Crnu!Qzm*dyW!@md=|(!mEE44{$DY+u(XDro1()LHBkTjvs* zL#ms7s=N0C7%^4HbY~7?7l5koRi-K>l(2WIMWO_d@zGru>W_ZzK5pecIh3M-Wc%fI z^2f7P&3#0481cHQ7)-lu7OIgF(43txlS#!HxLKUP;c%jG?0?<@P}n+#F+>yOCy1l( z%vWt!Oc4f9IBRuwC8D{?Q3plXWrE6;Zi48-DoV~?w?2KmMfGn?-bWVAt(}r?$C%GM z=B_jKM|bp{a2N4sEM6F2SLtslNVZrd0lv?4xfW^~HK|HG=6&pzyXe3ivjq*Q-ED4- zIuTRaKP+Z8&9IjjFbBKG8UvNp@0Ft=! zsP^YWFT)L{I8ke4{tn_?{d&Q$9nh5+V)Rcu>8s&PkVPDHjIVnZ08O`@euJjA+y<^2 z7KfHK9&vG0jRThohh&Av9cs9>mD~0naZ{}kB*XCM;HHC(oDbhZ@^~5bRIPiLOHKt@ zzp|z)YP(Wx@u80T=>CGO?|VhDVH%2T--Nz&2(QDPekwxt%wzH-FTY4Ok>h3AqcWth zlhUH-JeCq8vLt#cDI9(As_pGV3na3)fNUTy?-Mp4%T=bEZ%fWSJYyMHp-NkTdL6x| zD3Kqa$p7j?%VSNeL7eM|j;Zr;yq`T;*s}JVuwS^6g1nKRh335P$WBSt3NX#f3Y~3% zIzL(=QsZ;nPPsw0D@|WgoF7R+&gl<#Z?1Q9;Yc7?rZHT!Uz6an{rh5=5Sv9#S%f4B zVjh{lLD3+sTqzNvzVbqAe6V#P*|@zaubV6NnBR<;zZQvON<+y-obi6Tv+{i+xA9&c zIAu+^1J{h>h17QVpj@t!DsIu@9m~w^$gTVJ%=E$2@T`j&fDoPkzS)Lyh9Ezy^P8V1 zi`=H)OhXor?@@YZ$dq(UkB(oWV99BlcKW>^pdx%7p)ZRSp3@%c)=6!9E)PHIn#x{; z9geSBoM1U`C7R6V9!BoByR^zODev$fr|a%7mL~{~-dj|~P`(s{}_o?0yUJ)nQG}TNCRPRQ7$}^x7y-@-Ehl z7qL|Ynl3np@^RL7D%625UYZD7K+?gML7HmIIY(?tPg|pI8v#c@k&S>d3&`S+w5jS0 zej3Rj#`(4;1FKo=6QXIdoGTSKJ;e6A8e|JZ|A}&i%_{*GTLuRXXVSO*y3HBFSruhU znMZ{i->|obb-o?l3H*^ji5iB{7Rb@e@pFMqNO6c%ZfH~gM0O>7L@NaFm^NP_Zb)+Q z-h8S>>(uj-eDx7zkl7>Y(6{W=lrPr)^!CM(wJ}r9@V|d}`=TAwZi@CU zCX?8!4P)EbOg;mft&9wntnz`xu&Qs!N8bM&O|{*m@bk`Tb{`I}`@5*YENU`eukUXf z6zbRVdk}yroWvrQ*mRwwkjRP|9oI~?&5Ql8O~$A9fGOcNGXe>c8cGSUGNf9XDcE_v z$9tcJBR!gd+BSR z4%A|vH_{YA?Soa+;c$OkwKZZs19I1G~43$;R6!0AWOrO?bZDxA>Z2x$_TtK2FkQh9o}EG)!W##|5YH z3^kjrxN=0dj!%ClRe!$#N|y1d5;P;e@KD`yw(<)Uvk*NMi6E@$ImEAMk&Mx% zjmt*6*^3P%G6op?#t*ipo<0YczmGh9?uya!F#kg&EV_r3hOWSaf+S;&FK6aCcvzqt zyHkuY%-&RZQ=(|@CGlp!VA?dn<0RFWJyaufl1~d;BBNR0W|pM~n(BNOuuck%ut5ch z4V{;!*xIcoq)xTnq3Dhe=_G~fCUCvDNh>)DI(y>B5>JU&8PU@4_G$Vc!`ljU}Z?)BC2 z>p7T*L|NHOK4dyu#?4Ixv@Xm>MCcE{*ifD~@aGJL8(Z9K!41k+yGgzDUGR?)Vcbov zGF}E!47KAD?qOrKaPMgMavSQ>MaHe+{Lk6lju6dc%C_d5XpY*KOh7~`tflFu$zfV(k% z*edIhgVM8TJw4vB@!1nBd#h}QTC+KfF%>J4cG6Se8Xt)ZGM~=Vhrth|;_~`BTX@4% zZM!v>HC!Xr*MEyi=mS0qbeUyU{&=G&t_qgYdSV%C!!4!}l}r`TzL*t%)2DQT(`Dps zY!tP598y!(leE;CW0xLANj-ym&d;|xV_4#fR+TYda@grCYMDklNcmTy8Mtr%=y<@= zD?oszS)dg$$zaobM&h0Gr{T*#vp#?^{>W&y0g94Rzqmp9Tb2Of-B1#MrXRLzKgEv# zPBBr0rxuyJ-FZzQ57wTyu;_63jw>mKu=i+_1p#+P2F(W&wrKJ{hdBK4KU-qhtSDK{ z#*`fVxI>qmlf#uH@1^$6^Q9{@pB`1SX^|xs#i511m@+Ofvy};!8ZY1Iy0G>t?f4Ph zdv~ee@zn-7S<#&%LebU`02*|8?@*Lh1X#B{bq6wpr&)~X2eIP4e3Mv8$-aO+QJEq% z`F~;%8L_Fl=vDQVXhxM`fx4z>b@c334zR9$eH$`glZdIY064CtX|a+O->shYJ-mJi zHxbtGYu7|FS0mEL+zUcg96{#$Izb8>uIz~L()@j)P^G(L%LI?nEQn;i&qFva^Sf{QW)e_- z*{hq?7S_Z}yGP-1ioalgN-a@B{0E)!Jrc?Ljnvd=oQ#;9ncm(XQ{VY`8>N1h=fu1p z@q&gkMhjc=E{%_}IC_MuKRRdB^fcLJ{^g;ghuG+^4<2?^v~BM(t^h&_lc*v7<>fU=pML{f&_brd?c3XcveLwRP#M-kx)&L~TH~ zWqaAzio#q2O{ZOEDBI>QGXwn}#WJG$qq?p<&xkyQ{`23t@a7oayUL`HT67m4xixk? zuy7%{Y+T&_{ilpv(cf4-X1W~iOP$WO==q`&g`EgU)%K2b2)fiA>w{a zuw{%GD&R$QeVYsq0e@S~yox^7yo%=U)mzVZfiTBIai?yh& z9ciU6+)`|1)2)(}ntyQq*OlE~Cc7beMa!M*vYyMOuIqGlKy7qJL1$fe;-9wwc%u&l z@3TKKEdI62KyW?e{g7=D=iuCOhQW)jS9i3D%+n#dbJL;7o3|+C{X@bsa1@{MK!EtX zme)MR2sxmGG_{3PCaBSO3VKibN3#-qZyj#aoDDuub z)7ow%OU8Ko2&KDfEaxx?TRixQgd`lVol9?tFqG_|dFU+}7Tb%V{eM5GKcAY0bSexw z-+*Ik{$EcM@I9DYHW_(fyVSh6y5usELo@($Y(9LVRwnRc#F_(na|-UD7Cz%FHxcm} zt0@Z;bW(p?em2?3@8IApr_Y#O(#z76KU1VJtr){zNp|Mo(`tpL&N0?IP-a~W z<6>(O1)33|7qgaOZZq{4IA2D(*XE`-FmcIz(tVz5)Y`@AK;^nXBBqXmy+@iD?Hg|F z@}EfgSJb7FpiRXJH1l;$>iqXrA_7rBBZEXEZpIIjHy>Emv4;42E?lLR^Z`@Kz#|xc zie8@5UZvEk>2!}mtuesqDd|b$cZm4*XgG?e{MAEd-+K5lPpR0F4s_?oYq1>cWc&3j z=HQD0eqN7&=~?`kM*YiNqjM2>F~o3$^qfZw8meiid}V*HT>XqS(sE0ga=TlgQz=;eRTBj)nl<4e*Clk3`VM>^#U(lEi0{~JcX8vndI+KwW4yAL54--@M_g+{vXHf-1uBm%fxLU>GZ~73dQ|&vE&Hn1<^2Sle#1+h z$)a)+C$ner@Db&E?AW-5u@J1#N1^z;m3b-C{7b^nQ?6M}!q&(&e;(8-_I+_H3|GIu`)KQ{Qib5{sj<3t?28E6yf%m)$n4A}1 zh4mmjw`1eJv3 zuaxW(x2W_@)Hf=}sAF;cxf~HSHqhQ(rBL_%pZq(Gmxpx% z=&R36;nQ$W_jL=VMWuT_Azj}nu3`k|&Z>=iRy|0*6cdO2vI{$36c`$rZ>t&_H@zHW z11@DD_0Kr?uam_8rP0(l?ySVe+xY(HqQLv|+|Qsw>W-L;0iJcx;$02ytzcLme&XPR zX+5C`p^MbDjyn{j9Pwy| zyX^sUo!%%BD`N0{9%zkso1~@=h$?vupQ@d9Ba9$u6Y0|?43e|bqL&asrT2&L^Wa#^ z*&=3pd;t&vzicL2U_-hh78A$@`x|T^$bXC}OPx!?-aQL+#Mw@CCmW{X zI7kmK+LSYPXcHI{JvQdt8xk{Oht$}EtyT^~T4laMkoVq-|XEZ{2H8p&ag;gXvUc1Kq2`5MNvrtu&xnAD13(#wIlyn z_wfX1ZZb@rT^++E;ry3bf^}=5t)A<0X$mc^+q}pNMg)3?n7jB;xsK)v6vz#%c{|aO zURp;+NI^Ywc4212bhP+KBLav%c&RRr*LEMfjmf1riNq%Fuu5HC2+lg$q-}ovxVYWO z40{-&d&y!)9HeDX^(7$t)@r|4J zGgw)ZR2=D=;`q_{dpD3L@p*>{7?zM+^q7U3SO>Ox?iFyk)LKxB8)kT!P`|kORMeGf zPWstNQ;uXN!ylqXAyzJZAqUWdqz#A289PM{4j1)STFQSpU|Y`2Ry&wmylI8`a8OG5 z2oE{&pGN~v=V0A8GG_hW8RKsIduqFoyp(44m9wZKOklB%ul6ZRlHp#H=rjLdf zhon#4a3)=4l+`NM^V(b6Q zbn7rfRvvh}A8#>5jKe)I2%l)`fhEBWs2BuD{|s;kT^vChrMngQY&nhygUVws7N;9I zmbyS#StXf&BfEMK$cdZ~Duh)HR{`zeAb^(kPEMoE18AhO6gebsM!VWQYV?w=Z~__; zt3gO^08((eRA~^&S!yv3pwJ)(+zKw5wg0hw`ZzsFjl1WTfbqC5MRwk46fAFc_~PX~ zPo@#MXpWJw-?7k5f=VJQd0&jk*RfE5n7*`C+-ymD+(rkpAG8iV;5eq_& zfreZQhY^bLlnhO$L%)XRIvVr~%M*)bSMyhV$P-?Tr4VwCPT^(0vBQA(x_)+eds%IC zo&cnt#F@JjPQECc*p%HQiB34vPDT1}SheE97yrNeD1gpiZ>`I={yvu*OBJ3 z+{pJ{!F(6sop4PLHoJL0B0?ll3E9^9z;X@{XOzG*uB(qhZUey;Z=AZ3ee8wUN|>3? zzygEle&B6WA^BVstQQMAZ2__M3LkbKP$x4R>A3)%*2l#zEi8kK0k^sP@0jf;L*two z3a--Q|J8M*k%0B4dxj87dc59(5&c)TN{r+4PVdPb{DJxR1GX|cAKpwe1Wns~#cW3j zx_IKkc)`9~d+wDoprzpt31O;*;PqeL9nDRz-z5JDd$vg1vq_1|WYXm4YT))(7{fCG zx6?rb=C_o| zz-w?Dg(lF#GGqiu45zA>z*tc|_rjehBu!*l6Jr-(*;)a)=P#|x9aQi%c0Yax(o)^-}!a6XhLD6Bjnigi(@0p@`nS2pjpRH7KW! zj|nSkvRpmIq3;{RHV6(aO>5L8@s3nm$kZ1n7Er;>?!+3j+4L;raqlzbm8*7lY{`uC zWmiREDiPWgKWwz4kn_Suc=h3>I>wbA`z|@cPdxn4lY>m>C)ZEQ>Cu>ChpOWj%02Uu zOnU!bK>hc8rW4X~L+mrUMDJq66c9Z1$_S`Z1kne($x@jON@krh;^P+P8|o(wzHl(3 zZR6=kXS@1QZPhy~$U10FqLLdN?B;ZtCQR58^&uGt?6mLXl&GWk%VDeb*;K3J9>Akj z3-&sT^I{5JW`{MQ_9ENO2M^r)Tw1wgNr>(%0i7|~J3^2rr$|kJ{o~5|x~kKK;I7kO zwIAZdb}z{4_sNzuuTPnb4?KYx%Ga%j0nqy{h;%A_BSZiywUjeJZ?6o#9o+_|^P4#- zuTgPb#I=u+Op}G=T@UkpfW2AeGTua~0r|0uPv<=ShseA2Eu_bb#>_jG0@8DDqEnJ~R{+zCsKIBT0uX zfSblLJ6YAf22165!&h zkrK@SKIv=^Q2>tJO$j$%hQay7o#itkuYID-MryF08?b>>F(TRC(Nf;TCMs(O0lC-{^&JtG5f7o?A?f*VFIZ81u7vXhk9$nvO5g5AW5mxNp?2=U=agmSXm-Hy?)N>*<) z(pVERpYJC??d4AAM0qn%Gyl+53#rlpx;_Bia8BVso)GmZt(sgCQM9@*ftO=62|bR& zL6`cgf@4e%*Kq{bQV)!XhAyA*rx~O9X9*I|h6Xb-fufm-rIxrbii$KVRkmD>BsIChl8E|daFk%6 z&5Nd4f!h>t@=3#FR@_eiJ4cReC#B6O>R*<5?UVveF^z_$rR^QwE0+k%i@~bg z>-eKqy-LYGme%a;&!%DB`O=MNd1K>?e#OY0qX(~l zGAUhvq?(P^kX}JwdnTX99X|RSL-cWd(jds62SvH}7Cv`!+9n3S%$-K6qg)W9&0MbV zfqH@2^9AS1T759EQI=N$vKci;j{M%n-T{l7o`3_-Q7343e+UkEOHGR26gCA1Aa!wC zkWw2a7suNz0Do0I%j=|eRv*d71om8nTC+#TE^-QeAJ8Nyt^#r#O;Svo$CqB%hL;yH zhG#Jg6MYb8aIo%l+9_aTrG1S-L-g{}irtX)(pT)%_t~P@wwbqAHIn%(PxWQwcEGNW?&$oP`=FfF zpSHLvtE>po(e|HuIGz%1AVD^tS)|pW*mAAIYVQHdr`uT(_~q^CcFU`L_(zQYu_wSL zW~;|L{lp;r?KS}hh?3{a^SktX)iSxxNbf|zrBp4tdHC+s^BmuqOKj4IbcX&b~w7QsgN57EZ_ z9xor@poN=?r9vP0mob!Zp*=#~33dEHu&gd@shsSSx{!(xLxRANFci~=L>5br$9@zj zz&Ca3bN&&QCDoXhKY7SfEkDSz80~e$T}kH;^SL5G*4Ew8do^QqHm*!L(4%GQ&~FsR zIQ*AwzmQ`os?OJEt8s=~LI6uLSdI;7>Mwnf2nq(lt6({F52f6RAN`D4L8{Z>s(Dz0 z>R`(#PQW-GArh`;tG_%b6Ug2phQrj+)M3Jx3U4=u>+*loW-2G4e z#8>!)X!{w^YAhdR;sym|&5Ld>z*YmjSH@n^D&=E^OY-(*&uRRpGI4k@7vqIMEG+L{ z#f>V|%zbVd%im-TIm4XLL;c z=$CMXSoQV1C5svIs;a77hozQ2d@7eYpG;Kh8V)Z=>nT#n4TQgO08aXI!}3uO_jZbr zqbJ!8lp+zFytDS=J$}jh_s}i;iLvIcCGPI)7Mp9#;bfAH1@X7t`=&)xdE-6R*7+t} zFWowc6HP;-!9F?IIG+uE>2KshmoF;5n3fQ#Hx~NAQTLgjdh+ zeUKTF@;-{M&gQ-Px5fqx+xOP z2fW3+g2B(zF#?GRXZ?g%P=QVC?-&5ib_z5JG86-+5%m^c-r!&ZY^SkiWBb8iFWF#^ z#6dvmsACA0sK*#JhOD{_R_rW%g3o3*G%A^!g5`;zb#*Zn}_nP3jjsAJv%nm&QGo^KDWQzW+@ ztN>*ZF#}pbLXxB_O-g!m-v9^IcmGU(NMuiGkIm2wW=XjG!z*G8hwVf6N=O||E0w8| z6eLaIL}s(FPL2<4@um1;P${q#pX6bqKjy0xz|(v427|3U%H2U&%IDMAvX;Xh%6(#EU=3(?3s zApV|oYDLkvgNeSzb&>y#E%sMy3jPlfP~n;&qT6!a7(t^{Rau2IS!s>tRq5H0r+ zM@nmY059{FG7JecnNP$2K9O&+pm=Jut@`|%P`i&0?PD~hy%l|m9gus#k~WbcQKjr8 z{6Ifb(a9oPW45LOFF-d_K_UahvMvVPm}pEp68a_){SYitw!v!b*B%*z9NUzr2IU)n zJQnqjXtO)X6q5c82ODiUFzIU z{iSZWhTOe!nEH#`fu0p!RZ;eWJrRv~eT=lGIbMhTJh$TdQ_Xl!Pegk{asLpEE{GD= zhnD*0UyH>EJ@3&Rc|GqPaqDS^5#90xrA1Tz1B;pFLegU4A&)_Jx5z+d>kQ<#mxCUI zb}=bS5gM7Hf_n-_gN``2d1i=sJ?6_rVHdW6v;yXk=^`)mDp-iW`+}sY$ITBg)^7xe z{V7_(54R4=hZoJ?4f;mFkT;=258OzIA4%C%CSbDDuifN+n@wu_8K1&^p%mKks_nJ( zSPO6muJMd;8pZ^+b)T4pX%IuKK45JLAn;-3p@INjT+@55S}`ho^@iP@W+bk9{KJlu z$RLyTvNqi?#oO%)q}lh8t%`0iSAg98AAho#~i%g{rgS!Vv8IxbUSy5*Rtt)D_D-ykr4kljNKkt2>Gj%#<9$MLO`<3LlBGQyGinAGuMYh1-G z<*sEoHkx)^m8EuEx>aE)@@(DhT79^KI(%Kz?-xtxG~2$MT=*M^O=Z_Z$GW8 zJyzzstZQ|7@P;@n`O&+NC=N*m2#Ivfq&FPc;gz}|v~FB=DUX_`uRW6FMs+RY{>0rq zX&XqN`NESTX6Xy&3w(+HD8=b)SAy-5obUK7_CYzQP5s}kwZX@Lcp(nvr1W_oa%@;8 z2Rs8KwMRxXjbqiPxPL5Y6Fm_d29~tRW%V3#^5$-elVy(E(q5X?2puVUD+qmMP+KD$ zrAjK?8C!GT`e?&)e*D{`x$5^}7CIF>Cyu+)TEecMizE4-wYmNKs#E3A$1jSu8N6$) zJ|6-lh!H<}Jpa-Y&_tH9K67u)nMXLl&l#9n;18Mu?C`79EmzKLP0EJd$@g= z#liJv7EKEUym)8`nsZCCVhEq|WBosY!4UHQqw2c@sowwpkxJR2V^gl3bBsgCDyx#c z*TF?r5{EL5l@Q0sitMaoi)_azd+(8v9FBGDz5Bh7?!Dj7@6Qw+@7L@7oR8seJA#UT z5N+2exWiSbBR!^A|2|$bEtmXE@9pc#5C(4GH+6k!lTj)o*Onc2MqeO&#^m5_3c&}( zCSej=C#`tBQ!yT|O8Sp4;wnOQu1N7%7e@PQrXl+`6p}!xStyc7JX7-vHl)p!!u<8{V5M z7Po7cX#Dq|3Za|*OHXTyFczvB1G~$2@2Sl8TaA5?E_zJDc0o6i2Pw5}mk5PPX4JT2 z*7B*T+G<-uW4nDZQJwUJFme5*Rr)_+eZq=(g6-9>xQA?r)%%2O^$L+e-chr}WYcX2 zB!WJBt(lMKS?j!u)dpz^4{C%rV!pABoX+Nu81(}Yt%f80T-EeJ#ir%@EjHdA_cWb& z&@RMmvU&b7h@9w$Q}M7S8w2roZ8L^YOIi|G$LRB;<)N;ovEf7dMH*diGR|n3iq`X_fM6PIDen(m)z+TW-0o}p=c5o49*}$T zMvLrl_|P!p3Sn=Rn`O4W3-^=!a9kBaKAq}qJ~%O}iNDSm8ijXaO0VHF9)1F)CogxJ zS!;svDT`3wHTR_>CwkJ&7^=x0a#Q8J$a?9X3Xa?o>+xaoIc{|V(^ zNWN%(!~M-3)VhSBQr}9F zS{R@4iFf*Q7Xj=;n|DTD)NI1|2sp~BC=6E&BnIz~o43|ImReI;=7|}OLtb_n7#$8< z-|KEtSE9ejpRG*`?FQDK=EKwj(${yL<1ACG$m70Ad1eJR3otJEgb3*+3t|lISbi8^BV~#!M%x=kcS!of+2K4li zJYyj|fv*kgO05@$7DT`K#HHUxRmLkjEBFV3fd^Jt0>#_CC%o6DGk8m2oRFGPK zZNylsXm93Ib&vL=SoLOIJ1yz&ueqk-ebca{Zxbk9mvEng%W<2)v~8ByGi<&&H5`j`vpVYop50bGdd9V=Kx93 zYg#g#RCQ&D`l(#qq=DBe$*bHJxZs$zQbe2kihA}sF45uLdOmG32rCq4Gv|3Z+tqvM zcTOIx%COhj1R2uZNi#AU3re;~vuc@Q@VBP=@YyVD>vc1G(?sjzN7raFScjW`ZE69H zRl;?m8n04!v)dl7N$`Y05bF(6BCQ{s&~Z?+yz#=bC?^lXLj) zq}T$DvmtO;etwIO7F%_b7fYt4C7pLdxtToz_4DM->~!Dks2^9BNqV;mWQckdS25f>}UF?~~XhmIP@M$XqM z%O#1q+=eh(NAWYk);WXw@g`DdTTf<)G`cX z;)|r1kmux>4(9MZM+hB%9S_H`<31l&d9!ay0ZDrK3Sn&SPwM&}pCg zancxvoa|gWIA8ujUQ4_CN}liN(#C`YN7Kak(PmxBUez7M?E0$<;7rLP2|)9MC?y~H zUd!exbgjae+d(k>nH+}`N4otq^DqTr+sCleyI5wc`(#9GDS|IPL;)*ubR^p~-*_Mm z%N>$@scbnd=R$zP$1^t4!dNm zoeD)=0Bv~w!?uFhxpLq$^w9WM2j8X)ml9C7wnGmXX2rMhZzjOX@`bbNZjPFuc_>Fg z`EPDS3*6MtHq^gA_TzyB@!4G8^rQm&FxGhO}jMrsoe zslEjF+~MXn`-@{HHhEEfkatwNh`;}uzG8M7(uoi00hABJ+A`e(?&4z)jbf`!5PW7i zCMY2ydEjvk9<6+R5gy5`+Wyfsf1#zlS_Zd{4sXi2OGSP<@uu|^LbKZLNbpc0Q|4$n zq)G%}zEF{$?Pa71+1_U+qCZxYmkkUq@K?Ea3 zLm{5=%M5ia4c&6m4<~?^F7AQH*WIfaJyJDI4Gvj7KWbLh(w(=GFwK?U%MMZom7y7fOUv@*|yJ z_7I8>y6+tRcd>cI`)l6l*f0y%wJ*HM9^PsJ|CrqJ! z6u~BY-ykX|61I6mOUZVWGxEE6I^yGC%H5#i-RB(X7eBAxbe4I#lxN-SJu>XUNO*^e z{I~_4$K+8&{`4zvtPoT9@!fu;^7q$D=}thebP{crl_Uv1ZF>;kxcGKtpdT(rAGvdGP=7ZBExMh=Q zSGY<<%DwNcN5Y?XL1_B2A9ATj-rn?ZPoDQ0c2NA?VvwnVI@YxG9o;fmj06k0zyRcX zOT5;iy4CiMXj(HmJM1cqsw;s~W`9?ysfdY&sE`fO3-K9@1Wo_kBs+~Yu(Vd7Y7~UN z5HLvuk~>b0JyisbwsMP=Fh1q)pT)U{g1@Q|yIk`>drRsS$7MzWI7zSfCed%%6^00d zt?5bBS&fG6f@k)q-yhO%4G!1R-!9SPXSM^Y#x*Peb>kXImKW_R|EB})oA*CYcBenE zMx7uJ25&EdOp3auMW`v6sIgin1%J=|(JW zO(R&AR>ihWM|+Q<%Iu=6NQ)D)In_6NOSS{?!14*mH9^2)_&rQRc~$7aT8w#kjDH$W zkZ&p|$eRM(OX3OfXLVVbb2!iUa}-^dq;rdgsBDj(O>y4LJ{mzIIcSWVl@!# z7TndOVZ)WX82))x7VF}gCdbr*)a%spVAS)>HHlTM!Kj8e#ZsHY>@oAxwELWq!dvVw zs#uBN)h|<33{2B6sdRo6Hxx4AEQoI3gMVvZPmm>A-IcG3RgILgMn;xD zjZo7yNl8{=eZcYMCedoz!eI8d8#rYXd5k27M;1oZhYptQ2!GtYJ69t(_E}=n(WX78JD2 zP8}J6-xW4JwRU7_G1wVBO?5K}l(8H{`O5%+*boLapwg6{NJQhf>`?jsoHZ}z1gBe7khl&cw!EazB z18J`^R*d@=Zd=TZ$EfI`oq|x83UU({=$2bvg?XW8J!zTV|0;CKvS0kCDGj?~A&D zlim`XlQo78CxkjAkF~~*tx(-pYOjuY$Yu}{>8eZCKCxlFnzhv3Ey1w&ts8&;Eg#AS zc@O?lhrDmsYFEv@zN=)vtn6Os@SRJFrpj4_YM|z=^fTjB=uVRVM`JtUMWi1Y(deTC z#S=+$Hd%&3mye&LP5Mxq_NuaPy6GT>+Mc`aqrPWzG$qR9!F!GuXmD_n^_pC4#K2JW9@>7Jy zsP%BXQGSMAz}%Qm7oso(n{^JkFXR={{uwKq2T&~okBaGCe3Csd;$Wm2c&--9_r4@U z)%CG)w!YGzf0cfP7^?2}LD%|r?bd3*$*rPW2jLMKW><(|>GyC@84Nb!NHMm2XfUxM zmvtmbMm1#{qsXbBO3%`y2lQ5e>CfxB6Dk)Uo4fWk#Fo(jcJ z081kPXTNUh0`2-WEO`2f$4M`v0ywt`D6v+6+=1#^rWJihgZ+%!k~GhLrW&I4!)`hm zp`x1-EoblDl7kf!lDJ5l7W)1(dmCagASo^pTxnB^4dGUkf@0|?D@2Hr!sJYrv}B4c z%(K3@Yj!GXOIRdh6iebLPW5*QvneY22O9i@M_=2LA^U+ma!eJBJa%IMiyX4yrzUb& zh!f1tC4#QW8R7@K_0+kOJa%7Ww6suVCe=@!HiuaL0Rqqx_U7fGyT4E@O5`IOOqk0Q zx0fdzxrkOp_M&uCPNH@~b)+OG+uz%tQT>D}PYJbrUKid_S>eKz*`1qrtz$C`AbA|4 z2tjP6YSBgyUEXK5w(mV6ukk&zIoo$Mv16CYo8eOUDSkP_N;u{uC0mEIl!+EfzxZ}{ z#=)*W(uRLKD%y3;f%#dvMYHH+We9o6da=0=@h*aQ+=T&`V1uYtlX?K zv|sCQz6RDMVSXp(Pb(k~!KwN0BX<%imsTacI0VIsoQs}}bGBx?=n8YITXzLOh14vt z){|^RJr3HhSMQfUJkN7T?DmSf*9Pc`Aj>)+p~~aCRaOekiAs-@L%#C@$7pbk zPyjJdy4_vgZU_7kKPf#SX%WrCjkTz>#74$P*fG!HAC)SIP}(x}E39z&>P?-p;?%Dv z5;~Afj7qxB$6+?o$&ubRZ%Yrx;`V$J!RyO`UwI%NQB&kd0yrsTOxsd+mY@8T*<|ty ziEZPJBFua$CpNgK2p_bXNY;>YQdRHAe<;mKCa#vTA${(72``$BIsmX>PAXyI7J90F z)*RC7u#=@cu46m$L5523^yqQrllnpta^mtTKPA}tW0%%DAazopA!~{S%yQ!jjMHd? zi;o_|cP_(@pdK-_KeEEddsJoK@R+78L~IVU0uGI|zBgn_JIhtVXSx6sK@|Ru4OvA2 z?apv{zYzTxr>@~K`uHFmCX2~UZ?3cdpD|KcQx1DmVxHSqoEYv^a%L@6Z|fan+Q^6* z7ZMUP!7)-Q9+zAzdCO-Wd@+*cZmaE{53kXc_^?_FsUc#7J5?Y_kGW;Efd9_i9?&(m z`UT>J2hMDf2%?s42r~y~w?v&olmMn(f_^y)3EjzyzcZsDXx&J zpBAHtbFFO?^SSc@ecT>~A7LA(ZW%Dck2hxORMbW%JQ;i9hvMn;1n56zC$QOSQNe^q` zZvf^Mn!)sJ$51disuq|7{09L`0`Pr=>o;m&)=;EtnD*o#7>kGH->j;wXQy~-5-L)} zy`XUHmFBNNz6Q^am}PzrCwryfVR*F!gz$_~n5+&q%R%59qB~6nQT}X@_uKE{Ssuub zdKF^@-?VrrEBSgCrNLVV*H>BnryFj3THv*Q`DDJXBo2Q0{7jI32OAlaY5IXt_Xj`S zZCK;jdhllBUO~SJGcY1uS}82wIMAdVRdt)I0SH3}k2=P%%lyg5uGdU-D2u!`QfhEk z$Q}6EPm%$f^7h9bbI)$NuPFL#PBRM=o!?CteGD~y^I_h_B|AzDn7cczr_|x!CV;E= zmf(~sd6~~cVjW6t?rnx7G#^#^{FHSrF2Xhpd6wb6A~qNkkvo)Au8J5b!VJqlkAt!8 zlskF{TF#vEaj3b-$ByeiNG|#F)9`~TF!6-Gt<$&Gv-}8YBpQr_1EUQO*Ws;vH!V-B z>V@ZL0^mu8eK8d!+5;UNd47^NkczF|BTJutx0@>M^;ehHPyzE`av>i@4N#}uBs?i1 zJG-m+^1c+{XIP)U)LI#KQESi~Bteo(vMmf}v|{2mvdFbA|HjnQG`_z0HB{hD&${DeklTuX z$|YVZou4?N8=B^35k?Iua4QWbOYQdd2y^+s{|w#Gay4@4YPhL1k<MEd&Xijw*@2iIU8;F?$6w=S;Re(^CHw- z=cEl*p{6J`YdamNGL_ai#oe)t9ko#`wGC91n`%QeZ zDV2|rxNJNzV_2fXZl+>!Y0RrNn|7u+r7Jiz@|3DmfsXsU7&m_gARR1OQKHuv$3dOh zF-Z9w_Z9hnrjD1B%K;G4)JfOGF))t%a-WD9>YU-#E(vthK(RYODB~!_jICcR!`cT4 zC-TPk3T4TNj4Ty~-GYs5!#h_Leh7-b<^^88e}^&4w+WdDUL#TPKg243as@w99I{B)5gG;quo_M?88wwhH%dp&X^ZCD>4>Jsf^DRaV$Y zPnens>v?!Ase<@XBYxds&z}I2kQv1W>03IAuAyAZNkok`RVxAEP>(V9&z|Xb+;4YI6SzaHY%=@(Q z8|n!nO(a!-O5Y}&QcltX%gu;|e}5A*nIXzys)AvZPe#N#n6#EBy-S$_%F2I<`` zNpWi8UUS%}sn0RC6)W4@sPu(_ zC9*))Jw?;u;EqAmNWay6b5wR%S{P>_j2R`jo zU1+@eNN+1j*mMN^6g1CKvJK)DzZKM#m;88cFd%1NOLBOQdH|mJyUJNg9-Ydq1KX*6 zl~lHv%M^Zgw`57ET`aO*b$bODVY)n|ToWEb? z?b4?YiH~piK7HqQ^aQ9pT z2)w2>nFux}D+>PO%cxWwg^Bd>YoBU0|x)k5NEdx|wtI59&@x4G$ETsJ~v8?Et8Wv7S>?IoO7DW%bNLKL2H$@9e!mS~CEBq;TBE);o+=Coh`iV;7mppF_{>7YT zIpq>Oc0CEt=aNu4_g7Y~*N#GJYozit4*?x4nUde;7Tz1QJJ+0QV$_45qP zUwr-0`pDdy{ca^0A|^~^EBurFim=mqo)}i==qg?ZK#Cf7!@0Uv^zlmlMd9g_TEw6A z=w&k1;m}AGX)Cvw@Akx(JxC@1x!DT&y#tA7QUfbC;6Kw}+Bde$tP%ZZ0s258Z@|TWm(!s7HV(brGif`)em>d0M83cF$#N zKNxYYQ?5+33ZyND(uLWhI|LNI?N(MRj%C>mMGu88VlwXS2$8!aTV-f|+<6!L+7m1l zSs5XgmnN1?9bAbO9UMot#)5|%xUhyO38Fpf`h{@X8dF&k+XhFK;p3XLNeO-YqU}8a zymHLfwof@FR^L6;@l8s2lL7UwUQa3cOXcaTyn<3&+x9Kj_2c|Z9dg_LxI*C+{qe@ zC*DgBW_cTb244U__csR5hu8+h0?T`nB+XyB1AkWoy_o0sbpnhALW89i?zG0F?qy*B zF)_$Xc+wQ@8}vL267Pd4M!>9c-9`Gpm`TX-;h|i7VMZ-N74n=@xujR|Mb2a7En;z3 z1$h~8`gX@wsCu+2U$s5ci0tU^p^KWwGQ3~x`A8#NfVHr^naiM}a2)mi-L!6-A4djy zYo|!}*7n*DAa4%QEsikfvZdnp2cGV!AczA#SGat_ttE1#O9=G^Mq_iWG#fqa#(?l( zzCrROM_5)FVwmN2HZA0G8rEb&WuA80*ik3iwC?8rPb2qhrVF`inOpA>?dWY0?-P;?sEXJ4M!N^I;QDg*RrGVc6pzP z0?>F8pY@CoQEa9rUhlG-oymmkM7(E?WZqU}13xar+nLY;qz9vIJZIu)SN8YrPC)?k zO}FJ{!R^r!AGhO}M!dYW9n^rUVKI_Y@6q@FgS`t9ty2vsX4!f^VQt1t#hI_`QvE{s zIU~c4wem^Vb-P|oZcrKQOM%)G+;iJ(p3((Rzw{t(iiD#YeqDz&-mFXpFz%$`qmQ*y zZ?ogNTj5`ia$+TQSK3bWR20V>h!LF=vRnZ$9aGtAKXpAJYB5 zC$fiZ5)&(ve;|eu*F2(8mnwxW&vX0X#Y^e%_9MvaQojfl;;z&4B&iN*IAi`DdgXk0AxA3%e3YNguYfon-twPGk{mpdP!c!6Z4MY4ZdgXz9tuc zZ2=^%c?{l=VK@!YQf+Lfq&+%9{jT-Bq&a{b!zwG%Z^nwp` zYq&h3Dnno+D;Yicl1mG2SnOWi2RFm#!|252B6-x4Pp0M@Qh$Caa~R1hi9W^*r1M~M zm^d&IG&gHcO_MGKS89hw>|tbHRU6f20i@@PFV9$pN?F^b5M-GJTf)|=qTyjD3f5cW z<0kaZP5}((?0r{OK^lvcz2rl8oG`Z~RyjZ(oo$coNS98R-Kx^l!spq@3X;E2sV#@7 z+nFZ2!X+gbPCOR}T$6d3=QCuaJaVTyv{UhAaZ)hBE&)7o@rt`gD{t~puY9)vpd4WS zf<&RYzT+$R)8@4REmS?b*K2gct%TKZ&Sy+I<9JTw=-btwXW>_YPXM zw_9H=7-rJG)E=95`=+pztUH+xZ@c@HO;OqNru@G$DgU4uXV{y5+6(-?nWdV-(<^~z zX9i2JS!9>a0J5lt$pDi88}vb(;~ajJ#iCdTCL8l;&+v<&TaNbywdemV3LfYNj`6~Kzay*pey-i81p z0UCdIA#PZ#3@>ctyu2S(>hyC>V9Vq=+>QxSK^FnBJna2;3y*17b#R1R{Tx6Dpp5tM!F~fe( z7g-%?OU=nE%3Wu&u`1(9(~_K@#yE=%HR_!2q}!B*3R!qISdg{vsxcOaGJ%AP!~g4R zfXzpMT+N$?=gs9ij@ze=yD@qKN2BK#voa^lk`&T=13O6XGblv%cQ@fGo>J5&`}<5Q zt>kDOsQb@o>s;~h+AcxH1@5H(M9JrLv+?)X(iG)`wJ*Zz6qF0S>L=0MC(H{+ zP8axf(zD$?lx+Rk&@X)H(qM3vs|(YZi?M5=mMWc)r7a z#KKo?IrPg!)+e>@+~ywtI5y`{|X zx|e#uBtFx**x0&(gRQ7??biu`%x(=GPVy&-q8$L4m>rxFEIxsK}7iwKc{f{z_ zTxgS%)Zvp>4<|1(J{bMtxGkl`!iSxW>)18ayV`cf0(dZqAmfO>Pw zlVVq)cAZL|XOlYI+zuJTP!>}ZATE2VtWu*fQ$eCqcqmb#33*09knS}Ls-fo2?1-ZUfVh@%>TUktZs=z=>?Jp4k z1hu7mn+v^|x?RyG`jq$an{U^Wt|6{&BA zC0D1X7X$seI*Vvp?ttiKD`Bd*MhfW)RmV_Tnf=vqXSFdzFRAjLN04|!?oeE&R2&BY zqti#|TR&jq__9LM;$gvX&t*QwnU8J0RXf6cPYYE;k}ttMYR8zvoZJ-x-5Ijui;+pZ7A0A?H1&l&4-J3c`(L^<7{W$xw2aqUxkTkfB7lNz+4mn zFW^s4f15Jo-msy9d`sG)d0AZ0KmhC;a}|9amOW8!_jrotA*X>P#*DUN_fBI4x{}-% z?M>p9e5`|5E17VRfgKgD>`9z$hh>bR^&R(PDSy!p^~*)%u>M660<=vz|A z!vzmb5;dH=#p4IB&}2@lxeka1&%TUu6|`&8qN6m&mm4G_;cSy7&M{2NLNC`0?0gu! zq^HKz`zb?_z#tKJc@Owiv!e7*{w&70iLH9#Go$VL#Za%+vjuvd6dEI?PHj^`HcSlH z?=sb8_8lEL7?v|+$}tdk$qDxLTRO$#0IaVcfn`3V`O#vUWC6$<}J*X1Y0> zh)t-H`YVFwYmV$H1}a2>5S>-_?*vkl$VX116M3VXGxI}7IV)`W{exZKs=&3)E>{cC(I zZ=D)6$B|!M>0vyJ*HJtvI(f0ytSj94`Sq}poI|^g@k2Hp*=M|t_IG2{)Eb|dq^JRe z+7(*vr!udH75s#5AG%F;W0#@E;xk4L^$zm!jsmBzee!NbYNj6?b}!!7kvu6`W$9iZ zW4DxJAX#Mw?c-@WUtq)&%6s+k12EyC{0E`A!$D>M8fxaBm$_++;jgXgI)#6S zUE$ONpDUf;veLO7MaE+O7RQF5$TLc%c_A#Cn0SvG*zHBx%!ht5gA0F~w6QPa0i%HM z*487^{%*Bx4O61~4+ABxb5H)&E9qL^_R!hhGPYScZMDVIfg(@|bymRR0p9Us?W*E~ z2%-=$%tLG(g;AVXo8o!R_V?N+bq17yywIRl<;lO)arFN0{Z8o4AA8?mVC9BwYbqmp z`q9)0w}=Xrjbd_iHmz~_+s{tK+v3dlw5#)cMv#bM#K$MYTuwV|NJ|LSZ;4Dk^pkUV zN`H_K?RX7es&^44s1R~$e}>18{k;OBC+cFSdg_}Pnpu1_Jx*i|kT$-Fx1ok)#sMyj z|00QC+bQc(&!@t#1K|3d7e9?0E>Xw8TS^aX66ze!R=lEEawNdO`v0H62-L3yB@Ldq zfx>`divF7@*SM3G^APaW)!eT9VNH{`;)R@1k`|O|FM*BRSO9e zP{3Bf>Cv1^tO~MWoHL?Ifanm%0NYjeV$S|D5Bdqw|bGSx?t{)Y~;<)i&bw?zgZ@U{707?~?{pn)0|NoC$= z1zs5z3RYA%6uQ??lxB|Gubz+Lvxzx#2U;-e`QR||@b9Zz00dYFqvFXKC!^;$>E zPW5%VY#2cDGv!pVd&ceUy7gUh)sH{bhBvH2R)d+=(6vhh00H(^$;;7`!lkdw0TYH{ z==#(0N_@R;E<*htr>{+1exEx0G_KnB&Zf|VF>;A*UQU@fUNp~3&7Z2foh!Uf>#~NV z*dM6SV7@|V>Cfm^;lGk3v+*4Rq<->|*{D7d%9i%=1KTSyJ^~v3hC%*P@FNKseQ6bK z;N1j>S-btUi|p%;<&+^7+v(ggL&;Ly+KJGJFPDp(CXz53f5#Vz|F@q5$lJfZi?e3@ zExt2E9qzm(j#<-FmJ<&xPMvR%Sk{sQ5%XF*RXz@O-`g^Ihg{W?YEyx99TdfJ0{4>M z|22sY!?9hsePYsBC13s-t$IV=y`YdyI0g6b8i}zRu+%w2Wrevqm%@ienTZRXB1OAW zmaXj#gMV$7uK1Cwxjx&!Vhlrf*{MfmdbTa!ibPs!BA#ZZVqD5hm_1!W)p@t`!d|J} zc8haj$?l~l4g4IXtSqc)p%dTuk78Il8?nI^gIR^SiV{El zEF#_BqgSC^PuV)W1wK@+^7{)@tnJ(r;UY5uT}D6P&cepgMd!pish%O~^Eiy&psta{ z-j9x zJKcLk#o%lSHv&zK@UNS_8K4>LYA;?d`UMzh)X56c|6+ znOzWl=TGo@Mo8HEDFavdbxgpv3vyF?6PjXWsy!Nr_;Uxn?`UlJw+l)fj>;`L{7KMB zfqr7R+G_3Uh__VRS$RV-ea}Cyj8y_e4q6$QEvk$j!a5nOZMMPTgneQ>sx$};6fEoD zcCl4>y_If&S)G=-V$@Em0o#%D*v^cX*$&Lx`K+|{8qiesxDTlK?qb|#l=S4&m8Pa7 z+Z6Wp&gml%Q|SAqX~&P?H+G+;AiaoEGtmd+H|4WcX7vyeIb{iAJ8t`lmdT5Iv11jw zwaZQVgEAN%q_gei-oQK}1Xq6acphQh0=zm7c2JD`?PjCQs3}!S!W5dL8x|6M@51;G zoFv)WDhzP-dnhHluYD>f_UvUsl5M&#Ux)&o@aNt!rfUIW29c$ZS~Vr}9uiwt8Wy;z z?){={Q|+C|QwuD_gDG)1ybPdrT&gC1ks%gObKAMvbpL!`#i#-^Znj@0a4vrQ;s#Al z2*s1cWzoSE;gZkYxUvn?g*XRa8bcbEQ3y&U86OAcANiFsN>?D~V#l#seEorO#~p}%WhufgODje&7_LX+013%Y@&Ten9=G!6RilD{#P1t?XukNkvU zjfX#swu*-6vkYRffEsyJrf`;nBnvTgwzHbDtD7HYLTxk7OjsnLs?}23eiV`_-pUa# z^e6^#UsoAdo$7xg>fR#W`FW=$K!kQWYCfw#>T?)rph`CRZw@jP*_D1~ZXqN9n^QgdRp#Dh3hlm{lta1)vURE$#BCGyP3Tg&w?My^((q2Xaz$Q9TIp%4O;M@ zFCTpy57{~f1CT9sn9OFt+nEo*n4!u|s0MvlQNmpI6WmP#Rbp3bcsimmrVxClRj)W_ zl9+t27Zsuw16AFz%g8KK7=}h8Z*3}4(6(XSWd;jgYwERTswCUS!A5xF&f7^`!~70} zOw^iDw4WA)BJ6nzr6h|JzZ%~n97BS~$@U5N1H|U?p(x!=Q;q(x6eSQMEdAG;zPCPzWIen$Z=nOB&% zk+7dQ7G8OO)HEKJBw0DCC|jH37-px{;K(|pZkOnk^kv6BhHF(XI}+|05m}aC#;4w_ zseg7iMR%UpW>iW~bykrjabZxwM$@<>(@B{{@|ehcmd@=$$=s0GCB{Lm^2LQ(6(dn1 ziDIWJ+E%};c;^W&`&AZZ3g-R+>^W+9+*%zQS&Q7uCIuj(0FOh0p(MUJO6z`oZn&gf zqFdEBY$Ow5Z~RC!as`}xV!jr?T8Fd}f*reRqEvh2`zy)AAq+`bOjn9W3?WZAu-3>) z!646Citan<5)UD9>6lOYYdPTr{Z+lBkJ`lN6P-&+OFa0=GsoNPQtZpBhE8HL9lMy5 zieFh@m@K^0hUUtB42t@};9pE--ZQ6A*Y#%RxaB-_v6!nYSB1&N?ypGbP8SEa18spm z)xTN*jP27FRS=>=LubdQ%`J*yz|=HalDPa@yW4d`@TH(2K+{`Z6CN+QDjZzu^z1Mq zO!2@^vdV&<$LFPkd)j%s2hcd+@m46B6t!Mce98h!tUCjpfVM8BSQVft;t!@8@HZ$& zO4aoQ^uC}`|AgFg-R2E#aJTJ|NAuKT6txZWUiGFif9%ka&F&xM2>=oXk}BP)6TW>F z!ibFyX{%PW76-^G)c)eAoeT#$P`f+}TYIFt=Fqpu-Ww>se77%r4Qj;|@E4A-9GUVM zA02E+6@Y~b)2?TqZ!LmR;Z>}GADE>EN7O1kR)^<(ckUaIl0->pudgM&WsvcnOX81T zx=k)@5R<7tO+UVB;&l&TCjqpM0vuvI&P=jji?UMzUZy)A^jMvNZwYVhGpoVau4DON)a_q|m4Ehu zt|0`cI*pFYw>0rEGC2ogS344x#QwPZ3*ADdZzIJzKYF8By#nZfMrID zx{UV)0i^>|@pc&=ne09F&Jq307d4FHaX>(SB)4m#kE5K|w$#zFG_bxKubu3wm`CYG zqbseQh}_H<$e(ApOG@c>6Jh9i}zV&+Y20NuXcsK z)K*I=UUXfwmD-Iu;xmd*oA-(ETRccp$*Z$us8jZ+Nt>^Bt;o>xFjRZf^&g zC7&X~#70`sVfN{ImD#5!u^}MEJ3D-$T`kBqsDxq6QF*e73~-_djz%7s_2^*r>voA` z4=Hl>y#A4jAI?hY0z|?=1t&)74dwUuMTB>GuZLZ&e9u$hN=4Kv*n(6yVFNuj|F)QI~k^~S@ZBy5v&R7oinXAxwnG`${AMmWR zmc<=B9E+BHODXT_8lb;NDK$J*cNt9J!roA3u&Z^E3Le2(|GHZ<_GT|Or@&JJj{yIu zEC*MTbOMGjym{clY8}cNE6!m|a`&hv67f6ZYFT{wW$(OmGn9pjYC zq;XfyY@GBSY+@NpKJwCsM6lg1l_uPaztcshOep4N35jxRnPim);X?V^-KIEOV@57DA{dJ+3}xJ)>^ z8$0M`(&NG=e3e~4Arj;9$0yC-Fm)yVT^lgUW8`EGmGt;4nFT`Q|Ke$@7w_v z9e^JXlY4nDiZ5>IdzUnb+ptFkJrw>a|Dt?ZmCuNb8bJb9#os73^fFEqUmG6On|;bF@YveU-H_#T z=2`xCq2JbSIzhHQ+QNg@{cZV4c5q_XgT&KuqS0BvOY#EMq}(pmipbjK35vd{EWL~G zxc;54@)N0huWBP@PcmisSOJOLc$-MP4C|#2HzrnFjaP7^4v-5p5<3T6bDufmjAv_) zHr_NEU%)%@W}~g-iY>NsLmq1ISPOTI%idG4~fs(0xY+Ec(>Kv%20SUTKanC2+TP=(ByakJ z9syf>%k6RCf?VBSK>E)D;4&o(0)q{jACHS`g%E&rHfyOtMPKFD;n zqPHOLLa({^&()L@AoA?|tIJ3io3#AlcD?GW?{>?TWw-66X=mSQ^>cX+g`CJ&IlsjE z+l8X6qLDX!2X%_SH&lUUggSSSp-tHs%(*{*SR-4E(C#=RSQ57sUo4!ce$ zt;}@ncKQtSlYd8dJ~sqC#VA=-$|2YI?Y_`$9b>Ibx!(B>L+&B>@2^KmNJ{L(>n-{Q{rX$=J9n{^Oirwo8 zcLDVKnhUnpocDC4$(CDUY+?`vtK1%0Lb9E~A3DW9fPT631WO)v@o4PkxN4F8b^m|X zdw^E1`0uk_B;G#MSoXKd%*3I|VePnpV2|Z22k1}^!MSBY>1A-Kcnk31)TFjsFqUkm z2AyoLnh0C-P^m^|TfwsT`tVZ5_-Gr~D?^)Nc9LzzM@}Ndm^F=meFY9<0DQ~f$BQUs zRzpTQ09^5AI=TpH!8YBgPJq{)d^s9a&FdI9(Dl#7x=CC;oxgjp(rvqCbR{e!s#w8+ zc(2Ep+vJWvxY#B31$-yDFz;njem*mnCC}Zp1*v4XKxf8Py{N8xA@u*K`tEot`~QFA z6fGl_O$e3D>^Mk9ge2LcBu>ZP>li62d+(K%bwtNHX5q-@9FCpB!8x)Wd;G4W`@X-Q z-@iQ`r@G$jwVp$3&J~}K1k9OaDSW)oM>K@9ieDSA*4s2jIzTR!e9{=HmRk}I4P3)^ z(=>uYszrs2;am7TjRUBDGdxy69zZKtvQrOGt=p$#L1k3#zpY0;N)l5l;UadJvyE;* zRrywx8BkeDW}z7OeDRiCxXY01k4*j;!*G+bD}{JRO~y%Df`cFs)IwCRXYz6@Jdl4W z&L6CJbCbv?-IV!xv(k3zSfD1vsJbSmc)#~{ZM0e{_>!ltwvf(CnYymK|LzSyPZVhY z%pW(mi@#eZnLRtz(~sDA0rHit1KMkhi@$d}&A0FQ3MUU|yqHbiU08ZS@b=fe)~df}=l~S&fB-70`QqVA zMYO(*YhUPkiF3)G^rhUtc27XjmBH&RdZ*)9m<1%(!JrCe?C?RU8KG&vikzHh(7kvj zrO=vP`r(H{{S_5|A#Rr>9f%4$;w0%B78d`y631=nW z<{{~Ahm8ON>^Pm($}I1(Zu4h=dA(@6J%D7~2=ir{6-70#y%V;R;EVL$mIY+K#y>J_ zlC)B50hfA<$=@`dH{lH%UFZ{h)-lAHB)m_8{dJ2C$nP)a*7ytU*3L!>fHz{It}HG$ zpB&?oC^>g(I}(gG+*?Qj@Lm4PZh(&$mZn62RITuFXDf>!x`H+J)938nCF!9xb%5Os z^K0c?hquOUFJ=Q{b^7uWK=fF1k*mtqS!3pI_wCgIOl@<4fIa9$r?Sio#2rI{OU`{n3 z0`w*}b<8G`r>j(Pw((l-mr9@k7S6vy9B0G}yQ1IMNo+uv01q@e`T_sDQvs+i9gYv| z95Y4pMHzRXEWRH%M<-aNLP?_rfLNzT%DeGa%Z9Ws*CLd%cME(Il1B`^$G;ocZ&|Yr z>$%DuyqI@gj%wHJce_h(TUjkNIQD33l2F7wFmF2W6riWb1s7OL4VuM|eA?yeO7>yD zHXNWU$0Y0m=p>abkdVwX=XAX8JCg)#PqNpuspLOpx5o{JTdXJ zx^^(a;pxxzIP(LhUEj%Bj&@3pwZM%FfzA6R%^yGS&Pz2LH2|<7AqODNF`n#3ofvkV z>Kvbys#PlLAEB@SUdx2NQ|;+yz8^X#93 zwl#|Yi?FWtX}gYevn>_)ty*lEez1H!r|o+i`!+o&dq+2I;KC%Tjwial&`jm5xfWV# z5Ry6~OT-#^uwHaU4ov}kf^+2Ctf6c!>PEcVOn+KQ^6ar3K8rafp7$2lB@~a+h0pp* zUnGjm%nJPYGra3Et#4pfm0(lk)|+)fs5U!3M&X3%y8-2%*!7iQ&O^(Td6Iav^V&Bm zMn6I34luRVHxfX-9ni|h&H&=Hd*G9&K|lY6w{Y4C6w2cNzyNggP{9H;M0&g?f_z2G zNYFeIfCgVz3Y6jT`Bd`V-8t1a-hou;H`_F)Q6!(U6$@=lJ=V&$%1;lBPaS2~h3fe9DZBx=OK4qwzWIsk7t}{AdL1 zT+SYO*Y79dxK7rTxkZ&iKcx7EUS??YgPaR zrZsG3|FRi9gion>)uOCtURmI6Tn;=8BQLXpbs`Sm*GvFn;zmSAqW!z!jvXz?hrbCN zHTitkBqL?jh;nj(h<2(^2h}H*BaS66oqrc2gZE6^-p0hiygHU z3d(@_!HXs>B2sX0bqUR~dV44xs%Heqb6q0bbnshV<}#kx5PVvv2?C zqxaUo(fT{(MExd_VIY+oN1a(mV_DV_C)>IliY@YUl8(r;isLFwK_K$8obX5?)@{MYy^gu2oknF8J3hDdw-{5IJU~m)Q;{A<)K5127*3%=#3^DM>SW5M@crj8+E|{cGY`(|q?>tC zw>2fq)<+NHbQ#qIt^zm)iHOa{m!3x`8DKSmN|-%pGWC1Ya^_Bh&7MAhJgGqfy%f)v zb&WdO&5q2z=-&ZWMXS$bPIS4W9e&N`V}`uR3|O`(vMnZZ z3-j=i%doXB7~+=-24K%fxfiLjji#ol*8mBYQ#dA|R&Q)kZ$}P21I@u+T-T-HY;=2& z@C?ztPt0JnZ*c|qD?IvaLx7y{VZioJ&|+dq@qt50=Hw&3=no=SE?-x_Ngu8B{TJ)? zPZSgs-#%yGJbzW`3g{M$`SA1Q6z=?sZ$cj4pu7Bl`pHLb%jECZ8-#CfK7DkKo|;+R z#r24AuuH@RtRnB#*L&{ZUA1>)PV}p?s%>rwRnT*U% zXJ+!OO60l>1jGWwt4F1kyUKlBOs!FfM}yB!SFf6dyf+4>IFNpEqVZ$RaDX~<@JF6CD@43e7+IWSw}#&2X0^!%v7T@%lqak7qou{9!ucInD>=%q0w-O23c zbm#Se*=g8W9VPxdItJ`9c>;CBx42%6-#rAq;i!EW>C+iw#c6*lX_xzfKL@v-2TPs- z>$Rc|Z-X$hS&6f+qt^M1JbXkaUlvj=jpB3a=d8D%?p(C`N;_CBK2YXAF@)$A|JBwGLDOJ1ytW}TMceY2QX8Z2B=#pId2;k387qFj3c{> z1(?=C4-q`z8%aHsTPoe019|Vhiu9d>sT&5X=+QkBg+=aeeQU2KrN2Q!pe+|Ex5 zP23=?&}U{>puhq2_IZtU)#@4(G{roAiRCp6<74o|9FfCrDn)Tz__7@)zw_x#6TR>6 zOv3wr*K`w@OEP6);uj5d5S|Os*O$Izp25pRH?brqvUgMc7B`d45%z6g`2I~@I_N4C zE@G9z+Y9hY|KnWP-?yX+{qaA4a@srrzJEZ|%si9O{(2~<+KqJp|WCgluMc2`Mm^aT5Dy|2OC@Bi?We$0QVXN+xAu+7BVJOY@ZgY0?}hrOF1ZSFswSdS zLz{~pnbHa*4bSf?e%-S6(=*v3W96{kk4EL|owTKOVKr^oUxhjEUzgwG4O?*2U~nw< z6!O`p=QLI`qCH$dC&^UD`6fKR&*Bj? zq9>pFt9O5x&gM2X42!kuQOlriTmHL4j}_i(P;Nw;jlCH!RHSH-XHE{s*90;-wj~7 zvj5Z_NX@qc5d}YRNfzb{5RIhcR4BX%-n37U_1XO$YEVRZr`~;y#3dVs0p>}6!kRpr zHZz6+caJAqS%XWq^-KRl_9Q_D!t@$%UX2odJhCV9)c;k?vZ0owQ`8KoM?o{}#UK<)d37nKu@^AY;|1I48Ty z6h07>4fk(D3z*4tOgSY_6=h~u8vx8f{LUdpNG4uhO2=dQP~d<=-+jB%r*6KYnLjUb+LdtRiK zokj44bFak(ya@<$q=@SGa`D%I-&y+a@c{BLK8kyK&iDO+PHdCxLb_{l+Rv>5_mieW zoUlQZR~t_;i9%752{0=37cxA}i9ka3#UpRUXQW>8h0{L1(7+}O=Hb3Is3tWYt>{g% zl`AGOZ}v&HD)MVVw57*$I>&hu{kJcZcsTm2zz>T_y}YX66zm?rrYZCRnzqJ(vkL?g zJNy5gV`k76g_KLPd-BLQQBSmcm;QbiX)sCr zz=uxe0Xv=nHTwV`_egen*}=1(AYielLF7@`;2CpK0Cwo9R9&Z5F{wiREM1yh!kU&g zeKDqEyo{-_*)I06V!>nr!>&+_H zJXK?nRj0E2WO}c4dWSWEkHdcaCHm3keZ>*4AIG5$4R(COO4d2bA>wC$pJxeW3ERBi zQ#J(x&B%?pWpghVqUvf>$-tvyzgorNljB)>a&rW-sY%g`sRr0;}2n&};;Z*IE=pAK= zi^oWZPA|*{{$%vo=Yh*%GuIee{R`m9Of!oPn)?RAUmAWd8I6 zen_&%i)so!76!X6zevL)&Wm~JC&#NN;9q3dL+U+O;X=s8PXrSCS&-9n(m4VLJ{aqV z2YdoN!2F#`dI45$D}mIiAR-Fo%X~J|rcK#_p&{|PJk2GSfmLF80E`HXUKWODId*CZ zZ!QRZrac2*_3kg>AbwUGrXo`~UXL2FkjsOSdk*Eo3N6q|na!TNOMs7CR^bWJ;zZvO74VzUKgZ#o_1bVV$&P1#*y3my**m0Z zRPIkKDS^W43ikI$n$6k$*1iz~;7tHct>|D6;A=O5$vVmJH~D!1zJWtjjKp`}mfU*I z|CcIR^3b;=1#Vr|s;uQW4A00+EZN8}Jmy0ygQfclXJXq2m)alb8DUw2a;Qa~%?9gn zByP-@(Q5tXo%Oao>MHb2-rtX1VP9eBL^>j8U`v&a^h|HaHf3^XoplgrRuYn%J{}-+ z4oKNz%RJyY&___Iny;^>^bh0Ctt&^cRox{re2SvOpeJpqbr4bf3H*47Z=9zP zuP4Z3Z!;cMF$0mm#R2g#b~NyLt)QA-b%OkfW#Xm18QkjVcV>0jF~Ile{)H6btq)GS zyLA`Yk^p!!@-)n40Ng`jP-AWs7^xj6cv$F9C@kie`+aVB0eG>I4QT-TPV(WM9GuH=6M*i9uxe|x|6J4Ow8>FyIE3iIa0QdOFp<@?iD z)Mz38!+runFg^K+Voq5P1UyJUX$i2z!(ZjM$9!;Rp*)Ikp0g$pz5^hy(+QJRmgU8JPPni)r)G}wzsPAn!)s?g?%M>A%!;3> zu{l3D;%y<>2*LqlK!YLS-;2||%>j>NltV~e%JzPyO-wu6TM~`*R%L-OU=Tap>)>%$ zrT5<=9^l$w755lp)KXMwt5{3YJIQMb&f?!$%785ymT?hcD&7^f;rn zdZ`2Z>B4JA>UP>^<)Y)P7i2!|8B1G>f|yLSgr(525(aI1$~*VFFb$C;B7_mOh~uw` z+j;A~1bZV#aoQKX-5$LuTBg2~&WC|~b){#jySTDNQ#GL&=c&27OXJ@MW3IVZ+6y?U zvp^AV4%OP*4+sg4>b~j!Y$ae$!2SetZ*o+iEY0cg9+r8}yOe7}+wyvN>X#O^=o5Y! zoR)WCapBRY$$M`L#J4JSaN^8B@BoAO1Aiz_;yeIYxcoh!z zmC~C+NTz4QH1GZS42j8qBMGU`aj8Vo*1kdTl3Pe#4X!i)VNHsG6}a*HLQA>rdC?55 zJjvKie2xk7A+Q^Z&#fAj9{Q&;W)WGNMz~T>jC^3tjF2>&`+wU4U&sQ)Fc*gwkHlI*&E+3qyjSeY; zID<5thknxbxp$0oze^4gloSA8ILY_;>(V#w8uH`@yzh~%#kpceUY~gLED#pSAI${GH*F-|m7DL#Z#_4tB%LU(Y^x3ce}d-Dx#m6e-y^`L|pI zyw*AQkK1a$6uJWR_btNg2mZ`a+W-JTF%0qBqnY`JuIS(}A@zQV2J58CZ9mM5?Y;7R zi4RFV0lM(Ed~a$|RjPP*%*_AE&h+u}G@&&>*t=7+7=Sem;qbKHGyrQ?_B<$W^KRqN zh`x};!ct|9Ul*?BC&_M8;YyC3KOtH8P-*lrww)2^J?$ws%Orcvr`TYRP??m$bGX8H zBxsZYo5x{F)?NzC`_AcmRLx8-napi}M^Ae{8Hp(6T8GOK;_B#IvIdj|ydH|YR<+ci z&Z9kx@k;L~9CE0!CQ`TUX}MYq;a>iEq-r?~Y$BdlEmCc&cw&AZ!0vLj$n{_7+`^T) zt=aQ!Cloneo!x)k7ccq|+64D>=uL8nE0;Nn3R=208~g3P%ZE$VVbPfuwnq$VI*(0F zWpO5X4h#RzbVaU^V|mq=Vx(nhkQJG#Sx?prARah>ERec8#;`x%3bUqpV2Ks)+i(w>pTE0*2-feNqqpyS=;45#ns zu389$OW*7TdU`><_Z8!iJ_!?(G1rI?xdIjMNA-sAm+OW)X>h@^qKjL6BtG)ij0W4Z zvP&d1POhT~*_#$cw(05jnbF!@nn8*==m}^I_T;4ShNHtuxa|BOliK#A@tacnRJsAL znS5PF(E=h`*-8$T%Lbc)yJR?kB+rn~TOB@MDcq%a=Dbp4&)3V z*ysN=#SPiWdq>i~GpgKJNPB(hE!4~uk*U>1%eT8~=KB)sw;YsKd-z7{JO6SRimpLR z8-td{6^E-}(2}zM7%Oc8>gH~0G*gGE%{A>>v=k(-_h%#&(o(S3{@x@QW_MdEQTGMI zvgS*g?F=76(SiSE>9Sd1U|2(Yl0XJYOSqm*^1{|*#q6y>Sb&A|YK6C$c6Yf^Opa3>R=Nl{Qm4Vg z0-Y%9)s0JLk%DG^&i%Ws1zLrEGeg04x78HA&MlzReH;p^blBvToYy}Km$jabL4x$J zXYYMLMz`mtV+2!w*7P8sAAhAKKMWh{7){!RaS^!?eJUKwf~#uTZUH^=EwcUYi!%qB z!f|sG-*9(h?-H!!x;E@XMxW3S|Eo&wTp-8xuwT0+6W3jA%Z}S{ZF1!?3nis(4zi3l zd0~6yy?f!_PhSn?1Gmpx<4?h>uU#Gs`iiPv64FmEGIKasT}+|3hti zug^f|-&IYKR)Qx=IK$PybMoNNHY+)ur5$u~N9hLq9B(DZ`JJNH>WQ#G)YDrOa(CYg zE8n)?Bk*jVKUv@kYAYs*w6pUYfJnBgqy7q;2wxs~F`n`TWVHC!B6I z6VI%QmuwOhA5xhzwHB2+U+P~@sbR$}ugGue`st>0@me{u(uIfae=7C07h;c~i+e25YhZ7Q<7s{D*-0AycEA90n$qbcKOEg?xFJojsLuo^=bO?2QOu|~- z;}PKni8nEKK(aFy_cyq_k%Y%7?pA;VbPsiqkOEzR;*vf?ma(T^bJTXeY&&ihG4|t` z-`LTxX<=15^sNvoW-#W}-w6-!R7!yi_4p6YD6&v}=)YuI<9Pr`KqEc`tLvq1xqcaI z5?JCOrBts$TY+ToK!xiv8=q_XzYqj)``92wznkr?H)ni4GvVA3z3xEwI$BL}#~v(E z9sxBgN*C@l=J~=%-uRU8J-D$5W!F`;lD7{R%K)2!#D%upFVC~OZIl3BEb>#R^3T>R zt98M!jc?C};$g#1uou6O?b0rs?9~PBfXN`<7HiK*9BWVo zeT(0uo~ylwvX20?hcUoa zir(TjqD@45nKBkm3)Bn%ks*hAk;^sf(AeN$4UC)cxJTE*h{^ea=79F1Yps_xtnnJ% zh$$~>5q_r@ozTM@t7Bhn8Z=%RKOqE&YRfEk><$MtgPW4J#kyx&N-J&7*QI4#&#n{G ztx-+GdZW%lWbeEW??d>3+$`RBTfB%IG|gM5bBD0ty5Wvy6Lz$(zDu}>=*o310*|ON zhCuPRURi^I5Ds~!P|vhX8FgO)te>Axt)Jf>#Rk>IT_;d2%`f#u?lNBET-WrakPK`d zu#5jZ>Z;iQ|G?}X?C#i4;M0~>13yq+p1vXO)R41#y-+-}`>UPVl;Mdeg}{SA;lMRI zC0d>vVRwbEC)s^87_~F724N!uIeNaOy)09LI+n7MZmS*r0OdTGeSqaB%4h`k_%m(s zgiXo19@-UEg#CWA5Sq9BgxA`k{?hJ}QK-Q`day-5}kIzd9O{ zG{Zb%EABD}>2qaQm>6@CK=oqj>uIy-7zbl8MOY6{k25IZTxU_{*5|l}!=E^4wtV#X zAWC!jscIib#@{Rg6jb}{byK7qJVkdpA!@5a9jb^)0*!GFXfM-7SJ^y1=y=^NnUuRk zTfm#=F0c|Y*j@`S!}i9y0O8}bc{CA02JNH?i(!mf=Ad3_1e#{P^f%r63g0HpJMy`y zvr=`=nBnt1v~Zt)CTph@5V`22GS+i~CAVcV2`E3N3h)inuk?l-6h3qe7B@Aql&rOX z|6}?f@hBoB%lvvCn*ulCkJ^@!yRRKH|63vS=nzsvj^8uk>pA?dd`1r+BgWY@ukyjd zYBD}T4eD(A(P!z|chVkg^PBz5j^C$#fiEfr`o0l)D{kY;`S4nyT;~kZF?0wgcSKM3 z^Vf8exP_P`qUjT6@|-~`D%$O%)$|ox9icMEd-i`p7r~0@j2+PjL-P72z3W<3psEmP z7X8jv9=%jVlpr#I-n4_ITL-IDsuWl z^Uj!sEg5!i4Pw>r@RPPn*TqI;a582HB>?rwBXW^^p0#muAXvnGNGZ-#_b+o34MBloSX8N`6*kPApWG;v`R+SyokU{Wc4 zEw){Pf2+(U?oZ0OQa|Md~pp4Q(_WG zws~T|T&3zC&9QTS48i?J`$e8E62(meY4ZbM=Um7X*FiGMjYch_~+-$2d9$}_w zm#)ORhZ{(P!!(P%#OzM~bk_zZubXK-;Vzs$cWm2vr-$@~w?C$!5}}WBNz;Zn zkpgWA5HFhNU$>{whWsd54T1tn?RX@-F=AUDVrrG=cvQ_9qS(NB5TEV-Y{YEIHO?Fl zBYY_Xeh8eNJ5=QJFFDCfX{v>W|F&GP;enFUyuAbSdfsih9L%pZlsP|K(lu#GbX%94 z;H#JY3XGBjRRevMX?bQY9PoLE<+zNTwj{nP1=g8S1!}PLYMwbn{BU)7XjDCB2V&yi zz}NfS=2Ql&L~3?;g*^|a^mp!8GPxRx31xDO&CTdqBhpvPHPMfP6J(@LzBeExC4PCd*+hSja0_krSwMcVe{^QO-ekFHD#{B1}Y@>3WG z?<2UHUPh<8GzKo}!R<$a6&);sB76vtj5cbrLp@O zc4QT8a3DdxGnSC}p%Dl=L3>;+^4ig}f=2sHg`Pf->4}3L{?xVYGN9UmXs^Cky6kC{ zwW4}*E`-OXY4_UObG3#9$H5EJQGrgw7wq8gfFyb%3lsbI<^p<_*IRs4Z|3Y)l20dJ zH@lq_Ut=%nyn5Z@?l5lV*A3WYoI2HltkY+m5v36oKYhaz!67x@acYGN3dMq!da~dN z!c3rr+#I7dK?H~lUjWKu$IQ>9NmBGi5qv85Bq07AXgw@XnURc)R@gchQKo+bP=7z< zu4?x+c41g(a?mizyVU1v2C8ksH;$`?mXSUY$CWKle~>Ksai?PYjdO&SyS=rHJqxw1 zKW_XI?_MV!*~M}}qO5Ar-*&36#u(&QW>vPx?cQEt|MejT`6o1QVIw;%9;%k*W>@%F z)qu4C-B;~#INdChHJ3o2yeuM<7&wA0Cdq>djzLtjRTVE({8FD?)O0IBbXFCW9VheE zedwq>82^ze_GscA*$DWo59h3wi$(qN{o%%p%M>iFc1;~O_GH(r--=EkS1Lz`HhzJA z-VlII*}EV8J=Xwk;td&;Cui9vd2FPL zM=k3KjbqV=zlxjOaNlYYrW0MzOi2wFt?a%_XiK~T*r!NfuYBp5Dx>By|HSnhI#N-j zmV|C4x&sz{F>x3K6X%nx&C+Y+@DZlGI{uBDQ1eehoxFbG)nz5&Uc%Qs+h^sg;ga>{ zz)eA^T!%O3A8boA+Vc_wEiLbW{Ezk??q_$(A=&{LVBb($Km>l^{-PW1%1iiZ!o1xeDkbx?v zKpfvusp}o)gv*#n$e$9W#(a(|4ZTRn-EzuTF+;@J0ST+348^2Qw3m0Em}VJdJdZv~ zY@G>-?s~mk{p8+f;j`wuD6jZ>&8mM~;7Nr8H3f{u{CE9g`y2CT-*dGqYY#KrEa+zu zob?5PWp=NhYX#V5?Ux+q`)O&VJ*q;6yneiOGGmD8XPezw{@GYR$ao_|><1Jms_$Q4 zsrghY%I>a=lllF-+JTQ^yq*@Hfqm+{cV_LTfSD0vz5PJ6a-3S$YLrPy6gzeyI z5~>2P%o4WtbmiktQAhRZF4HZ3<$MwGvM&N^BLuJOqSm<($*eL?O!bCZ zWY2D}z|B76V!CAnsw+|)kMnUt#kGP`fWC4@MkY!1=hG~dH3Jctt3wtGALaY*7JzN1wVTi3 zjZ;z9BkgH~ucWeuc)dm5q=}`hFai#w8bQq5bW->qkBgz08ldN>Zy>ow8#lvE`+dAP z0`?L1-urC~x_x5fwpR5m10TJyL2$wcQc4Eo@jWu*ULQR1E@gw7rcDI&gLqcu$pIB3 zVz*?0N{ZXRNT48lMMdEXoQ~A`<@%vlc87*p?$V_tc865^pOVh|tOsgWN+Waf#OBKB zbe7nRw+rUtrt_6OkRiL~u_MC<5%h2)#-hdXRffc2S*#II4#LEkU9X+ajXmedqS_;DKtwmC1y=t7@4Uy^vnqcb8h9!q#3 zr^Wwwavcs>bY04MD<1W+!~Jf8#Lg23*=dwa%odYcq-^Ic$4Wh;L30;97JR`lz$->2 zR}uDw(xS#*C0Uc0TPyCIWRr6Of12#iP)Iww%yH}Bn5dX$bouKpi2L8ZId32{W42c# z!M;FsIZKR&6kDmLgjh})uQbOp+3ezaFRmRuxWDN?8k-OQP%~O`D0Hz zP6`&AgDaX>5U)_aIhqAB_{PD4uzght1p2@$e)>B_APYLmS z(Exo4=8?f6_=o9N(Fx<94-L)Z>l}d}8Bss#Oc|8tfrABvBF$aqWDX=vHg21F_twyo zuaHw5>t3{%osyb(lelts-(*82na`9P`@JSYDqkYOkyO6|FHk85c^SRawxFm`-NW)N zwjB;z;z%sZH#4gJC-9~B>|hLF>tISXGwQ;w2j5T++v~h*5UY^e>SvkLYRC5FhQoMN zAnJ@&?%TN&6;oF+kEI(C!J^Vf*-->N<9XQwJn}Rn2e89?PK&>ls!Wd6wp2b%CUDs6 z&m+C<7?uq<+a5+qmVVPP@tjzAqOhwpV^vcR>D7S++VWfE^zs!%Oub41h&k^S8ysx| z8q}W105HrL;le27xaHXZ#Iro;hZz~|OzYl=yMaAgrd$-^%|&#s=i+~!pjBW+aPg?F zrPc}wds>QnEZGwn>m6~k_ZVsFgvI^QAJ62lTxMMpcH`)$+G$#;*EXX`Z8Ts|^=s`1 z-pD;HnNeJ7ZcfrHRi|1;yZL1^0qo!dV^P?Z6&+Y*>WK83ON;D%x?kw~VzAew!2@6Q zXx6@3?_2xw-*N`fnrNQsX^l%(=geC_V%$%b}>{W{spIw>l*if5)(Ru^8V zZxT`8V=);jsU1%!xCH6`F=!$2rP5_0KjLxiwv(-OvG zf+dIxP%m9j-v~}&&52)dE9x~|H#J-2E`om!b1{&uHj^~N^IFG`Th`dUaA3#r2L`N! zPcIE)H_g}h%eHyr{6Abuh#yTDjZYl5PAU-?&R;JO1O5j-U8HOexYs}!zq8MKA;9~? zRASGxXfzKh?PbDbJO}Z%p7(qU@%trs_EK)G-GHW4Xj5Ip?DZC=1@b-aD;-bS)PMpW zE8dqE`WK#T#YH#b&?#pcYx;fSPsi)R?i4VE>`;f7;;TQ4B7n0<`B%o$ET6gocf)88 zbX2UOg60QFIydQX33FljUu{|?DvBc@p=kj^bdG7NTVA2JxTi+>l7O~(=V@_ET4i9g zX()yK+~)Z0Z1Y76t$xV?$kZnqtk8_{!h4?Fu zOz0K8`<`Dj75|bygAfW&Jz81%N$8E_QKBU`7NAsBLQ;FPo zHgb5LtshUyqr$jbM&GnC3YH5tA3m|u@Iy=1iwj0{Ak|G6C5Q~1RgT7~HfAuF#)%Js z+xp7B(+I%mr)T}C6vAQQ6rK8ro_zbpy+@IH&c+4Qu@nE)%%`l&*1#qhMpH&!+}VTNw3sea1y}ni zuj1>hmbP^$$zhFAp;A6YY}P0^q`dsYbms-vy<(>`dwDXYEp(6C>lV^x>JD8oogH-A zmcekxWRvF5>Y6A=szlK9iM9#U?0|=C*59}GqO#&j(b^<9K`1|rF3HLqCXQS-aKkLL!Hp0UWTs5sAw&!O|;gi{igWJxHomY)aF zzSi_;|5+gL_;?Mp&nmwg^7!}} z(%PvGge#!l8i}tgONh_=7Cfu z`5r~M)th<$MlY;RvomM%Hg}6`yE5zyR$^{9u2H$-lN}7QkR43(S}+9K<~~^U`eT&y zvaNxTtrq{KH)amBtsG<+GWRF6v+sLd+|7J(=3c)Hbs7Y?BWJB={!+{s~Wl<4M_ z99oIJ{h>?kl3Rl+6NcK_s=!~H78tlvm;=qk@~pd+eO$i@G&`q3rc+eFlYqmZ!DQeQbE}EYmg()4p}nbF2UlJW-Y@)FNOB)sA8{=rhySPmg@2cMa`b zWq=3n5ob_Fovbt_6#Ji#kkhY#vz7CgN9neHqnzcbZ8s$g%7;9yu-2=gu}jRXX@&w; z5h8DA{6f+%r0%>Jk4Da?tI;~P;lt*B94rF0p1EF^1n|xe02r=PbHy;%W-bBC(`8ZI z-)z|T;S5bww>Se4d?R3cNl(rOQ(%IaVo{i3QCoXzju2YN5~k?kH?z3*bQQpgb6BH% z!n=Q-Qfz<`76O&OWGWQd_YyQt7fn0z>h6t*s-$)PERLU7KdX7C+EwVL{Fr!vd^R&k zs}}NUnjMp%hM#yl@${hB`3`v40}YpMdJmO*cug%U&J}5!2K3*?H!Oi_>b|1YQ!9i0 z967VN8RM(CAvh2MNXb0re&9mbJ7c4JQ>DfF#7bnN(SnQkO^Lv=M z*@}p%D0Ds-Rf*2Cu*ovr@d3ovt8Dr+u^~=b?b)qC==^o;_jEvHw*|{XHUT3itqKR0@{OJ60Z0I|6z!M8of26d&I_jAi+ef7FZy;z~ zqIN3p37R=l^}U*OC(^vqD+w-OhI_zajf+A)<#R~oqcCmuy42xdjl3o(0QRUi%cLKD z09yOzN9M#(UV3o)E|1;^&9!|eeo;c^tP8#!G|ObBV3`&7Ev1TEC;v3Z*Y^s3%l_44 zWE~Gsj_qC1G|dyoC-A{oqV{gIVnRC?T{fHT9^O687=U^z-(5Y#{gIgIX7u2A6q&1# ze+WhBFg4o>&)od$g84rLGJ_NBDS2<@1iiyaJR8EM<54h%$K)0bQXq! zZR0e%5hU$BjEbV`TWnOdG=$WV-{-n+QELHrfi6QnQ+<@{s%U$vlYVdyTpq)=X+#Q^ zY#_GzV}pTB|_idO^T*Ta*&mG*Cg@ZTGGqC!5Ie_X%D^WY5RG7m-6rye-J`7(FKbc z@(6&;YWC2dj~A+YyHgQW{e!fAfJz#;t(sx8;=Q-g4I|RT-I@>*uf?Hj$AeUvy3V-n9=RSBhjwj<8B(znc^rnjK}rI3bVNHI=EM3G^10Rqztj;kj^q= zAQ&WL6oy&|ePdz=B#oS)WC15hMqWSt#z1v%lTnJUKnu+nOL0to0yo)MdM#81P*$y#{V8o&R zycI`~ct651#1oBGg{iPxSs0k%xaR~>SrSB#v&@7cwiw6Gru|R%0l*~6oa>(7e9%!w zI=NlcKW>F2{n+BQ5$G@jmwCM|N#gKfzVGex@Qt9D+<4JNx7bdbaSo8%6f>N;S#}vIj(?w2y)FFKb`+j(sdgt|AqFBcpiLwCk@d{7ho=M#T7sAB(dS;>?$3 znB42amsJ&)j7M^mu-a2!XOKz1by3)tAVo#1T+k(U(4#ukXK6eDnKA&02Y5L5=ejn2 zci4GKADz`_{ALoBhW=p#j8)9i);0T*KJQ)4JnNE&toWnv(WH+(4c3yhEs2T0?x~LoFzm#cfz*vt4IGs4Uro8f#vmIg%+&+o$A_w zU-z6NpO-u{E@JZ@025G4V9;EsPIsxDwcMtuh%cibTZldtI z&tDZ;?c^v}IP=+uljDW+y^*)fjp56H>u0Zm*`#k>F9+4iR4uhFXUkz2+rsbI-P$r| zh^i%#dXd8r*1NYbXWR;KF(C>bTGcSj`tmaZr%{%=abB18qHMM@;4Tagx!5F`g8($dWcQ9?=yX%LA?cc&5~B%~W;)Hb>q z_1;6zInVFCuKnjZ!0!6iCsGD7-EWm=Q-Kf$28U&3H^(Dlm>8eE#o50BI%|PEkk&&bG|UOq zkfeB_RNlO>Xv&BKgHiFj(=567^_j;(*B*Sji-jp>;03(W)^_Z(%!K$Tl zZ$NDDA}CY6nQ9rKlY8me{UxVgF7h6!VmTpV|4xr{AN%gYAvpRs#|z*@4=&*f{3PwE zqTXuPfIjV7uquskcXUxKq?!kD{0eChl`x>>1!%(DIP*0}wA%X0`fC8N0`$fu07-$w z2G@~*D{uDl2e|@@`TlG9y@$gQQguC<`QHmexXj($X`C(U%Je6k8R;cO;kzXdVm}Wo ztXTN1tsmUg6CswMfB0rx9+BStI%;6hXYGAf6F(33N7E8iojKXMKnq=P4Xmp06e}3T7lZtlPs!YVrN>TK z=yUl6gw_{eVv`F_gEdmm3+lTkzfR!g1CJz}YChY|Ww{Vk6@(5|_=aIKzHTY0eGfg` zx#RoQ!4F3gLZv?V4$pxucKJn_#t%G-VyJn&%)?4y+7sd#P3i-0wy)(@^U*~4|$;UZps9vXZNTla@H0&p}2}IO07Cm2UHD4274o2 zmaK2F6>`{W7|dh0^t>JW;sTxrTOW*bn9H!?I}<#1L5)SH0*WeL1!*WhV*IlgQ*gas zTa@ABw5U7^6%U@FLD8{Kf)wqs1N%e#LyeYD%$5knl5QMu0>?abp@b;={b`!h-)*Xs znq{d|lJ{2L@%)IGq(Ly5j=vTqHCl;2(D;&O&qfPnbK1&`?ztnR{b66_K6{V2DKFl{ zJKwcc>JcycaO}$dX)RIVZQX30uomRukkE*FpyK1WRpV4FZc=GXUYHNk(h2~v^)G>G zF*1IAFI|^^02J_0K^9EA#*%}a-7p~vX$Regoor=m*J#8Ox?MHS;_njUGS#*^%I>jh zD&uP}cbd+@}5g|=kG$#ZInhas7$EbJYD&`M-_SdF{jsT7qH3&>ln*u6+NBICfyXcuaa^G_p-uc-9(a2gefV$t8KO!52WLMSjHJ9I+cwX0|GE* zHEnLGE?!s;h6I+ss_HpDK+MYZc@s;x6FCO{U-B$l2_tn`B0w!G7i;4?bOmzo1NHM^ zZ(VqI%?(j+p$5)nga5qT&sUEd&*W~CY;r6+G&nZOdWFctn3FTvZE!z3umbt$?R+~M zJB8|QiNfxO5IIq&iN`e(Op?ijvMYfD6b?MJ3qfgZ(q5nPV@Lac-CB8SN@OSAr1xlq zIV?8j^*Iy*%LZv?%&Z@%AsP{3GPolLieh1c4YT=~kDQY7i2LD8W5$dSCB&`iENsIg zD@cmEIdkjA(eHl;HRjUv63riY-2o`k#^M)eXQsQBMeMK)wc><$0H^kAgZOjk&Memd z(;3h%29H{?{)Iyffz;|)7qq$5dt)Y%-oM{d4A%g?(v)%<43;ar3-ze1oyzaQe~0a$ z>p=HPQ}eNQtAoEIl{m8-clNO+pfx~FoPErpYNLf*HBVAx16cA_)qKD9<{1<-kXoQ3 zaX60}HHpYCZ0j=IE2%M9fgTEd$C_Lw3>5g2NK^sFauiAokctI!5fPH!*R|~e71|Qf z3i?+BfBf$q$2NnR7@9)v1l33ENrG5D$c&0dex(q-LSZj?O1ai2}?CR zXNO6g9rkKtExUK$Qq+CHG)o>gV|-@;{dWL`b@DtXPK)eMI_6?YXW1&Et7RzQQT@&E zc#c9F>-@d@jzuhuGu8k7Il~@22w^uNIP*Ba*H5O`pEqt&HLAh39UHJbaZB+);s~L+ zAP=riiGxbOPs>Gwixx;wy8#5t^Io6RZ8ZLSSdxR{ZY!6!{dGc|vEBom*@iTe@BjZ+ z$dDODrBT>#2vl=~xi~FwmFtwLnk+2+P9%;&%2<-GZ#y{W?b&z;lUddGEhf;TYe*wV z6atNYR4quzLmp(=TMN8jrZP4&Gn=6}&PY#992`-ue!Uim?Vo~%{^FXxZVsSn?LJ3a zsH84BdqIiX39parBG4V}A(%%YJ5RkSS6P`>Ddug1y67~hIcX-#K(`S6Q!L9Z`WM;$ z?o6K5`ZN}RKuz!!fUXw?lTbH$!5m<`lQLmg)0Hmu?=FEDErAD?TT33S@h8 zRm67(@$b6>yn)mjnE#Amqo(jJ8WdLlJ?Pwc_@sSp z@A5o?ao5?GaRy;V1ie(Y7K+u9G04Q8{49Kp++543A) zI8}1n3Pfi#to|V{hDJ1H#v1C{v#z%@M&tp>fr-^4REi~FtcRcqF{{9SPHU4%YF@ix zD`p5Q=rsgcIRa07D+=hizTB-DsV_MFF#{f+eZZCu-2XzZB9w;Nt86srs;p||u#EQ) zyxVa1+fL%-Jo7GX0{&aC+jMtVwSeB01Xbt2=`&BcnDRFbrE3?RhI(0UW9?j>6#PjbL9e;Ym6 z>hdJsdE~z!8|sfEiSANGp90+n=x!z#G~46%708SAVoedq*-;GSFb2~^M|ftqoiL(F zJ&h=u0bav(*a(?NEP1?v+Klk=+68C4TI6V@6z%`+xU6vRCeR1)L=|471%kRmz>8Q? z8_Sc;DfC)k&zZyN5mA*OzOc0*xttNDl7ppDYGLDY4mJRIz(uFk&aAmDMkhhckx3#@ z#^BI5ZXRQG5r7`Zboq^<13pA45E;ReaIGGsKH6C^ldPq+6Nar}$C=^phsx=02xPw-FIMH&euEm6VT#p1x!&ByzE2GN{UKWSh z2D)!_3xV)J)UfEjklV84tqIHz>TeVrDu5~8P9Za#B$L~0M3$ZUZ zKKCq(AC~)HUW3)P-Yl&p%g+W*kfBeJ)hOtFa$goH6Ls(e{*b z-Cbtl=cRgsC(_|zPqTV#=TnUCIacfo89Z=!4%*#z8aDT(7s@sZN)k{-0Lyeg+&RB0#2~T;h3fw-w}QTm<9Y2{^bE75$JJCE`yf%I`PP}SxRFL-Y|h-~d-g3#f>V7BJVJp9 z)(U^jP)c-V%x}Jbemojj5x%exoj=Mry?i#?G=8ZOlucHgiE*$pTEOLFCfFF6e<(OM zzjF{R+!z`A{9xH`Es&4%Ok6w5S=NY?Xt^P+K2l>k`qr)*vrnj;a8~iHxPK2X_LyN9 znj{kNeZ>41XA^_)p?6>Pr#H4TNU)-Wf3na(*8O)L_(B51DzMr+GQLg+B=Lo0AFzt) zT3E$&ngu0aw&HaUP$$NVAIFXt+*R>(S)RYKtQO&+c~ae02wk<>O%rb@bIsM8vV;sA zWP|z=WcDHsN%T%`rLRYjf6)za;mHBc9KIw;QFAh7~(OGYG`e9JlHw+f=*8N@;?2-5A+GpD6Z|z=MpNAtuHbJMM(|wICSCSfx8D5kcK+= z$XUu(G;9#|7;?H5tBn=&A;F&pn$O?DB{0kQDtd-MBrDek7T+u5eb@!j+B_0UeK&t^ z`F9|lp@CyJ|Itp3k&O+()4$`0r&PF$C?N=#K(Lw#1{CCnrdbr%SMz2;-%pDCSsWnC z;(1S?kbowETy+;JSS>yNkdFkdmC6;-C$}rUT8Oy@4=B3&wfmi0lQNev;!&09pm;fN z^~HBPrVisO1@t5;J2Oio*9g{_>g)urzy^3DeK3(Bh*fpES> zp1(XT0h-u3{^xLYJUwc*}MkL8^LkXKr~(K2-E@NP3)Z^Y&em zY{wY)cM(F)N^8wZ+xaE71zrDtxo1J&WM~;kcu~X6VlxgdmL-wo>Q6QKWqNsf?pEb! z;B^w~9uagNX2~6EnU4CvUhs`%&-h5GaYt#0?rf881hOUy>Q^+O2-X`Now~8W&sM{Cl0}dq$UUrOz&=ug;Z50L3)d=6?SiGGc%G0UD8K3^os{&rq^d2&BTb&ZS{gcN`8#^t^K@^njG$n5+}F`kx}6@8W8#Q$hFu)u!3xQM4KsUz*^=Tw-DU?dryy#pPijP<74fYW`1 zut};*WO-fcS14WQK7H|OvQ_FD7#{4GE9v!=4>?Cr>oAln>{`VCL9G8z7%iG2nnWn{ zerwoi)84!IdoF~lpf{_S(2uij@>39cgEo2AE)|rboH;Ug zl#Co{e9Yo|(nA`jLTMH@2I}q-?-~GM$=yeiTS1zqAepksMwMvNk_hyOZU4k@(-Un`SV7tsuh{-;h5l@MK28GSUq~%3y3hE4JC)BCEIZMPH8IgE^Ot83XXw zpS1(#3>#=?w_Igrj;V>f(y7E3f)e;gT=l+TK(8Us-yA;;4LhP_xuap=MRPJbFH)^~ z$)M$h5n zmeL^#sg71>HC~_(RM_>|iV}Y{Xp>pEJjCR&`zw41;VXTWGuoZY6;}0>Z5Trz6-=t3=q;q%<)+OEREETG5d)z(Xm-& z`J-k#>O!!gu`x5yM0OMv{ZS-Vmb}Dx^;@Vz!-cD76 zj|zkZ;<$Z2cDiR0X+M5LUKLutsf&y}KNtftzWg6ZeIm6*sz&mWhP4eGQFJ3f57EK? zaB73upd&7g<6V_RF0}vqr9_q4n^n0R0>n!!q4KKjF9iZc#ga zskNV8-k`tKBhrx*)+ky8bfNz^)1~;dUo6u_x+}{I8~)HTzPk(*qm%PH9=A)UwQN;a zJ-+8OtF;VBLt6&i-8@NAYuQEH+81hzbpzio#se`NKCMD~I(S1tuG8qB#1fEbXcxO^ zwn}^XVY$_qin*iWi=VIWU7r86PFejbzdQ6-Hu-)BFW$+XxX^Cy%#?&dvkbgsjpkT+ zJ{l%+m}%EBmH$8Fp1%h{L)u7=Y9X8d=juPSe?VD!Bt;SnXS4BC`qX=m_y()`aE3uVC5W0&n=03Z_w#Th!IQWKr5ETb|pRT!(}$Sj3m|QL>vzo=4epJ;N+b= z|1P7Zuj1WpW)nz|9S<%5TYaOG?O}>;55Ge;SZwMb-(ya<`{etQl5|0_Ce36<1_R@< z4*AeIX-kU8Kg zF$^|#wEv`x;70?A5HF^~ht(qT6vnZPV-FHk#-zL2%dMUoXw*756GiaZ44FRjDd2#P z7n|R%#%Sn&z6mFsF(&sl`sYTUFOj*dluDvo>B3@WyReF|xU%zyePydsTlP|pnY$MA z{JV1W(D=`7yN`7+Hg&0(^qM5$})+*Axz*!594q_YXvYZ~r<^L#j+i|Hia zXUC=Lhn{hI?Vw{{ zUQw5jP1Fzg^Sb|}^BXLDRZq&_kx`k+CaI53Te%q_5E&DY|0?EFSL%3n5~s@8eboiQ zh2T|r8^wW_2E|twATA=#i>PNO8~PsBRw`PwF;S(T3c6nZazfn(e%-E5r-uq6YxHF= zDI&&KGNKSwt+oFaSlFB7SItPzV&JuQ!e-GZ;K#;do1;<3k>B$A*ReGE%QWYBz2`nI z1U1mbQ%^*`ejkqP0*i_4^j59WH1xL8Z=NJI3MEx+c+j5krCKVfT8dYHYtk-G-BW2v zdu_X-$r%fj1fJ~XYeSD3+Z865>AW8nZg_aM(lTN`Zn2jywv-io=Do$w+?q~YZCjVy z)9-}QnLBIQdZ{-=SM@l7lM2WX6)6*OFY>RA(LH$Vv6fbSDW$VN)8nvNtx;YtX#|@` zaSH$oP}bP$`&+E+977fb`40mmM`0Gi$(>O)QOcGSIW^&wT|o|(lmlOr#T~6 z_@-)q_&M~^>g{{YhuKGzBZha)YJ(9sAVhf>^$#SYEu;gP6g}q(kU$l@ih_QMC1g*v zmm+TcepZdioK_#^#=2jiSef_M|5zH1!ZQnwTUr#r+Vfh!t9E$;G(ox@?w2^no@ znOYiR6Z1gjFfv{8rfK>2zsm0wRJdMhfEcyLGdukcI{!~`Mubbdw$T^(CHB#rvPTV! zQ05`Z(q6;e?&9}!YG*aN1rY?vmh6$bjtiSM&9dVY5A2G##>BACh+q|BD4PukFYv>V zb|J)N8Jo%}U0U|tLVQom?{+KyRIr%Ejariva$ofm8d-SNI7t1EWmr4bhMOopAs%B!n_ zBO8Z*h3&1`@bW@Lk3X3dWeQbI{1b)#WI{%nKnpI6R*S75mhTgL*hSR$VMQv@&CYkh z2Q>*@DT-V=&hi(h!0cHQ?Hng7Fz)~B4e>l_EN2W-BPXOnx<7^xu-AaY>t>#3Mm`5{ z{i2SF%Cur{w&*4@m#`GQC%?9%^!6t(Lr6GFu(JO^Hp)rB$7QkBxw-ndr9KQ`pW{3i z;B7)Re4KhaTWr5SE*j5m&WcxL0W8L;`V5d2Cx_oOVGJ*(jKDPPo(rLz^uDQD`ch0x z*|vn!e);D~WV&(=i!xfaOglxn&ZRhYM->_)F~q^B5>JYFv}*n1vdZn;~g2U?P&0uuXu zaY$af3#s&_-ExZjUGhtMGT^Hv|A-;i>BNdwylm}|OUGQ9bjbm)Q|?M3IzbSxhEDj! zgzTi*&-$8~d$79vt1y1XzFnqA4!n|WU)=dn308I=up@)K)k*RM+gYBTI+O7S1XYXr zAT9o-pc-MKYpGNh!e{FcM$+r8xY8V#H0mrtSD61}mxP z_nM&0J4jG(Psuoi)S`#fZ0(sxPDt;QR}?lGoN3M-PuuDf{3; zydt!(Z1F9Q3u_B!iIK)&tyOF=S zfhL0p-q0pWB5svS((?N%Yli(N1lk{IoB^z^se+MoKRtW$7R8%U)-PQh#Yu6hdvl}6 zhIewe6Wto&WvnKI**#z?$*~wqBXYs%e`f;LRCr>ATNF5zt2woe|MR^^xe0b=d7ys2 zhFsT|OK4>kVO}1T)y75EUamVv+FUBXmv`O*V2}W9^ljH?`|E?XO;vCCPQNA%WFn!6 z`x;5dptuC4W6$p*9@3sFnC5dCEC`)Q5tPva>7k{jccdmOj>-N0Jjn02u~L9Ucr}fP zvqluPjtxi#dLP~`O%=q>UDS;Tk$I5Sx2AK5w3W0uMkn?=R4i9XJ}IW|P(J)> zc(~NWI9;Y)C15`mBsf+cqjV0f|L5Be0~6jt8sP_M>mW41tNu4#95p1^X;LG(w>wK) z|Elj%xX6Nk;$j5O(e?U!H|1ayS!#hR(4WVJht!^5+${}k%qH3UE_T?|4TkeNRFpe( zotd9n9DSuXVHD@PfoL}@#E{(MCJ(o%zTKEdVe`nOE}t4^LyLDua$}M~awZmG;8+yh z7@h=xsOl$f9CsQsCiTgEzPcXYG5AUb{t~CgYmexYeO`(sJKl&$66r9}7025enazG+ zkOPZRwh)F!iF*{(7C^ywSJzuG77NZ$vN4IT=35Y4kC199^;t6(?zhh_ma~Xt>_s}( zR5%p>u^U+g)0ehUl5S`}#~s`Ia+vmknwNLBz9LOx)YORulm~|*vb@w96-@gIvv*it zPHHdSu4u#=rQo0VKXHcI*O&G+%1_w7!%qC*sdFcc(2?*4sy&a)sPsu++U?D6h>xd0P@;Ux%aYfd#-I;l3NWw|}IiX7k_UwJ9s8F+F-VOxKK;ZXrZEcD&aan|H{ zwmn;v4W1scD_KI2)u4;iN4|k$Qge|}ShM#w`j@5N&w56>tA6SG30)+Cgeo+9g?J3z7us$aFZJEm9~>&~1|P5%J*`Hf;;F&niiLUNnHPuz$u%lakfZ zHc14$3hr#9qT0MU4HE<^Xl#N_1v+e8Y%Ge!R=#LfntR!1pIQ+J(D|Ly=~nM~2Q}hU zska-%|H&uMla{PNU)r~*NBs7$T7N&w(U7RJC?zw}Pz~ae3`Wa*a~re~xVx5o9#Cdf z(?${^YJ_Y!ii5d?9rjH2;)#lO(?TrNm~9eSr+aHgoEynm{**`n*qI{yl1y=F zmQr=eWUfCr_sw|Zl?*xP9hQ7$iUIZNp3#j!?-pO3P&pmwBbEO5X;dGR9vs)SSd5K-tuLP z$qb>^0ql-011#G7E0MXlp6NCpoq+v#e>5``FfryzmOdIyj1Ux$_das&LBni>eK4_r&vx0Wk89 zlf<*94c-#*%o2_fn7wkJlN0i6vt5CaSRov_Y*H8T)KGo_CKfC3M2_~~CNJp*1&%oJ z|2ACuyn?{kZe%B$N z0c|U_;n5`NvxRgkPV)}~vDXuYcia_i<_#I_)|aJH*BGa)CLAP{ku_BuvPPEyYGzyEy^IVAzOUs{h|7k+tnWO5ew`MRwD)YJ3O}0&l{@2$sdsO zb$^s`7+$p_Y>+^^t#m75p~VYC(whGM630C?mZRdTMR?jm>#3}c&04>6W`s^ppTs|e zMxf6(TyRqBYf;{v|6jQs+VqOFFFSi3JzLOg)B1K6c7X$j65Zb z?e~BBp&1tL*QUB=L|4OkRm&b~!MN(nzKmfFb7^Oet@z;b#*ro=mTwPH&)}Z8X+IOz zqz|-WyzV>N!fF^c(?-q3gT#8y)Fd@&$16sLRDQds(>NX|EpMJj3+2t^1N_kR;#-#Y>0G z--P8f*YDZHH(a-QaETj#O!6NsfHHA2n6oZ&ZZyjw|HUu5W3|T}u>{WC_u;W)`UXck zzNvF%KJho8F5{0s5*J`HH9SJON}-&!RJez2cWPTSs%vlP{i*w^0VWXi_}Jv$6gn=*PKPYKafrT0+C5U^-uz3wR+)&C@V_Pc(ti8Cgox5&|)$xAC)i&02RRdHl zcB{8#jLe=ALL0cN7O>X>7u(rrsv!*`L7Gd4qSKO{GSW5b*l0unD;=o58Xv#B(2CWH zsHB@BG@kgw(F9v*G!-Jr7>g#9h$p@@2U7^w_txX|Nql48P{Etnzs!ys`CvK&XmY1k zt+f1tIph=hG)4Bg$Uh6ro;_ejs%Ct(y-PEX5MgWYW$s;@s!c4ZKsk4_Ub$P)Apco8 zH1|zrxz=2Z0)10lh{>~(ojXfW=6O`c1g`^phe_60I+TI3Zmhtq8wrI7YA%C$F?M4~B#CA-2|AP44hw;$#m&C4|!`Op7w9 zBaM{^&+lF?_B1BwBn#TMiK5<5f!cKH^}o5jwBppDGVoF?1|e2hwwI7>NiZDZyC%HW z%p;FT@x)aZ+j2|I5U{v|Tpm;r8oZ&cYcu#V_A~i*d1ki$ze92QFT365KMDP+dMe9x zT|!3lazZNvdKgyEgm+nTDb7;4^766|>LqZhc&iLP{f*Op3wP-Q^^?Z$!SqRz9R!r~ z>%gQtF)~jO?+Q@AeQOxr#1thPR1?SSo_0~F^zZ)!R`EK4kPrc~Pp3frSc~Rji z;F8~gk!+84PFJgi${=$xe@xDQ6&8k=Z_h4dK&RQ?=|LqtWu>+8} zxhOp==|;#+p$+DE{I{1&lJCc=a?wGkp(_!@`!WR*5x)e7hMwOVkv&Z$zTCO)pUzkx zq8To<7ulLg9!tv^$vK{3@k`cX;s>&pcnj&lR)8+&cu|$Q{YYy6f&LL3rfOZau@I9SfJNW+%`3Zz#J@HBd7^`z###_+EHclR!^! zgJ;u{?@rMZ%_oJ$`pVB9s1@=s508FTD{i@Qli^8vbu7Dwf4vbGPupjDS-q7Wtyj)$Vy7*msyi9U=51gMl(pEY) zQ7iqOEMa%^y{so#Yx7KkkF;4=xBKTB3`V0LUiz4Mb>b1#Wc&1fn1}+I`9-{>cK`YL z2d3&NmD`6w>_NUne?L;WWKB>qfyS@&sTWw0PK1F11x10ElnVoG#vjTtykt*i>` zE1u|2X>y|KW(;zCcddb zI(*v$xU0PiU}MZ6x;JWKvtGY1khc@)(zq2Kb_Jka^8z>lT`*bE8fU9ak3P8^+U4OxvhODX247`;2V4YK(6G58|}t_F#G z7V36)TcFMTnM%^a@;|$sko~Oz#59(rX1&BF>m!BUA&VS@?M!y6!vTp@Pi|n(+ZUZn z_Y9gNow`LDJoYCfcX-1kxYtZ0Q%-hWBvQ043RvJCmEJiTto+18yU&t;b=Aw8!$}8i zHeuLiir}*@U9Jm@fGesw{^ZFJtaW+eVeg2xcW+5d%jSpeL^uZ`I&T=|wBUIM_LE#6?{ANVQa z4u9utiz_%H69u(v<0|;Xo}(LM2J_0vBK$v_zo(HB2ixJalr93h4f6=Gtsn#Ug_!RO zWT`l^zS}e41+WG7e&HN1ikkWY4P3@rXO{yTzUyoPx(rFU_kq(!l|5nw9OW|p;TO#{ z&gffhzQPU7#G*ejJ4h2SgsJF=VwtF3N}-Za7JJ!3X&33tI>u%>K>JnFPz=uH^N?|XS;sSU1#n+YHz_9 z9-NiO%jwU@>_PTI&@__RT|)B4ARQSg?%$93UA(%~?@y@l?$JoG)?b?&)DUo}TE?#! zMMj8B=l&*d2^t7rc&TtC8=f6Meo@uh!f$_}3a&E#Q5F8ZOvEzVEVYmD&EVm4Hmfnh z;ck+{KV0HZ4E4EW3Q(`~4F-wiCCP21&20b@%zLPTVZ~ z5f?^y+RNP3y@v%}vEY?6lct)h42!Lwtz_V{r=biK2HclN(DduAj(v zr8j6MdW0#j{&~pCh;j2)_LtKm#$~B&UIG95)G6X;>)hTP(~ z%zn1JIw2^8{%*exkl0tlJ>m$zo4Kp40lU8d0BxEn0r8kW@6`2hA+~bjJ6o?PP9(iV z=@ob#F8)OVxac6Dm^;m#Z@*Ec63f=Q2{h&S*fyTH z*part0xG9m^gNF@;i~|TnJf+VIF-Ctth?HuRpth|5p| zL5}HRy)a_lbzEqRtlYykHlBh!-2Ir=q?%``uGd2c%qyb>O84@guQpw#s=9flY6c$A zVcf=+)_ls&bg)FU;tCN=K2gjr9I~#qy(7-Krp`#J(Vck_8>a zv~Ah?b_s{6htN>zr2^F&-S}{hZH_j<2eP1E3Lgnf9iO;$|JH}{d@t;AY`=nQxQ-g5 zo4(R=9lBAY-mhb0f0p+5vU?E#mr+YkM3~?YGbbdeE3m{2^veUrqkG`$1qwCp;4DcV zo7}#aw7+rm_AmI`AZ>@7wx6s zi)W8g$L~xM@^gV_D1+dD)OT(BtL}&J1+Ov7WiOgMDv=9d|Gcix5)j_Ajl7I}nJCm< za!75nQ2*nAl$E`(SlDuQSM2jL^p?S`#Ecp*VLGl6EpmN&*q<-^%nHxxpvjg3@iA0m zePD2NzhD@er+q@1?z(1@P%@2Z71z&JdArPhbR}3~q;_U^98zu_NG(i8Q~7@Ba^v^n zD@T{l_#{3eMjvChZT+6BAJwabh^LT zw2e|0CX_c4ow`>^ z`ij|HBo$RNY>W4Tij70yr~!9MCBq|}w$Tl-A7SCpr`tLgmhPM)sHm)8)(Rz$@CfEW zQuyOFzN*fy*u=@w?nd`5aBPV9BMAIPUZrLn6B%gSv zxC_f6Xite;(yX@loNLMQJpOMYcpN2_;%#<(LR|wt5^7@$8N1n~ngbi;{U-eg`*5HX zX2JaRO77x!GU6Ujq8Dr=sV!3`5#n0QmnDWotw*{22WICxqMBL!wRKEmW}O0K2y_Uq z%MNJmbvC}szIp1YQx&k--RgKP#ks0g z#bfi81HT+<;$>3gN%m{c@^9?kob!&1OyjS_K1-17&hsCjxe-Ewlz}_;i)tj<&uH+( zRNN-hwAu!?%$sR%lx`nBP)aWJuX)j*HGX~74pANbIiMl#+SW~Ms(AjBZE3}nUZyzjkexEb0CN%ef^Y!;cjcc9KJdOP?}H6z7NI!o||jW=bKL#yJZki}JO zdiYEU&u!&*x~j1KPJWU@Lw6Oi^4lSYh}<`1FH&%NA&T?7DKhmU z8GQ)r@pgJaj!?Jw{FKlXbBp&A={F0hzy19extZ1b^<)of=C| zr3j&$8!V;kV{kiJ;$PV3D!eB2hU~cbXH{hy@!sH)s}o8GK(;5PW`*B61}t&(GG@6q zFm%_wpnyS79!SRv-aBt#bH^!g#3FRKKrK1o7R0K2zq>IZx{*f z`Z3sq{g*GnL>nH-Bb-MR{Yg(dziwAh>gi`idwn*&ede2@{RX9tCf|T(MhtxNk92=3 z&D&#Avum+n`)>MsguTdt@fj;0@Ila-kX|m-IG_ZntdHx;_0QowPS%v#2H)zt5B71GL;78Ik&a#0(OF zUbown#4GLcv>p~g_u@|;Q|Tbe%kmdrJdgcnVWh0cH})Uq3#p4W(?7$wG1XWOMU|77 zlYE~rQ-F=N>^7;S?Otl>mFZqQ4iqo59^;*OQ@eVV{WkOaYXYP6KNF2xzl#xwiFV?0 zlXyOcVpCr~;-|h*G4Jnf-00kQZ?zWXw!CI=ewXOs1P6Mb;hZlc1IYP9Su*1TdAL<> zqx<^}dxc(!8RH~BZn$JOGhp1T)Mnf)#MJfZ{(LG9iM}GdM6*wUTGYy0 zdvK^QVqUmyZwSv$A9NPKf(*CuH=+EVN+r>QpzTNBj_3;L)SV^iNfRS%g!HLqS<%Re zz8@!w$1t83MCZwa^S#-1Rn(~UfI!SnC?#5#BP{tjGA^TY<%Q*SY!>q8L=jWASV!-fOVO3$oi=b$| z{$VQhae^(~a60@T?90IIiF9HWVW(#%v>A;B85Sk|SlYfx;rIHk-S;T2k@{L2chTTw zrs{r0_VVTzAh6_Dj9acEb!Gb`Xs?R8uIZ~kJV@CB*wzyL+fN-_)~BJuG~`3n8vLjl zbV054;5Rk)r>AL+y<*#;yQrbAVO=W@7YN9>zbF#eU0ZY(VxwJZwQ%xZQ0!1D&F?SL z4EyI$gSsZvw=p7Hhw-N7&)6>?iJl(0MRNA+TZqj3V!$VUNc5CNp#ryKYp{6jS@5Zo zIiAqxW#3kz0iicHp3n#56f`@nq;JzL_ERo7i)33Ta5jV=B0j>IL4!SzxILq+(qwWX zb2shwPF?sBS_DzNGvYIU^UA2tYzy>C!(?G}E9#lIBk8RBcgLV5!d=7ufa{Dqd)+)Q zcT3W*w>zv_T@(;w`K8@bs`z=^^aHZaX4vI%;EvosmBil_O(WQtoDK-2f3kHwTm`@grA zYu0$zx)<&*zH|0Id+#GV3_K+M1OQIc9lVrR66;bi`P6&8O{l!dn&uIFi6NKN?8gg@#Y zDue4>%|y^~z|!}35oKj%Pa*}k*T7cEi_WiI%3po~bl3L51}b7;?<769^J?9Gel3_Y z@X^P5^nK8!rso=SE{UOnqbfIL{J-9#qP8h%s< z)-7)X)8dc4t)-zObjkT=nwKH6t^ihLeE}GHI8h(U)*4&`_SCn_0KjV&ur;8r0Jgv8 z5vZH;qrIhwn8XJj=JWh}M*uQBU1!|^o7$Q8q9f(=*eSTJ+(+zFM+kP??NkzaC_Xa_ z-4sDjf5;{8=~(zi;j!!Wh}e1k2qDA+a_;wu+#gC7iHdvW6Gm-nxdJsP|K>djUEBrY zI-QM#ZkGP!SJx_Q!tqc!`opuvn=QzfAM7j=-qYGA(0Y}IIA;g1S$C5lg=AW5owL{O ziw+0|KOmh;obN|U3&3~2Uzmd3b4c%R4_PG~@y*N3Q1yFG&dx|_lV}@f%wS)tr zKY=KagW>#x5hJms*2`x?dXdx%R5SKp_}lXP{j|$=T+ZhIMvA{bd&bEw9uK7C@BvF| zP$8g8FY0t20uxrLq;0F{P;G90Rl95D%o#$j?6Nuky}^Sv)`gtQGk%h}oWN%XY>yyu zZ0I-Mqix<}F0xHjp+lBIwdSz37Vy`vBR?Laxr}zIK}ZR$o`3afT{Rb2PCuBV&$INS z`m1h+;z&2ROFciq(%A>#Cw(SxEb~8s(cFN1aRflY)JHnu>RcpH(&)&G5dO1w0c-Yj zfcWvahX#kNYr3_46Tse26_<(fd?*~Qm2Pxc9`)93W=o6!j@5y{!n%af$`r=PP7D)l zkF4qIfSZ%A-t5(G8m3p;G^oaeQak_f8Zyrkzt}L0W3z>o@1#HKE&_b2`@E<7$iCSpgEC?KPCLrc7CH3zBBE z?QK}h*R8dRny<7^+7H5PQ)0EBUD9pyZqKu5ktZUCSXf&;qYK1q+;Be7Fk1}|hmfH3 z+wZl6OY&RgQ7A_E)SZo~R7kh&ND+GwryfBoKimVo^;Y>MWiUs9rPa$*?$K+k&L@Rb zk6c^2eH4wS9zC<&*p3!ji`V;gLT9d#`2^{X;JGhGK$2OqO_w}F5Eni z8cNluRtr$e2>n+r3L~23dN@Pm15W9akHX=jFy@{!`zu_c4uEVHFF#7 ztNRdm8!A7-T~FI=q6&;t+BR|A2VqRKdg13ID8Hyp)*tWJuYA|wp$!=G?9-*yizbAq zq#29{9Uq2W;!pnx`d+T}T+74uLMqJR4w{GaWYOLP{P@73Y0=-^vl$E#>UQs$oNyn^ zGqpagqNB&{jv%&97fN>zRi)=BYz!A%d_u~XE;6Y?$?Z-Nep4W=!5h%})B$^7$27QvQ2ei`jz!R`H-!7fSd=l?lAGz5a@AkGs`q4MnyH{{Ri z$5if-;b_>DHH_+$XC6!!=||peUhugh#3r#8P0ukZrKcg@>0&^>fX(Vr9(%T>(|1h@ zPBt$PZhP8;tl%9tl9X%DUB^9)d(DUj@^n{$WI=XXQhAWYmiWW8hFO2h2GbVbPr)gw z8D<=;FP-d$LTrF5q`asg!LBGHlM}_2p`X+*q`CNfpx_sc2FixxX_A_p`^jt5PMj45 zh}QTs8i8`Ybvl(^r!LGbp_mFv^VMcv{6#i_BcxD~cpOrQ9m z9PhaREuzj*$g_$FZp(~=?tMF_lg4mv^P*%#@^axQ8oY}4mG?(bG<4za?M){mAh20N zeW!shv`jMq3QN>=DBoOtWOrjO;`S!4%{n{;{-v9o+no#sMy22hu?p8eoJwq_aDS!F zMUf`D9wM5h0bdigA={+14YB3h3q|2=#;hZ_)XYUAYQTxl3oB;zfSi1bN;9dMdM>gI z4w(|dyk(Pq>z!NP1JI0RiKd?@dd>^QM&V{8g5)5p)@AB>G!*2j{a-;Pj6iWWSu=^S zs67VVMua{?5S9Z?VZSy-48L~mxaZ7r0?9r+$(Y)_cC3lD-RY9{OTSS&E{)p#q%zC0 z!rW_bG;0>K)Y1Ro?kl|1#N!>QRg=4!Nm8bJd>oAP=gHAYp)uRSsM$bKF+B|#!AaDo~4mEH5W;V z6wQDbm-y~#b3=IcZ?ftv!>pQn(~GT8vyKMq%gT@)>nG4~(1NiWp_{aVDcq~Z>=Jcp zM8+E`X+=gsC^;C(*!6Ec21xJ_EMiX21hgj=oJKY5kN zjGVsQEpX#?7H9fiJW8Hbqt*nlM5mtKrDN`Uj>T`P;g(h~bA!dy%f=k-NTj{U3cMW*C=Q~C=E!i0 z-L&#N$EjXNHWl=`RV-gL^bfsgR`_(=6h8=4FK8EqxMX1}t@6X2wB{Mokkad)-nnFX z$CE3_@3b&^75%i?>gQKJQd)E3K?d5qN~rSNEMau9Lh~WMi;4xGkh#lqzTt$EL|?mX zH54lA6WhG^Ub@wyyE3>^+gbCtIeM>ft!Be%@w)~fp8u{@AqUKeLur{4j2~r)pua$r-86h@k|S?(Rx4FIlQxkf zh7Rx1n&YwdY3%WYuaM{582YH(rFx}&8;sbL-%EO2@?D?C?OH&}tgqGrd!9UGmjHus zl$e$%ma*MAxSP-$Ps5K68(p&W$8vE4`%{QS3V2=1=f|K>1h@}3%{^u`IJpNv*98bq zeo-rX1F?9%kZ@Pk$Q7VbtKfkr&m@0!~d{Vx?tihCtHSt!{Kdk72U+KW%vJLq>PND`TW!34QP~ z&wFMtW{+?H$a51PwcbmvKz85Pu8!5X&3^{a{fJ#{A+FV)IqdGT@1sqs9jmnf6FDUZ zCC>>}u4gynq_Igoe7tYY_~>pkl|$d0$>9p)=HTu$w~td}n{*HL5{ zfL%qaYI>oLeBKnBPCq%m`JLG02sv~}pJNtWXVXu}O5i-?)9xJka3Y!%J5?Dajg8OH zyu;NCp2pu(vEQH462xE>vr5tb2yh;X3=scnx6us0u=68pihB#py5x=r%KrnJIE3si=ZI{b$0sLG-HnP!Jb#)u{!qm8EL5owN% zadi8D`V|^DVYbP}{B*efBMWg2f*1=102I&SIAq215YMV`ruO|{6Flb{nBm#~MJ+}g zg{%Edjc}zC1zINl%)0+2*XwZjiwB1evUf~MLKCUjF7R5KYm@3M1K>Ppz|}=v&=zL? zT2Kmb!XqZc7BT9ZJ5dPGa9+zQCv#Uu!~nYme=Jd3X=rv zPi^WoG-ZlnyR{VWVWs1&7r&pIa^~0)EnKqe4H9t z(c&Va-j?BGWiv0QkEocX}vJ{{mqg z0y$MopYRV@Hjp5JO~!3A?DQf^Swgtz%sHQ>fsnk$W=ZWUdXn_; zlN|9+j=TDWK?i4+?oY5?{x{_L`@4ZG$(A=!vznVrTC<4#RI#fxm2`Mpu4o9eg{d?3 z$gOu7_xl^5MkV6?k)+ns-(FZJtMzjY)i<2vAmcK2uu~TJTMKa|lbZ}LRmJUc*^Hwi zZf9rR$s;<76VQCfDCy98`c0W=q`fstCQ7sXgz57y5;+y%0YM4|FEPxLoS&SIc);*x zzqp&(^H!)Q>)Q>={HA`sjHC|}9Y&K8Mg9?W!z1*z*AQz@vC&LOwm7B58aqz=ih)G7 zo!hme%|3_M?zMP=(EF_!);SZCrwfYO%j6lBN9h}WKSc_f(xyDcS}9W?DZaEbMeIxW zzk6%sCgznKTw60)iw%iZpBfIn_0OsQ_wf(DgrpJ8@{V5g&EzG)C%NAV4$J&k)Bj9m zh77;Xp1^Tm_xa5^L4S-F^ygA&hEvJ|nFaW5);Ph|Q%h5x3ACm?GHp+NlQdI3O;-J%R7(8%S~>c7eeVg$XZ{rXiq;bB|4cArr#8D5_u+8 znEMxy(cgG{+L!7HC2m;M`F8EypBxtS2?8!X7qChFEmGacwNS*`?xSx!#yyu-uvzZ0 z$C(B0!D46J>2REyRpJgF_eBPut{-0wu9>vEdmiWa^Z}a;{2r&_uOZcM+WVi+Oe_Q9 z!02W^(Xfa2i)$DDG(+H_G0-KcAinm@aL z5)qynbc%uQ(RfH&e4+z|Qg7+iJ6T>7xVsl*&IUPSd~mN(w&U8(Q71U#0>c8bq}Od; z3z!SbWHquM^L~k>* zCOncQ>B@B8Bb10E`dMU_-`A1F6>IW2`{=Eo8VzA)FaLVOd#1>6)g9;w6PtODKz0Aqz1!Qe;F=aN(Vk;N8sd8_K9VuLw0yPS`VszqH=?l_c<>E%jjqP42t7r9$r2bc~FG`e+N=btDW zf6B=>8BNtM$&M*Vl)p}x-VX{fz>mL3+0FDVPhJbbcl`qaC0%W3iai=lZH z4bh`>R@r>=%m(W|j#n((PIps1^N&305pJ1#A<T7Ax!d1@ zq;*ap;nKnML<9@~;hc8_^R%$EEOAEM(ZM@$O~bhrM&s+}3<@5kDz9Q8erJ|^>X3V) zm~htm&lgM10(BpV)( ziiH-nej|4td?~vd_>RfP$gPQtio1u<``p(c5;7}_Z~0)t)y@)fjR|8{zp9=W)mP{u zV{RC9nAX>ND^_=~s(93X!M~^q#s%e0-SCCdDUAC|KfsvqR)SqNZ%#$fH2`LOwxt}} zB^Cb{Z9Bk+SQ5Rzv9{Q6VIc;Co)@5=$+|sTLVn%0>HW#pNc41RSE_?tfvuuzn2Vq$ z$O4XtB}tFV%y79O1$SzHzn8tAPP8K?fE`9f`v>INS_q03#bHi0((cEL?C^C5+|lH? zV5UGau92iT7k^Drcdur2OYibfBGFEO(wLoD&6yi&aS$#t;mL}pOB}t?5@zM1x3C?> zFi~rD^E>Ig4TWJx#IJ4iLRFe<>GTsAQfsC4BRP321~|h6}A_ zJf^O^zG&S@B@<&?p6F<8u9bAfaXyh%K8g1c0b2^a{ubE(ahF z4JdKW^+Zn~1SE6Xj_z+^BTdT&(u>}zjRs_ZYu^ZnSm->KGPfH~6gF|(a zllMR`%c8P6=UW!V#Re$GE)Jx|4%2a2 z7V=E_B+$w9s}uT8m>$s_X};|+)rq^0`_f5YBA29>RDZ-k(71gB`ZSM5eZuC%xo!t} z@eb(?Q*!(CIM2!uuTZz5eKINJDr3ICCzW0LkhTSL$4X-RUnBW{b+XfOh|ZUnR+4V& z6)hFiPgn%*+7qJK9hC76^vif`+*i=D5BP1)7lpchi|3QM*U35tWFRXb;oMY+#p&yV4A7=M9}U4*=K7!bT-X)dBT6Sj9Fjtdw#{#^NaAFFJ@nHG|zp*{uf>4 z1N@b>XN>&I$;K9KPe^$)B4>!w>aoU3OoT3pPCq!Yi)^yb$z%A#jQ#givkWg|Ad6-B zrea8@C5W-q+V%Og5sY!wB*G^+YsSTN=TK6%M= zIy3>Zyy`;A^0%93Oe7#c?`yD#qSEtMROSodge7{aod?Z}C5a+E^%v1^OA7fx6T46` zi(E!bRl|cvUq;WDSrn?`>YwEC51!T*8fcR0Y=R)&B)$ zOur^FT5b}0?9lbfUZkMrVRgxmZ!h>)05qmWz3a=-r>FC=cnqdMqpcin&Olyi=_4Ng zZUPL5IH!QT&WO0VWabe{$pO>}Be3r%gKw<{52TGx2TrWf1JV6amDt^i9@o0v*&c2v zTPN7CJPnB2*;|)&rhHA|M6RrXmqiJO=kdey#MC;@ehb*nj@<4Ykyzw{k0_MZ?AN;l zgn`GQUC52z`fHnEL(9@_(CnFZq z3A-QZ=_eHH0>%p<%~-bWkJO1Yev-88BVi8I8BcmWxOeoo2N&HUp$?Rafe_Ot*k>Ns zooQHQSy6Q_?I zn2S%J|Mp^|iYk)jL`C4f&)|~WQb);`(Ri@C;R5C%v`h;GO(QB~osDTvF}xA2`c0jFfE+yQ zI%E*anV6FK_9D#-VcvUhmt>R_znLt2&i0Q#B?aN-&^$0s@p?5wAwCo zAbma9%ZtRt%s)=3Z6&>$8*`e0#OYplh7s*JMWb`rBm@%EChegXKuHJfvv^o9l$`47 zOM;=W7=|h8?JM$F)|C(Htg*Iu=pvyy*Z6d~oY@(D@@~q!rFUe;=74+{A5GVOU;pMT z2PHd7@Ek-mOFjeOTw&1);El6Adt^SeS{%apEEUGM9qRrYXZ5obNh)Sj{C~8#nec5N ztOQR7Zp$xz!crI%pfh)zS`#-dQcvmkX4RR**=qkOpMPV7S22kGPdCF#CP0F}jj_2` zvJ%*4FnP0vpb4}VtXEjAxdeO80-xKe0xp2gZ7|BNo=(sX#qt<2E(6AcwJr#%l}<5}V7E*N z$YtB$s~N5+9=0!1RWMyH9=7`=qOD}=TwHov_x1h$({UntIc1DvElw%gz5M$B{^c`a zBu9KGWJiHyy`YD3n}55>+aagji9w|4Q~vHd+!EIq&zo`|RH`B&%d9)SV+N?I_GF*6 zC=aUv?z(j3``{p9{gX50-?e7Goz!8NlSDvT+aVj7w&_e&vE1%7eNMdI%VFHzC%@!rQ={>wEI;-8745TBTqqu4j7+U_~~PhZ((jwYx*!cC0bl*V#f@wX!W+t8~l zF(7JwK=+qz3F`V74%u!XDmXs|6lp6UtDhn^gkYz@OS|WBNIS;=0RLD9s1KVKLk>>4 z8%h1B@}l07Fe&7Pa1!|Aa=@5ya1xQzu=ZGVfoNYl4Ni|M{#7qt;5he2!j_Kt3^gnE zA}a+?^L*DoZZToH$o4;&vu8jhjC*1)x>hVTn4L{Ma*yrW0TYwG4VUF9a7KBs)k*Kb zu^aM9Gt}NG**c$De7m?Wii2$(F+`f?qPxAXw>#auPi`$LqmtjJ9>w_VTIPD}d#uZ= zL@GAF{P3RJFw2AN;0Oa%bz_*Tv#tTqqvQTq#fw~fMn_i?e8B9Xj$wM$EVupA4)>7O z(KWw?RmP7Iu|tDK{9*grWI=FKt(t^&XIW1SJOC%3uKgd`12|2$_#tUjdI;0mFPY=u8x99|)}%`GVG_ zTCxIRGQ)Fd4{1vwpURgB#au{gS7kl52|xSCvtc`>*Kb) z#+k6?!?kkW^sC(rnUCVy)d0~KXqu2Y?gC0q;{V= zU-ZpA*rJW82uJBVUJ|g!a>=u3pKXE%$4!#3qt(Cb$=y2mVh?fHT9i!6)T=T6w+ za+0^8jv5%7hixYoe?JuNhiTY~5LI0u0sIE|3Lgj!LZ&|MNbBTb&F{0!B*Z?f$Qh+3`$OY17mN*J3F4IYOj9@ubo%x{srR3#L zTEvzAoNiFEgp4N%p2_3-;%4QY? zcbE~R24ynxz0F$#el%Sc$`*Kxs$!r=dXnq3$#;~GI^Z1j_)$v?8o^Mv5Twh*-pszb zYtoZ-M|IQBVm_GkR%$n%IbXWSD$}5>3O9uZoGxea+r#wNU7xn*f?Eu=MwhP&eh(7G2n(aV4EK|E%0bd3;oZ0rsRIj5c`CD1EAJ?6^n}g_Mi6P z8w>GxuGpQ@^JO>jo$eVVduF5chXYwV;iUxG+SjKur`B|5?vTSaeC9~KsZuX^(=@3ZHj_pm&j+GLl-=)fCB+@=AKO+)} z&ckyvB1HgCh5l;eb)jB{tWZL$G6lyub7=5t`Y^3t;&T_ptp_4Z7>~i{woPdVUU42; zaLen)=OsSTxvg$cw@}aQ?!jsdUaI+iX# z!)qHjzlMR}J>`C4lE?Ol!x+Jnbhy8x-%eKrM9^cxIq3;$amVd7pl%V`w=#*K-Hu2Z z!~g0I*oo_+v!(_7B_sEqkMP31`Q9p50y@C^m?=f#o-O5TL!oK8NFMFMt4h5TI_iEL zaJX6gqDdD8#xHc~)El~lshZhSOJoI)jw1t0Ua-H?`L)#QYL%$b_dk6T=U4P(^UCtH1>&E@1c-a;iQ{RqLpKt z>-tVj(gr7zbhWlvD??U{@)Lq!y{q966nksv|HlK<>t{PByLuj&4a zTN-!I$ozBByEHRnnAv^jkHCZJKyXLO+M2}B=dN0-5hd=@NYvm$t-%bV%em|FP$^jT zGa>)@r!!$N3^YbV7Qh-BACb-K(3!@#4v@XbKDZ0ZfBcxYPAI^ld;z%38$H5BD2GE_ zy}i)^j|nK0rPs%iU?8PEI=TwSHU0x-ugIinxs5tbxkwOAfJJaAhT5?GNI40?fI8I515FUG1KC9qfZ&% z&m@~c(YN}m?=WCGdQGM(BjlTR*Xd9iP8`5Pm8suR&{A>BVTlm|yPEKb$2hhDMqCgH z7=Yq6Y(l|NoeR{L16Qk3uqfUVpc!r<`z^; z3F6!3;i7V&9wc*+OClIY{$>n-9__9Ixb97#1Ax{3s|U7qgqWPO@a~~I@0B!pulw$u zW1yv>6=C28WaoC95PX>690WH&+hl+~)|UcEn<&@87Kt*j=3!uOdxc}P3b>2HGJ+D; z-Ktk)s()53g}&n(<&<_bFd0|{in3CX{ED*>d7GLFP#`C=f%BR1#sZ1$#1qxhFDfdC_}*2IDG7v7pKkA5@m62yw= z>2_iqa7dQ{%WDKRKj)|oIpPvZ@e%|*ImsomJbCG>@@|`c=ntU$IpYbm24?OVV?aPg zn~SlI=f4(Bk$c#1<0t50zk->+Ox_?^Gj%AQmpS*C&!|rU1fVxP4{|?JufR1@H+9jb zQvwcRudtn$a|08+Lr7A$9#8`wPbAml@HSG&yb`c}XSpN=0eza9zX!C{qnS}-SQ5#} zyoeO?*N04o8_-Fp#w*|NKmW^TZ1NokG-ww-mw+uGZrfl{hh+2Qd_;MX{N!S80e34o zIU?V@`XqUy)%!m#Wzglu^OK}Yhl*Z~wYC#8j&zB2|0E)&MZ_#tRKn#s93qFpNr8^2q^qVljWY_ zWJ8Di;Nn+PZ;8A3XqNX(KGyuzOSKF3hz>DqCI&p2zWf~>bd#PRItquW`(GmJPXW!58)59f?fV;3UOrW)0>YF;^c_^M!$|C8a)iTWFLZZ zpP5A2+4z|wfhgTCdpUkrc~NBIX@?JgR1c-3GBf4wHdL;HT1m%nPA{F+sQ;I|>mEVg z)kx(FIvNX;tb#{#-F)=9M2$y4uX!1EjWzdp1;~KC!BQnMAp{s%utN983M`fVfmw6q zFVkhkVf~JpAp8(0(jSD<9h@l|3`EFTCpZF#LR%yR$FAyc;;2A&W(*XB|Vy?@$#Fi zgS;sd9|82XCk;uB_>5EOL6>fVsTXV9*BI<|hk-2d?lnGwD!bc*PklG91B8q)ABDmwK-fN5-Pzq})=3MT(aZyL+mSo=7O)*Gt)$zo9A zZ}Hym|B)U^3JhBoEq%%}X(>#@2B9V@gLQ}7z%U!058EF@)4>-Hk9O*Ldfl6w=0b`d zqA%p;F>!8Umc9FQWlak#YH8p^t)F3d*ecjh_^28<2?ztG$-YXk_i7zLPfX8XrfQgT zTY_K*?LH7sQ#!$nScoN{7^|0x|yxVTOYF zSkcPMEiXmPhvtb@0~B-J}^3bpoRDwg4Qq>63{_JdI>J%Xc)ll3(}~ zNWQsMU1C$Tm-PFz0RWsoxJ_HnYS1v{=2wZZ3jMj5cR;)oBM|I=Aj)CsXI~WKa$gc! z=c8`+Y{UL;`n~!G-^m9?#>w#2!E??>dKbGL+~#Xzy)Amn??@Y1^5sCMjjOxvZ<%iq zCdDeB$lR5#W{Q@<#8!Nj!(><^qWfArCo(*=EX&hk%L{J;HufVA@m=`*#b>^UHhh74 zlOHqZixq%Ts}j?G$qL_KI!k^X@bV|`f+OuI>?x>d&xKQKri(Zuxx zPtnw*)DK}6|3p{YT&$r;daRMcvY)$Q;0NGY0 zk3{X+oJI^7nTbJ;8$QwHjyJF|z>!b*xzf?N%rHp1m7@;=L3$DTF6yG|I3$ep2M}S^ zslj(Et>Kc7e~ueSKrcsce?J2qq1HeEj*>i5+KBZEG@a`E4ncxjCz+8Hr%&rv)N-bQ z*gZShh)OTmfY5h}{(T7EY!cT$OjenJ|IXia0}RTLj6eW`vdEyV~-x8oc_E~b7@{L$}=D^Rgp zJu||YK`=feorj{ni}ibk$|U$VjZe{r5}$HJQ|PPxPL|GDE(5Vlz5~2JcQ|k-0zA`@0ZxB3koo;LJ>ZJ-4LB$@eK=ZBNbD$;)1hRN?Od9bxt}D2N07H@ULJ z1TzY7RT!(?(`NWrGmXTBR&LBvYYd?ggEqt}2h%eOVbv#Zp$~@Ue1T_LRag9$~m9>tkOYIwpALXEv&!}*+1<#~ zn~~)qJ~-3Lkr8Rb)d!f1L=CF>$FJ0RHh_@h_sBn!pAom?^XtkrT!`bb;rrKG}ppS-bfcSN4j&u(`il3 zH#w#4&f~HbSUvTCxhO#1e;urZGtQA9OKXhC6Xqi7I57dnpSaBD$)qI^hLkbA!H0xQ zBm0h&C#PvkQnI*=$eW`w@t0aVExiVAH_gSpOgRO#mgXc!s&M-1B-YE=q_X}O4-L)g zen7Z$N|I1_PI~|PQ3lb$K_Lx3Mv+w|qm8lKe7z0F!Q>VKH`Y(wb z3)&&5gua2q*4zm&Q2L^dP(EI(j7vc26*bRWhv}-4eb(fKesEuw#6vuxE7&LN#iOS` zzca89Th#EU@c5xwB`TvilSw!|e*d{BGDPhD{q5DkrxE$@d9X%gQK;QnVFY8*cp=FB z3B6kM=rkeS2lHyWKpQY&yT84YP>YW`S&EkA*rd9L8 z1s|Q6*H8V7g^8CPxJhUFTONqRb$4M@kwgn#MM}L?XH`h{XwOErYGvejNF;O*-Cz7{ z%CcwS8ZeWR`(76%|0w{ziZylFaGgWG2MRgxRPz%n!oALuBkqlBei?T->bZG_Oc*%HmSS zE&W%qdwX$r#8T9Vj$u=l-2yFT654!sD#cd6@?v8oWNbJJ;yUo2Gqn$Xu+I;ZQ)Xy7 zJ#;gtV*e9%@J!_rGp3fxegfGzM{^xawmGt*_IWj4ENd{t%8e_hG9fskg%3*{U&Zc! zO!BrH8z(D7RiZm~_gD%d;$f~I!ed_#?N*@NToV>+DW)TWl?yidM7G|)V?uawhe%OJ ztz8>TY-d7P#8U!gnpJ^6B-7NZ*E(LaHi5v6O^**0_ zWm}_n&%Ly0ua3fiwsNyglJ4nBk?#dT**3Cgro0jIcf z?7_pmcTyruM8>N<lz1!&%fO)md?_J(BP$%2=q0txxjkD2I_KZu_PuS@u0gt-wCyT=@Y` z9*D;H+~uar5}qame(6dkDFd#anaiqz{Vj#GSO|g70sF5+tAT?sSL*45{L0oU@0I)1 zAe11#PBZbG>0hdt?G5_v^y zC77Zxda1^RQzEJLeKBay)rgQRGIg23gs26SB6n(X=?}(sn}#wR9I{5hEt*u&r)8P4qt z-&wXXn<%lycuW74%Vy=yFTVh`VSBj*H42hU9D#fya2O-K0|HP8;&+$rk=JP25wPNwmDM#7+QN4J8oWvwYs9}aDhb6=ls^Zr>m#|}LajXJ0U>!^1&&#Omeg5rp&wQ95!ZCLe zHjBjD#B0gholcWL=y1J!QIB|>{}k%q<(lv&7?K=~G$KX5$ZGp`(&Bwdy9M!bWCh@i z-g?v>vUMp&3Yt+yH(KN@8c++I&bv0-9$Y#?$s6+J{FP^krQYW?zl<;2fQ3w-Q8;6$ zZz{_3yL$cwf`p03Z!Xs#A9)`i?u>zP%7ZxNql&|ykL#&-L$z%Q3~^4V=- z>tlimZ_;A7ft;rlBvM@h{y#Ye5wbeq*Rj6c@Y=0TZn*QcY(Z_*gS?_^-_Wp)t}(Be z=NJop5h9uAb)&yck|wUbk<1K9h$e`=k@=@>(0t+9U)zWp@yc}vTCdFMy2sUZVdy%BNY3wM=PKcr41jT|L4s6tD;>d zhTg>be(Xp1QcK81OwP%Ee0Rknmt5Jn3Fvc3XH`=LGX})Jh@-u;s;FkN2UfP8v?G15 zr?L;9vHvvFx9n@-!-mq`RRj?(i?Mx9E1G{$$k`r0GIx{$5Fvpb@qLXL49zqB9lba5 z1HgTR4%a7bilpxjpa9JMS3*mTCr}Pe%G#KljpxIH6(Ll;_b`Z%NhP**A5@6XJJO#W zqd0Qe>wk4lTS6F~c($A?Zmc<-iTUYJuW-!|dpuX~-ypj}G{mpxz7O%VKq)TrG9GN@ zSxEaHcSW`fMAjLsV*3R(x0So!Ss0RizDk5-<#wGMHtwt}xa0Df`+r^-h7gpc^ptO2 zI--*4dd;}qC^8Gnsl`95)6T}yfiX>PKz>RT)j_PTmO}31eCw_OL3f4DprRWK(j)!c z2{(|q)Zwe_@D%m|5Q;-Zbc5ex{=AR!I(oTu6US0W$E{_tg`X&z6 z+q$C&*<7PQUggB$T9L&Su=zbQ@L@$g!VQ{EQnb#~*psEclHvcesK^$12WTXgfq9S$ zPJ4~p`C1J76G!n-Bcje0R|BCdpjpF5oZ>+`!sNv)XsZ?bUr5(UzZrVB`qx!7>er+n z(sK?ZdF6r2Z7mznpXIY~#6m^vniPl6>Md-keo{Uge~AQgtyUc#XZ!ZG)2|SfzjYtK zzr2bf%9zm#$iQ+&r$W2sKslT9d=+JgToIcqR6!sW#9z#4#M+LNcW>O78n;o@Oi6%C zqfH~}4!aGZQNOgUw@bPnE?xL~cTlod{1ex-Frixk2`;G!5OGDw`J(8P>^@+noEt(<5M0!RWxy`rRRm#wfWdH-_+v`?p5baoRxb&V zBkK5aG4uWXMSs9mQ(DBMbT2Qp_ga)WulKCJFKVuWyy;&Rl{s&W!T~nBzeKJ~ z@D~dL5)vT|QE0JN+qN13rE|@!Q*Dy-eV$Mh(urp&{+bnZX70}6X@GzbEQ;|HAriu| z8*2aK5dU>deZCs}NXA>er#sd~KX3Q8btm<0#~a3KIw5n{Cv z#Tw>@B2qKR3Gq9n-~4udx&Ts<(jLQE0RkQ43H&V~a?=H+v{MQIbCQ6fS?iuJz~T0O-b$q(W#jg!xO)Z*i{hi_Mc z=UBThxoUR9zWra6!LQ5Cew7GSm<)YwL-+jA?X-6`uc`zVykgj^M^A@5q;wZAA75{2 z->o$3zLZ5XwU+WB$2t-}ZP-y_0Vn75yUS&+HvsYt03I z1N4V_qh3~d+QpjCLlDqqOVBqzI0~@y@#h9H#{^~b7%*%Tl+9FVw>m)nHR^@4+%v2I zBa_CCvhVLSgzUA2{xEdBBrhr6>qci@+`y}#JPo-ynz}?u+&uZHTV6&;rBhr5Z0nWG zA~N?n2anlzy0-qgga00w`~kea6jspa4Xb!)#^nOvYq=TERq803AN@$**k)4NUVvDD z9p2|Z%Kn92<6~5=iV#h;o>6`qLkCE$qV6}!k7+>C0ZS88q)GNFNG7RP0+)plMllAu z2zA*+Pm@=S+B|P#2_D60C(v@8Ea(7`pU=mfjzgw%(8QKj0sIS19c77n@@*&SI300& z>C#RKy02x##y8)IH#gXWY!1o}wZJ(AG-WPnJ8|K6jpw7lZb+zPA`rPE_8-^!5u!e!lq&Vt7x7JexRC_ygT2i@pAS1$H)I5@!w^h0LjzI z@GuL_)dlHqm2wh&Q=0ZxxnQmrNfhVX?iZt!@lK<<$792Zr{JpGApPH-k}}@l9UT{Z z(G0z*7`t$p9!zNW|Etrz3|oQ>0=l-jHJZ&9Oi4>>K@x*EAda4{ilRYUgg3~183V&u zdt=K0FOyxj2rlnJ+gQ2QpS3#{3Tw^GD`2rL1)S^W;1@G@s^-Tp9*;T+fsBbCjbGn< zk0XTj!ARNR=dS_I>QA4P$kkr)7norTfqLJxC06NJ@fUryuAh2{&tz0a0sXcG=BDWMvR6mu7h_ zW&a!;3nGy5TF^BTRN#f{6M&y)56sNbhu4MvBm~zHqAPH;_o5vL`4V83OpdGrNeS}v zYrfX7*&X!pAv%n9ubsu~SAKyx1NwgJPI^%ShEPTdf({=za+HHimNMX+KLa78xFu6$ z{n3ViF6IVG z+n0Cf1zxRv%zu^_n$~3RThO0VNMBjzsR04LB!+JWiEpLE1IjR>vo{j zO_;Nn6LRHp5bxK@%gRoWD^J$~hS?iIOdhU((hbA_wI&mLT*7Umm`gE7LC)0llz8vg1wbt0XW?_GY4fPjaF~(P<}em*uI5?36&M?@vc1lEA>)-tU54$a@1V8x`rRPYCSXHzDE-c+&3j%zEwAWmNs^4G$Ck zH@7wR!I2B{D08#bm45Xf2DBuV4~fzVQ{+K~>3H>J0v)fx6VOkH@g-8xOOP}!!xlK z#H;}8NAvbWyuey8F)8fEFS0fBEQzLU*2f;p0uULn+kfNS%*^eGKw)V>2WITGaq?(b z6v5n3==s@qncRePKZ+<2J^zuVWB5fJFl4_T{Rc(;`;vqhoelP7?fiP{l6&{Cwz38% zN~lk~JaXmphz8+h>QN-Svs=rQ-vC2vM=B+rDk$&%npoy)*@{eTPfegFXtP6lDKX;@ znrIW1DYXke_zB`DCkSaWpcpU`(p|W``frqjdBYf+LpmcTX#oJUI46vWtYZ)FpE1PK z0n%MzD|CsPZS#d76G3;-6_pe6w87!RY{at+G`c3l3|qYPUVuN#4)9X7te-j?PY&8U ztgVvXJ9v?}p3sQELc3T*6~hH#cxv%7c{Y!+qzqcB9+}t*cjG)G;ZptDf98>ujYvT zYVc6mwKDiuZ!DVrD8=AyeReBW%Cy(i8K%5w9Vf#&RsNgUwvkDmb#MwhOcsR`hE1~?{>6`BA zgTTW+;|b0HTKVZoOcTPu8_YWCyZzj$?mA^BH#!6ZK-^^Ye;E7jc&^v?f25O9k&(*Y zGfJWC=$*1R*(up8dsAeEgk-PmEh`xjB}Dcfk?g(q?|P}u>73K~{J!Up^YBRWe%-J8 zzV7RKUeD+A0xr_R=$AFK}AlY=m0{ch5YfP&^26>BZefFCI13jW)7JR)tP_F0WaA08O4AF}&> zgtf3oBGB;Itw%t)xllhbx6I41Ch@!bXG*2j}bNb=hSU||M9`7IN*9#P-OwIM7kB7$1 zk&n2PI+imrh5b+e2YQudY$z_EKrHl0JO+{UPDo z&(U0i#?g9uT;dNiDJnb6XB6K)f24DH#-5p7B8uc&@FXKv7}gDsLSlp34LPI^Px-Gg zYJf$aV;g9#(N`CzCKOp|_o*6-!iKmHF5k|slLU9I*ib^qZl7DY2gr>^lKWG1EDvgwSj*H#nHqHY7Gct>3?Oi7%*5hKod*;gt~aPUST z*)64~laxc}D}goBm8rskP&S@7nk!!zfp(ZUJSDAV%k-2dFF-`{MggnOWG=y;bx&;#e;6H{Xb zZ?t%em2OT8@J%vV6Ja3MaC4<&fwoZP^=!}NQa*M;<~%S&ndkz1Cn|MO@&_=}gB)u_ z(W+@T*opXrg?4;9OwhPZBf-|pkUnhXJ#xF+%=L9! z6fF^?FG5)-NIsQoZaUoIz@+y-(V)_vbIyjo^AtqIzAFViN{tNZcmX+4F5=V2mj6Nr zm#?`u+lMA!Sxk@|npax6rd083!ttB05fAAtw}$&?Pq+|eoWJPWakg8+Ao4#xaR+)z+=ziW?=XRjwE#VEG4=>xumD%f&-)O!ExJ`bRE->H+2 zS=VXd>?|EN8JO(+k30F-cmS?D!%njMtf>xnt$Ay)o{iq(HCLb6mG4_F_MzR}4~q(% z!6Gi{iIWl9>p%p9i(BefHmXnI#qH~Dg54p)Ff0aIU%v!L#XZDG90VuqlX<)p%0pmq zqKv~2#IO@_OLP4txiL=juWct{?D|gd4Cq*34m@Z)FN!$Z*RCmjmZ64`Ue23@Q3N6{xl?A=AyvtX{GJG`7U5`6^>RzcSa8 zMoRk;k76KEsFs)w);LLDxkv%L?8kzu^Mm~iGO=XuAGQz$K}$#rtn)OkppC|ukd_ml zlKXXE8Jx!$vTXS<^@604j&w5b2O9P3a{c@3LI(5$_WBR%hK0AuHvM{Ejw?_~wBR{3 zc@KSXiF9lzJ!w9(az=*b1HU%+$9WiMgPS=H-af2ZSqQyrWb*M{j0Ix&|f5V_JXc37UDd<}6;^*Fl$!!8ss$;msk{1-IM2^A4ca#%}v7aftsA?wqFM zR5F5UlQw|=TpKuDsv*X4bHKwpIlgpT=gE=r{0G$Uzcjw{J+(*1Z*MWV0iJ+&prcd> zj8giS`H57w-^wk5G8nJ_W8iGQZDr z4ln;7-=c~Ues)Pd+ve6Q}}Bq&Rlq#7a5DEYO@GpTD(9C3l95t9{)n*QyG6gOVDSdkm(pX z+5X}!otv1e=;#j4asB{nE3;?}1r)Rj$nX4hieU`Gi1mj69_Rd*ceS3amNB5>&HkyW z`nBdn3P}Zm#giyQ%j0b4Is65Ux3X*vP4Azl?57SGnc34&c+H+J-E|(wP*3qNHk;T5 zyZu}Gz^s)1N_%!um)f19{OkC?GSP&yUU;`G>~D8ho!o{*JLpbwMXvKWD@=sJM_8ec z#wf|KcQzKxEfjZ}?J5b$c25HHi8%V;^GQL`C!JeVVD?KEx!>1ac^?_=wYhph`RlvX z$4iFVg(p&Lf1=zYFT63tC>&k_nfE|{L2@KJx=Uz(pkMBm)h#_H5o)vdB9EC!s9&+R za(zuX-p$6T`KsPRvn*WVW(~apr9#T_WP)y0PQ2u+rxFL;sy?`pFF5qC^y)9b?5ms> z{e_~TqS&IYSCtEbHtR!GEBr?^;?pkIef%czf{FT$b>(hHBufmhO#vbp0KW5tWUS*f zGatwnT_m}Ifxe7P^^UoW@m6>E=(;x=CKVbwmT){1X5vG>qIS(=7qvOn*Ax!_5Df$0 z{m1wJ_GKLvKCvpjk$xZJFk$cGOZ23T#X_Be5m{j~yOVuiBs&g&N(X62I$jq*HHDnX zgES*nS7k4ODLW?z5JV%mthi8vJ8*81___{vwtAePi^f1|N0sQ~X?dU0)$~$#a7y7vu?{E#XWSaCtTBfKFi5YIU}^{iwU2>X`;) zz}DhO^_POQqJ8t7!3J5+gUnfyF6bXxDIdGKF|hZl;KP|83**rzMHMm09>c`ZX+Ou% zFhz|l<=>X-=K^-{V_X{uGd)jmN4hX(q0%`!+eVyvJbpL+@UIJhqWWBDsK5&U$=GEo z6q)q4(e9;d8(I>(9ErRH{>r zQQ`*Rq~@+2Q2D}A2~J`bNkdyouWm{|oFZ$sZ)#QL!& z!r3(VwT=nW^fmrLhSG%`g~r6fCvpF4!Gz=8F(1}vr*s^>{z3WsDtep~6_#5V)!X0Q zT4ExkUY{s%;sGNYCD!FayAPh&7Wml7vkm&YFB zvgZNWXe*EA%ya^Qvn=wcjD3ifVD<95b1Ii>?OW~o`VT7i-gXo7;L)mSpf~wN}|y`ungOsxurKr5D0(cikmmoQEdtxbFK+3~(%7cacD$ zl{#-dph0_YPSE{hHFOD|`beW}Xe>4HdzFgW*RS?c{>OR6cNZ?kZ1WvR9BVCJZ?!4p z=##!2o5s-@JxMYUDe?1BVc%{BxVLS6d&Sggp+vMNO+s^)K~0<4P9puLg6!$*eI2+y(bgI{ zY2FT?apE3+bUz6ymRj6{@VcGE?qzOAa&vuNtpSBseBWc#zkiKvJJ#e*8I=X@Tt=yM zF+=c-Q6H<{$}caa<21%Lc^sB|(YTqFG;%bRZVHR=hLy&%yT&v#^jr4+k3e@%g34$1 zGDqRqXsPp)WHs7ZR#Ke5e-IB1tVtJCMC?Ng+Hlm1BZphLjuwA^FA!($tJEqlVbo^y zrj2vJW@!%duTTE>p!$Fs-_s&PDY1L1ueV3cv@_+G&yReWhK=cirb;+ZqYpoaBTa=h zTWD$U0ren4Lti(&PnKp7-`%k`YS$p3|Be@zQP)2 z728=EBtcS+nA@$MF=r6t~NwS9YLfwjwK(fizgZ^TOMNnEn9w72YqA#16TA)j8D zHxd3^Rr*`P!vT*M#Kv_BY%}sFqf29Qlt#I$)*0qy?nN1E=d7^ISm~`x72Bm4|51E+ z%RHcucQbEr-5Mv^fifp<=3LVwG|c7e!a`5KguCA>Di-HHy{*vu@*M7pXiR|V3&Pwa z4#C=6F zqs(IQM3>P$v-&^R@hG)y3Qu0^K|F50aSunwl0ed4tL*@L|BPf84tIidu~^yC1&&}% zz3s%RD@H+*-8lP-N-_088GWqnHkl2Kdrj9Rc6oW9r)es=()Y1ORc0)+g6>j_+s;O* z-+4vd=9uH}(*q+~k5W~3*YgZZm6EwhR;RvnzE>g9e~gA{gyXR&pud@u*kNQDxI;AO zU;HR$o9WLZ5&r^nl}}j?)1BAa{;swp+U$w%M~NEW?JikWPL@iR%GHNk-@?;*>iS3y zIO%1t^39un8V%3giO99V(Coscdyu(O$DuvrsgS2d6|5d(?)uuHn(gOS_miS>rl8Cw zOpXZ%)bG3TRAu%}<37YhQ=0e<`!5Mr-W$$Kl?HBkjfEUN6R*0|emx@@)bShRs9We+ zcqhMqURfvX+5<`-?QSj3XtKYyIn^nTMU+zF#kFKo8SdPSoh4SMnhTAAx0#n3N;tB@ z2aT!CG&6R^J9FRWzS!5JDGrHCzwUIdn^!JMakShyPGv`5-ml1CIZTbL7Z z7;jZWOpdj_zmpDSc9N+BWl83&eX;R#_G|pG$A7*6CT@=c-7E3hKBt_B9`QSg{A*8V zP7`C6fwaysuVJ*t{UY~muV}yh@k4RxEtg)KgiH6$m zR12j$x&{o~N|m)&Pi>2m$Ax4GWu1Dj^ud2WTN&HNnZMU&k-FEucrUq3D}9@Ib5xru zIF5IipLIqleTz^CZ@

p52M6S^K(d|4+F+JI5$CPu<*I=j${Vs57*b$$0Jr@7p)9 z(4dvZ7`(W>sMom626_%>77V<$^2)zQ0*KVo)i}5P*WJ(~mAXH@$Euu_N9np*eGc%L zHum;v=!CtGgcj;&jY`Bkrmw_2&O*aA3xI58<>x!5w~E{nrBbD$?jyrLky0S@cnehE zFdV@YaJSofPfq^aAU_xLueh8nLdBBSSa^Q;QvM0sm^-$ z%flqRvtqd+MVE@&oe%erE&Ap0!q*7r00^UQZpU~HSbWWsoR z-6x1|{)!WL%bC7)M;F`$?NBdtKh442ARm{#^es{EJoJJs&_@u)5K`3u2evnqSs@GZ z;$ncnwn-^aE$yNr!l==ZNwXN6u8Y3Ciij7Kfa59?Ng?xSEJZ6pQER22a4CRbrD&F5 z{`;UE8KMd9LuOz_T;>9ab7c!OrB(J*SS#PJPPMxPAT{*_>vFuaLg+gfV^B?t4`^o# zT|>J@cIKsY%zhwsA;UkX?*bpC=qT>#7l}C2)K$~nxeNMur!eVXm~`jF6q)vS-EVqv zhtg#=827knq0myG--%&|337zhBUy7xpfJ$c>05+;8xHfJTx zrgtRoYvnZ}oRjHTKk9%al(Jr!58c`2vQ3lr(sPpTF4^w#W~ICSGe^-S_`DMmhQ8*i zElJt1AvofGgdam^vJ|>)QZCSQjM;-h9BWuM1bxi(bIaq6nrX%G>Z3d@U@)TpAb@{& zGP?aRn9!p=&5bf|dfEYbOAWY@d%V3Ni6^BaFMsrg=aUQSD}i}RZzdD{8&Wmg_SV>k zLVTdj?aFOF#LowFKN>#}D{+)P?lUi6dW>jr&8()=Q~LnA`HndNm4tnn=0&N_PcMM> zw*W#)wc>G9?LI&>ybY$5Fn@Q$4Zw$DKrdTcYyg4Wq44|67<{kL$D2ZYA#()}=muVX z-V}~$xaag(Dg3Ua;jI_G@70f+6KMpGqdwr|j0>7p?X8=UXhw1W8A1O#^oI*BYmY(D zGxdspuR2eVwsQ;y#&WvYrnek-+x&8D2gIa%mpTC^>013vi8m7zz2;pHNYSbgOBdz2 z87n0LFkc!g5*+TJB%7h;>ot!3W=!rP{lvHep|INZ0#-UkSjio$f3Y}o!O zCSo`k31TqxVJ{!3Gmf8+9p(EhicZy$YT0WT_|52q01=zWjFF1~1WJRxOPf?~U&ASu zeZu@y!CLBctS>`<&F{?%2J9X zvmj9196+1vSk44Pw{*@7b$B_Hby0|`XIY{e6gsM7?1p-!wl@|Q(rt+AWQ=!#y1k0! z?D%LCkib6Rhv;!sZYIVliHNxfr_cz6ZQFN;1@9v`-iGD`88nakS^S29@jUOu>Nt#I zMfq-zn7;bNy|U%_ObE!$Yb(=_u4=N%Cl1xJJo_@)nl$%Jg5A=M4ofk(s!yeo87Dpw z-3+8o`;WKAf`n{I|N9^+Vb{_Qf20`-pX4QcqeGbeJyE}j64plZ@}3ppQxn@sn$rnM zlewE9)FJ0Y`4#)G^x*rojB`Vn_sz3YXw$hUdJ$X^o}#txqy*2Z`(f_VZkpTRQ5Qke z39bQk2aytPdXZ_99Ixr_Ie{hmoX0VWJZwS+I?l|`sn%s+mHzBg^woBh9Xxg=?un!u z*YX&vqxTafMNoU=s!WKO9n9|Xolhi>hS|%Y+@#(BdpByoO&K*>F;caUBV|0r(bB=# zFC(~Bi?AYDZ!XJ63D!maRTYd@y>Acv>{h(_b-=4;dl zy&&IDVp#z??#S?pKTe#x2hI?-cjDy#U}PaEWiA**%sBnX(iZ2gm50}Bx%lCy)q5aA zqsg6S^VzQ`Ua?OG%>VcnA&OaWiuzTc>-X`D|at~ODa{X6HnI?}hRcmM28XMgj zZSUqB!(tOzM~&uTBq6N*EUosao$q#H@pcZ4AfpCjWTxOhUjd*B?319|wEGlX5Szi& zpT3QBijwDin59x=wuxUs9!PAzCmk`| zwzI0;26ZQ(XE+s&Xf$93Qwiub7|i4gXA0y`9r(IyX~0EdbuBTmuP!X^P0kae5*S5Q zc5V$InLd4e|=!0Ct^wTFuutwAPo(~dF|=lKnl$3gQh zc3MU5Shu>lz(V+28P@^0W!RfC)k|_VBtj2|bwi>xk?vlxauD}YM+s-BVq>WPs4s80 zJQ*X4-zE>hzY5o{B`R0_f$fxFj(78sb`YBvkR;W}r(`xdV zxG^5!>24Q@lw|b>bsn8?#`VQ6fj-*3{#K+c{#P1KCEJKiM>vey=~Tw9Dc^jt{jQY9 z9xibsVh^fV1#;qUY!PcVVc+$1@)G}b)Q3^ixWCG|ABljPh;Vgh@xTL#AT!47XMUGe zt{qRK(nUP%*IARsz|*w%k)O-{)&zxx6mJ?R8M>e|g@>2LKpn&R#E5(xr~l6a9R!iH^9$e^e0SV5PvoxTD+ z2|GJrvr%A5$7wS!Mi(k&B8#h1|4>r*lTNPf#FgTadB}yDQcx7A?%sr(VH|@mxg;QkxNF<7 z`Z3-ao9sl=mj-TrBP29`80|sZ>;p)j`47-UIB4dQTn7!j>J|I6)>w^V{Sr3}O=D0s zqSzbYu|pZhXc@2davIMV<LwQuG&C?UxiI%@uPsLS{9A zaFX|StG{CiYp=65<-yA31DA8YGb*i$x^Ll&DKQdVc@n#_wWbY_vfa{AVJ6O zdYSv|kDF=yUhw4H*D0x)7nG70KgZkTd{5Tz=K+n05SePvRmRK`8u-Q|(SruiVaV`U)w7&FywzvS^pM?ltCqO3y8?6dt;4FC z!DR&;q}GmL>6Xcv^8`nyPx;C3PbVZsw=EGkE4!^`6?BT~pIAvWo;uP)#e00B`<%Ah zUJ(liZ_+mjJyJhU4;-;{u4H+o;DC-?7TzEc!!fE_Y6J0&EcfjS!3GO-7orpn&vG*!xGV9{-I%7QeuCZ@U-njq%`xmy|}Z z&|4n#4SX=$kiRAtBP#}qu~>mMs7s2$ET}(ek2xvhy1Cyz4~V!AB*j~|iLz!vng7Z- zn-0x+GQnHpo@(Xh0S{f8!g{A0tjxG$7Pulfp%p>a+?IqU?qlC{@q;~gSM?(R6)?4d zz0BUM&>?h`tNk`I*KB#5Fo3l`y&z?OjPhWyr5kZk#1kf=IVMh-ywQxz@d~N}x+;Z^ zG{)PL&V<;W>mLYuKbY3^6$P^* zP}AS)D37n0FLCAjG*QKGpaz!I zW&ieUKLUcV4FkTXt?x8drNrga+oRmN2^Zl?mtDcSjff3Vt{jz92cKSCd}oR%6G_&#Mp@&C&dZ)v zE9ue2mARI+(ZWJ7daM1VkDH$wlN+Dytbt#LN{N|fNvlG^5->sfa5>6HbVc+~VF$}H zhEE8IUG`HeY);2$qpo$*`S1gwB@0lDbwCLm`g8(5rl{m<8J(_}wDRR77e4WiT;|V^ zi6aF_zDV|-D4!Bsul}o-d3x+00ZbF$M~J13K}R@%LfIz7WSqti@LtOhpgQ^QeDMk6QGx>c;$_rgN%dDM8IGH2ZpLZ~Mza$xQ(y$$NXy{uHRQ*c^~{0cB6#k&{>6 zsudZHFSHV;%b;{CU|M<2{{BYQ8yO=Y#J*KnhC-oYypc68!Z|Q#CJX%j@{nBSQ5^H$ z5;&*4P((X(TJicj7_1Z}PnQeMDz06z0#0RUdBAwAZUXxj193SGmT^1Q1m*3PFd6=( zZ&}S&^RIXG^@1+`{rLRcVwTvGv@h=krEN-^hUt)!X%@z#R;&RADw-JFB(w?UR6RY~nQYTF!NsbKx&? zAUbxHTHZxG(U_OxObSzI#=N$W>QC%!`BX}jWV%BVN|zTk>VF!0j%X?u@`(N0x~d9$ z1fzzg$w!`3N=06_Jm{@x*va6w+ zmsiX=d($RIUytH93tUGVzp>Hk&EIOH_=LpNnBS4Sq>~FE|FpA^D(L&zr`pQz{Gg|I?{^VkO>!Dl#FR6{8VaQ>jYM+%1T~3<#Wj&7?b<9IseD#HG+D-`Z;!@IOmI;$JeO{N zw2_=4zH4@IUDCG8B3EH6)FfwH=`v5&igAm1%j+!Ct_D4`o~unKjJ;et@38fe^rgQq zRL)pZyu0Kw=9(Nuy4(0NFgNC7v`F8G_PD{j-1|$J(>oChYqxVRYq<@0x0T{m`)}^~ zXdmM<`i^G{qVJKCyVNIHkdush?WajiDv%f9ac3a-${Kxq%l!uqlShUp`A_+*D{0f% zpZv}0QDb2yXfQaTOj|%(4pQhe|IO#;)t`S{#o=x(VmW@Doe2Aal5gMwU?0U3u}+SK zZza0|b6C-nzKPX6Ze41Q;oH$K0GX6R;Tr=}mm3->sm^{bezuVq=4@sk#WV8nSxtHi z+t56n9)6;Dm-jAnPqWt2rsmUPvO`RO|W!;a#aO1BScTJCQ4KI!H)E$8U884z^LwoMOfSk9bYQM5Y{ zA3VsYbj$iz;nErB-Zt}G9$Rkz!P2d`aR$NzLxE&^_TJj}ij?9h z=Q0^SypzHrdg5MrTprt|ARu$c{)$SORRC`*>?v}xk+;dish2q zdPrK0BPx7A68@mNs$r0PQ%S~ktI|A6?=yQ7uJ8~3n$x`sfATDBdVj$nj-y8S@F z@gA>o`)f-LP-nfIyXx zHF?5xm0BpV`(vNn8w0+uidpD43eQpDW3pwuqb0ni#T(UkEq6Ue{=hEHvuyp;NG*OE z1!hgvoNu`l7sHM`1&7|Sc#Njfj>_Odz~4t}q{=g6{cn4>hj zS4kHntjT9ZV(JO!1V-Us!wi)`LY_lI*8);xzNUPHs zc~@Syv6fC;`b;vbc-R&9aj*33pbeX3HzJu8CI3cWAV&$|;Ab5jZv?yxE%>!;y=2?e zvg+8A6#?RDA(l+zOY-a41tys*y&10-KDwTqJfydIG@SapFr^o!{;>DYf8xg{cp4~- z-{8gD0~vCrP+^h{_j)T=4gcZ2l8!am_-K;WXCcugft<3vNZgvgQ|rk;xGrfYFjrp( zil-UBi?~XQ$s$VSob%M^MDI(MCo1G|)+RREZM}-N<@io7SMu=%?-h8PFyZ?Ud^Rqb z1@7Qd)%(DcpZ`&^$cA8HUW{obGKEj%oo_%+?q94&SS5YAB>+K-sAO+pY~wSAcozq2 zYNV0GutrV(2el+#6|-g{qpmN`g<6&n%d_R6Ee2(qE?#E&NlIT~cFB2DHG(ULcZKPV zb*<`oxjgyP`|@ve_zF(_EkBVp#`4576Ulz$T|5mu7Pcqeyn>9HRqqdxN1SU^&S}KN zUFjVAPdOrb=pt2otn!Zi?dN|vf3ka6o(n1&aTE5@co=$)=$?4#EBRE$e?JGw*Qg>? z+3iW*zq2WD>@Cn6pAxSjt+&s9?YsRf`xbIPdWaDE;7&i&Y|OoP_gZOqeah#FJ3_yM z+kXCPcfpcof3!tk#vA&m6+hNMNrSj)+yhV+t>#nt6KV{>-M( zFj-`TtL(gnD-un!qtk?pLbAx?NOLaTmMLbbUhig&MOvbv6dw0z-q=_1t7xZ<0Wm~} zaQecxki*8@&tLwJlO}<^3(Q;BWf)c(PPhqXVy4IxA;3JFPqS+n0E4{bM@L)UEa^RB zZcNT+TL^-6#Yq{ve56kfoIEdws@-M(h1RUwm(&aH-D+?l+vSaON2)QkLW4U$aq)|vBydK81RYgx>i6Wm;*x6HWs`l%6r zr>fpJ<<7FOCFah7>2Ky4uU?X}CNEhJjI(|}d*b23n|2yDh2;FeCtC1t#+|#j^3Fbk z3(6@kn>1uiI__ayyPnE>{jL9cdiFB|wpBy^R!2iga=*Sg2ZEZkwZfE97{YhxItAfJgSgCx%AH-g1E6m*1T+Cw#BiY?;yyCm9#Z}s zPJI~sI4=Ris7*oeZwtiGEM0}Hzye*z$;ko(#rn^6A5!jkgp8uH$h+bXPrOG5 zW{p!(X(>C_f=HDz_T2TNS;Ag>#{Ynv4#SpgHg-q0dN=SF=4y(dpxnIO(9oc99BFGK z#>8LUzLF>Ip4YA%@N!2K*D(NqQjjl#wD_{`G~m|)$b@VY5Jxgxix;#S0yO;mIn4{s z-`H9;6w&OVhq5!&$2eHIEE9~)2sI)(uQkHj-5U7OZ|BDu4&<9Rh;vzwVCL066P_n4 zC_Z_nHqh_ECUgWGu8a6$fPIXU6THYhGBqIf3Kao&qxUM}QPq-M?+Dp(NQ*-D&4+3- zJCdL!LdMh};-MYISw>9vy2>N7aI1&Xq3<0F%XmpD!xet{sX}ReM)k0fm0T5H;2Yd3RNgG%?C@2Q8H$>z{HWYC z;kc#2dg_fxGfxSdTUCWGZKCSZn}D{UQ+^hUd0Vq-)a;Ru;&8t%iT483C_1 zu8F)yFhfxI&O-CK1cByld-X_2m=VKphA*SgdmRft%B77;_k{21^aJj%G0Ee&YCyty z^4#TmFIRB|=T0S++Xwr5*)}s>U1@T(B%*%ruodb)Vy%UoLKV^@ z`|s5Amnbam;Z8HnoSbvrNxY8KZ#yaYAdr9gCQLF)jd9(cDmG|MT&{N~ILgwd1?2Ja zSfB#=f1#W02U@|q2gF9cvveIHXtG6JPx0lDWql3eBvSzLHtL;V&cSnXy8+7-EH}U7 zJ;1zng1!mD`Yr|*!ww9Fn*}!yJCC~|3{sZ|8v&Bi0d;8A33A?!mc&Uf1UjCIj^w!Z zva1{zD6^k$7Ib!?nonP|b2D3g^PLxxv7Xy>0>K7p>2KlzH|@OD&m z!jQ8RG5BCi&N)PqLl~gf7j6`w@qu$t-S&`q<{^xa@|v-aG2=IDv6-X6*P^ss(x}Wu zam^}8*0dfZ5pKyfjD;;_b6V9gckX7KktoqGTT&EEj*2oW&*+jWN`0@$*~=I6DVKHW z_`hr2?{^ZBzq{Z&b=Z<~fOi%n@*C_y+3NxVV1hM1T9p>u1}uW)_OJ`{X-R?0SZ?Yj z`6!2u@~6pcO(7R#gO3v5T0r11SeQHwv@<4(jQxQ6o#xOo`4q*o;W*uq4g>LoXJFFa z1~XSK%`)@!FfwlWkO;FA+RQ`by9Z<~ZP(a9a=t!rHU*2VldoXKhdi)0sfnGLJcC#<22jVUEjmuEezjElg2_Bb$uh!5`Qg5;}6|3 z_rtS<+P5P5#>-@^W7iXZq!&h<=oPn*n`?;jb$k)HSDIs@8FKY~3xhJ`LraQQXCa!? zJ)O3jh0@$u?|r6`nO>KaT}ne%jG0++ZjANJ#r>+#o0qINAeT%dNVk=cg>kx(+?SY|Ls&tIW?&eagYbije z>=*bEMaXT`LCDPlBEL#?Cf1{3i|80!F~S9IZxbijxk$I3H>H|v6ZSrr*DkQSKYEuY zsv#@f$gH5^viA?ltGn_aaBF@tKG*()Y7(aVfLxj~&Al4-hVE zIeAzdw7W=;YiHzJTN&4sHpf2;e3z?SSJK$gGQz?2_pH$)@}_X>&OQ)&oro$nw}{YX zPOTAT%V6+e?}`LdS->Dg_HS=MoK%rbRGOJx9O*yxP&19VLI$G}SOI+~63u&v$aUrd zMYh5mX_3OVZZZ|crTb6BW}urKfd}1`9_}8UBQjiMG@D)y(DI7H-3pyn*}u740#R`1 zDu{m;41>8;Cd@k0g2m45LO(`h%f#0IoXHkqbx${BN{y?UojgeX ztjvV>{Lx?jxe*l%;7hJ8u%AA!_LZDq0txZ$qbvn+m}Us0vJ&RNsqbyC3DnKQd=e=d zzU1mKduc0xyS@dlc63EENJYM=6pgr~Fa7$hOg9WlEQSPNs}E4-3)#t7E|5gpsQWH* z_NqLr%=5nPNU(FOHto_!MqdIYM4ia3TZ@H&ym4|)MmZ*(&t;``QCn11N`;S`Gj6&k zJb__dq4S$hMNcldaEa0T5QROqhm;Ejbmc8uSKrB*jymA(VJBaTvf$)>o^hVCmiL}6 zFg)d1tY*5hqk{ZDE+}JcYQK2#K-sUFxY057jHZo2^Fwu&xtGWOst%ER`|Sat56)x< zaZY9#`MR(F+Wzo)mfw^K8CN~5Yp%Ul_z;+WJg%XsvoR-h0-(96p5&E^$b!l_=l>3WOt{sk(z#5PPjU}`25wNbJv<;@a= z>0+TZvas$;SU=5}4xxE<=iTF+rcjnb=%g5yI0BI^3(`)8BBOF=;IL2>=L3^)iCQI6 zyfuUW^Dhi!NUCiy9Eb5qK6q^qwC*dKUX`m$l6~^A%b+tO%&-bBPN4+*3zet;*2k_t=24m^ZG6Maa?8ZKXw20(h-wpe zzt*($XSVS(`TFw$GZOT$PVBKBWN zyn2#|u6SCdte|Ye&J)a{6gHJ@1$=j&`5qJ&%VUMhx7s<-)1r|Hxt6a@w!W*g1$V*# zMB6UPoFM?Gy5%5CqX8%GXHtEyF`T0W$f8dBV=Y1l14!zn@B%d+Is`L%B)2kJu1=5z z5p6>Jb)?Ym>=Sz(P5dPn+AOYjL#-S^Js!K_0FCz5@=qW`e zA&!sruvpT|x{pPnfANkA`+C?F0)_u3-6D4y6Q{=_`87_SwgAQ`F(FLg-Aj5tnlfn% z@Hgg-Y6v759m-bCuW9&dz`RR)35YoAP)SLRzCjxPM9(ad%L!UjJ>Wnz?LvT&0itKp z^)?BPUZl)|zf%|_!DNs0u^P04YR*Huz3qflfJ#{MLzo&Ap!Z_7E|3PW5V z?I|WZC7;8q3c4{P+1Hz+K6rNQ zJLvs)7~mfujS(-12kvB8-3PUct5Sd_uda#G`55cL0(p;C@Wlzil1(fJ)@S|xR*TzV z`>>`QX`ka!U5odlLGMae_|X203mRuIT=aej8j!GfR+(IZJoyUpwgJ%mEz+oeI*VZb zM}f$p4g;54kdZ#?Yh~6`y1=-!YP|#h*W_nH*MT3J`v*x-I05TAZHHmY;*J`-4ZGE- z6N4Hpv^2a%-wFQXF@6o`zW@X}IdQBCL!*oIRu>bj7lsC<`IoLofj8Yp#^i5dR2D#J z&IhLRYvXBu65rL+Y^ozx5D-U2gkvV49qSjR2BF*(WQ;=`*Wzi8W*|lfP(4q!9zj@J zci6QJ_-KB1<>&!6=kY#>6Fom6!q?%uEgwK%Ak#t8RWC-aExPk7`tcth_4}ueAb?C^ zDK0wfj7+*Kh~^KQzH{D`u8T~rKq}5%12y`$<{0B#n2*$Eb?b6kBCm4<1yA5NJ+tA; z3a`)_aYlY1n3M9`=OHBb6al6CpCZTQj*iV({Gf<4aii~Plk8XF7CGb0*~iSO3eK)n zJnl;RpS$GO&%DTrZaCi{e)Mtl)yut?x8WC zx1H^2J)GqPIhqOVfLF-TcyaMAzmf$wML1KU{unTOGq+sKhrZ&?Z`O4X94(hgWO|N6Q z0%d-Qs2WCzE_YN8yY6J?W^RGKPeJPbAdJJxf=*}3#W{|PqX0b`!k9}>WO63xWp_OH z+pwYZK8fy|M1cKrGLz{!t{q#PMxrsjvGMS|yRG{pl>bQ3{tU{$F8Yr>opv7afi+h6 zT!znfs0R%i0wYUG?K121Ju|(AfdeIF=U@(W9nM^vyr-i)5mM}X1na2w-5VRX^K?9L z5BiFXRp^}6*Y$M6IY{TCa*J?YOWUlydHRIql}h64h;{lmJiSfb=l*t6 z{(m=!qCUoJm@>AzB*vN$N(cXM=NX8WC@O;+ZC5q>XXTgbLs3dOayWvM&S6VQoK{EFyRhh3>^N#z zB9{5Tn2XNtlkO9z5Wi%{y>o>1Rey4LOLo(0Vv7|0yaboB^#{wp_RIf#G-c|iQ{k&i zzOS2DJIJ=wI-T^&To04Q`fpsp9~~~`f82;wdaac*5sU}xMa6^RCzvV^mK)1Q!7_o= z7~vt=DDfcqV!G8zQA>ed<1FIBFlaYiQS*#9)7lnHF!SKss{O^LtWQX0bhnnrw|vo5 zvyS9_FKPe#)1@(EKG=H38-8^(tg&3=)5p3hk*(*Ag0G`fWvz_*9BQTL!>x#QrP-ZA zmmSD#DeNa2=x1eqN=1a1u%ZD|QBIX6C%vfa$+rafNgs^p(gE77#{YnzjC=CFuoxq4{74>iGq{b5V5(?8SGUG1KdFnnbmDBZHJTSczheA* z(edY=Hu}RuqcZP&2iYn14z1q#kZc$GqOY8}vQ{{CZ&G3yRm@&DH$K-cUVptEHdC&< z8QvJn`)h9kd1b;VF@lCnj; zPa+}-L<;YMpmW?XM4oa6alr$2Ns5mXNM4qcQ$WQ80-jjN__6zb@0;7h4q+ zk+h05eY6eADCX#RZPlBa0qErR>uBomtI4A(i?o{Ac-;RL6mN=+fD3IQP}chWh*@WW z*jo%Dr>vS#u>b4_rj0H#9s=}GJzxsTZu~jk1Mvz*m_NNh?bU}1r}R81*(cWFz5y^y z)CuxeQ=lu_Hea3Sd50%tB5vsf{I?#E_^91+Src0QEE=1ml33j?&G!lA409P@Otk_C zQH4zE$O34r7;x0h*hpIH1m`{kMe}6R1|K?J1_?h$|OQnwG$x-!6z?^*rhf7SKxmL@ohjt)bWG)O& z!D6#s92|;qey`||MlCPhaO7k#Qd&J&Q*!!W>QFQckGF_;hM;cBY63E^EE=@(%~5}3 zvK=Sts0L(v0^-An^2L^*D;?x2>#yXJZuG(s!Nb6hyFix{^4Sa;EbDb7+CAW@KOejc zS%t39!LC)$5ZK2MBfb%uc_r{ZN`);zF}1vhxb+-rYqp7~Wu_d7kIJqPpOqC{0MM6# zFTi+zw)fQ>IFu|+dJ>QrnPX$ni-hihnOx(>^Y~wIuxv@0>u79U=XJD^1enX;eP8C| z%MQ#pkrsapW=b{(`%3LrhA&4;wrD=NG+vV(+Cq5d+k>3R=zj~yejjp(yxnQmq#y}` z9?QhZn~EE)VwATbYVE0s@Fn+zElLls-NHnilN{AR`!v`MV|1$`Fg*!+ZrR&GGChe>VII4{2*?tkSUIW7h!1DZeESsT zq&5sfFx8#)IjMAiC0<~*O}FzW?e#bjvnP@TY?jhi{JZcf}DGnPZqT^v5az{ z{ubIt;k77Ma;4v;peT1*pkRqnp+8bbg|HdlW$vCM#ixYdKk9G(RF)5$A9f=9JRfKU z$PfvYDYT3zuw}{p=1ai+xEBUpPL1-SW)NC@KMPL$EU}A>p(pX4jBcgd!Ok%IGU?R^ zFA*Ik9g(8n5F)f{U&%O$3f@NeXomJPotYEd1eD^kX$0Lu&|~7blYzT|q{(1%RC3EC z?J6OIWFCyZY->NuS`a7x6K~>Hb-MW^ZMBZ2V5&1Syu^R+<*Vo8?|k=m$Rav94=VSz zAM9LNT=pOe5NTpwrFcCXj&Vx#|9IK{8MT+ih0moTfdh(C11P1}ijdJ?GVM+SmLIl? zcBwTsD`z4)KS z?{D&~!<5ovl;CMI61>3oX!pULSj-0m>*+E@ZQJ6q-7jwuk{oaub<{(1aZ*<}c`Sy< zJeJG2t4ZNVX7*Kr)8b7j1}@^C-pIG#CpaAlT^i_TrrOsaft z82^N8R9$Z%#7j;y|M&oj^zPYPNEGWk&|)BV)6p6E2~fy z>Fz|=ksJ-k5)Wl~uHGH_hO@uLRR4Ulzk7D!qZkC) z>7HdZI?w#&^Rl0`5Oho(C)oEQ*rprGgpiIqMdzKH}}{XcG8k7*QD z28U1aB>_~zxB57^4(0@V`;HYKrf~KLj^RcG3ntC-gC?5=PNG<$w47P8T+-=5|*OHm%2?R6!ktanaxqIY5fQuzYff{PO5-b5HY3+ z(vn-7>+e2!Wd!XF+@Dgmz73Er7$IQ{+7Uvn7B^hFwcRR^w5b??`Otn)q)Z|5Xr#As z>$)A1UqGpt1xegEcLH^O{H`-7u0>R;rvX6St^~Y(gXMMV_;rwP{_H3dH8FDklY|Yg z68s)oIm!W!Q>nL=E?!?w!nZ0IOv;#(SQ~h~o$SyXD!F+dI9!M8Aj-W#6V;7V{I_TQ zlWVX=8;@oU9VnSn9l$MES4mK4X?P8xj#t#CyRwa;qCRZXHe(cr*@6UTk&v zA?{xE@$}=q0tWjteMOnzkERK}+Cn4;zCr8VT8OS}7?=}Zi@}0!X=d`Q;jev0syNI< z?I&XW5%Y3sSC>;|o%G(w6t8=jA{pATp{k+e$oG8xEBB1G)4HV+c5#F<|4ZfgJq9ac zLd~k*_L!O0tnj8=O1|vsmTiQm{cGpXvdO;O#2MZ99CyK28P)@A*6 z37}Q`|F5#^j;H$V`w|rmB~r>rWJXfI z*Eu7K2O$7J7C%Bk@k40D{fYUA57U1sKm;ui*<{ioxcVv=sOqch6ZaV7jDfmOIDSC3 z{1Za_Y^{^(oI#Cl<<6j{l3EM&-8Uq{9tD`LE?h4+pv`!*l%60|O2JKLobk|i5MKN^ zLt0dwe2!haxDD}C^dbQV9F=RqxEh672fqh`oC#*o1$-Dt>!W|O!5EQimqCc zFgH|+X^&nPId}h$AK`y4>;JXbZLZQLjX>t)kJ_2k7-h5Gw@K!2YmZQ;va_)*=qnOD zU7mKYO@i~cA0pmJc0fZSXcM12f+RFjGjI5JGlMg&qvIgX^O}J7V zG|P>OO3a6lK%lq3AL#^&NsCAPFdHlDYVj-rHg+%_fqwYVW*Q0FE=`8X5UHPluHtUi z%_14t%*hFKCy@%B^i`=rrbFMMcGmb{cS^)nhEsV>^o=n1F0N9j!XG82CR4EBCnhtw zL2v=J407yk>paH3_jGGleJ6RXRy>QeT1Lr~!*k9WTZCm_ugy@-6RBjUy zs=}3fkLA?EGzAVV_KKgPD4^XKs~hU@@aV(uiQiG#o8WL?e&i9&NMoF>Whh2;g5js~ z85eRQyl>$hEBb+8ek5tJ3;hfBy~!^;fw%c8(tr7Q#gxieo*q3$y{k+?0~JBVF~ zMK9PcvqlJgNgz9Zj4b^2jb~48a1<8F7lQ`!cUCKMhk?WkX^!l+)H_R|&CO}~O0REy zYLyHUmcG-MYDz4@pAExAxtH{vj+=QWay(gD%CjKQOYp_M0?{=7+dgo7m z`^EV;4h&geOS-=tzxHxVMFfyni_KEssGL`XJR8!qk(@#0f8a5Rd(|TXaT~>x!aju$1D5 z8swlq1-EZ!u=OnO&+)b`suCA6j>&)@>5MOh?d zq_3;rWelwW>vUgx3{S^bJbN%&bfnN!A@mE}9>Q+)8}M@d1w0aYJm7|s8lWHg21D)C zh5KAWN%+Cc)6vfhCQ_tUF-RNHougDLRk4L>NYQ)zoeel1ZOpqNV?|R~@0&k(c}mn+ z2^@~^_`@w(!Uqtg^%F%sI<~UsSsq^#(t_RQjodZW8Q3-F&Mc7CbLi0*mP7KbkZXQd zs?#9s$MFF?7TI$!r}rY@RjyXiij;keM~+G@5?nmv#q>Nui9kj?B4=8H<6HKZv&g_+ z4Bw!fL|TN6hTQQd{3OhQ^BEZWX{4@R_gJQom|17C=K@Tm>@85(mS7I)yu|?dM!g7h zIyAGZl*=;u{q2K zx)i_|G)?!>O7%DMsS5gWVrv&_PPqJdTu)Kp8-jOopWB2(n zvz4|A9urlhZ!*4gOxm%hU;Ger@wy606TZ*j3Y8nAU4On)Ov?U(=~w1?S7@+$z3P`d0#9D##zGW9Xtl;I{WAyuuq(BskUiFySoq3Ul+PNqc zjHU)x&od+~H$F`zxdyTHB8FmTfJFqK_I*>=U$~%_=BQM2>)|ccnp^AOW*aakTf2Wj zB`iv{_6ySFpf6V2insugbZFh!jQMQRlhU|Uy0|bjHv}M%1Yhm^-WJ8R0BO-!Som$j zPIK4F!C$E{wd;6c8VT`VZQv4{HUnU4%tgt%C~U+ZhLZdrhp7B zOD93JrxF^Z>MS$O=o*e+$y`2X5@SB8`7ps@l&HrT+5L-Q1}{4&b7<~~MV$I}Hspzx zpYEi1B{<}AH!xNS4-#s6+PSo;60oR;eR95=F+J(xkCUW6Q}~jlB}wgO&^4~VUNX?3 z6A}`Qx`hho`B;KZah77r6egM#B^s@@%GDCBP@r!+E}Y4+N!ytW((uK=1_s(Lu_?dB zYX$MtvwnFTBkKz8yx!eGkqUv2G(oGgSIVqpuMNe5GW8)4ULf`;n9*S>eZ)Uvahn zEkr3lRZR$X2AVEC{S6B1t z&q-AQ$@}-v87Hz~`>xnwh=JSu$I#Idn3vc!LPtnRq-&U{R8>0C@8?lQTrcRAs0z~| z{*F!*uXmmS(Z6h0ghkw@12<|O&ZhL(E{NL6p%pxVx~k2p=^mM(g)ho%&LACn4Bp{R zlCmt6ZEj3Z)IzX9oP5V;x@vbt<`?i;cQq&`8TUrYI0K!2`BVOl zRB2SI_wz6_K`m_h^JpO`Vyl|mTpnwcu={fT-ucYwXVTwTYjwd|SCneczuu)RM4XU( z6p3V>z^|FsYk5gom?llYL={^13ZdvKA7jeer{>K3YZRW>ecmAS-K5ud)34h@4R!lAF)r(~ONf-rP-}^> z?;dx)M|)l`F)8pr2QU8X760MFlDS3jd~q63!2+MNjQs!s>i2LW0nl*s&F91OnKh_X}BNx%9+5NOy-;d0tD)#0x|8z3{86M(0 zt*}-BJW@XZ^{u8dsbHKI3Za!NQW|E-&7nI$TI(opKKgFR>yN4;a7JRyip>Z50m0Yp z%?PapT_?{&rZ9h{HLv^WN2whW{fEI|9em;PzNcX;9-$XWJ5%ZJo8hJk$$n~ z6(t>t7t>hP>O|l{N^FKCXvvuQsbZ^=FR_q5U;;#BHPamlc3wL3F%jjPTads@ohE+`Z8o}cDbN&Qx06XkE;|B=T5jxAE3m8||%!+FhbrYv3Cz($js z733($6?Rw>WGTH?4?`W;;cNY+Z~FB18&au^%UQ$)dcE&hy4h9R>o;Vg+|S^h`_BG? z!oI4w1?S|m3(%j>MjFXq`^02Q(T*gSm2fj9%y}2BmYPUV>JZ`^>ZUV3I7+{j{qZ;- z4eO*)u1ii(;Jdd+{sW)-@9Of@rE*C!A!h0~!7J#WGdWAY$YVTK; z8f@z3h+Y;qV^;4=&W-A3TMCqv+y$~2u?ARN)O(>NhO#W~GnqiWXo%&rMIsJhjE?xM zYGC2nPDWvnPj;2u3rMQ#d>{Qp=Yx`1Bmb8j?hl*ozxi!q!|5iq4u;CFz#V{--h}-8 zF0kGka1R-vMCBL&l~{#=L$Hf6S4C4(vosgI1}S)V9Q7a7xpoR8g$x@D13U4s6Q4@g zL&YKK_DG!zqD}WKuv%{AsO|-3?jejJxTicRZCt@gtR-AN9$T6Jg0 zS1$s(t0xJ}G2dY3a{iU~V}h{t!v_fW4z>+6lHiHRjy8bcb~pfqI~plCgaJ?k=C1oa zNjkQlZgqpq!b&r#+Yg!%IWUSzN9y>Mhii+o<~>q6T?@^;xBg$$@y|=~m*`;sm~qOF zpar;M3*GJmVqdG4XeJ!yig8yYFaM|n5(2m2NukHo!)dmUz}z4q(Df|+K{xBSjR*`7 zUW^39=|9+yoG8vpOLGDYftA46F95X z-g8-bjeuDq1!F7&uRwS~?Y3zU^4%ItRN1d6vc+;%NDp17IT{Iih;RA&BPt+G*v<>? z@{?&%X1=RRW+v~7IDx7kr9D6x2BaA;EP-?Z{ww=*iWi|-K%l*YGKD$>JAm@D_us$; z)zf)=3}8`g?)%|>M0Q^e`4FHYHrOBS8U%E|710bhLq^U#BGILo1aQv+lHQ7e2H#po zPGV^lKsJO)DOqg3Lzt+}L@98>Wh|Bf6ud4s;hj!@dH}>q?OZ} zoI@Dp?OfGhvN8vG8V*BdwE*&mx3b)Qb%13XBcP%=u}EI*l|w8}iTKrKmV~IY9!MTmk9^vIpNG0-Q)?I>%-UaYRzK5rfC;F%fn5FHL3iKjrcy0jC#1<&6 z3j^Yme(V{s#04cB0+PZ&P+m%tg~?q3hCk|s+lm;+8WNm>E!H~*^PhLdg>XCt9wF6E z|6_sTb_Q%+QN5{!Cs#dWi!qV^)EE7?3*+DSu3}0 zJRu$Qq}%%nA8|Ai>1RI#GY@5IY3a1 zW1q?RCA@QDN+ayvwI6nDtoMK@U}pWhLF@FB<-n{jfTVb1i}Y-ZrCBR7g_C?noLe}_ znU|_$Lr76mI-aGk9Mcejql`tf_g!-NSsRb9>4J?<-kcF!-a_PmX6A%xU$;{A&q~H^ z0RX;HNk(M_?5DMEjD2X0acpQkhF_(DM=_5O#~#LCjD zMEGfiq5R4Z(xko}*W`GeB<3AWmQFe~LwKG;OPyyr(c@Gh7@UUm@d|_ZRXAuTth+<8A%JkqU}LK7?jI zuowz-7VQh{jOZEWm(9r~XE;BL`dk)09ejj`xi|5mI;ALg-ibryQnY@-s%#d9eqqJ* zo^}SxSFFjq66xvPpLQ<|3U2ep*$%G`4_{m#UV4ga6I>mOU3CmB_l`m~vz&Un;8%z; zt<}3VuxXoqb@}qDbr4pHj?R-_BA7S<-bzGeQ@U`t{oulw2YU#myVv$qe@}@l(aGE% zp2iY07_j;<3+x1C-hI>aBT|o8a_sI6n#Mjiz}RBIbp{i6?~+2)U?hotnn9dH0QQXk!?S>rHx4n951J^Ko2*Za-pXUe%%-* zYH@DM5W!Gw-NwT?!^Ll}G&0fStgH8Sa1k!oxu5yjfOOA>?1tp7QnBTTUW1M30z?i= zpWvAcIexC=(~b@Ob@djIj=QZAXrXO{H7;0r@Bh5Sbt$SeWDg#uqhQK|3zi|_kw zVkvS}WP8A68uJz08sX|T)X_JS-vjV9wzL#et#v8mRC2Cyh;8oH+H|bos zb+LR6vBep!@{recg{L@3Z_wfpV zU92=TBJ8Gmt0|CgT@`!z(?$sWnZ@3M6ldrdli_A+#8c*l2IjEv2iqYcZDfnGi!pY) zk+)VOlbl8>y(fD!es4J=kHz+E6Q8`JXKdqHjNP#FOj-ngK+8~tqb(mk&t%>C{`v97 zQ%>j)8<4pes=N8JF|wqvxgYLm{J|3rr{rX*l+VkE4nr;bSETBGjQ2Z}_bDnCfsKNz zj{1@Tmzas{Iz={G$``>+dnL*m?pO>xesR0Jm0@R1n+a3Uz|F)G$ON1hpl(pCnOAE% zVl!{ks7>XX~RH#@)srOMq04n$=YdACfXIAI2rUa{b_%-)3=~ z&u-V!{r#A`7jy=uOn6Vt8@H!U!rGvtqEW3`v3?tOMTG8v8j^Jv;T%~|%hS=kq0z_u zn&!%6o&V+|-h(JU`pp7%kxM=vuve>=j(|+GW$^q*p$i{30xr$9aw!Hb@0I|g@c2ED z;TAxHGYH&_k3#oCWZ39r-m{!2nO(^!vXtsucI?Y8`AJ*rC0l4*Wvk2Z$09O?+lL5$ z=gVdl;r_^_oKj(qQiX9L-AbpTap8h@)Xf+~U)F4z++I*~$y7#?G0482D}+XZ znv748tl@??#?W4Pv39v~T)ti-;JnGHIYkY>uR6^MhL^S;jO$C>pf=K21w@HOaxZwP zzOs3*rPOHL3rtuVZWi26jL^I^!Y(D~?+xkZWW(O3sZaJ|n?7pz6L*RTsRuT)3bE&S}bK& z@Gzh4db(rC)zzw23;HbG1UtjZpJg=WZ^S$n+;>+4J@&-nc28qSHCef_ch|nH8V>PO zk7=tH`;U7t9(qk`tLevkU_A{LbY`y&YgTZMBp0(f`X7SD7<_cY^Lm(}(F zAQ*znsQAb*TK>90#fJ`Id9n#a;JuHzPHKOq60wvl`dxy++q2zG!jb zM*P-G7xARbyN270@2$MUN-d-u;zss7$tL)PSCEW8M@(%Y_k z7{TC~6)g-p&-{r7&!o2+Yc~(~R(ed{4^Ngv^Hg@k7kn7XCHj0 z$A#kv<3GY>*tS2od|7U14EL#ZS;us5uzfwLD#?0Xus_sEPCw_w{-QieZt;EI?=J!S ze7t)Gm{SU@YSnkB#o|cku>HAJ-6a}~+bthPa&;TG8yaF=CiZi;9NP@@jUM2CZ_?)z z?iF;o&lcE>>26*yn_J)ITf3Q$cKMZ@9SOGfHqPicdlUb+=d-xD8|o|g&n*iE zL{5a)6rp;`veonKxZoI{SKG5%H_v?Ub!}g3-$C1QU^am)d&kyU$yG_DgV*G~5c}}8 z4u6f2-oX*|CGC963hUq6;gBEqTu6<}M92M*`39kk3kj7`J-D_~%V;M-8MnMD{*K_C z(Y}TT!(FB7(e;Um)}gkHgMI8ybB9|Qz0mE*V(jF&hJM_RxW0}pTaWf$u136aPb2oi zS%!R?0++b1ghnm(-`s2!Za6lp5UFRz$rz+9H%f-E*&CP(Chr zs<{85f{{+0_vAQd_wPO*GKB|j1UBvR^c_WNCOX;G9K%{3Th`T8PHm`l$yclli@Nzy zI~O7b6gyVmXnPS}7v78bI@I4f@0i6L;FKXv6mIafx<6K%Juhh`OfP48a?vU`yf+3IHtlU@8u zVKmypB_Wi@$E}jVAsv5_GtYHd(4=b4el&h{?^NFJ-dEnP5zxIcPZ|+iL22Pq^2q~D zTlPw+_d@b=s8&mgrBU|-I2pUoXZ5&ZnH?9=I#|eONxJJe24r@u_iJ5g^qR%!XUnohIJ93XrqqC7bHx;Zsb!$6v_Bwh)KXq^J!;yD2fyw~$cg8PK6w5=4%68* literal 0 HcmV?d00001 diff --git a/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png b/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png new file mode 100644 index 0000000000000000000000000000000000000000..ffd0d8a06a9c662070feef82d76cda5f0da8cb4f GIT binary patch literal 324610 zcmbq*bzIYJzdxV|Oi@%2DUlnMQa~i65hbJ>sUaerQc?pXEkKd(ZWuXWbV+v%Mu{{7 zMvd5D{Jwk6xzBSyah{{E-ygu2Ili&$x<1#tK7LSBk)s6BgNTTTD4)we(;y-u9G>qd zzX1HW@WJ;9@IsB%)2C|BpFX{<=Hy^*Wot%6B>y2QhD=Ly@<#ALy--qMG!-#Nn$Qyw z;>^H75_Wkq1{MmINGc7rQmz12-t^q}FX@J^5|eh2biTOSE=Yc(N#u6GN50E9Y+uZh z-Y_wr^+Xp{A&@BV$;lN-i#gYRBOHm;!zG0~bR{B0aj$Cl`8va|rhMsak|e&98Y;r% z?&QHo!S%-87 z7P$;tp5;$%>?>&-NpW%Qay2w&iu`i^yRj0Fj(J2{8m;be3Dg*-MX*`Z3Hdk5-|PCp zeJaV^aR%$X)V@f+ocm2PtNH8oOPEZHAWpqRF*ffAnUgN>TsG~ijE}m+=tSQ{92=gg zElqzGHu;wL2WXMqbun@E`HfDJ^m}^oGcJwjPsC(itJdqh#|L7T=O8tQ-*QL(8JoLq|9w=6i@nUJTm(} zxJIaT^Duk7fc_b#q=#t`Ir$NCmXy-9()C`=&~i+K})uu1+=_agRvl?Kbsr z;)jVNqtrp}^X4>1{yL%?Et8JiiDV9eJwFtVsHTZhLZHoWe}Ioj#r%n$k*2ZA zzJJj2Y(oBV#QV(K8n5qa_(lE^CC@bcxq_wY3-~qp@%v_z+?yA^-&puS@xAS(d#r7* zg3{G&MrW!KKdbiXmVJ}phsLGUg}25ppx!OLTqu>^cx5XAqxK^5BJsNFMJDP832B{Q zeS9&If%|5RpJMBSR+@RY`Smv|j!D%%)8x0p5zSLS>bK|Jxxrxr9q1dL-#Ei_34YgxW$gEFpLk6I*orE(@)n< zbJZ`12^;KNWAwaGAL|o2D|5!L>`QMG{}LH&S(01{&Yb0|hj=0|vip?# z0sD8cqWe6nSLg;Qy(xz;!RQko-sI1u_hk5Zsfs>Xz`K9&Rmn}d&wR36rW5i5>=W`P z6^Ps!%h8FJ$KTzH-qi0ry&gd^a;N=u!ShdDC3$IUwj)c~mDz{A@6DCUwaPcm?aa}A z4|*4R5eY)^lI4T2A1@a5NA<-@GGAa{m>cNm&%#>t9u^H3x#_vSXdUMs&l{&17qWfH zpv&;Jk2|5CZh~&>QQIR<3zw+~J3W{%tP%!sbE)O6t#k`@oFO;tN{wuIs2OL)eb&_1d54Dax-Fh>+psH#x;4(lzU{QYG zoWguFWejb$(T+i)#!!9RD!cfhT0}=@2ugEpZvxR4gZ64VIUhm3*FEmeV%pG)3%h8>x4 z+uOHIf2>jcgXxbLIhc%k#toUVXZ4>BK5>UNFq_h4ytn$G{V9prfC(qjFIjx2Sjflx zP9r zvskk_vj;t)(NMkFD*wt?Xa!5N-R@}Zdj-2Xl!ce|A&!(*%Gh{g1QPl{59eky4PO>J ztsC<=og;jnBYgELvu+%E1v?V%mxz@o65D#;V`X=ASCqzw1|y1bv!6Kjq4h=d%{0`a4pz||(|c=jL*e3H0SAIw@w(Al zt(!fQhp&A%@UWCvNp0pqaW(fJic!`1hc6eK$E~Z@*Xiryf9hB3sEfK=;?u%f@gWK8 zwy1;g5!A(SKh}?|1HLNQ8kFVgP_${UscT1H7+UO!xcVTGz>6)+ z?(AkYDbgZe*jey+8A-UdFLZYm^cI97450#sZ)<#h>c$wWR@B0TrX6dCM(1aTriBak zdX-ZWRveD!;4TTyIAoIWTansVa8LNk+rxuxujHA&YTP=wzkUc;!2FV#T1?CL1y&xd zzT}|OICcm>NZ#5**Ww?q@0|L8&pyqWHTvG_^pf__z>aQPVIFl@`JJlTSJAn$BF8kB=kzhl`C}-4cG(tYG{8#epSDNf$Sf$x9WSmg|H1j0{TM& znB$nIB-$aKBJiU}c;`vl0-iqOr;QRv(Z`z!fm};rRV3u=jtU?mXaP~`oz(iLWJH-B z*ToO2pIo0KT{w@wuDU|`Ax%lPqg0RsN!z5NqL8u?(&2UZ zurlYQnQ&)TJK@8jT^WWR<(S*-3jgG=F5r2J4X)zEc@8(#dC9IWg<@C zn4E|>#ER%Va6}Az(F0#ZL?o#sf4%}DN%E?vlSl^#NZ###rZMy z{qtPEnpf4O6#jl?ttse{Ga+%t{u%SFBrkODibs8Hx%i6kfJch+h2{R36@G)#vYVvD z=c%NKe*GK0;Y(zN)p)6?oXu>V$7U%l57-vnc-GLDA}LvDGm;%@SF;p7US`&V6uEfg ze#!FUnYDQAli#=il~igWiDC?QCB^l76N$pkN_*IhneTZbqW}&P^4oqy zzx>VY#?g;}E4Ez`cMCAmsER(ta`QfZ>mO&ql>3xQSAXZ(fm@p=pywnl zjI`B`pvH|pZvDb5i`tTUYk4ciB|Xr_G5!Z{DWF8$Z`mII=0mz(ov^E#DYO_!K3L5N z52i{b@v&khk=X5It(3o>zR*CP)0&}}%KD%9Hfh0T`9-^>mM4kDJb!_x7?vcfX72i{o@rQCt z#!sf-KU1s-0pGvBk>;>SL#s=k%wO5`zsM~=sXtkezlkq^;mvEYS>A%#swwt|)6U^W zKF3Q3K#f4Ir1&NmaoIlog>sd;NJ4))@YFAa$t^a^%#6EsgKtDI=*+%m>Bf}XlpSgm zYSefzR+6HM8kPU*W<8WHGehDBI+$@8QDz}MM*E%#4Xr{ncz*~r9xG4aPxc0I>z_4A zo2cu0XzbOSULI^r%3PrrJD4yPeBDW<*=ar?e#lSx=|y3>UuM7E$K5JN2VEV6&?6vxj{`qn|{lD!x|FG6QFQd1cIW{SInMnl*1z!d+X|Q83_0)4dZCM8U{^jnMXww6 zeTLbS*o=iWhNAvQ+#-@!uC3Lm+Upw4EWi<`o*g)xn>cC*6OLN+yuoLr>A*VWoGY?k z74g#tn%4+t#y}L?Q$DzlN*ifjd1%ck%G4nzR*x#K-N-K}E;AfRD*n)*Yc*$kcQZ~0 za+aL!;d_ri<-V2dWO3EOfJ?Z?)N`Sta9qZog2a^u2jt#0V>T$8v(d#U?7q-Q&1;lH z@k%=`7v_3q{rG1^ggF}HB+if@v%G<`$@kKIH&Wf%+lZcaUkIJY5A$|H>vk1xCTT#1 zdAq591-*20q>6Pt>|ivVseM~T3E1OS<5kk~GE+b)dMpYI?9(6VK!vSLBcT*O`ec04 z5+<^q`z3eZjUV#k9ofbQw&aq=lO>)54AfWrQb#oc>@^9L1zIr1qn$=K@W((_4A`3N zmV4?c{;;tMhi;s!0B@e|#x#CTxR~~BuU~XvRE~xTxA;xjHN}*A$TQ*!0H#x8|VH!@!O7VSO zKB+YYzn|l%!eCqSuF`reU+_oHJexpU=11v2*pgf{$lqA$b!cwq7*yG3f2AEeWOAEm zrsgy>4HdB6UmbGWX{Sv9E25Tw+AWY|Ufz=3!&!!w@HlL7i4w~pZaKjLJauKpLV*=>#@up5Qp(y}&TX8#=voSc)8hO^_JNWp;* z;&h{!p2HcpStT?R1Hv%*p6twf4fXIeGLnSCRWaV|U+xF=Cy9ojok8yFUg@$gMqC>g zBNR1KYAYI0Ui-JaHm<%E@3rvE#)vD;m1Ar>4q;m?bTIoF5!7K$1uALaG^R^=1$Da9 zc=6=c?zirPx>Ya#24m}Iy~d~Ox+b)zOld@kEt8%_PNgX^75H_`?y(rp`b%WF`%;wd zas{HHCOOo$A34_p)b~v)KE&a~WhRlp!3Mm_$XqiPzwyNLEQKmu67Rk}CcN(T@K>0{ zyYJlFqkB9P)f8uf^&~Gch;L+AmL!KUiHzpeyOGLxpA^2jZwtG}n;jei>JHRG!w@OY=f&Red16VJ`o5`2fat z5mmUrUYMW$vi2iBS$oJiBmImtZ9Ao9543v-VCPFIg^l0Xe+9MhlM)A=X?g4SJC9HK zCg5y1QNv%+-YcRkwi)X0w7dg_uh4lU(0@fuTBJ~BsIOHf0RtujeYyW5O15XXPS$rX zy(vUCZVCWm-rAd^a@8XRUIL&`{j9aJSwP0WG^rP}hw@ z(5Tk$QN}+lhH#NR{6&)ZZaeW$*b9pT??Xae#AdkYF-pXGjU6t^0=vl{W%Wkn!|5qi zGS)rRz1C~{l@I;HpjQtfG9t3XFCI&H`P^4tSW$yYk3p&z!bFkrHpO(Z_ae(>(=_GB zS|K3Hbu2t>%fuJv+E>>pHSjT+a;5`3oGj{lx9~vSQmq5=-rq@0UVwWtd96Bm%Y?}Q zVRSbIt!jmsn(B2qTXNbvrO7`X3iqLI5WvI}CSLwzoBX*5{+l4gs=&OZr@%qK1SD7UX6FE7Iay5eEwBE^IEd=Vs9 zhc5keX&2sSH7O_wbhVK!nTWx&wC z;Gy?WX=kGL>~Wo;_SaySotF$c1}4!^=Vu9ILP#FJb(FL&D0xjkKePUinlFM+S-BcB zF-BACo{cufq#pb9R?KN?Y1~nU0}8p3OFdW=)VNkV9R14|DcOSQSNWE!Nhb_ag8K`_U zQ>Vr1lBfH35#_YnMM;p*tY-|N);KzhrJ zq&}UemU^zi-H|y;e7*2U`BxPF^I7rWj=*lNtlemR7Mc{~?lmFi@_LCD;((A3q9=Zl z0bcit1Usw#io}+B?L0qu+y#?8K8IPck(5wDWT)6M#kd5NmQHNxrwcwoNf_ zG&WneDxdu*$|f_{yEuXDgnvDggiAssDx7z9ipFr*CX)uH@mS_CpyfS%J*S>`ilN=T z!%s0Nug0Di2rbB1t`U9E18(>n&AQXmE0pc7{3&RGgOwjeRkI-F)d_23V*ViF5e1g) zhQ`B=@SLU+X=UtQ)440J>y7IvuSk0QP(_^{57uF`EaZuM)1HO|<8+`&<6J3^XwbSw z$@SIBq!HTtNE1n(P{#Zpc`e$&mhCkmVL6z(eX5WVZ>=R2q=^xK&PwJv@kpz8;gjTB ziS-O=eIHwdy~nu%vR%#JScpu zh^w{!BbpsxgPT9B*fI*`%SSM-lmWXN_4-Bz5$BJb^wh?W_+sq|eJEag$oIBdwYJP5 z-!CCDClGr<>6CqKkR&}XCwF()nsO1cSV-41`hKxy(Ow_w5lO0w=Qm}+Qh+0{BsR8a zTd~@4RAj0bQ}e+w;|97En@2XH0`M4c!fJE2Z7Rsga-W5UbcVkhp)54>dL&2H*iI<) zAUj<*fA&oW(|l!`zQp2}r~F;|Vn1?-(q!X@ZGsnmLYcK!<)#SmsUyKg?tLvlh+1tV zbthxXQuik8Yd2fu4QLG=-^~CLNH%0C=HR1|K$`CUx^^?bfZ>p%`H)JV9d>IUcIGjz zIlTMr=Lqw~&aWnsYz~T=lEE#qT}^2%&pm?*{8>TM1mQYRj=H;x`u?Y>e%G0ksC>9F z^|TG5*od6OS`X}_y6!XbJ^(aqc}I|Wj9XYCD=f(&;f>_TGRnCTC!RxH*1&OWQ+fKB zeqNDgwDJKX_x>qDO@23f>AJ?ahN;9AGUJ&L8#jSXk0w* zva6TdQ%)5O0s3GN$h!`Cs~uY>9I4I;BN&g zTGOv?tkSgfRXnb{Go_Ov058Tjf9;V#l&j~K=Y+-g8_@g@)Q zT5bH|^_c%V?XOVtkM~pOjBJ|tq88POyZSvyP9h1q5%@@Q2ry zgK`|2>>S*n!?giGA)@u?26b7TzaKhrgEr`$7PXS*_H*1FMQ^1bn|*6zQYyI1+O-xx z&~RqZdw??Yqh@NW)@b0M(Y)jqSFg=V-N%7d32sV3*8K2f5=Rj-(hV`Gj3nH_CWF5a zecp5E?x?&o=WI5~V+M!1(okkeiMn(l+uofeK&IpmeVnV7MDoc~V9OZ48-wIEae7Po z$mgh)IamVhz0-cMUH}b7u;9p5-keI~oPoI*Vv{v=Md0VRmgG0;P3GwO!^^DqIbW`@ zM&yNE+E;iLYM7-K3FE5H9Ga+j&?jmyig49z8h{3cXo@vett5Mm!+gKfSp@19NE|PI zb6d-bi7$@wH-$Sy+?^9F2ev{`{oI02Zlh;X&eoVQFc+3tG@%);SlKL`-eE@Z*N=zK zi6v?P?evU1LkH^fa4Qp<%) zH>rS~Qlj&IQCkP1-~+S)yY7gmf1-8>Co|PnO>4lP;zD$zy4UbH^p&SI*n*UWgOTJQ z>6H3v@tG-4l`**)H0T^^Qag8YV18;2?>5nneiT$zcT|t+nas8tnCTOmV#>~CXKyM zTemYC(Eh#Q4ZI%zh@SkO$Yjh7(w(A zZXZii_^A4HF8tJ41nE&Pyadhm#S3r+x(tWqY|`t=vob}cvnOU>Z5ft5*W)S{dItum zhRCbv2#i2}WXMp&RE5~vt!b+D{Jgy9Q;BdDJL?K~B?2t!>V;G|UOyT3Ucd$&$k~fX zl8y|2Y?0mVgBh)8#@HC7pl%#$L+^~qhe_-j)D=1GKF&DxJTPfu?_(oV(GUkv?DiZ` z%jm(RhB3EOW$~Iv4hSe@?D7V_C_NEAyL)q`VgFvIZ97??#^uz>?$qv&wo#M8zvrgi z1+9jbd$O()ybCIPPfqx)lRYa5Tx6L_J&7Ne+>YKn^yz&wHYcAOkh$uRWV>96=Lf45|Avrx$r+0G_b z5wRJ*c=q?#f8i)Kom85rB;8AX+Bul@a$pd$Pwb0*$XYlO z1L#oSqaPTUTj6r+35F9;nykOwQ_i2U*!kHmXg$i#@bJ~M7iV2;DWmqEEUc}qm3&Vo zkZu5+n*>lUZCIWzdqEPQAWi_D<1M#MWMu&$o*Pv5yz%N~{f&haY9tj`aN~jgXma*! zJ~rIz8f>{pZ8m7Y;DN@_&D zHdjG~VQgZ&&CMnSc7i`Lqvo&it})<>#Kf%o0;Ly#N;W<}>L;I~zBXF6EdWhVodNs? zHccuXJLdZ2qo1F6VBr$jPJ%~5buc1)h(Zk90_1JS49j|CpU%ld{vLpukOxx^%=ugu z?ts@hQDHSgaGTT6A|YEdwNrpYl2Y^>#_~I#75K-vfrG$OH8o6hIo5Oo>U(PQ*kY-W zyZVQJ3EpcCmrQW%3JE6NW&>`k9AWG`b6N7lam3Q!%B;zcSP<|FO4bNI;JF^FFVzwd zY%Ex^&=xUKYN`x9&{h|v7eH^-qG1<#>Uhsza5db_u(O+jQBFnih+;-HH%mKiUDPWD zMuBg`_(M$zeHJ@cTB65t9E(`a#cZNfFouh_-X1ZRSE!Xc99i#m2y^hoO=Vvcx1Hcc zO;tgQYL?^kaI5}C5X=}17>|&O9O)oyxeU_v@5Al!;$Ci+y0jX=ZRq))Vp{ta1bu8D za0TT*%Nfo9o}<1?jba#ZmzwUWCNoAP1E$XtawfnTN}fHh`8!Lji((pBnARy-{7 z-K<2|ZvSYWC*f*~v0@Iou+kf63fN96)i3gE0jno2T%@||UP2(1ajn~C3uX4gYPJFw z2*(kdJAYlfU-ImP_s$Y9mJJH7aXJdsPElc{zcJCs`M>y)xYAoHvxu6tjRO)olGUb7 zoU$$37D3QyWl6Lq?5FIiiYR9|Rsm69jj;tzZ0bx+DdRss&0y zh$&}6_7IVPd!m!GNU{GrPT=#SgpAfXnISv&%+>=enmHfP!pbC+7+n}ld)m58 zkl*WC_0Vtn`JGY@5SIS%A8_H2OLVUQj4epHmrImEn9BkI2(@0i<$=tW=GreupaK(3BUoh$dC;MnLH`d4Ytimax1H_MFcb0-vkd8Y|l9l zi07=Eyp}W@*f*a9hPNN6$RA3;0S#+M^cdB1-|MmK1tzBi$8T1wmf*X6J*y}KwnCFg zY0?{GFyyYr^BP`E(RWdutu-A3JF`dTz2vR;2`YmLm_;mfHYd}f%mqCwKDs&A=qvx; znp}P}#g5s*w5W2t%)q{;-TQq9LDR6+C{gn*qY0vLy);7vm~qDL#ufIwK~$$kL&!70 zwk;b`l@vG7{H2u0pO}Qc(og!WXCE4HuB^H}2{umUxpu4ldItOIX6~BBYa<3G0yh(O zU(w&(0&*yq9bjWFDUbt-j00dLn*K2hN4C zB|7>Oip|0Ph$=A4W^PVH-{8pP0=EW$r!1eOsq!~?=H2p zSO#**72)PeTxWYoazJx58v8!WuhlIAs{w+-TbCZNp43n8s@lrUNtr*I?(YVOxs;!udL}xDY(pX;(UKJ?3)3I>j$}7KviYygKJXs6*UeYhAdOQq5?_=kH_(6ZR za>5O1AJ8dBLv~du%m%@|SivuqP^RAX4i7V_`K;xQrTgWX>OY1p>sEY*v2|2ZcIR(OL6!-<()3vfS;uS1^ zFI1<%nwl6*V&gf&X{P-Sd!EQ9-ug&2&?@7wNTKK2@2~|a27Rs)oHHXZ$(Gzp7BfOXX%oWBG2$ZxB4kw6-}-i!0wY6ZBJW;e`B zVh&QBeZXotD}4#Se>%UWv-2%nerHuLW@~~j(jrfe_jFNhw8q{h$mqQLFG`xCxsBqj zoHwyG&MU~pu5UImdr3gLuAfsIz+y~cx-kdA0M7t&lEzu$#<9F6{>lK-r;Yn9?nll3 z#Uc5A(WK0$82=?i#P5V3009uQ=`Duup$ryp=O9%)%i=QtO0hv*rswa(?yp_vHlI#LCJEZ<+tpXZ z0q;O0Lk{$8fJ`aUoIsq6ocjdwre1vb9b)vw;P#wYoW8kZy2UYibYzj+dZxTBII&)=*@9|QM-Eijf@qF!q5|s>M0(UzAvOMEqpj>L`7v#p*3p8#-Xk)7%(5GUE&AXEanN9KVuW3z;f{b5~w?H0U;kWw;$+3JxChtUgk z-|{&!Nsxee<_=fd=z5`ws;pxmlG%SJ^1pCo4iH0+&Ty-3n9{j9v+i$en}yvr_1i5M zvvpUpmm2IFj?GbH1-hv^(MohYOWpjd3CNV04ZlVT_mS{QqA8M~qap!4Q+I?nlr7o))#HBC zaPb@5u`M1z#H9KZ(7eG}%EmVznSZ9d$=gJT2bCzWNI0nOaRNK1UM!H-J3Rn$S${C% zA~jV?2?26UK01RUUlnx8dbBB7+$-2w)2YJiaO(h&Fhj83zoJ$B&+0yc`Z?)`Fgm@5 zoa}V4;Ic}hU^A=EJaaW zIzb{D%z?~~gUu9%E_AwTKmr#?y%KR(v1L6zK7Mc(m@Jk6H?2rxCJbz}%IC0U|Xv?YD}6GWg}%aMr<6j4`sR z0rLO{glKM&cmsu|0*J7VAx{9m@A6)kSe%nyce>S=Dj;Vx0iZ(*05KvtYj`O433MS? zsF$-fODYwNE7*t&TgZw4>IZ4*cVlN1)BP1K+UKk|N;q67BwBIeUa-IAsl>ZM zqaH|&De7F!AufNKT}{6&Hp$};NkDFHJ__Ddu>z198v?;?l>)L466cARzIc;5TDmO# zO##@qw+y;IW^c&Pmxj$vP{b9WeX%1c8wqnK=GEIGY^3*T4EIM@k5^KdWiLzvmit1s z^y5;Eme>N9&!oS@yFQZu@eCLxlgMas%`!cK4jOr`Wx?iraFYNFQ=j58>(GkuCtlfl7Z4k1DETW;SIdC9r^C)!ouf4xTQNm1YD!3UmpSNRC@4nK^mQaXY#lWry6Beu2#;$ z$lxc>l`?dWAivfEfFPc#sX?#kl$2-}=u~bhC5Z^sak~StJfXe<@%j{x?nsVG$eyc6 zSVSV|4v;Q-cxiJ2Xb&1pZt;U3SsS!UTQg&HSJJWcbZSDu{W9}y1<(eOeNER&qfJX3 z3fgE7?EDo4#qfML_AX=?Z@_PC+x%~c>uGfK9#Xbs4;@F+DMqU#i4;Y%Y1RV*ToAT9 zf#aee&?cbmWZWGOlsnTjqxu>hIsvm#MQ0X= zS2mw{Z)_pX1vUP_Hs(or6kB=A+g`YCbZ9#{k-3#ml=6B(Bm8X?CC5e3&$U-k8Fz4b z3HEL8uflL!wIO~looc~v3Qg|N-1#lYPhQCVrPrtk%(@qEh~umka9Wg~0`@}s6xE3poWt&&R4R;aUYq1$L&mitp$8KsdP|dK5COBt9#ma}*ipttIr_Ce31~Y)7}h z{ZelFPsIB!m}bzx<_h3eZ;vA?SatQkbJdZ!o87d-@ayZN^?ui*kfF&P;POLBo*1jE z27UkYB7fDs|HEsyWBdU1zX50rF~m%S{l;VI7fV*USEp1x#G*qJVX>6BD^djTUS4L{2JFj-I#_Z4*RuB@1n^_g09tHU00I5V z%}3p)0uvijFr{7j?ze;zrvC{x zIaQ@R{OBPce4iV7%w)>_Ik%*aNxcIVQ=%K?DCf?dCOpKyHO=$Y%8Gb=%yfp!m+x;tt+4H|U2w zsBth_Hy7pkKec!%vx}pv;!XchX>x0&G%4i1ONvro6LguBK2$$-Pl5O2cK$E$I2d~# zV5a&WtV9Iw>1LJv4E#wUz<@3M)z$z#K7dh^j84!t7PC6k`qMZa+nnODOoy7SudTG5 z9M*j(^XD@GErJ^)0DI-}VqQlQpS^txc^V|*V}$I485<3JIQ`Fq5fAh8A=6pNu+##z z6LdUJp&cN^kXWt=cur9Oqw^3(nH4w!9d{Ez4#pP1;n4(g3~0NCqU*UCWt$pkJHVwX z76<$Ww@#K2!X~I=gDxMCC@1(r4Y(O}F)Wnk&*3Eunp(9~RtW8aUwS_`9iNiHieyjw zh|-?xq-0Ha^#37%M6ptl_<0(|okiqM8)@iGQ?a1^o;kKqA`BWlCO7#Gjv*hD7w^KTm{<1@JH*0k_v>fDCHs#(ti0I3E- zMrost_g1EWz6M*s2P+5k^09`KUBv`IO@I&*zf(XG*j6$6ekpLzNZ^jRMYpOKq>s>m z+*R_GRW2BhCZynf36{y};am#Q%P$KT|FSkijZ=ruee@ySFV8p|5;@Z=qej)89^Gu{ z&zXefUH*@Hqh&8i_oVhHxgVh|kP>dl|Vzh`_us7o&Wq1e|qXc%Hucb;-~u~Za^1Bv70jO z{TDAm7p)hY38V(lJ^toBu{;f)ctXbXTG#N6%+0UcgxIa6BJoqw3uKpWz9hg*@>8pd z27t82`}+Wj##?f>WH+Q0%XE%fDsr%fgXoArmd#>}%sppz0JVg%UwLH*$Sh6}fU-?- zjk)_NR63A8?#n;c0xf?(pZB9F)|@-5A16rykT)C#Ec7me2Xc1c?z29sJ zT%`awXRY(mWyRXT*Rtoe8}y-rNzz{QOsX=_SzwN++5x7r-NnU{}z9 za*Blbc_PgE%X*a;P^>Z+NQ%=jXXqTzwK=+teOZ~&lpB1#`4bvNkRLQ~qZ_D|OxCyBi-!QDwVJ-MELDDwYA?K&R- zc?|%&nplegq{WTUBs$=8Q0~|SOw4;SOMZ72M^Q{|S`7ODA6qAU@5DjNtIT303VcKzF&uMUSlObk~er`QITWh-PqQ zUf`Ajm&?klaRd*;Rs+s{gv;BaH}-101h6->6gUIX!-O5RKfN5mS8eZkhi3^ZS+;M4 zUNR#X!okve^zk^ zb4j~t)b(D6MXHqe^ zz5ub9&wervtgjN45yC3NfPQck=I{d9X|)^#hR~)4M3<8#?xzUYL|0mk+$$WZV>SiY zV;a+0`oTWEiEma3RM(GC3+5lMe?qF&1MEKtcz|A@1CncB6|&vH-brZ5NIMMc>o3+bz($3azGm&Z+q0Tnw0cVuU%W!qgHnog7`y z{xVlmdhES7cpU7tNi{s;>0mH58dJdsEb%cN2cUKw0Bu`B`+DyrD{08i3r*QR0^p7p z`L$x$qkk@30O%RIxa$1bDMrae8q*`GxGMs5AY$EX(ccOa?CKA4{$i8-qw8JBnL4c} zbDl_dQ^!Ox3oj*R-w3ppvKZx+fa@AcZW)jtY&M?ZE1~CAU|G5pHq=l4&4!`I{m!XA zLU`nOJ>Y)SWA+#F?*|WiszR%im;W~eBb7zL{#Iv!Iax=4g|k!=KmvMZnGM)|btqN8 zRs@u_Un?B*51;$r@U4{QRe3yFcdBsiMGF}K0JX)wnMs6(8p&KA5dE>q~)JtO{91@Pldk!b!kctL#vEx}upL+kLxbO?UQ*g(W zg1roh9m3-b0#h!ODy-`42Ui1p{!HvqYki?3mc@e%? zc0b?FsH}0vDQ&N74~oVwo!r$nt%nNp{KLsoSxEPtO-{;9p5B@YUph0HJ{Pj1<$AIw zXq;kaWQag5>BrdBB2)j}4t6C+D!$K^3k$Zp#d^xA9Br05!Wa z0`neUsr7BZ{4`yzRc*k4tVH#6JO1f}p?awiXU=gF7V`L^O8Mfw0oLJhi>c_MqVL{) z!b1OW!vt273nqq)t7lwsIq`MXLFWxR|MxpO@BC<-9z1ZD(1eUz(7u2AP9h~SKWI_J zSY;|24VRu`cY|QQ{UaKTRFsljXp>ywcac1Cb~&}(lr*+T&^;@&8yJJb_MIasYsGW$`k$u(AlC`4^!jTtmm_+Y4$|7=Nu|F3 zxiu(mne>K(qJfHmE&}rN^yje?K08ZQ;_wg($ZSqm%FMbxkFKcI z)EL>n)XsM57ckqnu~k|te1N49!`adY6mI z^?tohJkL4Lb6%eMlei7Z>^4K}d0bQw!F9#%*mM-O>Y?<(CaK$z6JoR8?HXxkIrVl3 zJ4mR{vt!thSWOPd$V8da&h0eZp9=$2NUN*5c0BL>F3-7L!$@l*IV?0j3}P|BpKWJe zttJ8f{lX+flqh(tdyCq0U9z;LeYi65yH#tP9;veZm4O1Uw`z@cdXfdj$E`|&dw6v! zVIlFo)s9(!^@(oh9wBtirmOsV)k(UqBWe+N7cC)Jl$x$afN0GIBl|BGreefe0n!CO#_fvP%(BLT zLCJJ?h+{?J4euco*KQ|4-Ti$+ftRPpU#EKnu<{TmoDVDm(b$#;!^bSZ(mCaU?zOjLWV3@YsN+YDPM z=(q_v@4Wue(IBJ5^F);#R4INs=e}xK!?Fd~GRNam&h)BWPZeMirk)_@|M@hKaX60v z7`b{X)-fp$4vsvr5ahs+RH*nKO8@m}IdL?n*CM6^&D$lnoyGs4eEH+J`Ib9ss$69q zFyKJC%ZnYZUv{M@+hTX;uI5>j;OVYjwqpAHxGZc#ulv7*T+2@$?<8-rmiDZ2>679Z zY7jP@Dpl6Ok?L#5?E=4u`eV=JODWbC?Lpp?ozo=;>y?k75=o}c2yn1o+0GjT|L9$Y z$#sbs#M=&5R2o7n5>l*J2T0E-Jv?T`=`!EhyTqF4wcB2lHT)=p$v$8@3{pP`t2(b> zu3_>oD58D=k%ANU_Bwa9ZmZy|PMcLc#=22#Mbapu+Hsv$I>NS5e1zG|&`ksngo+hbwS_xf+>bdvoCxJ$H;Tv9#$ zun=4>alPq21e=jvy<+!Djw6?wGzP%;nGDtPZ4lFD00$3+mapR4BK# z3Km+T28~kh!5Ti^wm~x2bD@M96bt++wzb{ZyQ3LAZj*)7;B|J zjMM(ZhCoE8C#5@|mbn#cz~#%fTaWVQ9VMk3-ANf8@oT$F#cMAoC=c}Pz+Z~UBA%RLoDn{>o5Vf6O4NJK z-!LfDMxP&2{04H~zDp*#?C_XJ`n!%Vhy0{+_EOy$D0`z^owgCC&rK`;N8+vdT4Dzq zBY9pH()-2V%qaDKkOO5#z6Wk(m`%~3dqBc9%33mqw!?!U5qHi$7h5rqtQ9N|gSULk zEb}=^=vN-pnWnKgobqT7@yhF#4#UvN79RK?$z)u;;qr7^wcJ6-l`j6iu#*Vu_&Di3lFRoL*!Zpk3P{J#T zLRZRRyQ>yaVkeol-(I6JiS0hdraQCg@=g=VHq23)1!;onv%50rZvA0^4}^eK{Si4F zt(`=T!oTReT&(8wx0LL`<8ZA+{pSyu9b3-P1s2z>f&!7_ZX=O4@#s_n*GygdkNWTL z-%w?7t91t^>0^r5u+iJIv0%$Mn^!oemR)+y@%u=$U258}z0uUJ(jLQ^NpvbZCyfxRl25l1`R^L2X>##Icf3(b41MEZyq;2KY70@^x!&Mouk*WaDY_FRn3S zCMP!`@VS2aza9bu;TS6wz=1ew>?HP>qQaid|F8cg;tpSvolEw9zpE+_lxxb)toNT= z#6ps@fv#-qajqwWgZz)G{c>W@(X8MI4gl=0##-@lgekA>X(lX{xGIi_5%*uT(N3g-<>EQr@FPbqcO;KVMTT$(yL6N|tv%_X7O?K1JYUm9DDaG0nPV z^f=TqjQsjV5yo>_SVw&W&-=rcms+YqQshu*uZp&5cucz5va@|bPM5@mpQ~~Jxf~UD zt}Od2&a>ji=Do~ZCY1G1w}8w&q&-d& zU3*G8t9KZv5848a6QtV7--W@y4g>fx$i$mQXUZna*(1wNT7$vT`FxrXML&me$G8+VYe(-$pj2zLyPq)^RYFEzG|$F0TeJ2nkcbCd&7zm-yEBtgzoAa z!O_~{a*9*?nvteJPEZW@ry`R|Bs#!89j5QQNi9nULIC;jqHjrxgCx={sYpp8nz?Vx zSp$7G`9M*+p+42HwAVoC$G__=|1!O4+T;W|pOM$$yR$Q!Q0wXf?j9+7OzKB@IqXV$l9*FZS0yq_ za~A4Z-O#{WS5~-TZUwgd*qa^?8)!{ct%6u&BAqYv>_tjJ*GZHb?IinW%i=qZs6JA| z;d4*Y$WYSEklrLkYMYU&N+W>l`VIvYxf5iUY7Uuwr~w)-edY-}o7B0~y;V{uYPqc% zF`(^HAh+g6iNpBkM#M#_XL$e#mXG4dV4<}+P-k>D_L}K#@TtBto77N6Il-T}Xl^~y zqEyP~2DIvqO)9TS)hZ8;t(VT$jo84dCpKNOH{FdnRQif1HFb78DhttL(4cq5DNc6` zP#tBk*$<2uXmuevKl`4vDoiC>@xR>8{@)?$ITA;xm!zN9yBHmpy|NqsZX;m2phz>r z-Fm(rJGPD6=ooCCW^5z{CflN1mMp6dEzc?pr;#=ua??p-LKnd(LP5L<+PPq!-aaX4 z*P5<&%qv?EnJXUze!RV)KI=}J>K&#k)dVyGoKslY8Kw4mOe)rsW?_hV;#~&;+N;jY zXJ8PMeSuX68t9qr&B;wq7ffb)uy$-zQ&o~JaI3go3QH<|*SZIZluX6g__rwl^Dnechz)kXOuzZ9&@{KaR8bNcCS*DNFZ<$9V@jFPSg!GC4cI;NGsiw!|Mx+$G;5T!R9uCt?zlzy* z96-Y`KmZNI(Ssq_y6N=waKF>9ycj~Jp=++Ca;tT=+?rq|XyBuA7)u!C{}D?4V{|f9 z5>w82agCrM27^yBv*$8R2v2}2I$-sBQ`K0yfTBpzle0Ix`x8lJ`@Qb0Y4C38`fN`N zssFVcs8_38IYgpr&f$P_G`MiJwU>Et|M8r@@|q_vaGCzRbQ9H#nT8M0V2{a0Vme&( z{dd?$#41BRnpf@IU#tdMzq@5Jt=Y4-t=we`&4;XRsQ07vx}1`At{879qI(jZv2#aJA>^Z2TK5M>%>V-{NNG}TIVg&ttGbZIGC z0PP=Q7R_{nz+50mECfxJOGD~XhpbFAi2)<8us~Jtu~U5@`8u9hb4OiwbHm|JHgTFnpYya}PELFQ<_4SjWhk~Ab1#Q?zgzYwwwp3NeksBLL zTA0CZ=YOTN{-Zdca$Ww#?gM2qbwpJO=LdQu&`#r5gEBsPf!;*>4p5(sAoalj)QxkI zXU})D!~>nRnzhbsEFqFpQ8aYjq;%i@L)DLwAAWoOCgB&e#WV&5NRn$(z z*nnt}*tks}L%Xc!0!&V_$BmThE$FdMgIG6wD@&2`&uWkV|L^&d2|2+tQ^5p}{1|ac zvys0Pm~u~=A@J$>;nBh|I8d=P13Dp5q$ctR&*rt?xSq4zn^^htQ(C?Y`@N%UEOJ_7 z0tTcU3A|j5Fw(lM?9`(X8{SD(QXNdwfn0bsX)o!i0OY-A<2&eug0S_gO&gs+dXtaH zn2mqF_J3P7z#pW^kx%}}xJIT+`O)}FTsAPSBc)dCAYe?yQDpEce=v>NeS<#Q)0tw8 z1uj*|XIWl`RpAz1rSyz%N!6|Pyr(&OWZZ2XY#cnj+OPc>ZZF*B*|Pyxn7;WBI{S;e zKw<~-s*hhu7Fp<;nEOY=XwrDd4+qLT($bT?w-|K8f~Y788?Lpix?m4D%SE|iOPL3; z;q@FeBeXq^&P2RrY%etSM4>&JP#LV|)WcT6$@TkO|0Vb&Q*`rKYrHh3SKo#=Gd-Pa z#taFJ9gvDt%GA#OEGU}0PHGGyO?uLA)s@@DI{^+0-dFYb+hq@+!P6FS<0i>Oz-XyR zmKTqBN>@ZU17jfcEj&uk0sOfES4A=&`fCk;rYZ%GeKGssh%&iA*(#DNXE^{&wd#3Y zY?JU@o+okVPF?CU`bks>YQ}K%dM*>DMGOUrwXLXhl{3~k>A80LUw-haUiUwLTa+#YIEet3u#DZObb!dA|s4)LDy)xz%+e)Q|~~ z4QalV=%HViI{EV^JUG2WfA;*Hz}xSkqwBWu>RAB#4Qy(?s{yuT7J%!` zKFA;5ZI|1z#df8JkT{q{^Q565q>0_27w+(-^wfkWK)YK)Hb89z`c;biBXo?d0R$ru z#IOaled*jHQ=e|5^e#e%G9K8HJu8&BLb3M3K=Oq&GhblpeSf6vhX)wwZo_bNP3zl; z_3}U#r^PHD;X@MsiQcFD-=~?e^5nnf=FETf>W)uIBKemzid)Lcbk_~vmm~@(k?CB$ zMoHOxjWtQ|)#+XGtKKyy8ikySqpuTIySH{EitQe_KwW_UHl5zC#=|`_sxq90S2xlv zOS{D@GtEM-ehradnDH#d#Bz6pb)SKceD>>C2 z`G5V9A-D!;ZfcsHxA*dZrb>TLcb~YzqDSa4Av{<&2XC17eEM`fyK%EHRW^HUt(v}S z$#m|bwG_AsJucJsNnI;&G%=KkXl zi!L`=0<>3e{VqH%_+{OX_sXt-5NmW5Y$i?g~G-7)3tPt5u9F^7mHrTHIiOl$voKXw&w zQFA(%-u`c#>JfbN$kBIr>;)v`u4xy=R`sVZ-=BGv4e+C6RlpW84sNOub~E28=?t2#Ubj;+$F@M+weQl~gX&4BG zZ**{IO6;ZMgMp~GfjzWjr8mz1eZ*08zY=zC(EU){&+}uj^9+WgqmNMRNi}yF zfe%rE+X6e#k(E;-l;E(=6$W!?ky7&N9A{Utxc%z!f5;55A-yU;?1@wM8O0Toyz5k+ zg>6kR;J^l-=)oT8l&FAHiSSSKuRTBcJ4-)FUpGt@S& zjCJ1Q&mL5Kz0~;TN>8Todd93^h?{aeeX@d_pv>Xg66%z%r+5BW%=_pN_ZtNiSa6(F z=*f5d8qZ@U*+&@Q#i~?yOvT8F;(`uV)5Eq;5}*NPYcWDb3zx00UF~HcT=^e;0p8s? zreNSgoHOUyO$I2 z6C66!?q6SIjO-Fda@BF;VV@XO4^3#*sKIA7djNN0C_BlemJaCKPR}uKph+@&H}}~C zZgDxF0~&n7zEg9N1*TB2?1d=t-(Hhu&Qo8`l#*JFqKyCM%wE<}8^2|2=MCj3Zd>5YkPul-gHDuEPTuu{8VT zHSM~UZx@+`oqJa-yDs;c_g`KPO#^MN89Se+eXNmETiyI9vJ~b4h3%!AEI4%|;v~P9 z+r{0lFXajs8iMSe0apMBtUdjP{bM6~vu?88er)|8y3=dp&-DNX z=tpe_sJO^i20NqX<(|1RVtqDS@Ts?ewbFC6Xz=%x|M+5EL`@mp{ZNB&bULeU$|x#g zSlPgA>s$oz8iDc&Y?>_E=@)cyGvcFya!n1;C`-?Ny=Afv#=l|iuXfJ!e zHB)7`#gd4l1MkJpDS8nomg<)V^Zw(*vFz&C;sd`SIZth9oHJx!WPqaq8z2w5dAiXF zl7_2K2;zmxvm}+Jn}%J5b+@@YOW5~q7FevNU78Ghr}T~od}X7Oed_P+MG!94 zxSx@ZynIyX*xcn~eq6LJ+Wmym`@veEGkrQBnd5>Esg3W#Rq98c&A15XFdAP~E3jqc zBcvDb<|yBja`+$959p*HTi)rz^5=GTN`8dUaPc99h$+&^g=^W_Y7hqt&rVU)*{ZgO z-=@7D#=HFl8vg^n6i>$AH^fNntbJ_N#Kl4*t&_pqF%DmB{*tLRV5uPK$mf^fQOT^Y zPLKgD83x00<{}}$rX%1S>dglCcF~Sa?r%AUB#2`Il57X3ofK zKdt%fM(=Z^s}A3vAnHqpQt4MQS>PVxYiTU+jN}F1j$v%cs#_RaKO zF@05@^GU&*&bLfxXss*N(hxWhxY2k)Jb|?at{Y=nqTDrtlHZn@9Z#K-+b&zl(yz*L z=_!lk9hp0{{x8QWK!*?vKcfqdP)usw%-63F4x$b6h71ETqk5p$wA z;$8xD{k!wkJbntL>yVidDMhKVr(|Pdi!pCWjfJ+i&Oka*%)q$F~nQ+o;+yhMk3uA zBoWT&MUe@p<*2|T*T%Bus`-|6$GG2zV|oH)9w*a{9_Gj3mK)f-@D8-%^-?1lNaMSnjKaBO`FpvG$t z$cE~cDpNfkDxlCq?NWoKGcaSM3;OReX^$nrj-;cb3x_9z4SJ`pVfxsJfAGa7H8RibL1-Td)&(V+uQXj zdKc}6Vt3c#B(I9u5PV`6gl4R7>imurY@d^@GAa+LteJH|&`lK^Y&nwcfDo1{Sd^@F zC}CrBvx6=5R{jxb`MW>J%;HEvM@>G3?z60wnQ>d1N9JV3YhiNycKuZbs(X)4QDX&R zN;I(!KK9AwTwvSB<+ff=D&|NgWN&GMYOJe4g5k&C(MHQpke5KvtvV?#b@pI4#ed9k zJx5JNhV{T$jN1u&KlusoLStO)s8IG=N3-JjlxVM?C6}-!=p2g{-sbn&7iYizl4xzV z8tHm9vRtKaE5I6+C|qgGbowP;J*|FLj_1*ewdOuq^LbycF%*}{f}}+m+&Vh>@eZ+` z!?kWK9NF!L_`5Q8T_Y!oiwc^{tA6?!AE0jMxEft2zx&MI)rD_p!yT4glG!{P_+Qrk zVe6v=$fvRdlfe#UgNm$Ab8hQu-cgEt$-KF{K0N-etb7lR?#zD* znxr8~-W(bEu#P6o>Z3#l2%Z3*f$$;Ic&h^MCA88z|Hzl3I9PWiYYpC1uk=&al+eiM zp!=qbn6qI1#Vj$EKNd~`zhPw9mnqf(HdBHm6qy)6wh|h()!=NMYtPhH#WwUz(?q!H z1oa!MH&VC>-W~B2mS_z5Mg8?;hoy*I{WKI(#Ig9gb-ie<)<@f+{7Jtj!t>(a^NAmd z>ezVg44B7;CVZYNfL5uNs=pj^coXDJx%A+CTnFaDzHTDe$%m6NbFRlgH%_h&+v_)r zjW;i4t+`pZH(mPh%kt0hAzB$3MlU=1%H2|8`_zdl?ar`6$wevtmge794dF=R(O2>}+kayH-o*9x6M{han{7x~eWM^ln5FO1$Eo1T2IomW4@@=MKo&*(Ok z!_L-yw(PfbwfQvYaBu<#ZmaJJ=uM^YtF73Duh!|*I#&=-T6uOSZ&s=h%T20V=|T(L zibF86VqeCa2KcO!93jnVvt2ZbY;!VS=>=-16y-6%;otIfX2eq|_!mOYi=_}tcBO1< z4Z7zw_H=OUo;nC9FwRdCqpNhHJNLkcCxQ_bwL8e4 zNa59(l!vBcNQag(-1=w3KMge?lN(|3z3wHebZfYjMUK0Qya+R@L`_El+lXFv#5@}f z`DmHKR~T0$f6dl-q}$+FZ;#brv=~CUq(l+Nu&}Ln?0H+E95m)1`4FKJn6CK(NeMaK zUh(*BahwZseJ$Q5NRSWvYeVLw5fxz#Fc#!7#2flZK1|{XaY8}oVCSp9QwIzIWZ1Mu zsp-}VIp#Dro`k|8I@>N49#U#c1 znT}h}w{oZyRpUCD6_DJS_TWoFXn5zx~Tx(~j+3?^kK* zfohlvQ>-j7m#?=P)H^pYzg!2f(!r}_y_A2X*cgP?mR68XrmZ^sQwQ^fZ+SK0(kjj<#btY@A>TT@E_`y&mtqbV zZ>i<5@n`eZy50j!D$C*PYP56jttx)+xh|M|%HF(PFM0$SDQN+OBn6!VE7_<>UJ^ng zowWLy#+}kYrdpGBoiTo+0iN2e!Lf5eY8Rmv#DiA?J}D6-KOxZ%o*NhP>L!UVt5>(heS#IZ#{3(Gc_q^Q3helY8hZyues6YMND$0WfmAy;bqE=(G=wtY>uPukN zp6U=Gmi8mRRBPuJ>!RE6jCnNQWKAoy*{!3G__6+R<-w}6kn(ifQMKVjw7}}ceYMC^ z^`1`WAj@h0b{>#=Sq|^S6XmZ)$qnrqNBJS0w2BhWZ=74pCCDXKOGk6mPAT^FYaLUL zdWMg&1DV}9g%@0I;O%PgHz*vQSj1Y|?(YLh8x#4-rx>Gc)_4o9nTv9dYlprm!n&)^>u>oDDTV zo@zoVT(w!2Ue<-~8yf(=PA>>4JyeeSC{y0j1RcAFl!+OosyTZc3rq*9Dnd{)Cei68 zT)s0Tq?D9>5@P*`!OnV%D!wsFWbKB={WlMhn7E$s5-9(PM?Ge+6Q4NVp$vEOxzrFT z={(gR;4|3#0Uru4l}0*foYK0U7Zl@QPV>X_yWj2(UQNVzNZ?Wqx2G|VX0pD7ua|4z$N@%rvOG?5 zS)F|~?&J4&I2+jm@g(6lMKQ_Ge2jM()pnap^}4|P@Hb@xD|T-~AS8v-JT66sr;2dw`O`K}Jq83d5p`vGgO_=)l?$%lq~Q=N(o%Up5PCNgIvpvL#eqTyku* zoVgaEr@d#B(6RDj-%g#H!ul1YandteNE{F5qRc|?_zxvM-Ujk&WuMWE#uFNpwq@5S z(Wiy&FjNA$@fl6;$y`o=$BmO0(j5y-eu?Qvs_?}Q!NF$St8FE`gU)NFf{uTjGx4;k z=kwRn7NAD(J;vj3-f3r*-HJ6gXdCy4GDjB4`hC64>S0}yYypp25|ariuO}JSud1wN zdeNyOMoOaN15=z$;vWopeN76^BTycMU4!iBY)UK>M-1L)-3Ct3+=A}n!7KnEp{+)> z_Tb#3cV`UvbeF77hpFVs-Y)ITGb_$|&WaJBt&fm0GXqqU9zkg{wsEa+N?GNTTWRA5 zZmP}Y>20B0Wt}Qf>hDv(ylE0`qqM^vNyFazD##*jVLR;-!-AkwpN||k+t3<$il|!U z{zpVb7ON(3%yx4HmqC355xs^k(X(hI87;pnH3OMLhgybpH@>`8AUWzhMoToqLfI>(y2GcCzTgc%O-^jCo^rmm6^YbkySF0IK z0q5fMZXOV{= zbg9Lqu#o$x!|5otc|@!QD08#11Ume>D@4RCI1B%Flg@53Le;cyKhgcBiq=> zseH&5xxwTY&hOi90XLBHsgJ8C0`hoz^Ceo?HO@Wp5iOvqafp0wnogs8?Z}xuZC${OZ%ZkQdIadk+V|ZB*da9-MTZ+jVD$`6~EcMm5sl-z+u z)9=C3ec1xmJScA(mY;EZTcD%fG3vTg$IGRZM|O-yZ6Z6vM?2LLgoYYMik164M}#bP z3YuG$YCU_u_0sB|8mU z5!3gd3h@_oMJ&lB>*~y{8pNY&o|j=KXf6hL9 zI6yxQ@UzCx(d>H`wp__HYn^J*xHrn}DALXVW?-ntzCJ$-efnqgv~$n9)XvGoZ(Gpe zHAhFdN`Y6~QuvPCWYr}O9u>xgm9T1i{aL=%4u6~8_zD#yuJ1~%MH{1=HKU@waAhk- zdpb*kXqDI9Z%Kl3Mw7UdF#8^rKiI~>^>T)G4f~aq7MduRx2%ib zctl!mTg|v!-_%>l`~5GIkHu%{8r?XZln@GMT4b6Mq+%7((RFQ5Wbx=6g<+ zbYl9(){!U5pRKr#Mjk^fPdBdV4ILzKvXs1CHl$7*-x6z)N#d>I&f(sbsyAiYUpf=T zN-(37NEUfH&*HF9wwkNaB33Y3_ZPGPoNm;4pk~AG!C2C`B5V??kDBkWJ8cSi-Xv#e zI0gXJ`uTH*_u$bFzgntO6bF$^%@oxma>pd5?`cKCZFR!2^n4%e<$YO40m@)5gqHbu z87IYO3D1Z_u^&$C1u!+;R9QElwwJ$kvEngP7M7bSF|xM6clQr~_tY|q8Bt;5x1gt~ z7rJS~sDq2OcB5}dwr>)C2F@no!hIFBjUii&%YjpkwMc$T*?F>4ex22Z?2%G(5I&s| zx%Hi)GHEaLJWoEs2%K{L*UtTah$y0wH#;en(XFqPz%ZoLFtwrwb5CYB`3QVJ{AJDpH>*uy9HXbvo=hiEFYVaTsGnD~V>=dws z%09o}AMz}S0(Zu=DgZ{&Qg!zc+w%C^qrbc~mg8y73%x%uv3zuP%PmUpAJvBVhU$x zdtVX(QOYAw)W;mUy%fyEG+AS*&o~l+CVaSv*mKaf{_?{&v5%|M86=*$W^Har_4WDf zdu}pNMT=5hRb!gQ+IVI98~MAeLeJfl;BUCYE%t--8s{@VqH{n$oQNvMxlH9g8F>2jcrBnr+9}_biJnZJxBI zRpIM!)#)pI#19rh)gy%$DoGz9T&cm2dQ%#@BQ;RhPPxKw*#Ror=4O_+AMDhd-gAd@?-({91y&XSoz`G*biT| z7(UDcdC{CLbVZxQ@M?NeP9u0v0|oRzlXpGe#E#>5P;+cdmFCf?#73QKtD=O7gS?vM zX^ABZ!Q&y;2fEI4&s$O<{7@{EHHz;n==4+8pMCU`HGwQgk`au@JxWG17<4M00*GH@yUgl0n&yx49ZPRS3OVOig7I?oChsC7 zW%HWa|EQVQL{q$17Fc69($vdi41TLX!VoJcNs#$3IgGs`;Jc|gJXsPYB-1j|YT$Rz zt6=!UNm-XfV?ZJVWI@R?QbIGF+mqV@mBxcf7E}|Jy(z5?QYY`7-~LsxL~xxp&BdE4 zUCie{IMF+%=MX&k(_BXt7{W}<7BU3<9H3lBB2y_8uG5VA{OL7eAakz6B7pNS6MH`} zW?5J6CQ2lj9^aQGX7XY4SWo07)a$wQwh?6mkrS2OgAbguN*i2t+GE%RE`snT5nz!J zmQowybV`?jM?PSZ&Kmi{zgR7r_2YOsNPZbb87BqJU|s;InHZrtHP{#}RC*4c08B%( z0R{)ylkWJHPc|q3Uf>zY#7n5${x~P zjddFrf$vg2-JNXc-X@n6(pvm9UrnDc_BtyTs@$cvm+B;^I&t%dqSu-tc6!d9&}-k8 zwMVg8_Y<#phs#;StfBm#hW14i9aNp_2*}zqLGD+5xwNzLXv^GyWLSxeL(N zDx?5`NXWNN=vV z2+&8}lN}7u$=RfmBD$8Wjw<2Q_ziJ;$ZTNmO;Jd`87ypSy3j7L30M1x8O+sFDan z83BA{Y3||Al*mhU(r4fLa)|HZ{U*&>N}7{z4?31DJg`%eWf)`w@JO?(z& z5<`}saUBtQb363RU!f12DzNC=Kr&rlvR2<&HQBbpDsy2HXx}riwP#+9JsdK#VW~nl zQZ_toNTUTAt)Eh}!k>?}Qt)K*1wV>o9N&`7^Swm*NRC-j)@e1x%OHX3=2G4%(g|yF zEH&#?<-4yXu^4fWR00DCbC`ikH+5iqnxAr7nhl* z?y9-puk++VwGl|CFq_N|wiBaaXuY;xdX;Fe-Q=RFyHW4lVlbRf z>=szOI)H!2f_qnH6n-8Bko&v8WdU{5uF`e|m{JjO_ z8G|^x-cdA2Z#sl{7-o|(lr{hiTf7vp&EC^#oy4_0>kkL|I5L>m5|8$zPM+QbuFa_O zL~8ocZfwOZ5P+K`W*IwN6$ox4{JSgs_7}gh5Jy>hIX0H9jl&Ovw?0w#LJE5}&zdZs z?Xkz#+jqmahZhOM`X84Lv24k&&yNu2d%6-f3UP$m*FJl^M@89Ljh+$0_mKyr4a0U( zm@GY~*CN>t!|xmD;%u+S@{)40_x4KYio*@`BPI9w{GNX! zVKgYkhMr(36@0Z&-Y1UI&QZy9$cV<%?t7c{%RdTDM=phi9q%_~?!3j>ZdkfT8w?3R zofUty`wN9d8Q z&q^WC)F7@hLloymQE6;NZc1Z|90_>!X@eF$H_J1*b4uRh5*D`;+$nc)xx4z6)3S@{ zYHQ^^*>+G@APM6|oW>}5a_%nYjCYlDw!>$Tp4RjCx5SD|RqCdlG{`2pKEYI7dp#0d zs8CFv3r;A5@1v(&(#}$q&nDTap7;V=ruTA~SL_dQ9r&cak;FoHoL#fk1txSsFEo4O z`@G>lE(G6@g`JUD*H9d`dDGMFD~^9zb>V|K(?USo!OpQFvN`f{{SnH;SE9K2ABKa* zKOg;fo?}Oe)!+AKFw3@`txt!M*;uSw*_hv4(muxALz6V7a$l$e-*8O5tlvsHfrPb` zwL%0Okl9vBlb=&sR7}b{Y1OK>%7=v1X6r0YF;2=&R9KWWNkPT2v$P{T!u9xQv3(!_ zL@U*}%<7B;!km?kn(JrU+0Pwr?M@lUYW(E-U2hAB#_)6{Vg-P8vU0~^B$9cYQ+vAc zvcHZNc&2^0p_%5K^3NQ0lMDwf|G6~dTbHjzL>T3W%0Pl z*5_G+s!4-cT7m>Tw1Wn?$6y=yit1JAYE8nzBGHaednlYyvY*-sxUfv_ear& zq>YhRY80M`X~wQDPClu4r=_V8D&Qvxw-r_!+bIz*95Xrabj}fEt6}%Q@gMW*pBU+J z2odC)(vB)IFVx^Zg&(0V(dd)6G$=DKELDn+-o3B5Ph}FSvfmaO0rF;LmfKdz5pYgI z`0_6GF#u|sGJ<5(jPYYYI&jel+VcQKZO`G)`0`}yPB=Hjo{nXLrzL$hkC-Akg&Vo&!KS1 z%;BUSi@90AZ}k8Y`E&L*zqP9^HziDUtCXK_rPVRa z)6$F3wBaNJEHw zfa|&qxVs&Wu~PwMktE)L^w%XhGJI2E z#=ga7h)DBu)fR=c!gQ4~iOJUMxlB}4wOxxX%s-|2#k~3yQf2z?QAhl{vZyqlIRfT!%kPdD3ed&+iSm#>Av83*F-7#HTKdM1zclvKl^K!Vp+Xy z`M`a*6<)Hqv~JF_*OvxCgf;;`WB90k*?v4~t^f#@R6y`&uW=L6Q$On$&n|Xy21M8) zQOrdVcVYeG-ET_VMla1qVHAWbGuzNG#s-=Nz_R7e5;v>8)|W%PV9B#lt4pP(;f#Z= zNUv=qX7zoXUtgqEyu(x3)yL7$jT-bXh(atN7$UXy>Z^~Or}U5he>oQga3ZjwoemO(V%p=niD5FY4720WnMc)V2QS; z+fi~czgS>KdJGeL=(q|wLq8FRYwY_{b;D+hYTi-tdq`jTj zZ*oLI{(Yxh% zY<86!Z`=WutFkUW@8l%}qMia-MdV$R56T1Y1A$ocF~|R7>$>Bq{{BC@6{YNu-LeUF zZC4>>COd?LWRuNBnIXz{jS!Me_RO9sviIJ5-|O-_moz@#-+vxF?seYhea>q>pU;=Y zC?tW#Kj+WoAjcEux~TieNKI(z{Zsl0UwWR2XN$N$6Ywd*?f(=)j$8uhNw{%03dp2B zzwac*Ob%U8Z%{K4194#$l&~=newD6f3rzpfE^fS<*sEHy{3Gkou{Qxy5P*fEPD(oT z;PK|{RM|48qC&A^<3&Fhbvx^hh&6o0^(O~jB*C73O(YsRVUXFokKb5uz|E(pE0koU zcuomRmAK=i?s1&^-v`M3zK@z2Velkr{hH)gqYua6!pEJcEkdTy{ne|e$1~P;!imwg ztc~Q>@3o($Tc!+IJAq7?O-x5wzqDd1d=8<16f0E;sGY7tmuiCzU%6SG0LnG^`4%87XHghC(G*4NO5Yt@epI7lSAZ|gBA$% zj@yg|>3+gyVH`l!i*;ur~ym3kM1yu%Pb$}E8p!8*e^0C(%@5HpHBD{=N16&_^ z_bJ@f{dJP6mYlqh6FX1bYm7r=MFR$&2c437iAy7W=Nid3(_5P;z@GNjlJsfFvQNa1 zXg^D5@cpEQ`D>#@ah$|-Xb_dM*TqJm6A2a_mnu#Ahc(5|vbg(_h<>|ty|So{E?dqS zS7V?&GkD(zb9yI;8uvd35u?x_SVYefQ=fh$JqSw%;vF zB`pf5mhvF*KSfnzASEa1-(p1Yr!aJ!Q-QPc8Ip@v@j}=yEG~tQPux+1#&5fGpSbrr z&QzdL;-a^VEDtSqtk|Brb!QvTi?Z$vb({ftH-~1S>tzOx8j;*)s8qhAn~k{s0cX@; zZ=mGQ1{vY(g>W#=)MbA%S3GwOj=)EkgJB9U5b#7SO8u(nOA)uU{$mrsdVb(AqiF$dfOnz6Vuj*_}$Gjh!xRQ60*;jdzP| zElM}`P8%?022@_hmopEd#z?a&+K_lsdd6f;RRXv-cl_UL_Up+-$sEri0 zixkCL&{0#r4Yy>~X;tp|Rnq!(lrXzIDC)L#c;gus(^>sM{K7eNueqg!J%cVFtjoDb z>;9;T^FVfRrc`pfl_>NLTwz^oO`d-+GTpzA1IGcD5Ja9-*0Hp%$YC^a`>7MSao~$v z00<%rXOw%A?XMGZlL;F#ym#A7-BP5~FX+}A_kn9nNm)dz2Aa6nz`pU~98UiD-?iP> zamHbilxd>psiJ4}i6M&(4O@T5A2PN(&`gq|9*t$iT0u(0Pq++i%DzOmW+3%xwvckU zphO5e_Eay^#F*_~aF)ytf3A>|G{!pt^hE0lL2g8Eio4*ShA_SR5({$mY|w$1qUTMgMJfGgnjHZkk0nCc6c-v$*t4Mi~4E0LpX zfw=RXc+7YZSNkH5?9bOaJ|cSZu|n!Q?|mI1(O?LK%F>RvAW6BK?Quof z9nF_J4qp2P>`7SP@{a*05G~)wiIfhpcI%MWXd-bPbl96tQY5&sXNDU?xg4A#Pm`)` z$B%=Fr&`D~b4gd&$*{sb47vW?6ZbhnQ6`@RYUr6evLVGdu06EfZ%3B z&U4S>LyqENsmm{z&|e1_{HRBw6Kwrlkp(3YMyg*aF~GvxJ<-69{@LemoXjXcXyBOV z%%fc`%ezHbc~Q(MdOV!2>|Q}q!id1^iX2ta(7H&UzAh`|pj?K|=aTr=;{n<%R~lf^mpwJIzh-&zqCtQ0kGM%A=5{{a?d{ zsTRmXu88?2(B;yR`rk5yFFO$FOFWZf+Lq|{qMNFGbcw953vd;Cyb#-+-V#~@eQb&V z$uddBWA?73sT8k9xHBoc=Th80Vm8fGO6Lv4J}kXP&T?`40XV{Mu~t-J*h19u#(%xm z-bXk@O35ghkVH}+3wsSn%_a&{HC_UfMi<5=L#migkN<9o{kez00%j8X$egv)fY0(z z8M{A~lG|d{+E={Q*pQ8S(Ol!2@R5T21|S>8XP+%CT4U5?*|f2g)6z1%J%Je0Z&s9S z{XC@@mK z_WI_#PW%>abslI47Nxe_`RLD8&_$bH!R%k3BMuU!ayJd_`4XUi&a%;GU97Jo=I+R% z7z}4$UMHc9JluZLqsW(4AHmeJTzbdsaTFkoG%0$KZ?&^%7r?=9{r~*QSUIO+tVsi5A}CDfib7hqq8z zU%CFuE-;xn(R*4DqGa2LEQ}|R^4YxB2~xzdenkI@#G?vVRp`3KylVTw-w%)RkViBO zDR`r5!eZeCSXuOVs894+ewC1e{k56tEYdgk6^18<-;O^SC;sM=#oa%h)hDuzrS(2J zphJsCWXxf%#max3_K@l_Uiw`(Co+=4v`ppBp00a&ha1SD){#pEZ#z@jY9{2gcVxSq zMz)eWSubr~1+XEhy%f^}L`ss(7V|iQNN0?z5qmCp{bu#Jzrr5^ z(a9%(mrlQnr7k{s!!m^frmkP67ou++B|Oq-I$lB`bY^K=px7`myFr+yuA-KNd;vZ5 z?R)PS?y8K>6n~JRcr2d2X2*Nj*qEc9EHQxQ3b{_mr5J9DiY*5>|x8OJ*df=P8qGoADk(7Vom z^fP@wIYtg^DtcWUHp#OmGd;(DAwvo)QY&QJLU~>z^wSiI@Hxx41Va|9NVD?;)w3T7 z^fxdM+e?GfDu2<|kngSd?+_XMe&EiBrOvNOqf|+vHx?A3xI*$V0?HalUkm1Q?vih* zQ$4>cE?2tzOSw66b2`G1AO?(SRXk*vbZp!-J{>77x8dh!@*q#hJ^7I~Dat4C6`iHp zO4k(sit&R3LoS78LMaF{$5uiq0b{77Re%? zhtb+!CHXCyaU_xThOcwL1e7!~9BTnkVBvxOe>N33nZ2%4lb00_-%W&MLJ)-t>wYaw z@F+MW;R|_TYHRpvh`tH>-iyo}T=6xIH}EWTfg={3KB6uz1k4| zkR(9&Z+B!8hGqSdbS9e%mw7|lK@&2sNT$}o>9MbdKXIptOYg4FlE5`pmH$0vfFGxJ zXx>Rck#d(Z61Vv-SQM2+Lh7b93)<%{omt2(u_Q%HIiE6rpAz4)25cABa(MYrBS*{9 zcZ65&?eX^kN)3#n&j*K|vt2e)zkfN3?QzeXswI(}{;)bVYLbeDAdz`Nz~yC{r$GSM zZB9grd>naGC!;ws{Wy38X510)rhCKD^&X49NI~vCLwdp5pNW$e2H$_kK!|M@f`e-H zPUeH6t{nwne;e0ca-2q*+=})&_%ds2C9=^v$h~&`>VqlzuK01<9pw;IQ5-Jg+w7g* zeJ7J1>(xWz(;|Z&6Shf90^|*t-X%DKxbCN(cr8E?(H}4L71(T*T2EU>g~zHO!P4dX zshgKNvl#lHzV~C7Rn_E?r>q}IPoWIo^k*N-bnIVOxUr^>t_|G+V_(yjaw{0tLv2Fz zX4ivVBfsdkgiJM8%>Tj@T>ZW^H;Ox9N%TRkZ8cMhPClAXp%O*pm55J~R3)3PoLt{@ z@Ym)4^&|9wTfX2MxO$DeLVflF5mQsiK7vSD(gTAfg)dz>OP2gGoKh}K8w}@J$WUeB z1!xw|p2L04kaJGEzHwxQn#N(job2LR?2CAPjd*S-_Z|f)P`FYPO5M$eFSAMI>cnqx zxQ={bm}f;jENDT$`*H-$n=+riI(X}+yn{^ti|Ge^d$@L(TB$yAnzbe5z+npE&~#L> zy(xhsRIK-ZuZZ)^-Xm6_4 zA!>~!5n8cyoQV--!Q;dr8p7~7!ww|oWs3`t1LKYDHk>U<TkTx_k+MeJmf%X&(^MK>HeKe>Zyt%g?Z@+p`?8W!FvXrC_)|1Q!T<; z{pyw> zmadT9Oy#l0LHvY@QC!9IYuV=^geu|ZUM78~b3Y`FYu}s$se1m@+&YkzQVUb+ z*Z-5Z0P?TGwx33GWBqe;#De(nWpy!;2|wzm8W(y5q@0UP;Va&zVx55c@f^QmoT=us|1?fF>N-0+n)&+iv9X zYPwBbLz5^+5wcYeJ43ZF2***t6#~N=7rug%fxyA_N->B!=Ptx5Vf zJu!*^ij_Ho=jLJ*`3vusJp-6qZv<_qaRr{v0ghRcYiJReqvG)ZZHdsI$2UF*HtuaO zZBeYZ*`p*j(wfs+3qg!GS#KJT$S5)dEmSwTDd39Vv3TKydJ}Mg&FiobcQufj{}C#s z+Yt3v()+6#LGebkdr`eo@_<_jUHXt97v~x6J96;@uBjY`*LOHbJS&P=c4YTHx2;g^ zJWZ|j?s&qJDHEri$bXWl^%?>OL{YoyNY1aysci%n&bQR_{6w=l7SgU7gJqlg&6^p) zCA|903%y62`b>wBUHy5{#*)A@?h^orVN7La!t_iLTleLzye9$r0UglKfPW%vWOiqQ zT$%j4koe2i6ok5+bKS_IUiWuqrFJ!_zHA|D25aoPmQmZT zzy#!QvVY91nIJ54yFt8uOU(BoLL`KWKv5M}$aYc9g)ObwNr+S$cC~5!6Q7oB%3vg-XILj8)Ju&~JvN>)xi<7U#7_(5Pd1X~mfdu7V9cU|Z<|9##zwDj1) zon?~KBKt|#MRs}*epuF?Y^%vt_fx$21#7u|h9_tJS~i&E6k>K)?Ts6OpKoPbOsEKJ zMw(eOzJ4>*>$7#2qt)F}p!o5kt`wS7qq(Aq;^1|elad$OlX7j&FW#KN7`^K>y5^~b z6v?Lg0sk9RrAvu=R+_z9G`!arNVgon$h$32zEmrcsWASI_j$5}o6{937t3NhDcD%f zN{o?04)4igK{S8r1`v4M9(yEs@Aq=)-76qS7XPh`EOU|RshkksN&w8ex{@$w<9>VJ z_rt-Ub$fp8}-Hcu1o(f8z?DmRRn$o`*%&=i~v1KCAJ z7nn3|-x=f8Mj{J^(aZf+l|o-R;p2JTbS1J28quPYPfg!A50a~``&$OG4QojMcDwFd zBs#GlaYm8F0|_Tq5A!IG@k|~SQR^D@WPP}5LQ~x~N;Itp0zn-5xXHHZvp75PvdBuQ z^37mHdW4s{2PQ-la)KG|T*(tD|JaI1TC2?w8E znOw3W+D}yS=B9t;C_?8y5L%h>tt9^4!G|y2xmPZ|))9z1UvFxr@^R#THqapuYRZ=i6oESvUJF$gLnUIp@0hHD6EPV5J?LeYquHTTEryQl6zKg z%apuvK@>v^PAIfhX2cZJjJHag>J|RY#ZHmx^+3gYS)1C@fKq7Wt z7d^@$h)u8YVeQ#XDSY~z4M($*X*IrAl7bgHmeXH&+(SAFn?Ts{bHf5Eu=+MZ~{U<{z^8v4@$ce4bM>ri45JJ_oB^752BRRSwp4Z3d=Q~ zbaPQ_c;W?w`W2w-4*yHZ!VLnrxYqNh4*22+D*Oxs|EYtxcj9Gd8nHm8Uy5o%UnJ|O zy^jCnBMcOd%V}Hg;iW*V9^b`x$k8)l`FegJS+ht=6Imf`rj}tWQ@osn2*}|!3v*Qg zwey-|dc!Xp3W{hEs)kxkj4lmo2-EXdyCs+c_0ey}>2&dpNl1}8A&UDJIqHQbEqsMJ zaq0u%%`OV_S1-a;{_@saJ6HRjLzaUl^-^l#kp!p&44 zQ_7D->DSL@+5DaN(;MH_?Dj#uP;*LHRn3fixcs^aQ~`I8n>Ac3c=nR^c#tM9KC-d7WIK|_Zjwt-B~wEc*Hs1)$BF-`jpRxfsrbg zKab$X8y&I0(6$gOYhyB)jgg3xz-hLIo5kc(;kM1IY=>oKve#?5>UI96Ng=KiCPBMh zqietITgKr}=nwnp4!_lR@ox$IaSPx-qev=rTcbqazla0B3_s8~Dcf*+aG>^TPg(G} z!#}nFK*_GS5zklrWm>(9Lyb;AlIJUt`f&!xi;`_|Z#Kg<1x|`4<;r)TTSq zl^+UBymY9hSkJBh$tH*8wd#LC``P*3oPVO@^w(AH!R?~WM; zKLay5L*mXSx0_6^rcSu<(^z!&&@WlDTgl_cD@C;N4=STWdf4RX7s)yMvc#7oMVdWY zCYO57TmLrL?nBptI?-yS@q=N-$jks}vx`3$T(iCf8Dzr{*14iZ&IaE~{OT>Gf9-t$ftcleP80`O;b+YW47o-=b^WVeQq|;nf zQ_6mLBF*#LUR7B^)D+{@yz8>>1P3q;#)o8vEeu^q>J_gB_7>`!fZuq}=N^t9;qw6x z&5zFOP|*A@QYlfOU;{wC77|d*z$){GDnO_^rViK^s7l)ZtQU#5YSw=Fd|f4TdQMn; zl~9t>svfn+Oj}muLav!#E?vq#|6#_9Gp27`e1m^*+ogi0_isy8@D3c+5z{i0ZS{Zh z=m>7?kY^ZZjs~(RSx;Z=HU(5Np(;Z00&gE|V3-caiUsEO9tkV;t%tRE!BrDG=! zlEchKxNgFp#;Cx%?H2hvi&U+P`zd*>?bSl!<^tkw{e?vU6^01yjZ+9P0NQY^`f7+y z=Dzviws{xLihaX8m^RdGrYD7D{7>Jo#JuU`=x05kQ?n_~?T)XvT-tyEE z05*1d?BZSS60mg)PM@D+OHL_x zS2rFNjNQ_SZGD&CrRTnm1}^{dY*kUyo}gMT1vdd z2biEcOD~@fkR0DdP#&?ToAy>LatWOZGq=otudrwwJ6qW56tj13gOv0D;08FYrKW%Q z3ro4k9==d*`bF+KdVQvlhh|O_R3mEWP5Hx@8teTNBHKBE$}m(OOat8%xBp;k)f-lJ z1nVAJG~itr3dH;=ZP~(GP-$d0V8s4$BnJDM8N23hs8m?KWGe3$M-Jw~9d10f%#K{a zAl^|n;iZE~j^C_f6@^iK?PdmjYP)W}{|)efeySurLf}i|^us;oG<~R_*CIc{#H2uX z&xRVo$xjmmUun7VGe{=GxDLvGUNmv&^-=1ggL?N$U?= zX0h!2`22(opd3mknR6ie&g_NPjyWmXB5^T%XPpH!>)DOP;BNV88zS{})Vyz8ijs!9 zEVHc4)sDIFm;;Xgz^!-zF|t2BKS6hBZvzD9l`+DlFCtwl&eRI+#2w7Vv4pHv03dsX z0H)^AxxyC^NV0VVmETYanawO)Z$8!$Mhpxli(~{^X|JBvemJy0&6`!aAJtQqTmq;P zTJ1Z{T$%ud*yW1uaqTj_$@`IR(^rh7V%xZ~pah@+)rEn$OV&(4Yj6%lKdQlXzhQp6 zu+qhJ1fYeLqpR%zEVFw~Rnt;pOG0~7BEMra0HDDImKEcWN?QPlmn0Gs=-4|_MGYbv z$AN)il{ixS8Nrca5yCNy;Oghnp(QgL0h>IL_zVoz`}6Rs7ccel$iufT)YB_JB~cdf zuB(Gz#x4(wnsf^1xcqSeM@Ps>%B{|9uXvQYy!{M?pDZa1YNWe->X1KmrNs2c$ z;umepmS|L~`muK#kLECs;>}IgW*1qB%6O!A2i3wSCN!7)09kM`lW{H6&X}QpCY9Y3 zi{KG(9uqMYgjNN|?N2%`yVS)?DxGKnVDy}k-GI)4oUF8BlXvV0{)~|zRvAat0YvHfsPCGqM-ck1okVmCUEW=Tkn1moMsX?W1!7T}-wx zgqki4UVT0d0EE-B_M);^!e_FO9bxv**)vURSv6hnYDW*YE5p`EU<&ab z4XF;!BD>c)9Uf$qdE<;?B$dy33QhNfOq~~tz>p_0XM&34o%%cHI}e}iR+D$f?N9O4 zGED#w?~b5eMXRCP97=fEGH_lJJ}LRRbOgQJwc|3f(}g0T+pRkLd8hIaO(x|Lz28t4 zQoOSulK5~}cw|GkOa5R|zT4ClgCAZZKE$vDY*4K&bBB^5>+j|j0v(mM>|A&3wo*N# z4R&9VmwiBDQ1i&(uj!P}y)8dj`0V>B66^8h2W^%D`{UamAw6<&m**pvg=M31_ZC{ zE^&^G?xxCt8)!u79=XpnDqh*lrdI^}hl!XLjqvpp(%b3lZ@7(+} zgMsmu+XV?&rtXl$?Lvkh8xg=ZjKnl2F{`>EK!;Z@A9G?b0?w5sm0wPIL!nhCILXZVlZNR_^y_^+hi})w$^JHdWV^*-ZW*0FN`ENxbY2n3A}$v|(RubA5j@jJ&n~ zy@chZ`pr;zDoOVrHZzmvoPtd5k&qIICC_=AYxnprh0mM{vlyQ)7Eu%!T^7C71IMp% z`rKyAd~w@XHOIoARUafnRRzSoKbc(>HWSgimJZW_ruK%Wh-y0PPfR=c#ou!3E5Ky4 z-w|1QIF=Kxa|O8sY@3(^Ft$1cR`s>F^~q8u(eqgcFEUzpFpdR7$VhvvupADb3!iVt z+Gx`1a9us?nk_;Z*gk2sPQy5zL$}%}DUz#_F!4;Zf}(aXngsn6Od3#j;*kt=_G;{i z+qX<5&=(8Yf4<-ctkDA9fb&41i^dlF$T~aO*1dzG7_iwpJG5%Fx zF0EodjYW>;8tv3Q5mXD~TLpV@{vdh$!z2E|)kpR1yEnt1I2sLkW2gsPy$^*ZI~yq0 zEpWz^TED8@%}>Su4*!+pokXU+$~gPP7o5OAq3-ijOrtqLbdS^xDd9{Kx8%=0v&P59 za=}oNP}1~LifJM#K}NdLQ(x6B9}H-(qlB%NgL6VI6RPYeQ20MAWp&Mai>`S3_6Z>U z3t-2p?{N`xv=?y6%B`{v3Gp_V6s9AJAt|Vw%%;cbGI8(YDv|@4kUz{d$Nz4N0S!C4 zIqTe^nB|u#kD={@<+qrsj>O7gSuJCyoVzb2K}EJu;QUYI=Tb>oXSpmHr(t++Td8iq zY;DW~AU$F2AmGM*mY;H}=ZUqnzf8xt)yDnq$4!`iY`?+t6 ztbE-SsHwww{-=un6!I{gnGC04I7s889osySP_~Z2h787X>%>5UUawKV$a_%3j-HgY zk8#P*K;QVD#MFCy_)%WAf*WvCXCgXCSY z8&K&1QuJ);KLta=qS*w$dBydI_7mO=Ooa>!cSYJ$OgTYJFQ#8qtKfw`y}nNQgmhx} z>tP#UWY&~CTo(lG2%D);RTTnxldaCHEwgC+%{(S?Co!)TZ^jcGL&8z?h`~U} zX~bTX!+W_hu&uqaIQy=FHqF|pljIVWLWWR@U^_l4Cs5dY18hyeqb35_8&@sHB$GFB zze&BovSZYdn;jN*L>wq|BUdoPG`4CPMRE>EIh&tv^s9M2?0Yh@j=?(_RK9A5j7)LZ zF((qBF>J8~`Rl=1ZRr}=gr4=MAh+3G{3!vPq{yX^-p1yvpK50la2pM1ZT%%2wBK)f z)-gk=%oNVN6iefj*Z@V2g&S4Y);@-e)s2sIv)i<%jW)8J77Gf8Kmj6KFMRB!fz=dEq$*|+nCJa6Gv`|uF*9?7OK zjD7Q$r>|~Q7BK}76~xogbo=2`Ni?n>3xL9mm3#od%JexgwF5w+rOZ3*W_M>M9zE3c zQ9StMZ6hJK?U$fEY@e%AcT6FOr;TGYRvH{Sdur{1WX*iCf# z2bU2b+(IYzDvj&g&SxPj1*VHKoGNiYd}{C_+kndzsMT7;Nj7CE?IPyUZPu^18ap|n>!E2}fXT_bEK_NWHzwW=u^RlvJKo`a{YnT?c^PqTg9#v1w%ylz)&d z6L?pdvS&J_oLgkdV5{Z}`f3o}-iO_u9a?5GG{B}c(cx~DzhLUum#uj)zONmqYu_FU z%GJ9Yw<@`wYy68r{w{$PS+xtZnSU{4|DOPkI6}zX(M(2EM^7E3MpHtE${ zr#sQRJ^QZHakbNSKF5yQemiAmx^5g*3JxISnSJJ-#p+b z1VU7~X$S2q{kLPA?Ta|324fa_^c($NX|qYed)4(z=KIBk4i%Y@IX9|k9$^QTq;fX9 zsM8S@W*$z--prT%XL&kKxYL9siB||8l>Qds=e=OxoB0e33e^B?ox+GF=vOvf{Fss1iMs8n(;yq}-Z==DqFA3ud z4HmWp(xPt{RwdMA!RBN{CK3P0!9t;O1)-ajly6Fay^OXQI&=v+?cJDnVRMf` z1|FiwA}85IAW_+?9P5WL_?k!I36tBc(K0aiLj_xb?n|4m`$CBwZ`+UUe^v?VwH8%Z zdOh3Bo@h$`OYOU_qXp@`FL)GYA`~Fv%-MrJnyRkPy|*|J@oL@Wdu$ZVZn2X&-&l1u zj?&{m(U50hF5YWTYVY-Z+~l1g=Fr*)@jqxi9$sD8g&8ScncVvhXdaq7OaqF~8B_Mh zI^vji>!$i7Doml8+N7IS&2$G9Ub#YciL(s?^anPBye=psk$+AC7$QOVsRzb}qqU0f z-YIYrnVnR~=^?oab_aViJ?Gr|K2CbPmo>T9aYM?t7gK2)#u6u5o&AxR4?~~eR`Ks( zc}6o4aVTnssH#|o?s13>a})E$U>!>89ddHX#~3Os3W5Q)HjBME$dAU;KYN+>lZ47s zD)(}v$1S-m7pedlA+Lge%AD4zBXJf~wb`~>Sp~n24P5`vqNRyM&rC78h2J$~_|IR6 zXQkaP6di>DCIVp?3rGE<5MAKh$9B}?GVo|dw#%&^0*YdaIFeu)z^8pQ-0Vm zsg2pAmVH9L9MIbVHeucMad$J1n*+K((fxQXN zA@!|m880cFLeQYYZOr3f_Rc$Bf)rh{Uf87@{hwO4){XBKQTUsl=d1Z-JBjrKM#rvzmiJ)rCRcp{2*Et?v9 zI7ckn?2^Ql!QB~V;v3q7Q}=Y3@mg2@sMHXt2!kuR4A`2{M|dIcb6s6fxBkU$@NzJd*d)!3=s~Sfi-gY zG`Xh5ZmJGc4Uv+;_Gl&7p-=oS1Ca*j7uZHq9@D9WqE9D;E*(2OaR-Yoeq8G_(?q7{ zUi%tLb57WXuOlVv_cN8&Tjt;jv@aHjcJ7iMM38@L9#Db5upHEwG6WH~Uv3=CF|xm_ zKjXk+MHf(eUhcs-hdSg(@CxhNq^Ja3=Mmh}s1q0^I0)H7*xiXNSRC!`@BcYBn2H%T z&YsdLriJWk%TASL@-uTFJLi4dIL7#ncN&@mB@p`jR;G6Y%;faXqW5D=-@Ib_zL^uk zsI4y9OjOvC99^kT+mPanVrsP*YA!a6<5tvbvbl#H%5S4NouO%`Ih~_MiUsoEsyWE+ z;n!MWVVfw1SVadFFkdcZ2F%J}%wXp*DvEG$h??7IzUHb`e5}8F95h_SyGbW3dfuXk zeAWBvujwy;J%^yIo2(RDO0dir=_P1jPD3DF7O#@RnMo3(O@%+TK`)?kW5`xxfCI)n z@v#cAnkqJP-+c34TXXt_AH{m&rXA>-JH{5??~|RGiV8_V>MuvbEw$Div_lSr+60~f z-9|yky3oirQE0Qn~qt8-TT-6|$6# zyJanK-pm6`-Q!`bjpnmVDA){;RVKv|mer3@{aSGH2{kV)YsSTgZ}1dEeOm^Z;DdlA zVwiNhp>B{>3T_!LY($gHe`e5~0f%KiH$ReYL{q;d+a2R}mPqq<hD$kLWDl4qZbTR9qwejki&yTVeoAUL%cJg%<;bfau^NECNWSm3^HgR| zK6?2ngnK1k3D7K2$$&O%EFfpLo_-zxj6s1X7R*a< zlx{eDyg-v1b%I*_ZN9}wMn8re1)gjn8^E%J_OM?|ERVib^Ef7;pXpSIg5YLo;qQ1W z8Gw1V{i;WTHhAV=C?Z*l@%w1E3}9z?!*&nbhY?kS?@3>x&~>Wj_ABLi*{|svuY7nu zNGR?}PoF?^T0#P$N`vY0 z9sL_fm%WY6-BQKEl=k7F6jXG|djVxFWv!OZp{}8>N=34-^tzEvfrfWH;6$kFF0tnw zY-NGAc;oAt`v%CGV@NmV88Mua86Ti1Q9g#cqZ2tZDq&a=)<@h{M=0Z}~ zCe)lOeaK?Ny2FYVn|Zg6d`Ylag{n0kK6z~?ul(k99JX*fck~VO}v1bjJOP?K|B# zt1wuB5#5!q6B82~94qk9(b)NiyU2=$LUTzqS^!RWs} z!H8ya-W{dPCStoAyXY(i*sT? z1r|HB-f*#bpkDG5@^!1>!S}Cm0#ZM;HC)0kOgP-mlb8sj({V6cYz{IvL5SY8^cz%1 z*(%F#ocJYCoYH&r?HPWW{jyO8B_3JhVMZYEy~{_#pJq4WYk z@H>g(k4<&SjVgbZ-p&+xRcF3$kIGv=bAO57=P~^Dvp{qB!e1&n{=Qi4QhQfx2%lDd ziLtthtkH%C$wXCXLf}Mu`g_Bf6?H-8VcXYN81NUXvjPoAY&>7LEb~IHww(Ms9cM_h zVT1$xMXX^AS5a!suFvDEzzwHE?BzFFs?-ACZv90!Jo=FP4`R{h7%ShuZWTZgM9?}A zq2R@xW7r;>0lQ_`eYegnQP?Ft;&?-K4DnVqYpwBk7$>gmQ>~x6p}#H@wvIXzfW>|n z7F+%lZ>1y1T->#E*S~?kW5$77nyHbC#qjzwCzXEvnxjU?9&(v@ZXz# zOL+6Cs0gX{!sDLLuzS4kOa2%O=oP{oZG#=E$BW_3ie}=jKT|msT^}-w!*SQ_FD};5 zL-lMceGW>ZgtG%nA`z z&!QTKX}bC1xQ&-#L4;CTb)gOch>Cm+UKuQC4%K-WKCq^kw<}%9UdHb4(qFwG`b)Nd z1@9+Mr*$3$Qo?F>z4>anTwif~*xo2_)`ssF%NN|uwvkZXnVAB6R@C(@lW?aSCXu%m zd0DmABg2#ZYXTp@RZX?FN@aS}Z<&ti4TmC`xBUED55rvo5+QuX3As7Mimx@6my zZDL2pn>3-Pb^Yc&oH^d{M-98-HsO7SR0*?O>jiR0RwO$PVsB@QH;NY8E^x%cad=Mi zfyqzpz3AR}JBLYl@*oGg?TfEpOODwF??!CsTNguc9zqVLMfAbQu^QX?a6ZjuHrZm~ z7VYJpFaw1MPMzzP=o9m_$m9M9zp-U61q{e>gd#XZOtPv&XLy_I*IOICXImc^Nr@z% z05)|lL+qDDWyr&GqO61r@iRV{wQd&r&IZmzwJwHVcP1Lh4|umVWYEke;p#((iD~B{ zvuOoCw39}vVrJsVC5`i3Rp(R~XK9IxqC_)OiX3PgZ-=ewYg(B~&+L3ib6EHVxZjlh z06%PdZ7|{)iMfI9rX-E$8w(b=3f5a(`DbDY!%OPvcy&z}O~?vmTDY2QgU^ro1`fys zmt3PICn?DQ4(GEzXGn`D&Mz_Rv{MvT^P{$`ltx3q@J@*Q?@0)rRW^t`+JqZd?I&mWm= z_d9rtE!ZP`OF?)Tvo&mq@e_Qh%!;PK3ZN*7XVX%UtksORkFu z@(#x%3S^yoF4e>Hs(p{*eKqeY?h(EwUbN|)`FYj}K-=jr$hL~>sAO6ySqq7OS0zF; z8aei%OsD?4y@-j9y~5k#MIf1Zp0n&GqP$pAQgW;6jylv+%W*DgO9mg~m@sOo^V7VQ z4*7C12WxbPIVEkTW{Lz4Lju$oIkyi-&%)GtmM|coNUfzA?e`c2N*ZQrcG10PY|MMN zjS+qFCFQ5wazimAFZxR&rAo4?pSe>u=T795c?+v<5v2vwP)FRY1rPRqLj@uIK%?(| z$P>&tf^bw1x(&b|TQYoQoEz@Z1W2*V%U>HB8uF)J3*HqSMrc)-H74q)AD=LHFq=+@ z&~Mh9Wu?YnV?%}{eta)Oz|}9uPlzCNc`vn~v~GhiZ-33igXt(O)6vGe)nL|8>z=dj z@161_m2<%KU{DodX~J9cUb4B09nG8vmMeW3bXj!Na7=ry-|oHTtU}vlr{YXLHi>aW z+ksf^jA!>lXxa+600Jj46?R4FaIZZp?G@7s5KuHf(rDfJEIfuM|H%`M(arl7x&`K` z7ahR`=9RM?mDvkWm~QABflS2|%!s%&Ajp`aj8PhEp9-AGv@18>!c37a%hrctI&sQi zFgL6(D&(t|YQ=hJ7e=XV9s_gJ=%hH-@tQ5i0&H>e zjW(;S%%R&Z*A@38^sAe{EG*`=9#D#YW}W&q)=>u!$650pE(I6;-Ktjn2&Z=9K9gvV zz-_~C(W&e!G0uzi)3Ftn*ap8!=s{Jtac?TpUiwN`&A;3rHhh?+5KngM6K6X|uX^6) zjSB5I`}W@`7UC{^G8A{+J?=q&hhf(%7Vm@>;IpD11ZNlhv})aHw-SmjDKOXQN1Guv{E zSmjnt52hnOKkmnwsf}zQM`N^q$+j=r6^YqkRPF%~2buC_ZqwYMMMH%SsM@6+?gk!y znuy3)0R!zdXm;|oQTba9z8L&u7DmpsNAk@}{4i%QEd6Wo)rJq%D+kLfD)R94;Dub@ z-Pf?)*I=SGAa41P%VFz0F2@D{Z4&!&#A4cYz^QmoC*9r`;=Sk}%|t`Ir@FL#>PY3i2ZJgBZ!p z4aWw-VTJQZT(M~TJj77?Tg*&trE>I=c~+)%%z~2k#%*oI!-)-d0%x7A_u*w9k*RUW zR7dR-i;N&_tZKU9u$i-xPF^jmP21w?m5e`6=Q?-8Ic7UllLxB{pS6ZrZ+6$Fe@f)& z9(||79+o5^ATVz6#vdl&;n7*9M2Es{zD3&L-l1PSa-^%D+WKBm)LVW#s4LmCg)sY3UhpFbQiX%fX%cYodW;Wxg7fvdYD zJ@_pZmJPiXnG50?T$ZKZTtDp>nb+5jy!18q(l2Gt#IKQOp^<6+=)T~p!OPgYPE zS;a1JNh_5AGR0L>cq6hs9gabxfMMvvN?Kr)dKV8sp3XCXp1s3f-q%)m$QlOQwN**v zy20z3cLc6XK_M#l5c5acuDcDegF$#DdZ1Fv53&MqcwG+M)(kjx7_~qf0Z0G_Xu&cb zw_D4oQn{s3sf{6UBQVeN1vTuO_cmFzLy4b|QY?U$JKu~;R`vX{_` z!+-+F@(2$w!pQ8K6JqVaKERI0)zj^|(|v-k?7*{z4=Pqr74y=So6-xEXWPj>{F;%F z{TetMT&0Rz4Fv1E(so@I+ol7Ix;D?`Vb>>iExE+ZcYccJVFDGP36DP>_Oz-svxaWC zLv?3}uh~}=n~R=qgNOAk(Mk^Q5HO=lhwLB!cIW?);HT97Kc*-&Z%fhUccFPeTF z7U}dyGpU0K5+68mh-N}3;COxmmn7B3Q>g}`&{rSQ?sQZxT;qnz3rf4z+FAM{41+l) zm_N#@t+UrO7fot-c{&xZlw}*B^R#)&inWrZ4}Q(ap~gmzouaL^Iz^c|yZ!#jc5b*b zr}NnPg|l~SO=Pi0F*?vaI6`OeGi@@AyQBvXdIQmFG0PZi8;xxoYB1$epG#l3i+UZt z{g;62wv7&0f7}Inr}-SG;_0i!*YORv4a4qi>p}1K+hefE-{`o;Fu*Ql?HwC!RrGfP z^uxMVPh6rsw;F$UAnuDw!*ubxJjtJ^p#jFuaH|+Nh%GoSI>IK1l9aP#HlZW{7Yid( z;YYL}{+rAHT@d@st%BR1zP{EnGBPQqIZU@09~kei=5`Aza%^yiSMx=q201pS)sIl@ zI*X9mI0xRk4(~ONrnq{qOQ#;uLcv=F=D)3%|MBs3k04QP2!-L>=d|Q&+QX(umb?{b z2i0bi+Ao1KL>|_IhgbIz%{*9J&#u!SG`c0RA6-Pnt5q(Z6~+9Sq1pbHp#y7%ajE%z zNGTm!_x1l{>&nBST;DL7YLqgAEXOjIkQ!UcZYbmwQdG!JMq)5z%Puh?TI_2J+GNcb zON?D+21&LdTlS@~FT?cvMspF8f@ArMr`>gkK-%6k$$KL$@81)T%mHDGn zkK}z(Bj-2z;Jqi4A;u~0emp^38zjQB+`65Dm1Ng@GcKFn9gvB_*O*w^CclgwLqVr( zl7?gKJ?VVY-DDOFWnrkdEzoPby=90|ci=dVLdSW1YTF{VT>^t+V`8SW(LtjMfzJf= zY#fkGUq_24*ZvGnBpp;nYd3Zm5km+jd3Tr)Gc3`)Sm4yO!*_s3&kOvv7F3;apnw>B0{)4B0JtU)TzBxXqG8Mq zkB5s0Qm>cj(%iJGe}iKI-r=t=$=XyAy)d!}K)77~89>{Xw3S5NS{VMng55lO&$u9i z^z&4cnATTR*BMN>$KX7+&!G%R^yeXut?p4`5XaHnmd z6?(x`YvfwgWb8tph0NRA4v}DP=+=V6&Bom4Ua`-s5YJ`-l6Kz1KtFlXt}LRyD7^qz z-(XdKLGZjtNFjUk64Sh@u1bQw%CUhS)7e|w9f@sbZd}TkD^gj@!XrK zz6t1ct_JBx$cqAYv6-iNBVLRix4(a;SSe(e@_;s1<9GwSbvQoX-(GUwZ`KZGCcZy$x;);Pwhv-u9aEh{D-*$ZFmAQGL;ADOsqh#rsb6b zfO2CxZuqqW5&=$3 zv#Nf{ANwkMVB;g6WVLPMb~!tYF}CN8qIS-X~{Q|$Y1=>V}gBwb8n@Ny)huLDU> zZJAw@yX7wU!REEQ)(Bkjn;)GpJ%F;U0=9M)8&TeNuqK9j64jE~;fqb0R#R;gPM&go zHOD=IGXum;3jPX-uDh1Dr<7wdegRPLymIVAC);EK!&<#>LI<1GCVSU7Hd}-af^a?~ zxE4opN4xHv)@o}=+&I&JJAWm9`{d%7%H$DH@Xhn^bZVnQRpDk&6XJ^TGJu1!dKWE4 zn6QP67plt0t?>&8bjfWC-SpcaSXs8^G_m1aE4A7n%{pdeimCt1ygE?xJYvVYtdxL4;baszG}d*2-rYQA?S+< z0LnQq%eL2U(azC^X75WUlXs-8x_-l!e@vghMBVNW4-J6v5O>RM=`TLkz0 z8cb}^)@TXk03%=E>JvxRp7wTwfN3Moa(#qS;7=F>G(geLb{z*T{CK^vOF!FI04(5^Oo3;}dRyr0{@wl6` zXhgzj!40)V=Nc1b`h<0sm9cSOLDKptQcsACK3j2bO|a$DtQh0GCh%dku<*NkZzR*~C2tIq?#G~CxE z7MZctjx*mbw0HAdt77BM(5Uxa*6neg;D2|MT}SeHf9wH+UN*Ts;beK5Hh1`Xhy2Hf zIV^T&KgXQ-K(?N628wm$i^T19d|jX;oe92EdM}Z$r^GR#M2}Y7SLkd>Ty4?|HEqH5 z$A+ZRw|-WLcL<#gzdiEV$Npz#i|&jCJ8X>!irbLpumwm!QulAmmuB4?2^(XY_3%Wy zg7fC95`+%vkSAc7U9g1SHK;n}^Ie(XC3mo<($>@gUGe^3a9^Czf8idp6iZ_S_- zU=AKKcJi1Y3_b4 z+a0*cuSgWG&87vN-I2NQ_;~Je;zGsQ<|{w3iU5nD4_0Z2QJM8@#piYU{*KBgtv%Pr zlZzeNTBGF8|2nL2Vjog_8e8M@Ss!_z8R;hPB4^>D;`?TxS=3uEAVIInu*l8DW635< zGNwR#Aqx<+2MFm7ngg{)iH4vY=z!-zw3H4D=Dv`v1|2R%AASH#CBmzO*}#4WuNn$G z%}-ufu|81sx%>gUmVQD6$Ew0idxQKONiXOv{d2Ks1|#IYrsxNK)@0APsrgQUMH6o* zFLa-3*m&`H47Va$PJO}7+1GWwX*){Io_{@C?0nTk0kshljvHS+ZbL|LEq7$IM)Jcan2L8d z(DRs}p*9y2#Og$yI_xKLMF`;e?1h)>w&_9`mbCf~cmwr_-g;%<{-)mS7QYZl8Hv@+TS&A! z8fMc{pp1w8L~a1F_&jsxD9Cqx`hwxm)cvx)sW~|8WOo!Xpu^3 zjmUBym9I-7qa2HZx>rtAlfZ8CTK-M4Bbq|BOvppFe&E|ljD(Im+1v%OOSdqqCf_y4 zoX<0aSk1HcKPso$y&pemE*;@6F-V**JtA%*xIRiF%5`7TM}8YCduCqi^J=c}Guz~C zh5(DtsW_kzIFyK6M?b2HEl-+Q2Y=ZcJ=HeDgTK@WJf9aY&yL>KUK%0+MjrIz#rdtIM+0WJOdr_aW!IT zRtj$3<#+SuC820*q@ek9&Q1gcz<(OwVtZm~>QB{cb6jxV_7a0Q)o+q^kQJ?E1^b0( zPpj=?u#SLG!nVn0Ph8AGlb96wZ0T#bbN6KsPTE?I^Fn!+bp?x4j>A80gFa9pgM1V& zFyP2dJzc*%3}D&pf_Q?dvDmI}V|znQY1)nzuRi8%eA(5QvvFdrZSRSG`|YKUlS_k5 zsF=(3!D-#eJP?~bvzzj95A5#}$_4lQkWGHeEEsmTUAmZh0PQ;U=&!Y-^)AVuSZfAf z75Ag7?Gx1D{y8^%q{et^#{!<;0=-!S$T6PvRO(!-Ac>MXsU@a-t0&U}1s-|3_OIWq zL;79muPQ1Y_~i-^fW@Q5;6xUIqrYxj?EC;wm#!T3v|khwhKZ(-nKLKY6s~9vtWSQ9 z4+%i@p+*2s)K5-k8*Wt;C~SZ1lLltDYn+GSo)<3_ypgMMr&rOrI>DmOauqO_#5Tp5 zPuG{;;b8VxfO%S%@0AgOh@o`9{tLoXl35Um&pl{d-~iV>P>u$J?D1~F07FMY0kL1l z58Mp^;R8-2&|Z-&IL^@V^gjogba15N(38tQT{!S9=-DjbQLeL1j%cdDRQnK-Uv{-r z;V&OEJ-;S5w*U;*-DST!LeOO=JGufC zfd_@mKtV>FCD~)DqcW*hpdef-Z9!0OW~A0Ra*x!OlE#NYj@ZxhpOR;ewDm#f3eyYL zN6)X6li!JyUX7acmZ^MHFF&YBpwvUZ+u!%I-N$o)DS0Docq2u_u8Y4u4d4j1m?~Ky1WHR`VCB99@yOQa+6C#EmA|&VLd{mAk2@t{ zSCiiXT`n4s=cTFnVX24LEXC9U>ZdrKhu?BStDLjN~;@C*cAL z%}!;_yDL4an9;dI$!0H)$1t8%2zx?&_Uzd-Fy!@2YUo@9a4s-v)8S>qLfV^dG(4XVW9UqHFv6s6l?n`|gYSB`<)XS~2`nGPvI7H@oe%XmFLg+K?C4HWm-(=@8bT zSEddhpC9+L*j$h(9DMvjj+xc5HAlSn z)J125ehA}an=f2%-KqXL&=C2WBmZO`}agX74x$H4P z0;o9a+4ozPcb1{>e8N(88`7^Oyh^>8YOgdn9`Ne*D*EetwY}la1|)nGRenu3EXdhu z^$|mxX}07Ye`ay#$cc?S*oT~8X|j16biLt@_SB%KJoZQH_4^w(62!}GGD=(e{F*!c zG@@mL;MffREpr{g8XcTJCW@f4T@=`%PrAk_G=1-HVOu;lw&|@jT;5UKPV@_mAYU7e z;e~`A);I5Hgn2(w#?k47LJHY>O2+)lV$Vz__HJ)hSqaB|$p|NVsq@Yh8oV3YB{v_{ zQ{|g&J}}Ho(PJlzO^TeqI7n%-x4i4iIqn-zd-Imr?ps(Jf$)n|9Pnm=?aqTf%|EmB zpZ$b?emVcShF)N>wq6mImr=VHby7=n6EDZ z0g)^=O+-u?S6jKU*Lz~+rV!+mRlmFq%qB@9fzxt^v_Qm! zlYpyvRqJ8-339WSaV;AHZ|CSb)wZB#K0=`Q*TX{Z*#pk^BhVkXxQ~`J7fZ8$6&|Q~ z8u0ch0iKztXWvM~5~H~&LNd)~PkZzujSW7$CvhXeA;futFKzLp+WAdFcO-3Z>V#tk zJ7F#?(S|VLa_;}x4*l(0F;k?TFt}O8R?YqQ0dh|X!X9<^lyKI?!%pxdC6%Tt_M3*Q z(z&w8V9BIx)(__f!&xWcgP3=~@NnAn0c6*46~pd5dxi_oe}?X1Ei!me$30ElsQ2Viu&r`t zPxRa5p0059Y%Vqp^BXIc>^@)n=-_W8Kfp6Qc4Dd&5ul9f>U3Wx{dOn5n}3K{h<=#p zM)1`@6H-&>ah4CY+FTZwg5VA>wd;NTxxbQSsvahTUm75HUu%3R%WZ-Z9(f$#7Tqi; zHJpvw?;2i6@4{d;8ERnLw#bdq&Gc7Kte&+wgqIWNb-qHsdvTeCz*~a-R`_@_<^tSXX9FymNb1qMgfbRd~0stOnKOoz3 zgkd%`%XcS}|9=RkUy1f(A^wN0o$ z)&-u0HICj|jt{Uk3GJYPH2wYxTj-*Vl7jFwVU?pT(FJ@UaetfCSg~csT#8FF64z(? z@K@gncnB9}K1$Ia7FK{KsW>MX3hgVMX3>vM)!Moe__tVoeFJIlm*WS~%^yBoL?u5t zDz|H%>ZA8alz8PNM!X;@Ec9+>yw@jpeQCh+44Lap@_n!hJ7f3e>(g68imk-J(&f!*}p#CHx^YIYtr@88AxvF42*WwM9CpfIUn5-NE`CwNJ&O+z1Q zRXZB<(rPjmqx3Iep2mI+VcdZ&+vRWM2`i0YO3IRBnD(Rkk`IJQ0C zmpCvV4!ER84&1dPP5LPTDhd@s&Qd>5_rtpL#PEtyatxLv&3_~+fB6LfSz>%hY>Z%# zv1P4i4B!Qc!bR?83xH1ozBBx;wN0<`+ki{rZT{AoV^;D(AJYv4Pw&CG+6sf}@2Lyd zV2qJ>*zz_&uhK*Gm&5iVtUOU?l z)bR!zw;WZZn;;1jcU4^}<`=Kldbsjw2E4ibyKMmY61+-*9K$@}12+AaL;w6E{BSU3 zo=%Rdee+Q=V+4dv&{E@OC~$g4D4K!};9(a6tyngE84WSh8&3Om-ss^^S zRA?1(k=?3Mbo-=}T{9N!rYF2kaIex_Rtb2U=#heO&LwPMOL)4#g;Pt$bDHEuUgcbl zZArg2?(!Hy0vDszyyH}U5ved`x}*?(Tj=AMK%7HJ$GvuG$MD=l``>FSz%>JWJqBKT z@MFHohQH^Xbng?QCLH{ji4MY=ZaJ2pdq)VT31&^YOPQQ}q3kiP_f=Q%jCtYw6Uhc9 z45($!G_b?5TnH$;PT#wHV8T5nXi6MTMoHOV&?jctgJJa8l|y{wAyZ6ku*f79_#GT5 z_3l;HN&uSJ=`+=Ya`+?T%>$c>hD0uPDaRA3>_aK_?yTee_;Jez1n{*CWx_54G9 z-@)E9K?YW&lkx;oNsoQ%nq+=|MT&6M_$o*Ky8x90?7x`W@$%r#QH9FK)Dhf)t%RSb z$zS3UK8{%?!yVYw?>ngWjAK-Z4$9GnghC*vhVo(;+FfRkiq-Dxi3`N)lB4a+F{Vol6`cC^agiXCk=zt=hObalWNyVM`Z#(xaM zS_O2F8{BhK#zy6G>80ONw;$0<2Eml9HbYk));Nz5lrxQ}hFOy_7vQmyoMbx+yt52_9lkUv*^4nCtOG zRF_$ZPftax3Q6gzf9m?25jlU-jN203aHwScFX(C6qzN`Fwhj;5>~K~Av}D5_mu>u4Bwc-VJ=Xw6n#urf zzo7vNKwoC%nLcCwynF$mjI!Lq%D;HTghzcSHUGoIdu1@3=7pb*Chk)VQxqM$C|0{W z>>Tmv9jlM(A>enAVYrhp+_dYSiG43X0j0X(2P^bRoh=t`9Ix;%j zTU??I4ue33nLRtABQ(MguRAez+%-eWw974hV{b9;x;3yS+thHoveuRNWfvm1AqqmA zQP~#yL1x^J;ghLd_GyT?z?uSmv^OviRksB%G8FK?d zcWvB1w)x{Y?OY*rMb}TFiROkoa6zAG;_!dX5v5BZ0Q>)`YPu zfUucH@1^;#)BpCBOf4R7p+UOpHb6u{{an7q+ds7D{jJyYHCJ>?-Tq)JhYK7)+-@VV zEE5n6V^W|AR);9t(9r-jN5g_yt~%zN6+HdMv*9X~bwv1na>)Rvg1)V>WOZut_Q*l> z+)7PU)#1?N=6jx0$fKBQF4vVqrqIzj1@*4={y69|+G6bXuG2=jNZF8OmFy%|b=A?4 z|CF@&c{b~p19X0_9hR-D>uYp2nok4(lyZyt^A1#J6{%2Rx&mN=ssh)LXFt;s!aJ-` z*wcWX`)4CHfC=x703~?uB|0SGRfO8C&37`a9g)c$6=y)t8GoyvLssx>T67tp;T-!4 z01nPvrmO6}=BlFH*%2+h1z+2N6VeF=FN}i2GmfOPet_f>MQo=ct?0ukV>> zLbrtawLzashjE@Z)~#jOR|NFv;F{*$tcjBNO*zWoI9vF=VH0%pq;Z99Qswjc4h5i1 zP|9< z4hrlFnxz6l<`?!eSsA5dumFNg3~PX*9KiK$3)9a-$DEXJdRO)?*4ddZyZBq&%heeg&ouqKMjzb1 zTat7^NA zR!6SZiGG#H$nR7-4an>`H;Fl1&gJQJ6-1C{7Ozq#jAyQj&Jh(pk3WkyA`bWAL9@HvZul}m}eiP(|NEJyd?NWIb zL4M+7V1JKu=~%ZXx4rfxSLw|0ocpk*2gx+ZHfJ5iy9lx{eYs>~dr&k65LHSSn0denJSE?seCzS3gPdpsV zPPDvg;ni#ZX$9~})8c_6;yY-}6-7Lq2+3oLZd_OB1JpK@>qnzlQ~)yAW)vF!jrzdK z>LovHJ812y#JjuyPMp36r#!J>3UduHH@PV9eD9msQ*VjqUkGNIVaL19GYD}_D4muy z*SZ{#Y6xjoYV+#vFfged&AhCTmq_G@sL0o?uZ8k9(GIGk{gG^NDgiJq zJIjPURYRfsr%{a%ADEb`qZELKxpjAtHE6wV#fL?A*OIu?=LOT3drem#Zm-RuRm@S-qfq6cWJEDD`ZWXw<=-mw-BmK5u zJ3NAh!X}O;k2cHBeasc9AFbO}Th$>Jz18BO6lm__P_bNkXRK$r z*K6s&y~wxFX6n2+0CDlKJDQ~e6rQMJR^jR#iH}*X!Nk6zIoj%ACPCCVc zv9EmYdRt>Zeek(IP590GeO5z*}8d{^g;rGXGce6J9OchwPWcC*?h; z*}ak98r_fNM*Q33)18P(CIb^|0v3P#@Aza!DyPYPb-7zMJPxP9&1`>RE}dXI!t9pk z=5m~!Pimx~s;#YPD{l+)3bJ7%vL4lbT8vK#^Q1b974@B>{nNXC;?f zG4^HY9CqR0=T!c?1C7bH$-gE5KmRiTq6`&JM+B5} zn#mq%P&edZnf@rF7*5$6^?0|CU}%c$KGV%F)}M``L;jHa{(j(;H z0haRd`%8xuupD6d(TN88ZvpD(vHywxjrzfbe7l235ofx7gLwRWh+KUJo82RvaN)gS z!9*5}+jCb<9@EWY^Yg`D-AO@yAjKv~h&Dx`-nUD~iRZ+%y9I1Z5YDokK_qu*{E{=tRh1$OgS;Uyu`ijVf&xY4Rv;?rm~JCVq`1~C9OV`Gu2%Qrz>RWJ}d6VR9N#U2J_LvCYq+~}cTn0l0D}H%0@k#PR~F_{Mq23_JE>0t+Ou)R zxo6d`^oslH@Nq-Cipcim7@DxMSd3kaW=gb1GajrpdVazVe`Tt00{EYCi45yF4Ve8OU!97|-pGr^Z-2@7sX`rm*m2T?w zwpcqE6mzjGOPXvXhjE778dKE)$;TI7Xhhk3MvCNAL?UFyhi<f42)aKsjs|u(PP`7AG1?;81Ivs-~N z=47aPm?fra6R{dlviE_p#5(Wm7yE1WbGD8xR<4(l5EBd&jNOymEKRac-x0f-dgN{D zp$UlSf{yk*t#kJEBq-BJ-ZX5c>`GC)bO->hv{<&sUE%7*?Cw6>ZTyb(pH3CCOw7#! z(moUHpS&P2f&vR>*o?#Bp~*y+=B8D|fejJWdcz0|W2$WiVy3{rS56T_d1yyuYsVbT zK|;)h*6WVX@XnjCOvpv+?jHqjZB}Jm8Pf<*WW~{5jLNgr3g>yl1GNH}o!I%&TT1g_ zK!3r@#+~1-ze~2W_-$uleqcr-t|(|-XNfJL9t794p6th0BY)~FB0G;Y$G@2!WAg%pDQAiHd-9DAbo`Lo8K z>WnvS&w;N4;4d7w9&3-=qiCRe5f)f$J+OAN<8p<_;#k^baceeMim5uM!Z~#^Q5Yg% zmWfviw)oT@^YG%w!RWTtp1$fE6~Np@h}rGkasa74Ht)ui+?8I3zmW;QQ+!2dGJIv< zyS2PsD;LPob>Y9wV|oaKhF~b`7t&1`Ba$TGx*!-oe>yvp*GY2sQ+WfW3_GJ%N|WMH zCyVeAz+Cd5><*&z?GCXb%eI{)dg}33$czyK_)2qdt{-KuFGf$grq;GenUst@SVE7Ze7xZBQ$+!0M$*EJOET&i265hf~tyrmNDU3Ii_VB`*RSf1jJwv01a6VH_$T zO8yIF@!h)hG8k+h?qmSR?cQGq)0!dPD%r<^&;Y#0G`=X)WKY?N%W5yyW$W zrqINdXcui3j;<`W*kliX4NMrrMerv}Ni#829Y;ss@4Fff4U+rvf5XB8UJthikc{m4 z53=Y-{$dirAS?*Dy))*WYlcrd*$Z_Oz(_Ae&}k}GOGY>kOmka06_7rPdlR4-bgss# z+b&-Jq7}e&KtsJA&`i)ZgKkT*O$l#B;`67zZsOoK)F-r&RNnGgZBKH8&*s5ac9w49 zD>2){F^6a!lG{roS0q0!0~3T*@RyXqGY5pF;1<&tU7FO7rLCKymuhFyxBd^c27U$x zrp;ejf<;tiuKb6tfgY9g9x%w=wAm;~F=d3lTtOY#RWYLr`go>3-q96dy+$g1y#-|v zglkf`bbG}1d@yu`uFFoIHIZQG$%=kC&-(#3v0oStuPw)HQN;PY$>APC#kUL<0JVZI zS7&Z;)}|$K<)BH{0>PKBV?9GcS)Y+aLzA_gBW;EILfxX0 zw+k?Ri^HY+uV{X$@nxW5wv%3>KNbGnnn~>2d%J@*_NDApl!`getYjN+SRf$ftz4Sr zWM%J~B0Zs#ux_!k)$`VQ4aUmYYLU0*d0^r^y!*5fpNUH%&0M9|dth+@7<=~wFxkzD zFT&#{o;Z7)7|G=7zFe*k8mb5y?>YTX5e_U}e)dRbEC^~%_+c==#8Z0IOkfbsf-vh79H6j{ASa;&%4T?U}O% zugY&jrdnuC|r|F|*fpvtGUVM(BL2|#`9+RIbIoIodGu#do1c2*1T611V z>nIpOXUtU--3E@4%vR(V?-SQ?Mg1ax>3DxU${)q*EiYo?W z(`*$EdB)Y^JK8)uu>+jrClzLn@JIoWcXq~2o!OoPYX|kF5@^5%Jt@jbcaSqd^<3$) z!kUoFM?e1mKg(ozupEO|-3Rl3Cz0PTMwehP1fVrBD&coOBZs)MQU+v;xrP*lhFCkW z4rFuqj9Vx!)^tDg@o-xN{^Y^!^K(p@n9aU^eO}&i4E#AhEp`^VfhDn&NwBUZ;t`Oc ztIsnBIozqQEPCo!FYQG_Np6LsaOY?v(owR5? zAHk=7Z{2ft;&Lmr$L)>qmQ%%*hv$I=D+$3k&s#R=i8jdjn$22c@yTjFar?PJ59iJ+ z#9sn7UTW*pZ>W^#d1vr8djn3iUzwVlYUQDhm9^*3+vmt?COTK-yZRtcvhA%d{V$Xc zxh=FeKj&Uc-|unx=gnr(XE^nN8~7Z2O^R2M`}saTR$@|jt`E>x_Z-vYfI~Ow2t}{8 zv$@(-r9eKNzJX;|zvwu-#9o?Wpq&CmOEFR-Vd?D7i{@wQ&n$DuKBW*O(?#rI2KW|cRpqUKbI%fm(Z1Exo zi;Mtin?3?|qOV4+8DtfT|MyrovA=Idp)Cy|O&A3)pP| zR)v7_`U67iPFdSfqZAudezbT{MhpIbw&UFMf^9~<9v|eVII|@Hx(9L<$|h>nl*1rH zmh_f6w&F;}&8QCNzpC|k2suIns>LtzmR9xqrN&OBwyeg{-g@UseHGZp&~)o&ZsH$v zXN_Zwh{uV#pkim~;^%LuHJOZR0GrmfHl-V$b`xj}bw1fI#IbRP$Ylc~P!ejZkrpXr znY$XtUF1k{rH-)M$=NShgWzZ*Sw1q{i6kpf@qVSTthH=snX^C|gj2VxII;XP8x;vk zC6|38kt22}Jm{Atlle;^KNsghlO(|Vl)HXh|Kni)ymGAlk~AT`)WFQ#!>R-Dv>{V$ z*8cBBgP;3t<(B>F34kaL#_y;+g8ZRZNW4Vk1(q7vY4|E&()ahumD(g9kdfQf3P4v1 zq(`c9g7~4_V}gxOoz0QusYOrJpzCP81qFwtRClT(jL=KVY3FP`rWFLXa7gbqx*p*P zP<9`d5o79S9anpnYu26PR#2v|{`qa(Cl+8VlIb}!7Pw>HGAtqeW+_tHu}W~yV>70J zQZ)0iy0bfS)`B6gQdXNE>-;&3)?oUk%TU9HEz@!f3mjt!vN)}&*GvrNM#PD<)ZJ^( zk7DKclDIm2UZ28sigBb0P9QIP`|345G)7W2la%koUg8ti3hH^8ZfB=7-nD~ZmIvo9 zIfP!GPNw>BxoF)XRbF`+>qu`05=HJ$%zIiay3CU~6HcJebho547=ZBnKGDK2gO4zz zeU>f$6MO-13LfBVIcf-*C19G;4Vi>i$p+!yS250|`q&O@e(APH^fg&H|Y=mF}AS>EET;2(IY2P;YUUdRgBv2EKX}-~= z$oI6&JzO2zgj^H?=1zCFR=4XmJCW2;AA4-IY=*0y%C|S)L1xRk!48Gva^~Hl<4W+h z69inRP4Z3BiQfDf`J_P?NOPHG#@ftg1<^@ivTxt_G!0LK4*6;@4$=6%&EUWd_)p3y zrq3V*falkjJ~&xEgDfuC$P<=?L8<`1)zT^4r^F{NozPU%ZGeG>j^y4F%F#{Vd{uXJ zqxJ}s`ZRet|6w;GYixboe0oWBI?_Im5mcDw{WDY|pv`i2{X%`8Ird2P<(!Y|c-!Dm zE!20S-WhAaZU9t{yp%2-d}tk+feVUwK1z)I* z@Lz)xC$mwlL#0yho#vZ-F_Ww+@j#f$htSY5|Ih&SpqPA-jSaRV(gRqEGWtQ4p3!FN zhFG3|SA!L>X-HmRk9qGkg}u{#*Q`U_{728D0B|I(oY|s)D|b}&^?smJ922$Zoa0KC zk6n6svu1RQuVT=a&xp}_&7AbForoVz9P6`8;Fn4I2Zs6O7KXnX4wYUzIeiV2Rp8%~ zCD7^66@A;jiUlP6mcSD98uRAJF*&}6tf7){-x8h zIRKOOpgTdpfjDPsNZ_Kb$6P&GN5&dau{fkjvOEoi>pTVo+FVzKHdS19cE7s{ty7$g zv+whzX`?;Q>}*r6TQN;JFdg33C$AQvEuKGwwWli(MteFxsL{D zy^S+>8Fk%nc2d>1GkHUfJ7iJuz?JK|r&J#`47GV}0c~2by#9Qij1$GH(rYN)M?MzJ z%i51SQ{yJATT;)N6jwPvpVtrasL<2dk%YvzgATT|`b^eJ{sFyA4}l6`%7-^;WzvBl z6!>LPt}CKdKl@X3I)EaX?q_@D*j+w1^o+sxOnU^ZhSSCQ{T{|XIu|@Tu{Q6ao_ZjX zdRu8?uDl7CnNl(lsmL)5D?YPX^D_OxlkXwPlBwTGS681fXG*j05Y?XokPD9un9_=6 zfz4#{&81unc{!2XiSF{69^?etSuU1iff63GQgg1ykB|%X%*{6K>mGh9nj&{drs@D^ z{655R&SYrgpL4qFTDAGTEymXnlEC z>K4`AVz8|Z>ESytX{hV@lU#7$ zb}%hN;^M(taSt5QofBOo>P4377yJL1CxH_4oA`BbPXVo7E<}oSv{nI*s`n3-Z`6h zPo=E%+(Gp{rgOBtXi}Jp*417`Fc!S|)1z6R13{K#6O8t43scB9@#(Cez%ULJ8kKkMh%l*vLQ(~uX^(kG0n2Txm7<}JK})0f zr~Y1|0-vEewaD4JxGW< zm0}^+w*F+V@wY+OO51iM{}CE*T}lL9dP}k(3RpnZ_R*oXpti>37tR9ER9^Pd&j68tiq>QQ@P_Kt{%TPMLJz9`A%11y0xemZ~nhg&X#nPPD8vt zY(pnv`27O!ywvvd!K|Z&(B(pl&C-pUw4&6|YE+lFFrpt!O$x|kaTOtPADV(tg)Y?D zXIjX=RnP@l`Hsb}z`Zbio2r3d^i55YL_oycNF!@fi9$&)4dP$hi|&y~vjK{QI_P8d z90T1thg-ePFq*%QVQtie z{2#2?I@?L1WfK{j`(iBXElSVNRNR$lPd5_Iu&p9)}UhN8{*kR0MtcN-9j%E*r zVCE-Z%Q^$199fRxq820G5s{L~Y$2Gb7am(^LctcZ2zsgCy*g`$^5vIT%Az?W$V#c> zaFF<;S21D2jIIv}oU)pNT~#b~AwWao<+d(ArJ#FbuARnmWkUyC5rBT@6f## z;^A@E8d<3+)A4+`k`CoUDa~WLv1(5I-N&A)_r(UyBH{gwt?xI_T3=^^_Qy*FoWlZ) zlqNjtHwPqLtHD~TO~o2pgz{eT`XU%PTm8FJ|Hq2)E)eVf9v%koQInrF(vKXLo>|~6 z*@i^ao{A!fa_vxcmn|cZQRH^gaO|q5qwtqw4)amKa3;3i0-)6ars>heEuJ4*5lg|n z?Z99`pta@)=6fYK49ut$u9+;+x*Bo)qlfO?w z3$z%`amr%qOmMfJQF!81DotwILEyKyNH5^hM+K#Ysonm6s^9>|VM-0cwo2Zc3o-dI z++E2k%-G`Xv5_h;6(T0f%|UJ|d=A&Lg>DzVnTo++$tp%p}Ni}sw=A#9>j8P-a`OXq?ldPG#wXQQHo%tb;jgR`upw7vS9TIRAG_lw84 z>5&0kaKqc*&I**shw&A_2F~i)p2|Offs%BCSvQkk=BL$2(=4`z4E&&5AI_L#yB)=M zsX#a4X<*uT=vTpqvBUZ`R){$7`|nJNLJSiqV1xh>*98GgX+f(>veqCDk5{S+k-(MP zwg9fAlf`}yxvz>3u}H^j4%8X$yxw~$8X({P{1Ml360SbqBbN6h%dzhLA^p0zrs1Xp zT3CbRD8;S?XL-~S7~K5w@qwu*Ky~lpz`!>gQh4rP{|e{|SqS3sn)Lw~hLNUw1A8xO zgasy#v2sSSz>4fdCY{Qo7AKW6Vy7`x1{`iQLQq2|HKuU`~hKrQ^H5=x{J;hCls^l2*np)reW7q%uz|Y{Z#H1!1Vgs8;V5lkLjHnoyeE`84?_R`BJLt;E zI{|l{#?_YPVPhi8Q7Sk2;jSl{$XA6GPa|5#00?*~j|0`|nQamX$vz|ydhc&(+HqB2 zI=9=u*V{{?aduY-W^QD$^5Uk=1s8?w`b28MPg<#e96|^h;yt4ic^uTS+uJ7Si|X~5C43=(!Rlp(%B<4YjV4fuaP zC|i-4B9fEo1=j=NxXEg$28UF!V!W=iW9jjlz!5Mqf|^ zFY`PQKCK>RVxqk}sx87o_o5WEzdPE0S?)_f!y)4|_V=|9Oc~1pO@z5s5*Gf>V@B#` zolhAM)%bRAY|DAm!!_+tRmCM|(?=TfQNEmb)MhFfx+;nqN%gIe{LeR@d<4+x?ICYE zXZilOC!eGDm;)~O#fg1g8!t2FwQ8~EpQaPW5n{E4lL?yxw4Z~zixY;~IOVcsIr>s~ zx<_*O%YlVW1R#8cQfh-v&~N-i`q$Tn!mzL);+C=H|LD33s3^Ct4a1NkogyJ9A(B!` z%qSq;(lH<+AT3hTlG5ER4U*CbBdK%>NOyP7F!;ZC+pSK4Qy@)E9g!{MehPT$VQ6+(MVqWI5fXvBD z?N$#xd#yfw39mqOq?k?J6Qaupld~04h3T(v{s8=Mra?>6{<0&4pLdLh=!+m80&Kan zURxf#&@n);`JaT5genPbw;K;FAlKUE->$1n5G-pX-Ui$u_VB%)w2Z@m25k$F4t#X| zqMR7Brwg~Q$F}31Nx7N{Jpo|K=$dL%Lcu=sB)}{?3)V^yV^244lht~B;&!Q3b2T@j z4TtbSIPt1cffA+vCgvc0_r;sk*N;9HnvMjshTo*bJ?(Oxe~O&8{<3Y&sb@-6qI;nb zU*;Qcw=DMceczYAizUB!t~h4+c(5EAS7^8IzaO2!y9ifefQW~lv&G+1ipcJm4naXY zx9ACWx7xY8fCk7B4iNH?b^>?V9hnYVpB&ZT1s7CLzOf01zK9laM;z5rU)H9516ZY4 z4cpW>pprZFo^zAx#J;+DV(As(ViP$J41f;tA1m$oUvzDlzH z=Y~2H#@7zzeCXX~tC_9;uPeA+Mu*;kO+a<8GkNn6&D{{U>{M+&1i!%1 zDAxznsiX7HIB@q;}NlM?=QQTL~xKuZ5igpkflR{y#c;r)&X%J8|{!k5I3SS{xA9iM5; zXb4dz+Zte}Mj%JlR=dyv8_E}_J~n{q3X{ooN5XjlxlBO84b-5H?~ZYQ*Qm}~o0&?E zemeet4Gou!xR3*_eH759RX(#EvfPtmP-*e2*dVhp7jVQ>c0_VZd|U7Md-;B0x*cUtnhA zBMua7VJ|*9WDr*EP>SBaPekz2jE@ zp=#ydW_#u5G73Az2*CdZ6Kg?GUvlDoNoDV3ZV4)_UIdtk!;onwI6rf$10vA+z>9&E zVsJL5A2SU7Et7;2G0N^99|;vZdWoQ~3(ks0>bA{hnhGqmb!P*`-2I-$Q-a?uq?!5p%*ARPw$R;4hSV?=->;>WUkZ{GZ;7Ode z8GSG5qO}3_06GvWu}hNj*5g?sT(x&}fy*oOHS4e@u-ZRc1piuUsFz(F2G$sr+2F5^ z z9)Xc$gvZ^JQw=O$;{@omk82eab};J%BVZQ_=rC$UJ@eX<>)%+Ls2=MAA)`UnTq^C4cDL{1PaBerZSr!X|LMj`3afUq1l`Xl6?K zYSt1PwJB)%-V`4(hjl@>fY`D7eSGDlq|wKo4!F584^vyZU$xk`FMf0BRg;?Vt8XICoTYj%GBPZ#tb=MBLEr%IMXJMEt!`PZXbN-!{92UCr!wgFeu*JXLypZL`u zx_}vjSTWtty7z9iv9GGX%(&iHpt{vrtgj#BjrFpFKWKj3#1W!rdfI@+i19?xYm*x# zc6p_F==3gFuKvuf#;SHki-L;S5TS^m_zql}Gk9_t6OiNP>t(FZSgTMhrA1BWNgejf z)Vp7wstgIH+SEX+MRrb5{L@gF?hp+csI>(MF&PnChJ3`0E~Nmy!#Q6#sJNL4TVRx+YW>x4E%EM~Ca1i{Zcpg(RbqRA;cKrkx)tE z%hzcGVVk6`K!cACSh|6T{?w|)Ht^0{Bd<_r5o=wo1%!3;`EP;-;1J}K(TvZq@n7+L z$^Nw=@G9PU_u}qlwYzGt%>|pi$7c%7g@77Yfm%lLwy)&U80>m(qf~&8j;&Qp)+g=DY&o~|F)|k>U-7d}chesK&8TH}?G>l!%}Dp3F1bDI zV>A08lS7Ya4%rcZtD!qUy^VQ`J^4>U^v9EX^AHlQy(J% z_O@zv?4A(00kxv#L@)NTxmeCZd6x~S1 zXNxz|vL;qYKI7DtH^A+JI%gjK&H;P7GujTcd(DZVSN@}nlmPNp-2=`Byuw(?gTCQFl$Jj!#3*S3JVe7i4Vce&LvRXs$ezc6epV8(e-428 zZ$3O8gJv8K^!c($p8jgO&~UKqXDNwWc5+72!?VV*J#QpII6{m&bH<*&wEK(99`U{u z?9aX@E&;dA5fWQc<0aI6qc|wrQy0hPqh`Bg_t*DRoLl&YnIoER3RZ9Jk}5&G_O7)$ zye+d1@vpb%P=ibX<$@|b#srgu2T#P0MsZrS>8P%mws~o7{3l%fNjktEf~kTgrf^_Yzaw*+7)gm{V^I%1$>urL1=v*AiOLzu$ z3Bfun&1jwp`Fffo6E-Iurin~`&YJ2O>f$ku>$nxB8;2tXc(p6#8~Ol!N|>YjMbGa$ z3+4*T+dKoj?v z;b&3z(KuJ0|JgG26G!|=|0IZ>oWt6G%C0E}Px0HnIRSq34upj}*K)&TxYpYXC#zOY zA|n5h3X!}-77qCNVE&qta-7$-Wa0ck#$d2hj5-LUG@3Z^lg6v_;E zmcp@|B(Yw4tqAp5t_Y>Da5qg|AA14Jo`$K4l|5Ua%j&a5|zH^qbd!0>}WpEP1uJN&uad`;V9RI|let0hySpQ(@Uyzb7Muyg5^LO8niO96a}e ztiUO?pswDM2sC%O!li(rbVLSTu(_m$OP5FeE*auEp880Mjv>Vuo$ z^_!PG2)$!l6eNUeI%M_d!vc>CK(UA@=|Un#2%B%V5kPU_RLWy{7e5P(c~ntNCbL(Q z08Irek9<`-DXQ$TTkb{qKEvx`V7-)KlR%Yg2tB1wIq42!f>_F(;@T=?qdrWzxK~|} z39uSu!0aTxD>;sfR>=OxcARi_bR%5!0oT=2*XUh&#Ng7e=d<4JU(aR2}x_g?uC_aCVw<=vvs3TU%QLd?vFM){) z^zSow4UeUO0a`l$>s;pg!0~+~=(N(pqPI1Y{<5Z!bZ8n0&aUx6JfqZ`6#rpR7{{;- zbaz{oDl+s3gJ0D8?G+8DL_ql)ux%2|E>DMJ?}^L!SKMc(*VGzSHrw4(?jmINOS@NX zhc-%@G~8_rn%N=k?bRRCy)=;G=Srupx)#0sey=ElJda4@PKXB{EQr{NX5s|SKP=L9 zrYI0WFm1rQyqYvNn0GG}e(O?z@wQah6`;AZvxet??T$AAaRiBU;o4_k18UC&9eF4? z`(}$=9oZ9A-OB?tx11U)P_zVE4h|b~KCS{5DIlvY<^IY5l8X468VbuUE?lsMRDRFZ zIIgUKsXs@qzW3nk@8H_py{v)s%ix&A3RIt4J&UbYaT!!HONN!G%L(EPg!iNh3*Jbx zSM9S{Xz*ISIVSjIj1bRzc=t{EMa#V1s3;$uoL$mYf7Eq$5LjV4w_KshZi&iT_@f%xOLszk>_t8c@9q4i+YUBqiRA} zN6vCR9NIN5DIsREkas_OI;w6_uU0zbwimn;T~wiv=Yx{qtj&g<&^4PYn-l& zJVm0EPHUHk&Y+U4r|J{e+uvR{jLdfRofK`rDo6hSfuA%LH$8z#Kt^5+zO-UB zulr$kn0TYfnswI5S!3h7giNXR{CCr3Is7q-F{&bv&gYNT2Ys`@gqdFqXyc9{Cj8C$ zL#IENK7kb^PnFX}yUw$7l;CfQs^}!e>L}%vB;f|^=(1Um`JiEut{SG{0Q$!e- zidcMt(~V$nYB6r>A+%H0lfGVLIS={zxH$?~e48jqR1V_lmI&m$_3?ZKOJexsH%^ej zcB%eqYn65+n)`Cf1N*Fb>Y%8*c;0U~OB1pl(x(l6XQWzoAVXKdX|v+_mw)-|uWJP8 z9W$m>Tc3^o{ugTk=xOT(nf&W^T!Q<+>W3ZrZWr8RRu)8O3y?1NHD-(a^1*oGptCvw zew!$xS)*Crbx9Fb>Helv$YC#|_~{^_psYYN=MfJp(p(*Ynopyok@CmyD{$3nFUp85 z>sU~F4erw{8RVn&dzZ41x1a<~j2g?~ThCq5dtyTpL0w7x^A`v%OuTacwUgHzd+4 zM(Th}nvD4lNxKZHQ_F$U3M5V{ja(f0&~y`0$0o5e0#OirE%{P9@G(?lA{?PqKfsk4 z6MYw%nZRQ$gOoeYJSsSdhrKkNUbOb<7WA&i0iu6Q?wZ=-}`m=ETj+i%X;L4h}aWuC=ED>ygTsfz%a!Z%&ed+q{lxB)C=^l(~FedBOO4t6y(LY*(5m#vBZB zpA)C8rX$!0PWZ0LE8mEyNUz@YTjs<08D<*32sCZb$dq2H3K&gX@^e|<4Pbdszn}QQUB%s z>+zF^seb{DzoOos00qhe>e+vqg{D<3)XD3;ek%jF_^ZPk_dPotunS`qSy@-mGQj<9EA%7FM5r3)&gV zk$ay>ptr~hKIr-Qzfc#Sl9yc~HM5rp(S(RwycvjjK*aks_d^FkSTwoZ2*W2bhCrU; z21^4G?1-ngpw`ReOc%3C8}Z9u%3H7}y6lx3E9Se#s&PjYpkok64ReWI4OC0alk*XM z*s2ZaD6hafFl^WZ*6$aSz#KykbG*^HvrjyiVlD~!uoKYhvQd$Zy6v7L|BYS}5`$De zh<*&NUT{eC-n0fW6FvLUQJt4ieQI%}X0Y9_X*CS4UEyvdl$MD*eaq|{i^CI4$ra`R zC61`q`d_dn=?-^mcHmE3#+>NI{O(|@@M6fVVrPq|ct6^l30VT6QC(R`{r*gh)u7qd zFURF8VMDy`v2<%}z%~L$`m16W)3_?W3`+Gh!xjJ%8FxV&g3qH%r%+oJ`-O$2ht!1V zC4R5@A#AAmSqd5CV6pII=|)0*cWUle3(|Sk)66{9l*&ZnE@xF=3;I(0 zc4hSPyGJf_gNKORYEL5R^<5m+-dFU?sJizrN$O46!cD7oup#e@n71!}|H%v`K_R`) zxz8mzPk>0k3iToY`(!(AXlSLun#&r`BvC_du~?Aq*TjY`=#kU0-!_xl6{ib5H^#aZ zrs{`_#2h9b+KeZNMPWQ=m_U*)Ywy$D$vp=hd5VT51Bssufds?J8&OHcOF##c|PTdEVF$vAsG z&s`1i*&n=52pB|~raI(=T%FHW`($+>#2wvlRNUyB{9SA|Ov2^XG{%IRH-V;3x^S~G z3SOO7ur+EXTx^Eiw_KzLY}_dN>d?Z`K&(m(71$E=a%Bh>%R2R0k!3;yf!jgxblpnT zEoJKdU4uME4tGCCH0Ml%hb(UOb2N+Sa&%woMi5g%f^kP=7^W*UyF4VTch%G7rfubj z!;S06^Wr~<>C9)-1}J+JF6G6dfP4r@O~laCs#uvbkH-7}ye+J4(hDTOmsJngI3ymC z`|vUtKdc{1a_(PFeep8{x~(=nx;4;$CR0~GMerB4`_-*_3lJINzwrM|M_3>7<24!e zYj%==II?$|w-w13bruc(|R*a&&zQ~j6g$dw!oZl1FdRJVhHZj>$$R-VOgvJo3FmA9>H zLX-51;a{4A@$-d?NMyb$IZ{q8JWTe&4S51DLWOqp7bs-hLr?pBg{ZIxDLzjHE8yTY zB>>P+%16lzotd{km{%^jdR=N5eAF#NWa-Vl= zXg`cgJsvB!Y>y80+|Xz&ZeD2na>?V3dply=b68AJy+aNq)0+LJK8e1|V+x|WmkfPc zr||Y#mq$Shy>&2PBu69}T+gc@)!0_D>t9BkoLLQIdhh&D z(W+(0cnV1Qt}BI;voIfo<9_t=+F`LwY-~y_BuS&pz0_JaA$*w@bD`CXbq8WO9qu!v zsN+0C(X{Z;+{(cf@DN?38O;*wgqTd9Tx0rk63Ij`#yLleqJbMuIe-XC*SRHO$T=X1 zNTrGOe5?AMYb>wUJEB%spBQvhfcHL(vc&s$ArM-F)8nj~tIN_;#eanW%$XVdLJR~f z-}uc#SVg!L<(JaG5e?ja`ZCY@ZkOFgeEnfdFk*R`9TNM0CYs&^#@JQzYD$_!s>R3MtNVsw{mkT^eZ!2$zyWCMf8ZPuuh?&jhHB8Uj z3TXbh{;D96hMaM6`tOeO+CY~3-6Kw~xgIy;B53|n(ulGZsYJTF&}fs%8uAD|0!i!# z{lxTb+7XC7zco_*JMfg)Q`pAFg#D+he9|xkob8PrF3xK9k##%XWn{BY7q(ab8U z4Q$`#V=pH(vOVrbQagw}Y|;9$^A8;WfofGemvJL}L`)?9>3_oMZc;;nemt-4`}>hs zd-yR5S}6Sp(;=mvpD2w|gU_H@WQhBql8hzdE``bTQq*yDmdwl3SA??Zx!oOA&uAs? z=`%*0*f@+OyRz`dbn=`kY{1$+@O8Z}tPk#FNIpQc4;J2Si;RM)J6KwX$=@|4M+vRC zDZaPe(HyX~dXr zm}|5cD##hj$#rn0&TG3-kmnC9g+R&METMk)9s5=}7-Fbw%)r&psTn!#?W}+s9d|YL zmyWY>uSG8>N?lW*Kjrn1%zPWEl4YSRXD1koM0xgYYQ0PXU7odH=z^dnUAD#h0VSpi zwVDec!ypXOm2@_s`*cxEhE09tZiR67+zMLEavB>OJ4VG2KZ+wObX9&>iiAzhIH^I{ z5SeoUT)63R)<{tGy!frbi~-zk(i6G<{eG{^u)Y*7+|3;{Lt{KsAed0 z5<1WBbUIu!xu;uw@5TQM^*6L};$uisdmM@3i*mO9WFn0&N<1nB>!;?XjR3CEwRjH*< zc=zUyiQX=kW?+nctD^8W8Mgb?SXinKG-8*sB>Hn%sz$_dsV8j-AcB&B$>}?jvd^`{PBGiI2#JZ-F0NQ0Tk!IxkfQI;dAsg0L({SVWW#tXgz zY2M)wn_^=9!OlHNh$VA^$_djCbBaFK!Z{YF>1%#N*HI2(0Mmtk=D&Bi%i{XU8S%_}|B+Vy;h(5Lw2)BI< zrr^*~M2~ROTHc+q?@v`nYYU}+8#S9ed$F=tRqD=B(iM=!=m909h93+8GBneNY z(=dEqaRjL1;{do=MvtW*v`|0OBw#e<88dY>|;kAyrjBG{0Xp)Hw zew9c>5$E_B~OOcd7U$pelVJQsRr_Wx^=?wIwXdnH44I0k_bLnAP~Iq+-xo-a!q zE)O$2?BPa!zQ~s@kK(d`A$kmj;<;1Qt|z1snBt9h)u8z_L41AZYMA*}8?}j%meuaj zuo7W$en1r6`AwIl$jVN5D}C~Ikl?tX&mJH)eRr#FHLbD;qoR-j&kbMb6ENHvuhhAn zy+tkUAOp$~_H9Yhg zY@$!$UpP1NR%?cqY1!{3vvg9BngyZl+Q<*&V&y>C&U23EQfbtlerZUbpBBE(GFuFL zBQLa{Oon*-bWo`X!LN-SZvs%w)8LP9Dc$Xd-3_%58Yi8rnkz0WaGV1fSLIqw%Z+@*YR3Gk+vTX4BGK#=uM6XFi@Xw)x7)*NZ#*7@LGF&vH_!Xi^j#*$csf=1?M*w) z(t2DTrUw{@J@99(^ikBcs!|s^8W3On)-1mh8RwZTn@{O8RXh?i)Xz<|lWTOU$m=rE4mMPw00o~PSNi2*yBr_t~IqLdbZ+4!x1oOCqx%n zf#N!Tl23CyYgyU5u;JMq$8pVmr3V*wu|V0n3HyQ260GL$`xfe$ar8wtpjgMb@4;%D zm8Z70GO4fAjEkvW9XAEAuz#4B85_E2O<5&C%P1%}K5U8clvAnGVIH6cH4>V}USzM0 zB|bSS$VX&~sUWZ-4T3zrS5wk>HW6ksXtkwwj5;aJw$K91l_AI-D8yutwT=Tf7A+eF zY~EBgSEt_@>j%F9t{%mK=yv&Z$jb^Y1H_bq`YzBz>dOEJW&S^EdTV#lJ48AvK!y}l z)&bErNj1THI68=+Fg!^(QQglchWlP z-4j&kgi=194iiJH?MJH zFrET{x9ppaMY~LEC**RUXJ%9rMoaKuDPn@p@lS9BEmhxUXuU7fpX0G(V28}vrd+86 z%zi*0PS zEV2}SAL0$);9iWjVtir>KiWZ!(#J76BUY}(2lsmscDa1-i-TScelq<0b}gVct`0}m zrnsatW1z+F+9G6xI^z~$^}I>2esu%`LCd!Wd?M0xkqQ5Jbh?=9Dwo2jqhZanc;rS= zDILoYo%5Wx7M4#p5a5Gcco;rD{~EQXl=@BIX*waV8Y8!4ssA~>j|QRGnW1IChJ?OW zsPefA3syC&xGwhnJxSWY_nz>fCo#k-G1>SVp9|C`n*bSa|NH$S_fu?Ja4U65rO+<_ zD9GfecrnwKyVkLN?A!Y|)mU-Y=Hq=nfFQd8YBZJ|3_pDu=2AN9x<2AXHK07xnaB_q zm2P?5j}cYFnQ#~q`PNWd*IajuTk{K$nZ|h`>|^xTRwp@+@^n&z#l8W%#Z3$D_>j?q zOpQTznXUX6?EPz_wL$a`7v^lYv_Da1Z^q7wo=%Z|(=rat$B;^~D5=ZU99e7LuAY37 z+`gidi8fwSjR8`*Fn;wiB746Nh`sq2odWf&-yI(1|LFI;Jj@pf1TjQauYc3Exo&9_ z@;LJ+T1;YKz{H-2VXN87@-SIkUDg!{uh<{vwbKPmL_VVkobXh#TuESe=%%qPrAy7u z%gKNO!QClsNeG$B(CQcNbH$f8*g#>=qOU%aGA%k;ZB>N^7#Y@W3xzO%o|&Q zQ71PYMpWi6k-Om-()wn?$Fr}86&BlyHE84~Kc257cYf9JbI4REd5l9>!> zs8JsE#v@33dMSU2niFkG^Y3^VFI=)a+`}fz%~_#69mYj-oOl?gpM24|gK@dh5@~&V z=)~DJ<3;;|qgMqjA09k@A~Wm%u~1wE$@Sr+^w0MTQABGG^N{s;AJ3imV6O)VWFM`# z@I2q7DZS`!7`@x)*VoC~H!@4$1+v~!rHnz#!5^16|NZ%fgRzJcjL^iR*?ky=8x24Y2`iAX z8n+47gWK3iOh+0kOrD3kE~kgaikYY)n;Kn}#vTY5z( zJ5-f^tzcCxH@ytJlrFlcEURlW=akeZg$bR%xA&|yqwa@JHoG%aQ5ay zs5O+o$i{-yYRGW{ayDKQLD4aR6bZi3avvkdlI(|a+c0C;RO;XB@hcSx2PXl-%wL{bqfe0;#vA{F8vxXoK-P2jJf~A% zd7SE$`Z+Iu(HcN#7eje>Di;reiSl*4BRqV<&!0weJD9cxj||$?i$n1^SyNumQ#1Ne zk^KOy*QwT=t*6nG0V>2N4NGe1L%U>S!KNy*f6pZxJ;lkHtW|u z;0qa1CMN&76n_y+985sbKQ5I>v-1}ZlHm?cXxq$Ll1j)C^4@ysp?cIhd*zlyy;aC@ z3W!m9X^$cqH`5X7vq|e+kssWvQccuk+20eLy!3oO0C33M-VbpGxXMZB{XL!*fduLt z9(oeV#-gj!mj>x{QafGMd;k67P6Z@v5{>pS0{*$DxrtE6W3dl&0n?c6!2+qmOHmfcNd|6i$Yvg10SI)cX?eh$*Y;dmwyvI;Lk}6i>L9tqFt<~@BXr* z-uT{6KNC8A+k0inCgt_+De#1;*0Z{tES*3bT!I9Mm4LC8RY3ZIvOw33SKWPwOd*$_ zELtdHSyB9TlFO_iLN%({)GHTZs?}GdP69k1yw*CNd$aYvc+m%e^9$&{?boO~gL?!V zCQwmfDXaIxzZ%Tn&nZvnEd;bC`mH~hw=Vd9M=5_7oz0EgJTDe$+bu2gTy+xW)NuAf zIu@CIWSSFR|5ezNG{FFbuN5-XnJxQP1S5!Qh>uO={WwLGdVW4^($BsE5U`bMZ&ugP z&eZA1MCfV!0NWVszl+0$Auz#Jd4~tf56Xry)c!H*V*pxV$~Ni2BK90ybSDonhNps;5!-w z+AnvlAF~F;8uEyAX5)X^@baLyS-`Kw|II@Jhq5Ezv@jo$1rcq2D&VQ}`boi=OJt=F z-Na%epCl?pQ$;nrjlH#4da7_ixwY`%l~{O+P2E8P+3}`>byv2gsO2A<^Ew?`I2B)? zvokRN3oEKzw1k*I}tFy5nF_&?i`Zq$!FxNfF zORaq8;TvkKm*pFO(4@J*tk|A}EB5hZ{%Emy{QHO#?y@xX-e+w35O-cwMJ#&Ww(M)y z-*kW(a-(cfF5?s%c2j}kC`Vv$8GK7;od!{-S41jWlpvv`BBi{ zJV%Xl_b&doDW_R?qxYc#I-IyZQYk)0SfkAV!Q^|zwbiZeNkI@QT4jchv}`^d$FcWg zU-kX)X~$d9Txu5#arLr0sJ#G=pJ<7*xjSOiNSSK>XA;5s7%JQ>G*Y!` z=UloZ%CEi^T>(psEvxVQe0RPmvfJbSQ~kKZ`&a*`gb2D%AWqLI35_{Sy714`>se?IUZmm;@*4T-?x0!1 zI5zx?Px~uEYyiOF#WDno?*vE?_5h~xlL6kHSDwH~I?5SG<3HM8AjQ z{D|O`t@v4=QxjgTW_N!iTi3H0 z{!QOp00oCopPMu`jdofe%~Wa^!-StvO>XGf_ww^t0vQ~B^I1FBf?-c?vh|J1$Hcqk z;qI1NwYHfRVJBBP-!wMjJ=K)qL$Y)N;eM8MS#R=fFjNS)m?hQjA z`+7k2rxw_^#W*5XOx}V14cAGVW_Nkpy7e;9F0krOG{+stDf(!-=1waB zDA(O}+}HaCf12J^P9+EZbp zBVZpAWYgJ9piPlU8|-YIU0#bqNY3Ax@OMeHPd?vR(F|-qW6l>>_TeG7DwTxRGJIOh z^HK$h?qOHk_lpYZx$UHR2StZ{|k?oQLzI<#(YK%DS8Hl4HK*#c!VmCuC1mE;jl zsr$A?>s7{10J+$+J()1=XO_Gb{;+utlPeG}aO4n}YX}trsQcDfO#|v!(Cm-P(*Vt; zgV^~r*B#thU_@98vD8iCnLU3&U%<5ts^IR2-0#Ebr1y0D(!Kd-LT_|a`8zwG{?+0kY$#RFIx0yq4bt--04hbkW%|7a&_O3=o8)3nz8O z5doHfFXFMO9v0omf4^||O*&#E??ty{IUy$WK)z=A(F5Cl(bEuf9OF?48Oq@DAU?px zZIPz@p2p3KHWo4Ba2Yw#@fj*ViL=TDhN$?lD<(1)TLmP6zGt6~)H?&F<&iAT0(7n2 zGE{?71RxU@T_o1`0d8LY>67P}aEEW&EOCInO}6v>nfU{5opf#@AQ5YxR&H0@;rpk9 zI9}L8ne@Bo>v~7BTtCc(G##o`cEbvurBAI3o9HhK!+!c9?+x6tsOIXkW$H=Cd6zzl zNY-3#N8X?a%KlQ`e(o8;At&Og1DfYf-n*=x2 zC~AOy=g}1RvkDO<<&(*^c@A%25`luiyGi0oy!k;XdfMm={4Zc{Un_qV>$WG-O4K8? zPDzw@;~i(fAYJ3y+|_rsiBz`bm=5{#@i_YTqEH~)zQ z?}CopbsS<%17gt64J>5WC_mtjlUe;E)A|eGFW!Q?E#doQStTSqH);48Tg%%i<`sZB zi(75HW@iBifJLDN#x2%hl5LgLZxo*GI<#nE?qx?S2C?tCpLfFuiZ4%rVV`_%K+k@- zte0t;%Zn+0O36pn>n0g~Ru1J8obhhNWFQr^=gQ#EefFL3rGQurZ8joFagAf+$yp#4 zarUG`ASK|l!Eg^gw=HA(V4oq`jxM)&n1{<$wR5bx#=D?*mL^l|Tv&c5fYHMRnk^+m z7U);E?8063rVh?j!rweS1LpkFkP}i5OgK(jsXx_2gU9lQyS- zB4;^gC%21BN3oDh>)Qw3gQ5MI_N7!Zq^fI9tVWXhYx&`*^sv|(t3Pag2WTQiYlk#d zOd^Fd>eDX=^>&nfp&|aj*n~*;!c?Y67nrAbOE`!q45vFp{ zvdc%_1-uOX*Qh-k0Lj~EkqzrSfcX{%i!Z8T(fl#NZ_bwbg;EENkB%{K4 za8+EYZo+@|-VE?6tC-d8`T)!`yk>~Cr1W4pW-sluk;QG>0(NcEvQj~-{`LBn2E3@G z{$-Cn0149dL&Lo7`SAn)b@)ZCE6vwhj6p@RyP3{>~uk(wU{^XF_RB4M_sxEI90Y^ zZUw9j{Hz^ZeIPg$u36NB{*G|mL{3)95Mra*tae4m*)NIE8Km>791wyzAd;&hMnk6@ ze#aHZitkeE-=!bRwMyCLx#XQ^UtTq!{)B>bx8Q<8#`tcXPLj7mOyjCkEW3sJnIt+_ z-%4uJsp5o48_iJr{mTb9$Q|r5mg8Sz)oiAbCbl9&<^L$^Lq;uE%ttPuRfL5#3pV6b-cg)%->&F-QLS&A|U8Y zb5L0?%!K)0Ca-Xm)~zHpxtIlLbyS%mIxM-BW$pPu%lu!jB3vqdn{h&hBoEOebfvdZ z>0nIJ10y*3E!v`OT3a|edvQk|-?&`dy?0oGCiyp^Mnt0lytfNdb1Uv7} zi>efzn8YHFvP5Qds%u`C%nRsuru2_MIYs;I!*Aq$GVSCC5}o{LG+BxeYQvfW`mM1S zF}%tM@kc(F1H1+Re~yn#%RKGkJ#p6E90D@IjoWa+kjp^DW_Ts8AN)0bW8Xn+O~?F$ zC%DQl#+ww+zyJRgy4Rts6K9;>uplWdm(S zK#Pb)O7Vc9-FJa%;g06@M_C^U9RH*wpftF9m#xB>0{ya6Z{l)0eU0xe-eE8)eOjPo z&75glv+nNL_>uYjk=>MQeIEm>)3+V+Z{4tUq#r4u5Kn3r2fqS56C57FVLW`gt>kix zE&(m{4dfYH)`|F_l(7mt{&issNX?85ic^?+EJ&Bf9lS=PY65?40%zW}=xX0H=q|ah zinT=318`LTF$m@X0+3stW)|Anj3?5%fa22+80AVw>8m})p~W)d)q+W&hV+Ef%;PBp}M+($pIUQ=J6k!YQ@BmBc|ZI zA*8<1y4_TwZyNUXHgm0Nj#vzhP@HGO%dPM7jOYkIyW{JsWb#&2d7ZD~_qj;og(yf9 zus~C@#b#|Ah#T}~t_^j&!$1?C0c@-ksP54i1>uc(XsB?zOKCX2Tx1Rd_^=FG@c*%O zm0?kCYnu`UK~O+IQjwuUMH&V~K)R%)L_k2Ky9SVwR!ONrx>G`0QWyd0?(UkQ<6GlC zXP+J4bP8lKLsl$&yR_VQKmW&;UGC*A)x=FQh)2MPQS}}Q=dzY zyK?+n-8mgJU9ksY2im05<7(?|N+}=lPR|@wc*T0boafiN0_QGyfH9mD*`f?p+4IL~ z<-b3WPoc$PmPtVlXK=t=HQo#|O#qvLzt{;aue|80d*y_1*+TaiK2;`r7Fco$`4qxV z?}1N)4zWP|L)gQrf%{avx6|9V=Zn#wht~i8AvTC}q*w$x%v`o5wD!AN!n?ySA2c-c zM=ruRs*CRi6fFDZ^|w-Y`;g7)ln~F!MdS6St}(NCjcRNW&wP~Hh~xSQK1?AA5A-aHcs!#$edGjrX3r48nH);rSs(y;xygKPXyYtiRj)0B*9whd zW{3Q`jfI?Su0@!TVGAsMj$AG}ysY>-ibBH>GgH)uo!-T=@R|V`Htf>ig1Kz4weoE|&qemCbO)OP8<<;qXuNNY zt+3JoN$zvlO;plJT4m#A-ZyWpe?Hp7CM-WrHhU1?xP#+zgNM;$BlnWkNUfoCH{ner zVpYl8Ii%2XS1v^3g?Yy4;6>E|@M*E&tOPzV@jHz(^?y4dDgXH)^)L=38skuhDhl(P zX@*~qJ%>6j(%wq7NVF31@4YmELqWcS4<@Bv=lsJ!meP;Yhq*B@xhjRq7S>sj0rZyd zX--1=s#bb2%-T+wgX#l=0 zs-;Oa*>|;Bh)IOvd|VA4Ks`lFU50GBE`b2>nlRVpQ9HC72IcE@>XZ_oYd-Hf@+JUk zk$)l`Xej9oNDBcL>^#F&G^30D0Q)u1%^23r4z8XhlP_<2#Tqtp_pNncuSO6Wp6>qh z(_EoQe5L(&TqIEW)K`dSB@sqRzV}R+t;hc;K?stW<59m~rpuJ5WmiL2Slt|7Z$NJ6 zjEGd;_1Ud=Hk>2)H#49b5_2PHV4V=NZaJna%=zVs&1jBLkJ7Z9-l}uwp3Pau{lMXU3?p!PIqE;fkXr*Q-ka4ZqP$7% zMHS@8BqG?yBaUADqY}hD)N8OQgaKD(Q)Qwv?-{M{+$}3K-Qyzz5i`iSC@pl_6P^`d zNW(kM_qxOBJfztFT*Y8)3M_^zsONz>^T)bZbe0XV`enUg79*#hM019nwWi<&@hi<) zDB=oG>5?1c7)_5L0BGJT6*KeMRY-Qok{C|2?c?f0^2LI>f!Geds zkEj(2AAOQ%u94F6Jg!(dMod*N9aUl)xmWK-I_q-R$fn7Zr1if^nTg7Gc}5)h#6BPPt`<>6zhS4zI7A|h)G%E zfjN{ti{SmMl>hgQ2Su{Ji0wES)*+|=u?>-rVK3i^0}WXrehViO4yqfaUE(Uu*aq9=GKPgV5~r@g$i zJW+$EE!5Nw=cwU8Lf#R200arLJ-q<3PC9f$Ib`5ysoK0lZmE6)jhrSEumM(~atf~QJL z%@Xt-is#;e7TxeaxB&HlUkYttbOD+miuwyw(V?tr4s%Ck;!&F)=q8&- zP2n_58bI90C%ssM76fPP<*+{Vy$^J=r_Uyi4i|-{kGpMV{mF5=B{Yu6m)xiJlWpdL zm|pvOyu}J5Ns&M#7?&B;KeUL}?W=YB_Kn^R^u1vQ{8z63RQ$R?lf{O5EFz{~Oc}00 zf>~$?*;SRS7hbH74|cAgxtmh2=#-GF0MEcP`nJE8#r^had)1vKtIr_gOgO~(c6AhpcvnFMgT!K-2h>04tax~O zE(`sP>bJe4Q9<1fC&1FR$k)#!d+SHT z-<~wE=fqK}9~X5>p=kn4NkQBM7#M9I_;^T*;YQ}8M#gVIev7qauH^ng%X!kVS;_09 z|ERYC&B(1g~`Dl)!)CgnHs5Z)M;W*pX*)O6UW-I_#_qrjWv$)$vD}jPLggn{Z4X}16yDEz z2)q#~MBl$|zRdR(6DFBgB%5(2Ny6MGteLr_Yf;Pf(q(D)EP57wH2tHW{ss4TC?*Zr7OdZ0>Q17vs@-kNj%}gfs8&2% zb^VZH?>D8M{DH4p7uXMu?Y@ud$>)$vCrEaxNv-gxm;2`L{VY2^I0sNajnky#@Zvke zHva2wrq^$?zjxC^NK(vyC%}GWdHhUjb|bpT!-QJwXic->&H(`V7k?$XZ+)8uGgxi_ zhCpY;6z$p zG`x$(5Q(PGx15cpnCH<-xmlWR7wK9`CxBqb;qp$naJzz9X_)PHEL}?!<=inhS>xTmVAlz3Z9C6nKm+=Vpg{ z;Kk-K8}_0iij&;35&IL8=c-e(3_=`P`383kOwwU?2t|adhl00EP{h129}L}k0k5j4 z;`SosP3kh9Nk>w7-VHU_2)NVS#!{t!d8T~9gV|BRN{Vi_Wfyzu? z%U;{Hwxa2hrhuHlesFGlg<~A@XNLX?+zr!jb66En4rsmKHb^(~IdPU~SrwPsWO2%n z-?mGcH^7w4Xi^oI-M#CET0QzF|Mn3h$4d!4gkK)RRU(2 zq+SceqNUpfu*wWRzPBztbHD8#r46{;Lg}V14l3JSn0IG^5Igc({-;1s1V`o1H^!f~ z02Hd`GK$c@%{6v;Qgj*2J3g)tD3}sm&Zju7!Wx~YP2}~&R||lsvSMfY@_f2vBQ0Ez zmG$N)&httJ*^oE@G@sl0i|l=oXCXEdqq!WeKPb0NS-8UGBK{m>;7wp4jg!{T1k6xE zz6ox$@;exta@fGjGrwI!$C7Nlr$gd6+{JbC3Fe3Iv9Z8gp4*P5^jH2$yi z`CN-{-Ia~it!2hYB|iEX;o--MJC=Rkz^!Q-5gOkGF&W1@4%%>cRX5_WB_2)Jp!R;?|pTK$%ZLlx4(;# zpV~0^hg^Inv~_IBip;dbHEA&0(mr4NzN5Q*+`V=!RNB5i{#GY|_=f3FpB~bqO_TrJ zb#dpu(}i3!b* zminAEo%mFc@6kS(R=Z;-X$rQd6pwzhYNESAN`7p#Q{vIvf&jN3*3clG019%`b~0vi zy_|NDT3^S4;6)XTU;=0 z?tUq5n`uIGdf@^~aoNuTRcV*Vid-aX`IS}Y4T7rUs$TN<=4VQep$ppXf+2(M0Kx6z zq#d*h#($c7D0mb8dVc2p7O^g?L4(F~i3Sn_#d zV5VOwGfA_S5ZGubkmC5<5X3*5Tp!+B{KpHC&EtS30Rp{qWLKMF<_eC?uSvnZ9}L(v z1*sFx03^q8GCBw6*qu!WzTw)*r+iZ`^E*p?4FYKFUM;4KH{@H%*bHC&ZrDoujf@Qp z#GEX3A0*s^c%iQDCc`l)1A_`>ybP zF^=O{MpL`&*pg6(LamLk*4`gFZm&b>7B*QCan9pT@|IoGGg@6li=rAU5?ZVu2M@LI zhz)0G6qGonqDIgZk9L=u2IhTHrq}{I6^OB%fkwGRPy{#O$fIjMHh1R(Q)$Py&@6Q* zHr89lpD|5l5$KnLAS=;M_Kb;vd=r=IEq4O1;6o_8W```SWpU0J@iL~ySX+h@pHmob zQ<8y<%0w*4iu>(1vbHs^w<13uCK4DIf4xtB_md?#Yye*1uk!S4yy1L&2gnvg3RB+S zW*f&$Q%5K~nbca1P{-2#tla69E-g;HsiA>ujHgJ-Dj+@ALFoGqt@BN(!x@05^Lhq^ zt2rmbo6PS5nF~dXm|#is!VTZuqmHnocteD3^%Ran#9?@E0dXPu({0A{4Hz(`Gk>ah zmiUz#0Apy2wSiu<ZV&F+WIC^7QBn~5Q$7~P2-$yn(=?7hWaN#}PcLU3D zYIyCk*tKZIS(oR=E}F9Ix;0Zzl{ua1_0iDd`4&>|N&`~M>X>i5?;uvnptI@`u7(@a z%ZCB;5P$nle*OT9`|7v@mBx`MTkU_gbKF)~><^tlZ{xyZtagIu6dJx=joKrdb z&<>q;8EwhmuAs{uqDx}(+vC8!6+7Pb*qTqpA=J&&xrBM*QSxr+QCPrKKN(RGR@HK{ zyObwaU-2EfkhCu!j=m6iGAq-%DX`Ns8KT7MAB?sgfkVb*pssT(Gx*b8LF2C>(O9E* zd3Czcl5A&nNlIJbBnYo*`E-ar*Sv2r1pBT$UEE}q(;KTWdc#JFA@IP=w)F8PS~ud- zeK6&|eeKAx$X!numwg<&jF1ES8pdm=S=8BWb}iz^oX2s}h7pS_9f6kS$HSuWpS63v zHvo@*JDh*+eU$>IFGfBr@B^+FCj|FWz}A;^((Gy^^@1(3S31z^V%aO4{HCloGgs-u z6dV(wu@bz=>F-ZL`nbafvxWy)>Cbz@ae68tz| za>*4r5rwBz#LU6Z8HMKumkmuL$YPeK5kFXdBYn-L!BFbv)Kwl_2Iry7qK=a= z)z_MRYm^B4+qle_vUiJDov(BnP0#}E4PWe&l#2W{e9eFBtpXUmlLx)d+P(Y$YtFmI zwz%k9O81WTrhOgrNJ$rGO>AjAE!*mD7~5)jLUvOb@Aa==hFf!Cmq?{cQspMVnNP}% z4w4#*^E=sO=prMTW4C#4+cMm^UTGdUj5dyiuP$z8b{^7AfBEY3V7=CEgfb=?yLbBu zTHVmj6D9p3eYmjH&zzd*u!=F9Pt@bp1(Z7Nhn)W2C)EzLot2XuzDH>$U86>hpRy3x zHG{!~`%71udFv(}v=1BZQnkC>G`~=0$puSiN<8}ws>EXK8J6B)QCQSyO<;JqBb zD{(SDs>$Z&-Ig?ha7y_U`KFxfpdUK|CHo7kw$ zNy7#V$mvmclh^Y22Nc+ueHnSQGi(5-TS7R1-749ahU!!Z8saeTB$)UQ1Hn(HzMw8i zTg{u>nI|tQIhPBo(7fMRcddBV1t_JdeJ9eaTC(RpRi{w>@(b-ej~(PNw$@ed)8AkeU_xVBLunDh)&Z z%p;j@_IR`}Qcw@!NOLYcQD^3Ew-jiNagB!Ha!3!#hybST3Y_P|@z^Wtxd5Lm`&4-il#Z2`{HR$AGK z!9@IF)E4L`^Q%6L12NNWvfYt_Ck7ZjDF)T)lFqMWXu|0LT^n<6aw0RK|vlJ;}z=JsyHqm}g5@aM*W zlO1EKzT2Zq<(@FF5*_flr^5e1*(s8HBx2_a*Y_maw6g$aD_)|E=5E#l7KInXZ>jro zx})-{auWD+EB9K6TA$vh(llXC~P!36riQMrN7uh102ZnIEMFmoYiUrhnb72!cPH1Nj zn+~dU(h5u006CmPxiGwS`=VAHJ?h^85iX$Eybv?VR{P$ImnfVg zQi;Bo22IH)QV1Ztm^hnQO&lcpx)-^(<6_U}3ACzhIM(7;P<2uL_;m3H*M z)Knr|$M#itcBi>Cw` zf^F7liCUZ8sDcyFVXIK8xEgQj*W`e!_8e_zn8!8^>}b|!@M_s&3Q;R>8~S!B=K9g7 zUR~v#v28V4&-SgRQ#FuxGwx3NJS{EvO1GKk0sF}a&??Pc`^8B1_)a4=DKw1Fw(-TU zlntZ%B3p~y8a0}=`WNZjfEgjT`a_&{uZs2YNB@n>j-6R>^_le77*>T_O;Siz#6VU+*{7TH7L*suiCVx$cO@bHPk!d!3$bVL(KfpWfy+ zrYBKQ#cdH${6aLR1mbpIAp=irhdNCDL;wDHGiv0L+ETW;ifwjCU5-+2t8_L;9=fsRSg9K|qj&8bzQ zwbDa0CZeDr`~urxm($glj`RS4@B8CY*?S=a=HGeh3C-t$>6Wfrc>gDGsWCl$Ez4B1 zN`^6u)Fi?##!Bh-2BI9FCpas-s%hUF2e^7fm}x7(TmiJ$1SpO3{qj7xu? zP$fAR9TZ7kd82_xseJSgd5V}VmRmqNKDk|F$i&F#JnPx3%!}3gn7D)#JBAo01I{R< z4hDtr^n;v+GvvlLxSO)T@IPZx;(ny4>UB&E7za{;U!GX2`)rQuHLR#+y|#u^hY2i_ zY%rPGK%`v_9WBvc6eGW3mhdtPb3gg;yib!ximu7sU*lT7pcja>`$v0c32EfJIqhC& ztn5lXQ)aG&;o@D?M*qH2kK*V8n=_4VK*1qT=Da7|i#zp9 zbFsYnv2x|kRBOu_$bp;_qU`B~1)MPJcH#?enP_&TThW5lN=TCTR)6(aj%n-$Qoz3^ zv|(;YT@x6C=jj_g8_S$Yd-wTz6wj4$2V`5^q;qPBepG~3ta0hWVrit=x7Psvqb2xO7SDbBusCp^0f#LBEMQ+)-@EDg?>O~XF5vOWDRIkE1T^}Zx1=m9wk_)6!wcFT)lXtzSj zIS!9uYS7KMlnSXxAuF%zdd=ocjgIUxlY8r>NuurM(hc+A3e?By5huLY_anjsEu90Y zmsj2UQ=M<2Mn_BtP#m`mc00dDbl9vc_K z7p}YOxt06Cj>s&P62AQLBm?2!dF17RUsPw|Aiu%;O%&>ad;zC^XNrq7$qlKIBpe% z^Q%V{xEY#G3f?CAP4Rw6O8?N540(u4laK;xCLf;7HbZI62!7o#2%XDRI1fMdH|f6} zq;p>}o(l~i0hWwMTi3F%&c5IMIak7o**jgf7pP!4mRje#YYAnxMlXG#`GvlH)6-{N z(@`&2vcI!xs7CQ?e$rNi2-9(EFJ3t|Z4RbhCN<&$H3}DR<)Yx^TiUh_N^aF26WIIa z$jYgnE~G-@8fDoENV)fydx&bMSM&nO!oTt11{G@RP5aIr;`dAiPDC(Y$%QfF+*`FT zA19tjV(d5_o$4p@^?fD zBoTv%QUUizlE{#sPm`nYtl5Mx_#5#U#pnWlY^gqSjU7DQpm!eJok@Ra;vAF&f-)`d zeQ+jvP?8_CW2n(eIUsr>OgYz?WV-CXeEd=4;w|3$KDx{t4w%s_$yu6xvycl}>bjy6 zmkF-F=_U`1H$V79fC&7CSv(uBA&xst%G1w>Xrt=4PJ}LpP$FB@~ zimZ=yx!3RAXk+zZ#x514-+o-R|Fq|c2pt-BfRKQ*K?B0lykH&m1c(4#Qi$MK_Ie(^ zPUo#ogr^&g=}*^qF;^^wi4fl?oWTgg*q}UzMV(Dj1>%{r;y2v5hLU1+bYDV;zwVrU zxk2IQOg-iJRj+t0Z9Uk+7SzHULY}xc)@d_175^QbpbYh)6lMO8fHhKF%-3&ZHN}zA z`Hu6$>?*Q9ihr>q24h<>?kb*L*)OL2+*jKj`>_5f>_8q0Zwb1^y( z9I1)bYwI3>oeE%3UKWZx$$97`xTSi3(x$*EmmTv$*IOGT6jl0l?IrDIfhtm}e+x_ubV@g?eK4&E46pyrOi4rQ`Kg-*h*Xer* z!%{tIUaECgaEXoXGa!qcbo^>?s7B*zy8nDyb)Xw|om3{4!#+NuHSY~>&O6^ixXO8M zVH(=2lpWx`wfn{n(c@pAYvyHeU;hqMLrcfuVv-SB$omT_M`@$Gt)?rTi;c{IM`4zH z!)@bmdu@0A{O+o~?Q>@`lr8PQ3bZ5EBX!SWM;gEN-zCiCn(^Bv>5wt@^&jecyOKt8jPBww$iWVP>n2xs^Cq|642nL+K)8(7f zYq}=~$_*dKqCU|AX$^cw-_Z5q!Cutaz=NSo3kHiUSrR{!2>0WRWBCqq+eZp1WjIv^ z+8l+PGVLWR3!-jc%7`jt>~S&S^-0MzXj< zHc9Z5BFeTn59Az+9g-oxB(W4G8%9MFV1(c4RG@shG>@qE0!vj&oAWEi-uP_#ls}}~ zvb}a*5OoXWw0y5h;@VbKy++z@J5a!3vKHT4&hw@Y87c<{7x5irRjkggR6)eQe4Zeg z*n>zL(}tK49TPe5L=5|pwvaV9Hx-k}7Fk1#tdQDYo|BKpt(kM;kCVoXkHzE4b0mg} ze%Qu`2fdEE;GU0npp9nC8!!u*&V`sr7ew|)1bP=fscSL>@F1V?~leXZA#DbG~d9MJsWx?N4bUykU!y%U>&gJ6dX%&E{470dORU z6&we`=euvydEK@$dYb;bkZF~_1pd}_0g=re%zEnq_EZmIX%}77}GOJ?qnM`Smjs$^Z~w+YRK9Q z8uvzHVr4pIQ;U^ttt3hlv5WNYrwJ#G`3fSXFxPh1UEFe?iqz2eX!XeRrI?5G6>YDt zF3#_`;jMURI*RgdkDBVDcdR=$72r2Dv_fb88{qOE`Gik5@zHw2g#{ai)6&~-sU%9? zJ{V82u}aHez+LK))^hisYQYo%Ecrf})d;)^*e4hhGDOoP4sz8bvJ!GNr{*b1H^`IS z+we>6OXuB<=0wH9_kbAH67p*R^+vL}LHCN|f#)cUStKT5pTqgh;lA}gYQ6e1I?U7? z<_#5CiMYXxy>~+swdU3VRePltdAg;ITk!3`oA(%&016 zK6?p604a2x9I#eSgaI|Zd=rCeQn_OqliAe?rS|67P@U zA2ozJiV&3si83ql3UhT`l3Dy+=Xr;*v2s9O2m1Os*Xa2V(jbYGd%M9cEb*H9wQnz0 zZ6~+hdaf~Z>EQlbMH#1oC4cKdFUUI{Vb*a_p%8m{%1?T!_4sw4-}XF`>RZr(2@`Qy zd(JCGUiNK%s)it5V8x~g5Su(t*)rXD{-F51R|AhA$DH&txOuG#~cC$*IBcVvB0Bsiy@WO~y{Jysji@X$7- z!e97*eju%G-QDk&QV>95(JZlYLu+L(HYaSo~m5;SY8rje=`HngIK6-e=)DZ zPdwrTJ?J#x|@#53?BmIS)6RD{Pcs^bfxPvl2LG18`&dsPm9{ z54qO(-@=+RsiZ%r4LlZ!_^%7V8&CL&x3xkADtdLTG7I7{IyEfo6_Gg2S?#inaNo`= zMRh=8-C!J$CjVtR73s0%4GHkbE^FHQSWt! z1h(I`5-|qOSPMP8zC+0WLah>tUV?B{}HGvc^}fT zoEh3rNY0Knzu^JGaS(B@g{3RETJy8`m{cZfn`qRL`IIy#a6V296e$M{&5QFr$z$SP zk)$#0OIUEQd9|=JdwLUHw0KPZE1mM}29{Su<4o#$rpv=LSQB!!A%-6u@@7~yA92X} zttVB#L`P<7ULEN49&wzoFj-$Y8oV`=$WWTsRW~+)B)F%Uj9Kdb?W%v)>fYSaF-lJ2 zOontU->LG8ryHjl`8UV&1HmnFJ`lQ0yIezs`b!Fa@9b%# zy&+pan1r}vX%9itc^`j{SM(O^zU+EJbx4djkYs=-ws3RiIIxrsWT450KyL5NRg6Z| zf0WTt;4wYzkB=vkof|v|{W2k_D&Kl6%=p_sL}nv{CsgDTcb{h6D90 zT}Ov+{Sql1%Y)p{`Cs2yu+ zNBt>o%udGWIsLp=M#CX}(d7iRTmEPxD~$g}!Z?@LF`{@aH!lE0ht*l%{;h6p7zUGt zNalgGs{tP^!yWP1@6vqc4D^2iKqD%Iw&1e)js7LMb4)5_6Z?zhe6%KfgFY2kJ4h$6 zrqCl~;BnD;j)6Yy-nvQF>L#xbC&|pb>#e8@{;N5Ca3-@F5B2ITaM3%?G}TR7G*>OM zs_EwCw?=c#{nPAj2dfv|V_zB8vtTLtrLVLhtd(!8Rpw$>otd-bJz6bKs3V%v3d~2H z$eXEgb<6M3c}Y<&uX*pJdR$QWii4~R;_Oo<+>>^k%uw%>?-hJHhpXu$mNO#Zrfwgd zvWAVJe6A*7`c|$&z2x4`sYJ`4p>)jh-iHyCv?+Ivm=jAg-vbq%6F*vm*+ykejaDex zY?QPIp!q&_-7ydrNd+Q6W3z>iK$HUS+ltZGvjRP?iqlB6e5Cmbxh}Dy=3KG8Hq=a1D60ymc5qTQhZgP1Cf5;v7) zeTr$aif6E1kCvM3xq{^%8bIvBXq$Wv_j2qlF{F|2{6lG}M} z0`1oJE2x|jd&53Ym6j~7SiC|=l6GhH{YH4us$SUEy3!ZL}JG-!?e8_<$4MdXJVHx8|0`ir1}y6V^G4vFL%14Y5Y!7-Ax zGPiI|wCiv{khk?VLO>i$V)E2x2?)~sT_AQmp22TA5`UrNcvC9t=enA0v<6k!CUs1t zTOMV{ES!XBm_~lOThQ-0g`g!3+(CM392@uB>T2}Wc4l&rKsO()*;E$L%_T(~ifr1X z7rZM`X`#w$gmZ*Ekc;IE8)mY9@UtU@c*S|ZHkl?C46{;7u{?K-Y(cSS$tO4DOoBTl z8a*7HFJ0|FN?Xc6cB)mJm3-8{@WQiPiZx|W4HS8k?21}j9U#;8&FU&#>$E@1%wf{SL)LuL*u@QW zf@EqF99K=5_$f$vOpFT{!~xRf?iJ@DZPV5&*YVR^KI&IhkZCvgrakQWXS}hS(CMy- z`6F~j>Iit3Qs-t`+a=< zDeB@VMrt=ANd5OS=TY#k*FLvTGI^BzSTz9x= zBH7|IHIq&==t!mbd)$L2O8ae&I~1E$o%O5EK;sW@lbbXb&GCXd>ud4S&HbKWlEMNQ zsuyV(Ev#(;s99*$remXTx+61`The@mBflB-KIY@&9<@jHF(@SZ3e zqg42R+sYf5M(FfETBwP;to)jlKtEPU+p^xF=o zEZ5Ov$q6*IhNR4u+3@te&QY@<;W@a|7wGVp>|ArxIx<08&DrHfooha)wsxhS^~h|Z zXME1*n_*!ouf*tCN8Vox5I4Bb?@f;{PVm%H{2B$4u;>~t>4#aO_bM$EN5;y;$}L2s zPd_*RdWs`{MS{5ci^<#WUtuT5DD}YUt(swN_3|~>+~hmS)3R@-Evkkc)CY6RH709l z|Fyr2%mH4{hQ7Pm=AzyxF3qiI>Uik9P@AfEU1}Nn zJ(|o1f)#3cXZq;s;uo5p{gU;5&mwndK*C%=aF%sR z{+CmHiKU325duj4r>OTO_jivN$>5RYeZ#1PzJ8ZIZi8jS1T}x#3rd_lpKDY&Lt1D5 zb(r6l6ze&Yd^{GWM}S#aR39#Exxb;6iR=?hFS8U}w0a>mmRWCz`eI$5-C#+{c${$5 z;^c6=tSY4`6!;8Ek^13oXBAjColD_ue{y}T$+_E=*mv}kZWUQo*hNpAE2;7k);+j2>m6bAx4C=OM&B+2!jdxCHV=*ikNIMkKv zKoDqQSv`n{FX}%|oypTj<(|m--q*G3U>vlxvZo@mHvcD43E@`5VwCcFWO*Xc^(yUM z(U%`9T$5GrFWH?Hm##cr@h?0n51W8+W%G75VEnX71(;1J9`}LS*{J%X(YM=zW^%EL z^49&+>J7RPcm$GS!w1TXma2-04CbNlOIwucRu&ylN{p8%gG+TPti5Ql`WwvBv}45k zY?fwt*18LY{7cneeU7;;hq$B`k^N_^4CF;s&xZ(i|BB@J4Bq^e7#D7yJTlNx(jRCh zp*HS(lTO=8b-Lo1FUP)LSZ}x~bMZ!!H7~ArW!q0bwRLK?I#ua4?% zyA7{t_qb`GfG)F!Gop24UH`E9#U)z6n1^;Z43Z@8_*E;yRFBY>iF5lVbYOUa@9vD= zmR)Sx-sV5mq#&J+$fun0eAkX74I0j98H}vE7h~1<%u4{e%N!IU*jr@R$M1jo%wZD^ zrqWB(hd=&?{4$^FLpTp3Ks3ml{n~VJg>W-rsr~#2Cd>zTpLEP1*S$k&_`~kz$oKcz z=)fFo`#$b|nR9_Riwo`>hhn0l8wKaZym%mYZ6DO;Kx0VGStI!Wm~{8wQw@~Y5kIjy z&YCxv$Iut@ma5lWFZQ6E>(Qv1i~WYu6N&@0YXj9oZg}K;ycjzSyU1IY;;U9Ur*s?Z ze7DbLa4g%ZgkbOxtBFz z!VPDL2Hi{K_NJ9H5cv+*=%WUUVCB`{39Xcek5`qvs(HCRfH~yXiCR-GqVL~!SP!gP zib18g-6zVxa8%@)cqg?NUVrjXQGwGD4sN!6w7)8DIgQ7NUua21dcnIRPv3w}5$+vo zdf(*fGnrtEJ&SJoIv(o%#lB4L?o7*`8@DguA-P!%3mfe#h2fhA zo(E;{f+Mk{1G{M;8ut-bXtnl9HUm!(;`h65-9^z_+?kc`1a@Ze}Zk4DZ@fK2Hg7nv(d@a*&m6ncBT{I{817LVO9f042J#!0mM zs{WnWC9ba6el>0LJG#Bx2BO4s&K^aCVt%wY z?6T)Ne?pJzbMj=So)YJV^OaDYzFSLdj5sG04^@-f<~S)4n|+qDJ%jC@A@y4Z$BZ3m zUo96$ObQn7yo>yULH;dId5ZP3>qgv$nJNXz=Ksgb>k-c|`oPA>b7FC8vfV=k*IEzM z4Q}=hE7va3`j#Tldy5ZCm@4MLdvwUG zafERVPBLMW60Gv0=gz~Q5MX`YHX=AeJE_VjhXt5==|8MVy*WtL$Pmaty2)U zZ}2-IxbUQo8Rh2E6FNWDuFUk+hHke=x@zoi7-Y0oWmEMBwaZ8~%*if7Tsz8!EJEr> z0sZg?%;-SE|4Zt5<(kc%_?H=stokG=JbH$!EQyhff(sHwd7`U}YeBXdj-Rs9yR{A~ zF7$@^vP2bo4=sCQ_y49fYaGKXe!!8|AmB-8thSYZb-w3Rv1DlK>t^>Ko1XiIPez^J zU|EQ1{DDAPDC~Xo3Sp9I;-f{?!z8iV80W5cb?e9!!Uu|mGMmnFOyuwoi%zCVy-@9k z*Mz|(ExUpjD@vTc&F76#68|jS*vC6yE`;w#-fM|F3sq~Qx857|y`}o*8(=OfslFEY`vKOR2;CGgvTMrcr%%S zZ`3^SzdZ-kl_P)BN~;aBzx$I-kfNG)!ctQFmvuPQNA%D%XY0A!FHS)GWPQzNLCLOL ztyy!5hRHQ6ICd*=4bzTU{rQ^zitw&K`Yu;5P&cza9FkYAt=S4c45LD7T8#ep{~+j# zug}BpK3j36I^>f(8wk^nv@)+oeyGyYC&+>|IYB<$dOvI&&ubD_SBy*%G1tAb)hE;c z4gMfoO~8Xp8P^Bc&o1@8^2vQR5*o|$(of~=P9Vf;&)nKmc=;J8<{>=(UEz|=K*4e3 z*v)1=SYMjr3w?RE-S)H4Lwtl1g%`V=c}oX;t<|M3HDu8;FYC-)=auJC!rF7#YP(IO z!D6ehistTr9#joOqrKBD?8Zq)Tm8qY-|O#ByGIzax*R!qqqJM zQ*f*uAx{@357Bd|EXw6IYf*AJD?{fkk;R_qEdJ>nkKole#+3*6m8V`Zh_V zleL`H4D3vDMdbi-ws>o165(ULvzXkN^Fp*zftS|UEv^Io6khT_PtNHMTnB|YF4?;+ zcI1<;u6(7}D9%@#RC2}()jGYX$Lcz-rpiu6_*=K*8aDJWB6RCR&B^k;gGXPpZk~T-30Pc~TSg%&eSO^seN^ZJk?o(}80mey z{G$}aH3r}yg~z#1@0Bekf01a9em9z_zsB4_xBNnHGgWw1F4Oh-a>Z&?0-MC5b$oI~ zB}c8yc~lnf(q6m6nE=fqmy9$xY2vR@FK`w^OxhC~sqwh!+r({idF{A1=i+j`3Wt9m z0yA&0Y%fU9H7!OopU{goq`_++2nTqPI}!|9ja4s^`y+d-j&Z7{<`Gpc=UnB^q1zPa zhC6|?@b=S;WSPo8hjG3n#~G@9{k=-X{yTFTU7MegF4Fo_p|FaS*9(|@a|3U?E5ERC zKG8|Q$D|?pGwePsT%X?tBU_A^gaT_@At_>+7rgg(f2qVDb0(_3buN-%ZgPAq>5!Zk53fFdImP+*U5J6 z^Y4aWU)W(09O?POT+O@U$80Eccq?9z~p{ zA55znV6?=(p*S!2Zo9qX52dlFjUBQ{p1As#X4&TgC#HI9Wx9WDMS}e2vcQUv#qH27 zB#~VFx}2&KGH;5fkS*Usg?CZKgB7Lr-NT2vS&RV3Hb-BRzrxTGc2ao3Rc^gH)vDXP zqc9%R#^RZ?+_d?>h%KMTH)jy@rkcbSYtg3wC+5C}dD~iob4DvJOUfsqjj&Ig{7)K^ z+Z}6cBsS60xAJYul73G5$S3~Qw`p($;`*a{WV;V0eYh(PSB8z}y2Ac||Hcu~Fe$E= z+qVj?;gRqMHG+TnPT>5f^cfiW_&DVM*m?`7rrY;_oKiqRP<*6ADG}Mo5duR|5XL}4 zL8U`L7>!7bcvKMS93`lfNOwp~>6VRhlyoC85C)9@dl-+O@9%#OXJ3ypJf0yo_Q%)H2lcX^kr?e_`=62eplz}vJ_)B=B=nlzpoPD3F)3SB|1`~a~3`t zGYv~nTzy_7H$GS_8ct2gYSzB+{gv|2Um^3{4JB8mcA=5x5HZ%nfg{-`14DuZuCJFv z2l)_5yy?(;39yiwKCSxqS}X-#%BG1$2OD_^ewkfUcyYW2fLRB(z_ zGU07twIf#|V#+-6imG_lEkBR~GxJskQ8HyN!t7__dkh9^5kznO@Q3cpon_}Pwga{t zOA?vI;GGca7DZOtOYWEJMY^ZwzE}^={@KSGYCQFZQz0xLeeY*f$IaZin$Wuu2jx_b zgUKevRJ6mf=6lG)52ZpT`f__uki;D9Up4|x-IH7X?m={m?_*@emwAw!k>Ipe&#eO7 zz&(c5N$zqPC;WAlu!CnzCj*oVvvIi6ImyDHK!uNnsKs*DM!4^TzSzO@6Y<$v4?e&Q zK!XZyDIJf%yzc5D8`Uxx*Nsf);)9Ea&KhC9OD`wuiHNOMLjQ!&)RVi`1zg)BKT}-O zuT%x@`F8XcWV_T2+aqlZ$k8*w68?$c)sR#CtR9v0?9&Kz`X3V828Ssh53GA?S z_XaDSdWnC>UN8)Ma{2k`|E7lo(9^CrpGIm^96jK01=A7FFfc5GiD3g^L@F%&Zm(g- z>VsBH59oCkUed9Dr2o77wqXIsPfY1g7c{5kFI!iCLDMerS&2&YbsiUoa7c&EMt_d3 z`<5l+`!xDT2uo`UU8Zjj25KCZDyoRSR4nYS-dq-~hzG>1y_>i8`2XRydWd7gyA`(h zUIFcZThydmX5BJXAADeM#Dq9s0k_cSqjJ+7IS9pJ$gS^J91FP02LM|DhBy;wKn7@m zr8U@}l5ng$X=aJXt|(3_)?D$9k+(Gl!lfb=idwzVndVC|I%ir;!WXm}tIMQH+|vHI zPhkIy8ben@M(4OMwm2dh*bY`_{N8Pu6I$qTcaO;rnyc6KPPMTCX_ru3p) zCblA4E~x|D9udBotI%{S8_$9wz}ztMV)A;#EXAf8oudl(>{$fclWP!u{nUuLxdYB}w)~_UfU&uxx@X|8l1Lxnnm(85JmbDM*sP`qn zQ=@%=zdX-6zHI{WpMNE%F_mYgUofz122=GoJ2XL&uVr1c6c;F6&FLr>gXk#k>;IQ~ zNHV?j7V3VW`LhO3Ys&6YGy`9^-KgHtp)j=2HL>3Kl;XXp9c}y}eKX3{5RXcgUhaJa5P~t9*8(grWVaiQo;{cLv zRm@(t!o4qt^_Z|RcXrP%stanRl{5YA&FWrX$2QZcwoG=k(S{?7D|H# zdDZXUrsJ>v58et*H3xeGA+!7porlltO}Tc)-&7i^ngNaUC=9x<-*^Yl7D1wr&PJVl z=*A#Rsxlm1a9*D z6MXYSpG!qBHOY*=?E-!vWS2~NxMMUbial$7j%D#fLty>$yLs90vIu1; zXIBcyS18h*#Tz&k|HohRp=-xY$WrF3opbNq^l#ow0AH_)lEsBm!7#3J!gBO$>RJ?r z3?nS`n`!U*?uat4edWvjlkHJa)ZUKnAqErFJL-WdI+E$_k zcJ>kCguc~W!NbdUtu@uP+-GNSbiJ+|9KY`>ByEwj*UE4)fd|~%Z^PI&gbPkiPd6e^ zOO!_t+b-X6>H5L-Vd(#q_}(Owx$aUDpGJ?Qi_Hh_(2A6H{#BWdn3O;Lg6R zqC>gX!%v*^+iW7c^FYk>R2%c5ROy=fo`2#wDW-3-xoO+CcJ@kuY0T?g^Ro$|#99@j z&-Y!~fv4NT_okef;2x?C=@mAN?VAvU*C4l0ORxy9m0vEz)E>wd#ZsmtLx6(GQ7dR? zcTZ4Kme44t+2S9qUip9K_797Y`;j|U)*2-Pj|%+-%#lP#_0?_uPK~8*iu5A0JF65V zeydA+s_gA~dWsRclYOKoPYR>}Ux99mV7|(6z=6fhDam(Qtp{zS{j5r$H)(A|1gc}X zHbMtnA%d{!1xMZcnN~CS%av}2p})ZIXmOf#Yz+rV>~qYt{!|-|!9t0+0qcD(yAxPZ z{73;%!PDM)O5DXa7o{ zAF+tAopFRY!bSwX8_YUj1vYVs=mJ{PTB+w2>r)Hj=k;nVyV9#J;-R2xb4Y0`amUvoE9qUr~nG48OYuK|JpKShiEK z{X%jSf_&UrSZtTPX+`t0|C7p#16wbR+-vjDQa-fj6{(~z`X|!0--J@8uhjHBqWv89 z^PWvmAfIE~$VSo@wr?EF@JECa58heBt%s05SDo<6e=s{*w>&+=cyOMh>TV zs;v()XT={sOp&J_;YdjC0mC>#2&KF}2dxiE}3}6psHPaN0q+c8k+nj&tN84K_ zBwO=ait}gqy>Q8ihcXG&d=Xn$EzZ|%);Tr0dqCIOpV^(PG}3b8A)7_^8p?+6HiudB ztQ+3EA>ZdcBBmZbj#viCPU}`W#t_=ER(zK&K38mxU;<5{Pxkodhva^GkX0)D_Q`sD zkRM{pF*WEQqcD_MlL}ugO9ATWKV1xvsCQA$DKdXCyxJkq&~r;eP;+uY-|)(a8xC_> zFrL3*C$Y^ASYwd`1uoe#|Ahrdr1NV!N|chnbBN#bX&Lt=y)jF|g)j;+-<16=3);Lf zmA?Y%fWwPUXo^oU3x~3kYXDUeEk$cnicc!GRz}!rn*{bF>us2TBQMNx*w5XF33eZW zLMh~Rt!Ub$E<(lWdkeS6M6M)C#4g;yvf&561ikHTRuuc1UH>HF%%BY~T7uI8TKDZ+ zF1?*=ZAw%hH|DRqKK_Q%mREab*|j0fx!1T}HjI`LeD;b+mawLH#S2p3B*2VD)S;O8 z%1)GyTb^9?`|kkrs#^OnY(?>Uen@k7i+mkWQ;Ax3HVvmpqfmW#rN(Ws=1p= zEiR9l_G86QS|ZgAhKk+hF5B`#ox7X?09H+KsAHbOfa<}6ke*@&hnW9e>MJrkteP*y zQ+-Z11=2?XaT-h5wiHjSHV?btCJfS?A2pUjpz>fn1t|}K=iBgI_KQUpzR0TiSq?X% zVyR3r1lqf)<;@aVTjDPY4=xV1J)b{!(vR?&%B=48gONRsQm9j-8e~9f9$!&Wp6R~r zom@K-TL-TjnK3d?u3UA*Zm&K4>01vkp|>)Y6H9Jh$|Em3^RlE}iyOB2^43*V&maxm zy(?H3PAYtF%ZBx{`>l&<1MA?~QEZPJeTTuNXG%!^NX6ei>gTgW*PoTyPYmSz;UyFj z%xU55PFkNR8+m9Gf$h~S9o07XyU^(jCY5tL!-@xs&la;({w2KPaS2OaYQ#PXCjI9x z5o8vUU=2%r>(e@wR*C1#w+OhjG@+3+tL{X;N^l>40=k4&w7@M)WF#XZ&Li1H)&j9N zf~n~K*gk+6Osyy@sL*^}KU3l8!u1+xNSpdlgbEv3bnfP}6rAkV*Hxnj%7$F(s#-pS)_%`-`@7_xRYIJ0Y zHsALz38zx+AA?H5sR(TJ;su>7cLD>%FKO`C_@1G29h|h`{bXiPbDm`9g-dAIS>&Xn znu*%Nl73S7hRjJn7UIUcrsIRv!uPHGQ-Yh(j^a7fiSo`ivz|p)2lgSXVx{9Ea-wy^ zbVPAE1?KQ@JY_3;Kh`#(BHKARFwW7Q6&S;AZirv*^aQH?H;0gM*Ljvr=~jj(j9#VZ z=Z(qE7|fgFS^RDo*#0V8{@US1ZWMVVXA*Y5dyz9Uv%`WGc7BF5Dg5Zt=V9kL*oBNh zQm49Jh}lmkPR^?^ckj6E#<9@45M?Y6SQ~de6ijTs$LSs~-G@Rx8P88{%)w}!p35Al zukdjjThHISA?4!qJJB$ur>YG!ZoKct9VqanBCRi8<=R||;(SH->#^b0>CudN!NByi ziVTM(w{pR?_L9mr0!yP(e$0JdRnFNMy(CLEEtxE-y>#RL;Emn%5U)+%0biEV3D$=5 zw+%EsX{7gDZMF5S?*Abak=zP;m?|Mi)0|A^NvmP~@##Y!(6+VZWn{`1xt#o0EFAv6 z0G=IPvvZfaECn1f4>&F8&QrlX-7qQx+EN`gltK zVb(RbObrxU4SxqGzd{tLb2bufwy=l1r+>6lsnf)+Y_uByK_?@0zqBtd_6700*NPza zkt_~vEErV(>ZP7Z<`7>p;R~dvscb=O`K4`+$#%=GAu+``Daa|<_$o9$ig5P=!S=yb zUV?5NOE^$`&hfilhbh58BwqjpD)qvgvG6|)m;`C^xii$heG=beW(I!-4Jl(}=INPk zrTlCsr%=vsrv3>}rk)#xlNlpa{@rex`amz<6GKn+(enX-1vTZOVj043Q_Li#(T8@6 z?>6ohx=DC)S$E|Byhs0j9X=5d?Jg#lCv^XzOIvYE%X?5q^k%HzV^%+i0m7_MLW=I{ zSoS?uSakK)K!x9zc*vs$i)+aT{oGj0&I|@qC7CClG!+NB{~^dX$05hqHnF(&siK9v zRaAUSvz)o{R64IoQmU}<+$V1LSZIq{L+(L>G|FW03OH^gD|<-KqC+V%taPDw);Wc=N$BAI#p);h>{ldu3crf=nBzSTb~gy8Pui{J?*BjI z90Wu~$XPp@?=5hD=ldFDg{VK@T2HPjc_z=KC=h(&!-P@bW!U1@IUMVK)uVH`H3%p~ zY4NF4oH1>8bVc049sLieKnrgiCTq~{@k3o7doWQzq1n#x#uZTfTbaf3IM%I9ua3ga zcOYkVeR*eXX*kMQw-ys+BsY@PT9b5OCnGO`jESC(SEhy|Szi1?D6!wEC^rSE$=tAuWt;j`{bkRz z@haQt_*C9FtK)9D5V^5a!T^v`R{tJ>#uTY4G>mKH{ zThU!^ub+(OK;Aw&o3t(CokaL*dKl`|KE4%ASgS|^ZijEakeyjVfRC71pSaR3F ztVi&m3r4dZvnI-f3c8AJC`7qsC|axnmMvVD!EI#amWk>w&N<`GEt&(`Q^#b*$)x?d zPV)kka?Y}~1Zc!Y$|_$^{g}~<6fa+P6%bfowBTV0JK-_*h!jkSnoxxDv+QqVLT7NT zhJ&sw2OFiZHX-K3t-33t;(vuyzfQ6&qiOdmXO19_<)^Ha>L7%OmUp>ep6|s06V@9Y zqM}Q~znjvoe0NHE4c|$$)>p#?L{A3f%okEai0sp4H5xy!RwVzMqlFd1l#b+i%V)xY zRx)jTW=p}J4SUiJ%2;W{v!mV$`1X*SHTPzM9RMgRT87GUC1}R#Q_^c=uC{N5%Za6r zF03X0Pw;T;+%{>8{94&i%Haw+tN`L5dy;=G4)bW|ip-3`5Y`&_r}9D}PM|OIJLm`A zT;@loU>m6Ks&{(*Iv%1t0g8eDiZg0thMq)KZAkHS)LdpkKjWP*%gG8W(<_-I6;q3@ zQrHuhh5cYi+Al#e)BPLMtr5mAccH@jUrG!9-#-jPV{;S^9_uV-=CM%Pp;!kyJC@mk zb&u9tp@epsFGu+7(*HqMeYxrHaXW^EY{}~GP)3T|V%F=+?a6Z&H9Yw}Kf3AND+q^A zB@wE#N+63NTgNM}XZoVT%&rJOlP=9lJ$I0?`f$+n;IJ|sF1{ci64)@hao~< zSJanhYCn4D_3*(%?+T&H+#VpEk}Ox7zpis=(-C5HVa?Y<;rBzbT?1JryFbZ z&cdh9k4tj93m10ie54T;snA}OW$nB(&VuUHeefjNxF8%;l?9OTCTVDa?Jt~wK`91 ze?9!48OqPsB6|w)Q}RH6=-OvRlb*{?Idyt|3%YJAY9GFfpW?GW4#$7H)HJFy2RgRH zOcO1oB8cI8Kp`(7&_9$q`7v0{5o-&xt7&lAsID?hPETD5cb?HsaNE0kkImtgYGIDg zteVx;Yjdh#I4=lc(tlB*Mp7gmKxL%i3Z1T0s*#-f@GWA2oDP@nbE~qJM|XPqs6!E= zMSVk5{8lWL&wb|zkeeJx>ll$Sn{@lN)2x)A@3TAZCa)!4#Lb9nm?EfHT!DpZ?VmIL zXc3D3q7@{)H00Yd_WxjHMFWayiv}MtNUl9Ascao0|LPIx{0;Mib4u}50>)#9&x-N# zqJsL_lK+Yq0q*DCAIUzaeb-=E+QYPBOiJ(FL@s^Ac#MrJZcCLbu-&``waT7U+!ldL zBg1bR^ZOD;GMGEVN8~o16>~KbI$Hs(&c$Pzy1?Z?s*RS(&Nb%7OWaW4u4y;6M?| ziSudw7U68k&B4CmGaK16~-*RJZq0BJmIw!mRtGN9f9tVMfM<`J$<;3!%Tg> z(x*(V4229vjZ(X?G~UwJ;UefRZ0GAHue3&R7n(qvzN-#Z)+R{Le}8D4<68h~{nNN0 zo&H}bh7N6!;t+H(he7;QKU4{J1(_3#yuV{EGc(*fK;(;7FnN*rKX@;o64+(I5Z|X) z`8$puR5j(Mn|c?(V;!mJdNW<=pvmlc^pf-occ~`)f?%Cwim))s#7kLC>?OP72UIr% zQFW_PX=-O+(Fd~KAY3PuHr>k4V*xF2g>+ZPac`f|Zi zmkghcvk}lIv@sE&*Q9z%G+|`3GSfY7EWT1CKX#%J0VBGwhFF&mkBj%mZyg#tzN0R>_eB6T8R z5Jb%Q@p{H(=f<+&7CV)>_L%)Dv3<2`fma%GQxuUEf87%=$?Wd$jLW>UI1*>0ddw}c zbhWQHMg6EMrNU&tW)XVz`-tN0j%Ks}Jaj@5nQIj#arahNfA;6?KXun4?F@JZ2R-YF z$COvpk=I30j_<4RFU;AvKBU4s%@hTvm7M$i=I0=j=d(RcgG1%|w32CCs|5T^#^9bZ z6Am;*)tvOiD&`-boROs);PeJNlYQL&7sL-h71Q()6+RE~_@2I(t|HL4uRptxG<5CI z?*TX%v1=cLQt6gGN*Nh!F34TYXMWCfoB~DAlp#g#KsjsU3r{SSm3(i6`ncR|#XUpb zK8z{Z;WGnTK>wXeQN;iPr`TyP$5RmuBT|31wY{oR^5aFnkq`YgJTJA+vFkKVTGyBE zpjhw`^)4CV)y4`NA0P8-Mu*9Sl3CFt%q8j?ATqdFHUKwwm=w}Z#W8Ovbpr7ydu?=kTP=eJJMm| z9ASK*CCj^IWB-=FC#5%Pk;uRAzVfz>hp{@#@7)aXjTT4Wr)PC-{i2abwf_OPnbC6W zHV+gj83DzYUx!qZN*yTf+F(BkBts^KGahWIfRJ_)XDqQIMC*bBSI@7q^Yq#*!MZtV z*D~}xMoKE{*sC4B`2oEm+`3~L9nrKEb!dr4IgC4Y4(^ex{O8Lwn5s<0_ObV`2gF9} zwkp+4ax&8o?kn#vC`-~b_12%>Kpfh^k|~piPvVkBVrK5Y_0x$@8S&*iEc&k?8j#6o z+B2I*X58`B<=UUmsJO&K`6=s$SeKzoDMQ6wup#kN-GikeG<+v+EHqN3H9Vev@+ zM4Y}ylj352eqrWeA5h>r4)6@P$j47`bs+)(bpic;KF>)=mPLsV359%jJ^Fi7FjINj zZiFvBePGqmZ+~#90pOb4BcWo#jZf7t!W};V6WjMK{ZzhpJNml1M|h7;r)c?B-urg* z_bCi`ea=)lr@dGy{yGnE<3@4V8OvMjw)NI!^#le8MR^8|&>2(*is7-vxy$$M7%h`c z7PYByVv6%atZLZvU^O|Dy_mkVhD|~7{|TV}90^^<$i9)u$w}?)m7yY!Mnz4@Gou5F zJ=)4ozV&h0*9d;KWO?uBGj<(o_=0vtDgd+bh~CR?fs)xy4O-(?gMlr5y~S$eZl7Gg z?E9n~1z@O@)PYXa%+zG_km0BW_x>=Uw721hcZ2lf7$ShNFDLYIANDPMtY7~40S2nh zr~z#7pblW%6o@MM;IOIp>VPTakMob-g@8Vgrf1))tl!QT?jVp5?MuEtKAIpHBOe_A zQnLn!dud0W+i$29F2h}b(Rn|2s3|#Bg0Srtz4F_I1Q-xc?nT+X&KMCDH74%_c8EDd zXC#!ebtQ3)zFlnjm6a>lVx|?@1?89wBDW)Z8p*}5+2Vx%p%#KD`pd!pxB1aj1V&Tn zjiXwYt!53*1-TkePC}s&E!bB6A-)|#~tNpAwLkK$4PRHOC%vycR&KS zj%ZHJDR>aT;PZws=y*6t;8`O<;DNc~b-6WPyB+S0Eoyqyv<&Sf9E*%p63vUK`gQ+l;TwBW`dX71sbNfF`IPYaR_B34xBgfnJw{ zDE&#&2Lp{Q(Uzv{}lTq z%#)m5*By@Jh^)ve9yco2pMQD%QuLF9W|M6( zzZ$s6={)`}KbsW=L9Ja^Ivro|7jyZKspD0Y_Ae1$ z2IIKtDXe(f>SrEs?&sooLy(pxyu=jIeN}Nfd}`}J#uU-b1e%HOX4FPaM`Betg~CA8|1#1Kplha(ORUVP#tlGMM;wJ%OC zh>H85DY@>TIy*m+2BvSryq20S?g^RK@BWBJ)n*VoCJKp5g@d*eJBW#UFrp8jkC9^{ zE*_P^slm9XKA8)sb}YrD9atKzk;qK!?3*#4es67HccWEwW6YxI$kWYud?7T0Woo94 z=+!mKg5X{E1|!x4#L&{DtHh6RpvHw;D;*hdM?aw5vEbVG;TrS!?e7078n8tBhmHcl zd`ZK3hH-aLpQz|s;@p8)xZ;#Khwti8$^6+cDq7c>&HgBpqnLR8=nGr+4~P(}3{Q*X z|G?n}XUJ0*OFgdzej42eRJL(#XP%Qb>jy2LDW>v`F+k{Kdg-*u?v2A+DWv)C<)65i zM2w$K?b@39&i&{#A-WvKG58XG>~2gC*copnEs?bBx8JKOd?18P?1n7?oG{1yLMRlk z4AncFH6dCl*zPw16s4s~zx~Qwk_L-<0YA75)Nru51n8^e)^D&k6cf+!(30lZ`NaAH z-J10F^_WzA#*#8O;g0+KwTWuYp)?s;y_CQ5q8XYt|6sp6w>jBe7WFA! zHGmE}PB1z{agRTF%5pcW?l%6bw>mu~ZOq`^R01wD+pRroN61KgDW{;AfB%g5shQW{ zeeJ2hvcIAjAzD#=*>BBZL_4#N7SoqMQs(8ey+)_G^ra~q>!fQZ>HjgIh{eJL_`*<) z!)v5lej5ut#11qwn;0THD}7+w&0ybRqU|^R+NpoGnJM>NOdbRK{cNLRig8A+hT0+O zvmr}6;b0sX#ZJ9KQGA8nZ z*e+fflzmwTkaaxhu`t1>s1|rb-fm|CAXNjuM5?Me5BD+G*Ry9@P=-FC!hQGWSe+csio^V_(&&vDCYwE}& zbt*Q1XYsGIBnk_Wx!@c}_u$GVcHlXTK z%MP};q&$;y}b|iyA5RKo*J0*8N6$(i(JazPN_2a6Px{fGtKygplbKVFd-?9 z>+n^~O7ft7Lq1mrZvK#o0DXZaS`d2M7su1UvCLzemW!(Rai}vkvsx-sf?f?~h<61RWy=lLPO;($u?tpBOopvV4ps zvGGtsI`qS`dV|taz3SuU?9UpR=vnePMR7WyLmRzY?(oI75pktascNay_jY)B38*el zmbvRlj9_tQwYyixWl$nuF74)=qItfFm*iq{@&5$D~T(YR} zl_c*rD30$e8_wY@x;#8cR#aZ`?@a+umd#tOLtE#A#JIvy*v(e?+KjnuA9DY*X&ix` zwken%fIQ|rld=>Ud*dpGadN^bj+8Pnw5adl0$L-fv%>A7-{pdj*}XJrEqwCo~uR05(i zes!c*;*}g$x|_rLJ{a{&RLa%NQ6UzWV996ZbA;>Q*=6Bk1wmyR_6v_Tn9#P35c+xI zLq6t8+r632xFG)7x;6Jg+?zO!^?HX3%U1s5 za*o37FsU5kgGRh`B}oZ0wS|Q-e7Bj2bk^)|_d!>3CWjO6Sh>Z;R3OwQ ztGDwHS7)Q!G)D(@vo)nE{L{EA{ug1C3~E}{GZ)EuLrFhSZBtcQ1RTjT?l4Z};;Q-}rA4?v$as-oeCcLGN4RregKf=x z`JFO}fK}RE_~;Iu(Z7EkSoB@0B*=W3Q1;Q_T^Z0x6h&l(ovZet_DFf+Nou5ssDr(l zV^}mjKOqdD*WxQ%d~37-pV@g)bf@Lw^fimLaLK$~y*a%3s0+=<%wKM`h@B#jefoO7 zSf2Z=f%C`3%wM{ZmoM^ALUz*LSsVsAzu8$`FLkVJuZfg0bQ9w^TxfE_S7yd7HA6Ch zCaA$9e72XoaP#f`DpB}hp7yM{XJ1-8C-$AmtU(g`Je4FgW6iB6p`}|SZwCW9Hu>d? zB&_(SMuw?#hQASK$D!WvbAL!w{BE-nR%o)2;D?I5BkEoEStQX zJx%(K=>)9}RP`L!Cak4uLBGZllrIcI)x$l-z*ZLP@od!RtUU&K)K3OP1?@;!ISIAb z3;1GIcToOwFBosnBV9vACQ4^gf$HB1FL{e4JK5TFqDR_)7ryC9DoeT{E0*_1!vlez z4dCI1YA$DQukt}0I8nb=(t zE&D{-8xLu`j**rcg(#M+MSA5OTS7{K=VGemwLSH^$#_dtXGFz-KA%OJDNY!fg1|fD8f@<)_G*?cjyW{Y z*nAgRmFj)8%9Q2ORel#2E*q3MKeGcFc(itF&>h|q(A2trcW>^h6m>?}rfOnHZ*x?( zR;}Bz20-Zix!pymo>t5qt-j@tiTQ7Fq&;zp0?zJ=bHZd95`fKjFM}6gOgan~e@Y-e zcTzArs=1y)fDQ_<&YCZ%Dn#q1SP%V7Ba1jFELaQH53o@ZjCyP(!bWnPZl(DiKxXhZ3!kY#|r zkc8HuS>Wogl?CQ>Buy>4CN-$(Z@?yJo&W}*(^BfSQg7k5!e`sUCky1oOp+T!@*Ofr z2qnvp*{%Y^{oO?enq#^V4 zHtv6#FlY>o>98QBEMKy{{e|}8vu;yOv?uIx!em%aoHLF#<2g+4=3tTS5X&AC$jE?v zz8&*6tzk{j{8#C3|E3Z+WKdp%1Yk><_ZRoc36&LecDDmwD|F<%Lv1x1>)_2a>O-$=$ zknwZ1r56StU{CF{>!sC}B5Z9F$|MW;x9lThL__fs%9o)0PdX>mAdk5z&WgOW4%RrN>tqH>?}T7z!b+TO8vY z&I;!b)g6Ufe|M`z{~=G+g^;DZsweSCe_9=QeNWp~q^a`cHl$?k+wjcfOX%435u0?# zSU=II9HmxEcW7BVROx^?*WzREc1!sNLBr_3>~8rY`t)Z-5HeA1+|8GeY006`EgT$^wF2&a zcxGo6V(&-aT-8sw$nTXgk|S~0^{Laao0y-vUW25&`ko7;+~YPHjT$$aZ}pzrO^ z7BYh;%C0%MXzL&BnL^lpdnPnh)oZ$CMC=6RoSO&-eOp~ob z3RyyDfm#qIty#+|Glynu-)W%Ds?&vpc_#)RVr`LHgrVEucbGlwh_{X|0r zIKrj&IGjbhQ6=bhv@ryKdf0xl)lFm^-VKNyKQKKo=jny8I4{VyN{Cr2RVCkxAG+Z! zTgl;4_L_LB5p7fm?!WL8N)&J?fOfsY_3rF+(!oNOZI_jKikn`+tURq#k5k*l`E*>P zQp+<>EjnH4Nl9a8b*(K%$00b6OH7*Krs~)85t_OW9{RjKjy8@%!T>7C?69qog`rxZ zY*dz-bpXrg3>~00(^A*@zJ5U#nUKgc3}rr(RV<*jN0 z&^NER_{RP2Rpp_#Q-f?hVRp8LAUcL-*sFoa$$G24z-alLaGav`*MYeLJV$a_gK}(C zyzHZ0G=sm`M$L)<#YVs#7wjF);Bk<<`baFcv%Hkgf$24T^UK^heI3Z`3_fW}ckF4t zWO=as!Y!jX>_6F0Zq_L}Kh{l3d)l0X#KiH{3BgzOWwpTsi~GB-wwUW5dRUm3U-`P3 zv-r`qimw3Z%waG^=m~KO3yXij`^9QzLz@0#1}ZN0A2v?Z$uI};Q`_^SMAtAaz|8Ct z#hd=V0NmXRr2eZIZK^Y1=JUsd8VOQNb+0H*~EjkuByM@jy<^#m!pt`B?IMI&rGO0{Z7hsmWoD-j^5tcS`ZC@YViB>9fvKz{*^>Yy#ql#O>mDuPhf zS?g>Lj@MRcz-7w3;16aFST+8+H5hmKLU^!c5!@+7>6@FoL(v%xQDrO&~h%VGETGytXlt`_PE0&V1v3?HazE{OnMJ zQ!I4C;VFglija`e^2-XT>l(^_x3dX%9b)0WOW)O()%97mwVZ*qy9FnIUL0Ne%GkJD zot=bHpIX%I?|gxr&~%v!iv*D*(JS%+(gT-I@pFtsY zHALl#jAq=;Yfn~&qvU=hhf?s%I@T~moVT8NizwUZN|aE|H(xm?VL;!!@}A<@kD5|b zN$y;7zmtDZjh7S|h*+#XxV8|V3q092M$@%RK_+H<{C!a-a@AGuc@>&7w=*`>CsZEi zEq;8W`_SM=M}$UWb(Nm#B!Vzm`ztgSy%4aQ*Hoxb0!hrWRfOJSH!7g*dQG;a5MF`Q z*`cfUh}>)~T5GQU;oh?vvh@T9fXx39%#rD#48mK~?Bqwu^}fQ&s$MbjRvt&>*0k#@O=_uOMspKxlPX|Ia9d`v zDa>8{vwPNOXi_ph>U{o|SjE+qmb?K|zDBW(GsVg7)7BW7H3ruF_64mUt;6sPhtOHf zQQm-OEV-ad&%EBy#nt!_!vR%D@NsPp$akG@P8;IWlS3lpjEIsAczz4c>IzvP>;Yyh zcb#?8p6Y#JVW^w>J|-R;8}#&Ju8GaYC8yi1hDoj$bGD)b0s3!pPb9+am-J~T9}ndP zzO6k>(I8dlO8G+sP)guQFLo=G*x|8eX{)=M{;BF-#NOOD>+x6E11UoCR0gk5UD?&< z^dMb>P5YHyKdFKidNdlp;l%m7umGZ03WYJ12`tV|>J`F+{Gz&M{s2-jCjJqAW@o zRD#kr#UiuhNvS~NxtG1E3)%534qL5~<>6H<6Rm~fBt~i0#W&VsC5vyY)I)rmXD$0- zhtF3BVM=!A=L(ds4f<9ui!cwV>hA?P6n6fMaY-W!XS=B^Oa2u~qf`I>{#CQWEeh(-nX2;Cn<1H>%{AqH>-M8k zj~hNKrvoS%{P$fxjQyVPYamT7!h-6DiQKU*y^zE2tgY|4*bw^BKx10W6no}&bAdEkc1@@M*R)q(kb!+$T{et`wtq;8bL6gqjt7f8fxqcD)jD}i+iwWrjs$*o zn(6ZEYqNM8BT&7NW}Vr0<-h>hoGP*Th{v6VR-=OIKVD(V>dgpw;;n1YA&M=B?z)i! zZI6{F)yFa?EWqgX{eu(e;)NQ+U%Lfx1`9lIPPhc&WZU~}U_>$Io~!=O+EZX!n3m3L z*^BgW!i)NY=!Hpr834T~Oy52QWs5eua{-HWv8cLdILvL(!6OMV8afvZzIADS$>yOw zHxj_G(n$mPM}Q&~gcT;qbz3K`BMJ()J1xvWBA@|7e4Y{|Iq{dXa)QLp_cN$yN2AkE^DrJRs9MRd@%gG9s|UaYGCQh{OBr_#1~z|hXk*jDrYS%D1itR>&uX988a_5RRJzm@mujlXpXuw{j)D~j((L-@i zxW1?PzgIV63iT>8YC)Xkr8hqdjFDm8gHlShc{uQKXrL53FlN-L&jIThV{>yeX|hZ5 z6;dRmYbJpd)G|?X7fBphv>a_lughl++u{@4j5SPNSIV*G5#4E~}D0R^rOWnh~npz&1rm3VtEkVj?=cu8ZTbgzsI zLa>S}xYy2jr{z&-oS5cx&Q6wIYRf&q7eQNuuL^OhO*uzj8F6u?h;mQUse0tJk^0Ja zEE}MA#(}L7b~RINj{(G(k= znatzWXxx8w2y``-fnHXi@AEn*DC!@v2hZnq3&X8Sa?^qan7^0NFntGFY*R~g3O-j~ zm5E0C#W-~dL%Smza=QuyRPVuwRc3sC1%^fuEEF!LksTocPxB!IXQy4QBF?s|j3i1} zFFo5d%UM=3Io-c4s&eRhS;A##FD3(tShP*rRQuqUWBjnXBU$~JCjpQ*y|Vf`cT;qI z*s^!u;sI%%I+9}w=%y_?IsC$6jkDEjDRdHubQtOkM?r*davG-LR0bx_+{=-DY9Ngy zl5KaXTzYfxvO}5JSViAuBV~5-u9!;FfJwc2sOSSBj@6)tl%sY+J4kUESTp* zcuH3sSOMddqX_~n4JOv77T&DP@rfhha)H53l8=;?h?oBXt0kozU{K+f4{O}`(m~F@ zvh#hIKMH3dt5B-LYT|PxUu)E0)xBB(pbAK((*01+9E^oq$pJNwZ-x|J0+Q{crI(hT zWTrh#x5(mSq+hN%Fj^ehd1e>%>~L8idn4`e?!oOA0*k=7Lmd^~eyH}1me#bvEyd5Z zf_>$v81)FxK3(IC&(Tde2arTquEcS4_>GE1;^ZF1H(lblqt$_`?1T2w-o7EFCz_uF z&nm~ymvm^};MUBn;RiCZFP(@^^{wp5EIFMz$vmIxrVF?PxZn+4Tji@lxs zSIwEB2bOs0tm26s5w1$Af}EVf+^Y4V7JoX!?TP#{F$cPD5!A8f2EKi~97Zc1IcWzA zk9p~#Dgi$2>oe^!?rbXlhFa16xtmtQ??rX2T4eU(?vdtLB%qCffp3Wiva}pi{OKfX zXDzy4{8N1OzRA#rez9#RW+%({Sg@1Rnw0v)r%%|w|KPtHze1f zi0PrHuM69YU3uTk*S28Xe1N=PHEkWkF$7T^f?ko{xZ@M+WN;}Y^Yt}c?gqd5!A4~2 z6fhPf4wNI=ZYXbyf|t#WGJMo1EEt6UB&wH^tRm%N4<| z<$J}*{7#YUy@`4l-sT(+TZ0YR-~bL#ce>M+^a6|}&C5@(LoMR&G5ZdrVfF);DY5~a zveO6QLqmB)$gE49nN7#G$J3|7)q5(5vZ^N27hkgs&8SLeeG&@XfMxIOlFTVZvyduK z(rE&?&@HWiu!Sk(`>uERz`V}zbYJAVrb2hzx&38GGMbS} z@=SBFFhMP15_H6faAC!s{UUQEr!HS66_V*G-<>AC#u~1}yr;6ZVYndtYyp&bZ8{{S z<&xUeY($9~_iID0X0;D_4?G!&p`Kwct%5^)!vzlYtQj;TK2~K@Re>3H%D%5v(B_4c zF_ibuPU&z4W%Dx%e%XX!!~t?T8}mpU0qcZAv8Vvkjpj7`n~UgB26UZo+J`not;6Oo zwA^EP5N5WLz4bhlI82ZsO$XozH{P(94L%NH-iUXU}XtUtW`M)7(6 zM?H!f6#w*YQLtYQ$LQLl|BtGx4vTX8x`3#plyoT)B1jC~cu|m+5JVb8KqQ8t8xfF} z?k*7+Lb??gnxTiTff*X<#_t`y_x|qpJoESuJ}~cl&e>6C z-&vP)c4liJ;W{RAup%$~2GTtO4p$xIxBamSW9+O}j8Ls0T1VWxfV1E`JhpX|ig!t{ z!7a35BkrnNvmdol=Hk4;C_yoqfQ4AXz9A5h`={K~wu5i=m+ZiB>qSC;G0e$~8qg~8 z7>d&&Z&kfEII{((=5uC|~)wl&D46=%x~^wNNenYbs@# z1#Q`}p7Huw)Ry>{a%GN$LRG#_;vO=dy{s)7yZywC=mluH-(WEmzX}47LyPSCizik# zUZnhIHVGg^qdN2rB}NLi3bq=uvPuxmi-(nAMXrZdH@VxK74a%qCj(R7^Vx z64f z!=X4Ge@-b9#E_%XTb`QG=6`F&I(2FK?*bp7Vy9x2qukDKYkZZGr1ay7PmjTDfwiqg zQGZ&0r%*hb;YhHfbuxcHstkyNZ$6F{1Of8aPg_T#t-Xj6PtEnL-L+PNzkiPOde2Gx$RsXs66WE!FH9kE6V@r+8}L2D|vJvzSK+A`kw&-49>sz@i;%%pj#xAkgW73rlnZftl%=RWL%#X6kosG>&yT4yaaVU6-VyY+&db8A^ z_i<@C&EYTU%7RKVuRG;ZL-~2oQ65za;?wuKI~8x!j#6gAj_?sTjgB}#q5UgSAZcGS zufekY$N%f6{BHWFESa{I%xUfm#($6U2`+1#Dz39+xr;%w*EV*F9P z-xz|hN~&wNALP@+H~Ahraw@ZZj*JF3#WANlZ5h1to!f!nTS<0DaPdROeC>!2!c4hW z);32j^0`XRZvKV_UN|iXm}OeA|K8XwKQU0`?;I?U%)~LZVMKI4exty{+AXzG=|D}x z5^7t)sCq4E{FScHjF5aw&h{!RglTc+tjH_(kKap=eCq@43GYJ_Wq`k_fKPWWKR}{{t z0K{BU$rn?aj;qTInI8pI_{+)rhtq0_PO}&CKS2h-1Y7QQUv(zl;kV1r^6GQw2~oY} z?=Q%V!<{y%ov;^q`TE5`VSb-Iekzry;?XBQ`RAbhQN)C%)H(^LJ8C1#Kq??tWfWps z&6$`G);FFH;zp~=BG^FZ^>&oOD@}d(ULcjH1=cAt&+}7Z6p=gyU8mAMS0xoh*wKew zb~HE!gp&F&2k~Fv(O2N2_oF@e)z!Kdwd?O=3&2`%vp}{BSZtK!|2kv9mqU`7w8S&@ z2MAHOGo!)C>JGM40CCmb$u(CEkgH@0waKB;X<9a;-QR4m+&11YyIQd(1ek5BjE|a& zh~w-LD)%D&8`K)!K&;i0L$MGt!7zVvUj~VSceR^_!v1b^wxbjS%{9@0q8_F`xXI4f zt)I8++l}r#^=QqLV#}0u zn^-$lQ?o)9P|J?%k8+Fc3jgJzCAE>`^Zkm(4kJ2E`nC2V4W+2Ol{?d90L=JQ-FmX| zX*0xtVO>jNhc<0xY!ah zIMYtwyV>c^$#!qT?BI9@%~oq1zwGoY(x$6O5t3r@hpZ7=b-)n{zWD1T2(=DO7R6Dv zk14+be*_bB310S=9e$g>IMOw){Y?nILr8?(3LM#Sb1Fw3FRRFV1 z`l(8v7kjW3`poD((Gty#s^rL4*0fO#qaJD78CG@cGc)$0`rscLHTjCX8*w$?{~?|O zAoRswerLBJlB|$Bw3QEyz$uAwan_7L+W>15mk~^ybrdjiH(^lNS$ww~mqsK3RhO}{ ze9jda#V>svZSz9y8qm0v`cW?$f1i_aRS{|4Fmn7XG#iE;LDIN-xWjL^0N zM2eU?0r}K~e7EJT1UXr>Q>gQk(I3nns@D81Pu0cWlfPdcDZ|x_Y$1ObAUiEsol6YT z4!`9eP6#46loCJEn~pjcX+})1Ic$UEJNLehWuk0~3k6l36E{xA!;op0(`(vw$~EgJ zwoym_<{nV?&jQQH`o8-}N~z$oL5oa4y4SDX*24;#2BWAE@aM<3+A_B#?~(!JHpZqc z`fK`&-k)<(Uxm^uze`Vv#J5hWxDV?*UYqT#Q@sRVX7d<4f`CVQ^5bFE@xuvJed<@% zqPlm)>yliAVZN#+zwjKG9XcvZo)dULp94Th<2VGqF5F25+;|YyHg8wHt{^}j;-AHA zCmVXB&qkgQ=CfSIpQ}nwboEJ&M2^~^f3$0{WbpDxrXrl&_qDxNhH!HMz!&cDFuOLq z(0ge5F^=whAm6(YwRAoq=V#-FK!9g)KQrf?((c~Xebkeii5jOq>z2U0Ks-tBbEg5? zTvOT`pp5Wpd;#YL!(Zok4pIUlw7uYM2;cSdcaot?YwFJN!rG~3V(*m0=|?gnoxOjE z&E20tRsNlM0`pn@0!j8i(?)&pd>BJh+a-Moq@fTuoRnSu!th{J8X(#V2ui-kp*zXx zkBANrW}@hlpjT#oQ0KD+WzRY&N)pXbkXh$9SY{SihelkG7D~V70ZyfSlZ?yMJE@C9 zhPXW9ePz`0^S;iLkYd80%s_W(@yzB%8xf7dpN)3NF(kOutNJ4-oW7FVqgm0NU)|XE zHTv@z#4@^vEme4(`tmM&y0FRnpfH&Lcfeq4HPmU=57(ru3^08mdBC`gDaHidZ}X<= znl@Exe#=RN)BBLLXu$FDHJje&9(DYE*B+Y*9jY7^n8zI=9B|Q-tgaML!o+K7?B;&D z9FA}f(5=}Cnd8a4>yi2U>97*f0JQlY&ejmuX_Yc6$hhA1R%bmW=Tp_hMek=#yUN zW=oZDIk?w$$#s>(Ebm)?VbAo~g$sC5O89GBo(N#_d6PT58RP3xp%=D20@9M&?+Q4i ziS)@J{0I@R3go>RlM$&5x>feOSSFi*bu#$kfD47Sx0sbD)`be#b;?2A-;e|4+Y zbxd`MD_O^ILai4_#n~+0x914%VpGHzHXgnyTvDADcuM=#wYyRqMe4ZbLab+`uV?*Q zmqrJuQ9_?uhI);0CfekVu(}q>rOS;M9_s@Wbt-cAgtoL^rAm%g^;VmQnaoG zFBnhv4m+mdm&9V7Tr4e_dJoA_1pou_z6&>5%$0kZ?6h*Kg7^8gVObh7v?@(^jCi_v zrvME*&onshvA^F=?){6ei6gUcvDW2=mCv@1IKz(bNa=3SWiA@$P#h_poBch31HuXa z&M=MmJgh|bmH~)d z-_v9z0e%^oF`H}go)x>ebr~FhtHYtZQu~FE+3YW8@}UoZRhA|y5BF4m6O6o0+!$2k zMeGz!BV35ZY|n73k{`6UJXAdR2FXAtQIFS`oezAgwu=YgvNmp#1cQW2iS*d5mLJp+ zI8$qxBg66;LhmSSO~Z5NXmD9X_y@C9q`H1~<=T63+bHQkXE3Fxi+6ZkeLS#0_Njyd z$u*UAn42keS9Z-Fw@8anr)p2PGbW>I8^RS-1EIiA%@b%QXvCeacvla#FnzeA*4 z%jKg7#~@M8!>;y0ENyOGuO8BN{aE9egEISz%JWfKF|zoIzdhudoBlW2vh&&F^gFZj zWA&TXFNbn%tO0Wa=opd6i~Yn_gSo|9B9o%ObXx#;c#dejhMl=bcY&8XLUKV#v%Y{$ zMyZ|e#SC-jswB0s7udN^f4&mKU3F30#}F(xMwEF=cbp0((qmDd+<~{L>X=dLRGgq9 zOE_CaQa=E_k2KR$^_d<6>PW^Zt=eTpgCmz~wwqPi?E?OCw|od@m4CIQknU~yb z2WpS(l7YtuYyyHD(f}(d=L0vM?$aGigD{g)t2WhgY<;wFGoa?DkQVI?GtzFHZ+-k~ zc4nfJFL@waZa*0!OwUr671c*Cxp#7Q4!v2jF8skJT3uJnB+Z|-X4#iiXF%F5{4l`? zU5x^ymR*00PwdVMZx=DltHQ#DrMUg{>JFq$1qyf(Sa3}g*03xGkSM-s9@a@5#RPdU zcL)5t=)+c&wRx?%>T1`8v3cOKpdBeJ>c3PpZ$M?}z^@0*!2YbvW4!n)(4czR`jfDg zZ|$rOAo}Wh18l~ry8g`bqRg*G$CZgMG4wQ%M@WG1q?Hp-{rIEDoFGd&Bh>28=qInDzXamE}zCKj{71d#hhmZq zdyY#E(eo;o{~jjtrO8ClH`u1e6}G&OseK%Esu0pL!5%|bXeJ7)9$&(UULTQ~v=n(0 zMPL&MNCQN53T!D~ci6@~wIK1(*!PdR_Atg};lAO4V1_r`tlWwXhP3{oYF6*9=r}(8 zKfSq!4;=Usdnk+}7gvWd9LLZyY=BH!Tk84q`lPkBVZ7AtWxDV6*|@~Q+2BT)@uitF zz|EU5?ra3CQ@`?hEfJ2dCK?+t|E8mz-h4t8Ez15`r6U(f=&=j1y1x?g)MZ_hGdo|; zEnLk-6Yx&^EZ+9X@(FpCz6&tmi}e@IClt0DR$`PlE=3xKBfd5Qi+l7|0OnJ5bc6X+ z_K0y|kFi?%YPrqy^hEvFq58NKKqlJYRCiZu#$s171YG)l8DL0m#Vr6YzjneKNx$%F z4medsSC?l1fW_u^~oS!!#M&uOsl>2o;K?*_F09O)1te}*CW zn*#ll`9o`I67Wv+0SRRxRaJbUU|!1_M=pI{%Let4Xl#aPW{jfVC~o@zM<6tiH&J0_;c2#Td87(ndDYu4k8Y6ssIt^-3pV!`@~5e<1_% zmWSy*ROjZze-8uwU8nQ`W^bI^o>IXL7`D%aentojUJ@fC^ZWcybIne*Wz z>Qlb{i~7uq`tvaF-}JrRvy9y$XXAqu#Ms$cz}`YT^Yzb-WVqK`X8<>D>_z!{9VOH@ z<$GLh8XME^Nu7Qtfg0_F>S{;1_nUuZ{@GEy}x&p z*X@Zjh9udrX}J5k86aR+V=t0f{Fzx2#R2Ckq{=)eqJMsXf}=_vG_EZ)b?m1dt)L&Z zONU=fI)vWNA7To(3IgQ~%Zpjq2)ri}lTWS~_&xKEdgiysO@|Be>A6SRbMOvf{$A-h zk`q88TH(!mg#L7mBVHGGgT4=cPSp) zD?tA0oYa=KfoNJfZuB%RQUXk*G@7#i-2O{VJp;un0{=Qova4AI0$xD$4Sk9$coO&I zj(J0dBL5w!A0y1I7SQjwfCze%toc1npNZa6p-h^ZiK94JGyMB$R>gf{n@idMH;}KQ zzGz}cA>dP)NUcMI^xn|Q{M!1#6?na^9!rGAuhfMBO@%1kGM}U3P2QXPI3Z;#Qb$GV zrLCYR^W-Ohibb8rNNk4o+}WTjv^A#-esH~_e*NZ%t97622!6Bma+SAi{;)uVk@9F3 zEnU0Z=hIYjJ`^miZ5rpc*GY{qe$a$*By2t^wJ#;cGCjj^k0(|w5DL=!(Hra{^@@fl z43%yJ+u6rAq>Wct^)Z?s{E;FR+Ibr@9x=kr3DtMzj(KN>zW zEH@x5)SY8^+_^4)vx#4*$)-wAV})w+ruHg-L#5%B({{q*!Y@DX$JQv7=dp=;FT!DdhiouAS~urX7h<(Oa7zX6Al}1p;>ue0-CejEK))`vKM!4$hk?) z+dpt?B;a}qhR81kKMwtZtt?o9o125($ikl^U+#C)@c09py!E3_&w6S-J+5CD?=KDJ z82gL*+ZwJq|s*l}WdExzAQ{T{1o{~DVX_<2Wz`S961IU2T&2x2Cx|WX6Vb3td zz6pt&Enme;r>d++6Uy3(h+05>i6$r0mz|T308OjZuIx=pVsA5Ohpy3?+15_uE>kWV z^_Hm#R6EV}=+?DX#(rF$UoiPBlKI!^Va}(ylNT*=*1RczJ90I~$9NPwY$d z;b2gLmyt#~pD-MdHPF2&aZ%AeKV5AzN+{5~1bLaZC9zXy9^;j8dWwwVy#u&V--U$< z(Hfa?mj00W=}(eprKaY6M|nNk>pW4?W}-wfQ${Ck|J&vwMn@p8YS&DuS zPXURJh>tL@F>WvQ$A0Tm+$4i%tJ0ijLTAekr*SFk{zh)XTGN+G!^`<7b>}xKqe1j^ zamAHrg)SjpI_yEt>$Z=b4vYt0WZRO)pE{1q<;1sF^^_!V_lZ!OIWm@PT{y+ys-lGT z>lkH}?Z;J6P<(yryTm?ze`&Hs1IvN0__;QEF*f22eMYyqn5GVChMs1A?agYom85o% zbS$oT6qFc9W=yG;HsDCMm;|wfsF~zSoO|1GBiy4;bj*2?WiYDjT4~7##+&vqy0Q;wG+J^fTuY745!(0ElR`dfdeX*7IAkb%AFsQ6E z1IP<*rZ%YBsL5O%R?*U@_Vtu6(OY1=)0#invCAB0ldv^cNG^v93_kY&WL%tH>@^^o zd2GoO3zLI0>y38rVqx!LXz6Zia?B}lOBh)ees2&poA}+sp-Rcq#emwMW;szy*jdn} zl?EU&NOYMnFMM)`2er!U>;Apak25pz1;~BM%jfhXh1$c6>%&6ov~sToUHR?r@Q?KE zmfmLud={Ae`|;#3AKj|y69xCKD+?%(WAoA+vOx*G(~w#@eT&R}K{(Tk}?MmxiWu+|wH(UCNtO^_~+p6+X&omG-!Tk_P^_5u<-DA_~$0+;cd3C47 zOHvy;X~ax-F9L~%d2fIb0`QYh*zZ3G7CR)6QMq^VW};L-@$Ss=g6}Wf&jF5y!wS@Z zqKx6__obWTtU-*|Z>|C5rFu7j{I1L3Jf_i!$w!@VeugYCU#%&0&MaIXJ2|!hDS}du z#vzx00t5IdcqLk)0DhG73Gn)sh+~@}hjlup&de5wuEuGFVt|HXvc2*fhYn=h&WAMF zU?vYMO{SG^ZD~$+MBBFmECQDML=4NkB>dUV zS=BSRkEGVH4kxZ-W;wq6%Vz%b_rl>;TCp=taefK#Mx~A2bNLkisTS{~>DvC&-X|akK_lXmltB<|=;Q z2Jn_v1;WUaQr+zbMa<5Ko@ILc-StSRpGPh^oV*R;CoMU@0b%jOD1!}-Sm1loF z5`TfoIKqIvSX*~IZ9^#RfUHj$n8D>q=nAdG5LXD-vx zm%}5SmVKpbjCXV=Ch5#$__WMo(U)c18R@(0`|BdC%=c>Z=DErHHhB-HwioZcR=JFm zf1FivdD!%KCBd)9jjvu-^zk&_eRw;^Qoi16?ig5eE}#9_%)`I1Ka0Jwzl)_2ZqCw& z_Htk92*G|XDCqveghRF^KODtD7w-k4p2VcLnsj&QQsTElsa3D{1KfqUsLmQx;VdzKg{fb*&|Hlzq=_M3IA%Uc@fku;#a+pNGOd<3r8^ zw7J8s_|mNb71^)_&7Cj;U!(pM+A=~u{|Ug|-~?QPO2%pz&8cARm6VL)>MNGifI0CS ztdhw%wL_?M3e*Yw`x)JQ+RfaO;z9Q`(f-qR);QnDhn&^K((J#{!PE!MBl7EIU@z-? z+UWu!@VUwT3kGeV+Rf(?Mj=v@2g@>L5;> znT-}g8yoG2tkq)zY8V=1!TdM$scIy(XSFEm_QDM4nBAXn)xwlu!xlmfC-2febJI&C z#7AHJJB2u8kMnSR?$`B+I)NXs#vEaV(nm##i6M*^(1wk#c@K`J;6637z;X>@G=Hjw z>2@iA{jnSV66;N(Os_aP4KgCR56Fdqr=R@90L<8oF=eCum}Lw?nEno!qz)=KlC*4D z@G6J;du~-Id~ogO?_v#n+H0mOb67SR@)>@a;kDnh{;l0>hYF(Ry!Q6LD z-d0GFRw>Leo9L*R_~A{*?9U3kOJCGi9Y5?zSvO;8(PE~)*@xJ$ABQTxi{{`tC#~%d zlQv|Ek{-){2K_QysCbvqNci{v5yajR^X8-b<)}Eb&v#b5%#P1tt6!jW|3xG2;KB7^ zT7b-7>+lyON6zMMN%o5MKhq(KDl`7xuHO|GYWt@JExHfn(AXTZNzn7ZL=zF<$(-<@d?qVFh_3oX3&A}S~>QX42jXtL$mAUcKbkf@X zV!Kvd$rCUH{-x&ysP8CA;3y@A!8a>==pcN;D&~Vn7@0UX4AlH7^vbCx>6_W4#^+_)1VK;RdE#jI`$09`={Ts< z8v*lMSg(TR462G^eGF3c`cj2{eL2<;UI<66jD%2bB3aRS-*6TmhGHg9k`l1YNy4PU zCF-Iphj%;otno=?GO3#KW`pAzc`4GDv|LCqx5!gJ&*i3_u<*wEKtra2y?Q%0qgy23 zBu&bUn#JSOcS%Ldu1D3TKdfSBT<`gnG2&qYz=-e*#wLDlA<% z!@NVc`SfuI?|0aBy@piARKhm;L2tNWdPGngm|H<=`L9R&mLtfDBlL1^9nF-KCESL?4&@@NU=?=1Vc2xxI{berCLtO;l z3HzTmhBldrv>WDb=E!g}126jDrkZ*GP4%362cIMDg(MK%)s_4Kc_Yu-H0g?XRM)rwvnj)qm+APV4TIUj>H7XdMzkvxYlsuK^D542GgG#iUym0pIF&z?ym zi#^7v@>OYte`R%lidhCWgKnBTDw&`qnN`MOn91e1J*&1;?W5OnlIxwV&~D2B{ybB} zcWD|FlnZA#_*Ol!KE;ropMLQCnmL*B>k^B-JHH2m(y&Ch^&TH~M`-hU1a+-94RD}OD8l^=$MzKymo{c2bE6T0o!APX9q`*$|$$aRe`2JvKAnBttzTQtH zoR(wEC2Azk;p^5lbDgqC9Gkx_wf002o3AEhj=6c;rdC&bV)i0;VU>=HB zDmapxrpPN2S0>?JX5`!a4riWc@sGr8807|;yPnP_NpdqYb123Vq_sy4iVd=Ht}gN% zz_Z<`$(MC~F=X zg3|DckAwfLq-Wk-o3U%OJ#UxF+_bpk0fE_k2BWy0@{%5P|i^#{l6T-5_YSz+le z)7E`&vB$n&Y8;Ko{NMrVz1oifHkKClGF%Lu_CZ(@=ri)v0N>zoCMC)nXU~i33;=o$ z)ufKbv|FYhpOptomDHUshpm=A<^?44aBeD}`(dOs6;M(DveS~ZJr5X=eS6&)MTw=b z?~;d!1}abeOp{YHSqa65$%8XbZw37fnMQTAKgfDk{$;YTlb3bx_Z5Y?$%L^{+?sfA z_+=sKHJhbc(kF6Mw87BlpSUxDd>imf9~xSU61 z3^yS?rGiIE1ORtbah;<6a?I?{ho)*E=<}oDZ>4dU`RP^*ocX0sxFUZ7trK`!*iWmF zZy07!3KoJKu3WzwSb*ZxN56W{d~zE`G9gSW+SbRDNU#<&{|k)W%;U@VBq+n~2lg{hhU^I0Ro zMpdZ(efPDLXMexJ^wV#-VH3)w>QX2%$5|Tq(bfhZQ|J7tiEyP_y9$rlIZmjDRmX+BN4 z&5VLHxwR|ZCuNhJ4$)j}GL~J>cTIdRs#BkTEm;!T3Z$zSD4@tkha-;A@JSx;uv9i& z$o+BHPn*9#FinLP0T9)M_zU@HHkDq^?t+Lx%Z=pmwbv0bo5JUile$yAnKAU`YxB=$ z#@cIIG)-^$m!*_xbN4qcIq4tt{Scgw>=XH_)|D(vi9N^_k*K*f`YM#whn5J@eD{30 zF6T%UMos^}_k*FtJ;2VOEspzTwR?mxUMPNh5Wdt3Kiv3Shw055lADl@nK5hXgz*9M z`ZX4MY3uuHYz zLsz0YbqUl|{^3a?#Dw$qJ^o?iRcI>oYrl3huuJaX3I_3^GdtJBElN5bCpX6PBzmA$ z^(NoC<2f{m9e2-eFwlXB)6uHO_veG_d>}H;WI9xZo-e{}Ds3$+iXM*@RR^vmfbN)v znK((V^_3vstz<*+LqdG&Iq)+DAFtbr=0Sfz;9xx%6gi9>LG2x^ zAwF+}6O4psgSi7&S?FPmoQL;m16C3}qc3(_{$97gTQx&?C06{VR>ZEJK$=SxfJMj# zjwYzRj8S^fx{||N$qgejiZpV2@?I?Ro97=@BaAFecC*LP#u%P7b^oB>!yEf7c1#C< zZR9b&lD^sICEM%w+$eI2C;+RgascvBce+icSN>*{#`;|mqF%ez7kCav>#Nt?wiC{5 zw3*04FnlE8S-*b4EOos0G{Y!WG`i_LZ}UN!aj1}H-LRXwTh+=a*iwyE3oQ+LL<~-< z6#Uv=Nsl9bY&%{6aY)O%E*PmAPqgk^&G;Dqp$f{$d7-MmL%P47!Au#viaJNU;``WA z7lA<@|M@(t_z^%)ai!Of{&iIqQn~AOsc)B%;UZ|A!VR|=mjJ~&d`+!UYbY2Q>SOQw zs$g4J)Iktpk^S)|c{r6C9tV-H3N;eH>p8pnX%PX5;+CMl4!%RwoA1iJ-+O1j%`50{ z`w{-*M1Paj0i)>HW|qW@BJe+018vK4l7*S>v{^o^|W|O#WS32nl{9nxi z$L=mR3UT&LPs^g10o9S#_0q+_>!oyHF_SXI7nUrDMFLs?N?B?-7VS>AkA{DHxm`+t zI8vH2sGXdZvSEDbEYW0$la`y7X|YN8k+-)w^Pu`M#V*@0W0`L~TUWkKaVd|h2adSQ zXyS{ESO~>x%j~|w{f#kk<4Xz+2E^(F4Yy2nC*PU*s|?{NE_<**_F>rGGjsL0!2B8K-3s*L6YY&W;z?q9;k>gDm)lQA-U4V@06RlwMzDXXj~gMccPeOVTeDz60{LFnHP6X7~(FRNMcySt7v-krjboRkGekh2v>2Cqh8TmfZo4mM~LCUd!OAe z^aC^yRQsd=o#oG?Zs=`rdR!5f9f!lQo;jM*#zMN7Vs-I44o7yzkt{O!f-l7{?(&6b^y<83fnSd23%!uE zgV($)(&vuMV7TIz!!eFtHvZTH!#>#2~sk06}>zWgx`T1xelv|7lP`wmd5^%<2;M~kB= z*?p6_mE{_>b+l}^8W{Ko5!7|HkD!~v^(~$EvNpdA%*V|!JIUPVX@K~?G)Ub}uX+!A zHQ+BasC37Bt49xL5Oub!!AD(}5NG4h3}Yv;JC0! zHq`X2(g?bgw=5lOEmvb$(93+swss6SdsFxhywR16+-}#h_g`cZqB`Fof-3F?KIIOo zCzrVK&b94X^DpOtsiS0BecZaB#v}@UDnU~STlA~9`r#-&XxzDDJ)RN=gG5Ggs-Spg zUtZiTHgmc_#TsYTj-!ujK{tGWl%6Qw4_Z_(Q)VNFl=7*MuFpI-_u3gmntRgT=#9I? zv#~MwbmY29Ra*yPMt;6z3S*Zb|Jm_eEupb-Y>#gIEG@Xx;dMe+Vqip{0FN{;MJupQ ze{N#n-?n86lMJhyZJjcMMZriW0yD4p)DNGELo)>>d3lA#Fs82otZjcKP)$Zx66ns} zlR(R-%68#awQ){?k0M@Dqi^07@0EFEm*Gwyu=D{49JhdJ;;^`FOpn6Fe`Lq+;mvnV z8EdRbnOvrI&S@Y_; zn*Xxvp^<<6J6L?e736sitIMtviV6iatam;9^`%bIKE29$q{9)6x#{}l{oLUjaTP{fM&WHc_`)P~Wwe46SM*fcI$tqO`(66xg~j#IoNE zN@J^+w4AT4$!1+V!eV>=Otwq@l(GGbaf;_<;L|Bo8DzXs&ya#HwHiEm#J!>BV%Bkc_Xd}3B((m0n#Do z5f$t0gq#bp>cxccotk!SpRZmA>PD{NQQI#){^OjNMD2g5GWv8?UE+8xB=js;OMM1p z9ZD+{{SH)8(*LUEPs{N=ZdKOc4@w=v%@~4FoXN^PurH>(DGP9cl^%KFX+jpoK?UE_ zFV=VKeu}%hVFRH{j-#zlb_?ir$7*6X;%q>^ABc7p2`n&@K5MhBnPP8lVi32l1uQ^J zD)2kNvGur0Y#;Aya7XJejSiVGQ9rhL9`Yc_rX?{r{*3bh2}eJQ9jAo%IauC?=yll8 zbh~O12PEX`O}%^}>JsTSn&z73tNnWUy^6^fT)}yzdfvz`18Q%9hPT$#wv3iEGBmM* z(yK`F)pfXV+_TpaV!5o=3Y}eblD3D)8h(EZ_ADEVZldK^oVBc1k05Yzsy=P87=E6H zzGV*Luyv^uf9OBw4MQYYk#C_syhKYva^}nRKm7TLtN^YX-`gijVH+CqB4Mxds{g>s zS6(=DyKEN|=2QgLuS82FP=&t8UfTHiuy}6?xu9Vtl-|Q>Q~suhqY;&n;lgtfOiawlMc0%9VO~Eg9)F`Y~VS8u+V#cR5r7q;R4gNGwQkLQ7lEhM%Zk z$z2jrSLp!@B#Yv-QD=(uWjZ0=`1^VJu<`g#UM;=7OC2BnC&e+j#-+)kO|3Y0y?l6b zslP?_cifWo#+gN**OB&q$e=n}8Dwh)!bc{%n)_$rcJ{WgXIm*kHX~vmKuiSw=LQal zW@J;-_4#IOzTGr?psWl3Z`AJ3j-Ms^gYE&SL4)sp8ozaP#5u>5jE9l`RyNd|Bk9GS^R$T3MX0O z!w%-OGZtTGJ9Il=28yI}@RG=|PxzWYzbbDyvn(=?X|k{$=lFfwPsJva%UpxA zA}zZQPZDyfdYEvrP9kf1as{ejc+z(SK(s1A*@WxSQO`i#2Z{OIY5M_3MAk*{3aen9 zh@z?dCdGMscHG5_tYNQD;(6RKFgwW!s8g$URwE&>i*HsqmW*_v$8lV})aI3Dm!VIO z;txfeQDSVUdI`!ShM_i_e$boqxD!ku0v6sT9THklTaQir%KsvUymwZd>%9JzV%l?j9xp{~7e?3d;;Xg5F~|{a4@riU4P>2T$Lt6ocpMv7!yja+O5@mU^A>~jF>*-MB_ZMM=j>?*avoWecMcgYa z^r%2gZkOQm&Q*^nJ83L)nYd*b(D&u+5h%m&Q$sAsS5?1y4>PBNqOB1y?O_ zZm)(n<||+p`z^lJWkh~LV5eInrr(|7X5>Jt&%DI$_TJD;KsqC{1Z$E=IS~-|;(LLk zGF?veX|j$FT5jNxBQg}Q{r2DdCyso`>x*DzfUFeXgPAI)XU@(;rO_{c;v0DfB?|8) zo7}bieBf45nriT~X6sgh5_4w@fDl==2VYSD<1@=cXbworI> z#JFLm>(3D{%d|As(%y~ha2ojt+x;bmUUbB_k3}WYpU2%E-;_DK07az-k)c1p9s}_4 zj_qYi!LPw@qPXXRI-edPsMsUx9Uqt*08=3rJ9_Qi_W$0=fJ1G=FZf6(dv8+ybvCLQ zsJ}|sire^bsl4)T6JOow1#Y?5tZg@T^J+R#byokWIQjGA9!fqEV?R?F_Aq^*XzHkt z94xX_rsZ|_TjQWXmjpkVMd1E zz04Uq71k?Kn`6V?xv3$>?ux8U&X66Kayuut@9n8b9dnfA=z(b2x`fEmBR;~EdO(4H zb*er(fPB6TAJv2Q? z-==aQjX8Y$vPe$TZsQ4mB)?yJwlmLajMG^f?@4R{*A0#c;~34u2ZO(l;!{~<@}+kI zUF$iEsgRBSdH;+prs4GaW0ipgPvDZ|UM|Pne*WxuKVSSv`Vg-V3VP-mZl6QU^WR4W zjJ}lo>UrfI4I~gHdf(Xac_Z^lyLtSZUeH<`{$NuYLGwl&xSg4c?~_62@DacSg3;@P zdJ`AE!$BU>N`64THfyr$i2Y{k8hf}6yr31gsvw%{q+;g@4LOi~jFJy|yH8(^t{|H- z#x$b_j}-USMq64|O=sU`u(Qe3)3*1ij3Pe_MuTY4kUD;)#4a)(IeMTz9H-y)Y8|7~ zreZ>dXLT8C@KL&mgs!$;#C)T-vdXz#TS4#K8rj<$YFhVCDFOfxC2O$Ot$VY+z*Eyx zuP1p4@Z2k&1-P_!kpG#i{&_{gSiD9Es9`EMFMA#)I$7{02^HEG3W+2rNgT#J1!C6S z=W}PuZ}S=1IIFphEUKrd6K!#9$z;hsLh4k$tM^jdj&D0!a@^~b<~-;r6C%U(wCS+t z!WuUcC#Z#7Lt=!V%Ip;U(0oF1$rq_y&->_Nz%zqIn!?GVuc0jNBIixos2C~Rs)BE_ zXM$jC)q?e(KJ^A!(uvepBd%0sN;C>QC`-JraPp;G17L+bR7N<}kNrD9BUuKe2>^l# zA3oocaABS}RU{Mp&D!bz4dwwyIUyDcj)RHFR5)I0cLHU(cc9@JiocJi>BAdlQ;eSz zyH%40K%tNU>(6vu8vCPek#G7rm?%ckKAoHfAh@#IOprJ&;II$b~W=a_)sjipOu z$A>`Z>miEzcXd|ZO2)=yq5@-j!?dqg)~2GSr{uqh7BK93MHZNsI&36(Y8%WsJt~fn zuK!1p`4es(eCjY(R=R6AO{FyAdW~KBym(eMkIN?P2dnuNL73x&TW?BrxW<`6XAv`U zq@Ql#Df76FGDSC_Jz4PsUPmDWw`#@gL8w2^Rt^|~QQQCT=Ul;$x~*8ZOf6x;+EO}h z->qn_+En4Mt@ESJ*yRY9TThSOrT&6{vfUdYTr$vUP8%HUF^+F3b>V$o|M1{DY0uI9 z+8;~rM*%8P;g(sTCbPLQbPIvzFKE@C17X^n;d}iG|3}tYheg?KZ5$ATP*jkRR63+Y zKw6QGk&$&25Cv9OIkn>kfBQ&g%O4p7&^Z_`09Dj`L4@9yv&*B*?X_G z@3rpViY^P^#|YM@jo1daiAVU{+>NF@$g25X%}*tC?|Z)o=vhzN+$`bu)2ek`wYrS^q z@c#Mo*HS^v;#f3pZqI|Y0p5Mc?!;(hUvJQ+0D!66J@?($^U{L{qwHVuizTpEY>Y& z(IpS6kS10PsAu{$@YruG_l4SppU|pq8~m*<34Qk1x%q^)^K$#n>@viGBO8d1ue^2K zoMw+&Bj8&4$W0NO6MSJgT?Dw*&fAe0+hG53&~j?--H=JY`|@4Hakx*od;fFgf~CXg zmKyEdj%N)_3KPi5S6V>2R$JVxo=oF-_JcKkLdx;(jdwM4fnN))UGz>Xpxr~)EPof@ zf5$4`fK<{Su+cP}BFAVClL2Ycj^*n7frw6ha$_N`Z@NN|DVI_kpdkNC{r~-Qa6Q2= zAl*h+D<`U6D)fKDzYGj<=EA#_%p$y5b9xeoDliOnW{}&}NyudtT%4DQEWE$qk*(!Wxsf zw8SLfcdi~LcEzTWf4@NAjespQzc{{&7W?+kSl^rKPt7KP#17M1TzZ9u#(lIJDADviotfvKOn4gL+0!4l++ewWk$;fciDH;*;9qowH_-Bd{@??M{~<1+uR1Vx2GpGvHagNK!b0JP z)!L?1jN&X;W-j$KMkFeAl{1TIztOTA7gF-(ayQV0TIJ4=PKc_U^e2UDx}(ldU0I$VUcpTq9GR;>%1eir-qd3 zJU(zkn{wbyESabw)Fp?mJoo{bYN7;X&xfP?6%lUz-=BS81gCCnj&<3p1G@%r~m$i&#_vM^62=mFAoP_UTL!0 zBb%Dl{61>nI=zaG9B~>#zg-l$kYl)acwLZ3wdh~4PLO@U#zR@&``BhpK=GuRuhItv zjR(tEyPrHxiSp~GpK=H}5m~E+&Bt-?m}*kD0l)v>bRK)Z0|o<0S#TG>0OvPB)ScT* z&Nd_fZqn~LhOSw}9>7cLv_bL+;>pI`z%iEn;u$15-1frA;}aS|!9g;D3i4YD*PPHX zJL|JFVd`mV$Cv54!;UhYY~`s%e`C|O0sm=W(_=Jq zR}F6>b>*$xJLP$XFc=^08uw*qJ!N=*VYP0v=7sATPrfnDNQ-N0ppXB4VBSl3UD{9f z`oGv-Ep%f%%kC8M!reU|CzV~YB*J(370sk**HLR4o5REDKEThSLcjdQ_V3>Y+Pg9i zRCJ}*b$P(ff2Ajv7OyVZIa9O5v?a&__9lowU4nuJ?bYdBkhAEex$?5~$O)p6gK?4e z#~7g-$PO!{CW1AByrbnsP9N$S;Tw_4-OA=WR;9~ zi(y^H$|PJj17x%=gkk!Gd*S*qDE2A~ZPS{~If3mbbGPb?`tLfA#O$c(q2G&rxwEvd z{Qc~##4f{U`1R+0^hG92oHa{{--?!G+sV>R5s)S#jnU+&vY0ASTg;8H&bLGURc$~V zNygz(ds1wa!sYzPZ6Wp{0}{uO%1F#+vYJnKUX6_{`~Ua zh=!`~t95Vu;Tl{zhuXel?R%jJ}h>X!Ty1(^93W z!5E)}JKFc-OAJG;0tNb#lTw>Upn@Qk_LC>Nqwi}mTYPsM{~Xo45WGpmlld{6kw3k- z(oE?WMLTZom?jD?f2O6Hf)88hkU}AYlGqpU`y5C}!P#|m`_W=oHRYzjE+u*{gK{*^ zG_{R26x$bWJKissM)=e{O-7JSNTIY?S+VB%8!E1c->c&@9@2+$^hbHAsd$@?J*B5& z9WSd$Dl4=ZoxkQXW_VSqlt+7qx*66@iPXuUJ&wL9SNCIHJhsi^YhDIHnf6 z<2+Z|f-L2Pi@*{K8VgkCPql)CKk`8k?Qggt(lva_K%g$>?I;7_`??KtvlD9RrzD)5 z8c?i-|F;e`7@G79iu!_$2|0}Ntxtv9 zECVa8QB-+Wor&Dg17_z5Is5y_5uZRWX7mlhE$NzEUZeR?-|*}|FT@1z`1TK&wR1b8 zUwnTZ!Sj%{NqUr%`&NE7)D-qMb0B_>o}$e_0CLw_XZk5J%yg5KrD~kPFT5o}jmBy& za)!M>#091o$5*tLpt@FHRKHRtI1keb0!6JR2Z92R2aeSjd++>HHU~l}OH+%|cY2d8 zhTsCH!Im5p>z(q%GV?!4iUjUA`q|tUAEoI<@iRR7;vT#Md-!t401NxMGV6D1?S>zj zi-Rd{t2cjTUfm&zP1f-{h9Z5`KHE>D_53>SJnfwIZYZSNTxp?vhi2#yF@;*e8w+=8 zC-<-Y@XhGmL3VGf)y;@J#J+Zn$M`NIdgpK7gn&wu^LEbb+IkMWWa~tGrFpAuES@?8}w zzD6WBr<|iJ)EN1GVp?#p5T}NK!B%hp`d1=KXu5s9&qBMb*vF!XO^JH$8%-HfQeoT? z%HB{MKKXkhVL-cs1*EqEkE0*v+X|q>U#S4@D@GG2PH+H{z6_juT7A& zl<0OG`vDC7VTPt%<~h4LCE7eT2n~Oh%_dLFXRrpUmb?Q|gpfBUZ{OmyD-rt2bM4Y> zq}9BLC+w@Cxp6G;tx_{P?BBC(z{Ud;+a?X3PC9=n*ciHNO@-Rb=h1R9IMGEVEo6>M z6x~tOX36cQ?hP(;vW(#2EYB~odh0UWAhKN&Z+pH;BrEw}@i#sJhxEIIitpPmOAqkg z)+MbY;;t?0%?^GxJR_GqxwIZb-w9GtRjF%Ny;?2{xww(c8s&OYVXR1y{@|v1GhdUD zS;FG26a9=}%dI8@X(Wr5lQ3XYshD`;VG>W5_N8O!zKSaGd zx#}!=zj1&lw7c%$oiMF!+Qe$*-#7k`)D5<%6c}~(Z3cp~yZXLb{68s~2_7Rs*I}Tc z_cadpw^^jn8A9qC#^iDdXEg<`bGT$h-wJeNxk7JxbLfP^45!Y$rTN4hJhOl1P90a% zUz@g!a#1kH9E61@U!u}@ zT(FFvZap$CS?;?*F7^Pwy9Gh|vq^M4&k}f`Tcb|z_-ZbwG)8cv<_Fdu z1*J+4;dxO~As7gO|uyHLNKvH1tOOX1JBwR*TcGSp8iI&|9%O7*8}<9h){e zDf&lg`R|M6IHSq=rk&ARi~q_=pTVWx-d4=Kp-Gm5$c&W_9SCbsz-S4ye8xWZXtK4( zwG(a=wnT7-EaDI|eB{;sABgMp{O+itnNd}v)y6OKI zz$OjpbhzCC%UScOEKtcwoi(A_UWeij{OT9rnj-u8Op7701&X;2+0p~tZ6@q{3fG_h zD}(sYOYZ}Wm_cn1|MtI<%HQ9E;?F8ImhG{$cnE&5{!nE%Ugn}wIROO=$_0WW!%KGi zv51rvUUofb-XKd7v{eix4L7uUqQghyihq&B0twsqWC!q6>dggb(R#E ze8sN-?W>|k-DU#4DcRv_Fek9+d}${66K5(nV)_CCR( zX@28G{aYyd=Oi@&isE@QimBnwf!gcY@LH?5qA=g;E2KJ)@K!uEE3N5gRze>cyj5?? z)5;&P^JddH0#*ejBn-MZZXzo_^cqSmb^l=+;ZCHy(Dh>VwU<#g)a`9hY&dHYyX4Rt z6O+cYThef|ZQ=dT;4=DnFTJ-P2ziu5l@d}(nAnx6yeS!{l$4*5;$I=IC<|3WDC0 zk3b?xZ@#VPQz^3yW9+F|Vuehn`O}8-&!yum>!6hJw?t1Dh?G-CnerI~$!y-U{+0#V zGjJW&YBlm&)iUr}{n!dtYA#b;j10xeMk8pQ8xW8@-R~tb!nbKW-qD}M2>!?=9vHso z{ailn;rk@ZaThGdrL>*Si>wY@PrE|_?vyhWIK>Kpv-q4HIU47`b_#ER-K8&x7q)D4 zi-8MFPU24fk)UnSsSP5HQ%*5(gQ|CS^(G%V@r258IGGgd%s2&r` z&tOX(YPtd*UMQoJ(g8`1^!*q+a1b;V;Ai~^wpdNrFbp*;kDqe!ZF|aoXw`kQW3oR7 z?4R=GY&*fY9!_zMpE2WSPM*Bx4hmFEJGP?sKR;fA)WaLt*|7HBS%7_z15Vrx`o*+^ zt70d6sIXXwM9m&BZt8k(gc)kw+4@Z4$ZqMDpNz~bnf1rw}&H=={ZER+%*BL^XppY~!U_hV;2 zllZPzXR)6QX+Z9r%-f&jJQ+_2IiG`_B;FbwVKPM!6IwY7oG(frWv@r$NSOX+b= zs1%v-)@;V-Q(>9*2`8tUoyb|;;Cv-;#7q#X?9hF+9d4^U9!=b-WhcT#R& zVCQAYmL;P$l}nPRkE78r|95@n_o}-`ux=KGew%ZdB&!SxUXp1RJ;k((DB_(M`!C1j z(!Dv^&G5RAAm`VX&RpmkIPROhI^L3aeg<@v+XV|BJik~Lj@`JLnct^+r=q{3G18YU z5H(LXI(%$$9VERvhY?8M>0$fpFI4q|anQPh&TkyImpr9aQFiES7lUxc>PfYou6(53qbe>h|A72b*rpVc{cKcM=P<(NYXWr7&i{JLS$} zw4JAYgZ}8pMGX<($Md2`>n&l7k-`~2Vw>ToSP+p%uh+E3Xsgu z939MJ;|r?FTI#_)63ZZTl!7}?;FLEoV@jfzZOHP-KFGKk2 znm5}`iA4p7BJ$M=Z&Q!=A7`x4Cj$oY)mc-$xmMqFkG+~+AMRG4)5GsPSl~vs+Yi|{ z(HGU+_Px-e-*v^7)S?vYC>p@vbG9`nL>&|KE7siq1P8zDcZbE_RV!6|=?jII80sE$ ztBfDXpZ&RvcT+QvR7l{Q`KWAQ1AmoK{FOpKl}ZGvQ`K9C7)j5DSC-Q^Ok@&yZrjv~ zWcl(qt;6JgJ4`=kh@vI&=iC_%5QIp|k?a-LjKf`PT!CG zSul4mPEw)@UULue#9%F#R?87KK3QCE*e#c2Ht+OLO#G9c)G{}fKPqm(ZNj^NbYok7 z$G|Er9QKWp+&og^U?Hm-i+(BI`&QKuCvh!2hK5H_hs?C9k=A{ZFWXYPfrzKRWE52U zHZtjt*;Z5K9u1pyV4a`vD8oy|pLWK7V(sV#SWmD*9bO6b%v;M;U9nXw$b>o%iLrNL z!g)sGP;ab}IrPx>NuSrMwXwd~4_p#?(etU!J@*$|aYYHr4(SO4+3+Ri*dgovAZW81-Cs@q}Biv*e8!G=|KQ}I0K(-SVD zxrx!GZcIOE%n;4P`P#V2!;Nno0(3~cCqN3F z3g2N;<92-$-wGo~$R7@-om6=2cN(YWWnZ4KZ}d6nBEh8=w2`+$9(*neV8dW>QU4f4 zO*kwtBvA9|<282*?D;py>Nj)kv~&r9LjcSWe^%U|qznxGn;TI+AK2yjmMu_9ym8~` z!vPb)Adx^bFOnY0Hr*3+Y5n5E6N8XBn0_d%+Li0+hfoy7jRRr&p9uXqYq^K3#PTBl zG%HXOq>tv(9y5Wc5}a{=my!wToB6H4Cmfpwg2u01gddwa=BfDizBRSAO3=xuNip!R z3mN!CLGu3FQ8!p>#CW$_OA@-d-Lde3u+xL$)7rPU85G@D+S*mUs~V42!@L0KU=h;< zEaJ+9?f~+{I!rMKOF5K$wctS9R6zbQzsV#-poOwa}r}- zfRDhE4Yhn|bo24_c?yGQIG!3!__0ZQ3n@g5&u241#+SV0hkphNWFN5-u&$ohY=jbr zW4+S@jY>S91o#wh>;t*-e_w$ezQ@k|Ed$UBOzr25E2tv9U67A>p;j8r%T7M@T6@}R zy!0#V@el~~;*r^KT-}#_N{_aUlQa404Kn?1o|jl^J^@%O6+*@WQWG`0-)SaL&wMl# zePdp2cRTR>Xn6B$MXvhy9g!3_!YjsSrEVVM6Ej$u9pyd6@~<2(z>T0_%dzZ}@A1Jb zAc%N5HsM~JEZF5rNEg_B6Xngcinmu9<-^6^PV2ijvk1Mg2~Gq1fDF2MkgIz+Vx4=z zG*0q*&rEkesRaezh7;_+;vw|4_h_LG#ajAW&FK$Ew>nb5&bZ@~lOS`5__P9Tm8RQ_ zo+UY5CkZ>WJ(=VQ4T1)VsIiFw%cTZY6#%!XPbhSvy-0xJENdks#(0{PAKtU zSkjZ5>xU1uf-N*cuc}}83A^jAac`y!y79`CqPxVLG;Lnyb=1i_csu?4%ozS37QoTt zN7wQz54o}_zNzW(yaCxQ)>_9KAakqfSQ$kdvk~uQ^PPD2rNJ1Fga(_B#EQ*zU&<3*Z**^WsBtC`3GA*7 zQbY5mmwa{>j6e7pPcw4C?ZVG}JNQPg(#S;IL|8{6t)u&Zk%e7tQA1QxX?SI~==OcePlhn)5>`BC-!blU!Day0K zL*>D(Xtaq!O=d=VGt)bNC*unBrC280*dR&zj7uTV;HIL>$FGt8>%Vuj#8t`{W%TAx zi&lbXjR$u_C1^;hKL1%WCVTj^BC>Dm?7*2j5c*haGHWJt>4V0%3Q%2Miv2SfML;3l z`n;52tbU^X)vmmw_5i#5a4RHpFBXrtMWSQG@^jU^UCQX$`iswhpVXgkK%RorD_^Fi z=<~(vjk{wHek63P=&|}w>5TMS?95j#rH4?&ZOF5>el};@zm<4nsYg;T8^&G%k#y+-PcpQk8#=KE1mqexE;>YH1bdzlqFJi z6*_8JdPiTmn(mcxB*m(D+SdT?ihYh{tG0?RqATJXQR0FIDRd(c(F<;Vt$@`QlobtNIJI zM5{x-A`%1E0_H5pLmNBN(ARn{*7-e-XLE|1X0MqjLrcXuj;xZGmF%hP3s*)EnKN$x zxs*vsg6XU9*LH!rm5_8BpW@Y$vdu*+U61cAPkinF_3Y^}hOErmC}kcuu&V=)EE00Pnqvi!Q&ONW3i89a-YJcVt1`sGtR z3eFcIL4MJt_nD!LQDU1<=gzLtJ(8zKvIj$tbmTBlz2g12Vo( zt+`dr7L@mJL;~KLOUW2>lq7{Y?851icB9_W!AK+b8o=)48se&D)pZ)KsoP+yK>LxF zKd_*s)@+24o`FlhOe=7x8_;+zykqA%0i?Hk*`hzArO)z{;m>P6#(T{agl^R+5|<%` z`&fx&Bb&#E{kU`wAci1%QA|^GUMp(eAa&=86rK%%r=?->E9^kUh~lB!|0+igWQLc6 zo=l=QeL=>~8I4VJ)W?oY6D2t{MzC>j78r)@90A)v^rL$)tsC3PVdH6?Fg9)~ao1#> z2|b4ZJH#%&e0>OV%1*>A9oKr3bp~9J-PMiU__G?lKg?mH`hgh|8@l09J6N)hl65)A zQSCPXv67jc>3Wan#rd$wd%}Sv)To^I7>|fQmC@NZ)$OuF}Z9K#x1tzDL>xn$*)tRme zXKOfKwWq)$9AmFzFI#Wnvvh}}?K!;g9e^UporUsv>5l7Oos)E1^y)IF)9_w@Nv}-k zH?*Jh{V0c_R~l@itPr`$54ov&^*@<;*dGrtk}RL+XWKOO;Ax^zcpTr-_Y{8om^-T* zE`75L3$3N`+3Ii22lw|Zo3BN|6)y$VMsP3EL?4_kLi}p6*9If8-I`t%>kn9DSXc{* zq}WGe=sULeWmvM%ES8nJ0U!rl><6C?r+#s(imq1R%{?MxQ6t1&8g*%$yZ98|*@HH6 zt10*i_U5P*`i4yvtO@%&Mu1V6dy*c#2f&8HrI3g;OfMnPCqrs-Gv1iYrQSjK<*Tqn zwYhsY5F@(ZxR;QFbK8hkM4VQ9(Q?^)9((;1x45~^iD`>OmQHm)=FaieGjPCP)l>ST{9Mr|INwt~cUr66^_H=#qqV7LI^eogJ zNJ}kWh*jK{+F#eMk!x)gQ_0QO->LS*ILU)<1|MdJ?t3F-K+4c2gioZdzhWf=K*#p5 z#*^j7>48b(qgjg9RtjT-WN>k|>ZXGVxP6nh!irJG8(X=OpFDb;Q)9_iPZpw1*s^7= z*#TG&5B7dQ9_2NO8t|kO-FCDsiJivE7!{ce$QORQ?tX}+hl^LG>D(Bz?i5_hz-dop zwsY%}eivnUEK4=6OpB$G4b(Ej)S6JFpI>1Skjl1=?c!^1+bc-_r?WgK86H3sO&;&G3s4u#GG0l&)aiFT z;&-QEBf>tbm$v-MCRPSLFM1Ixjj4u@sy&{6QdO%JYKi^=qaGJQ(Bj;Cuh>KVYS}&w zhr(8%9w(ZrHp%xrNetA7`bJLmQR3I*tj-oEKZKRpRDVjoP;czxA4? z2>b+2g#ZXY$Nn{zR*ByH_}zjTDpf@vwb}U^IVTu_>!7pFvNPQg(LIiHPK9o!I$~xs zS2`VY#o5lO*4<@5lOM-~+XY6%{>0`@&mwVly;c`kvn9JPC+0&dW2)CsMZQmVk|J`Q z|5N((C$LeGvNmzGND+Wb6uao}c1eIl{uZeRI6d{#dE3!;*&#dN@bgkajGF!eI94n5 zSB%ky%a7NjO;_J%MslQXgj)o@^7PJs~ySwsx_A$kS~J1_+wu zHi+*|mScIJ1l)FB6B8^jK!UmP%Ns3PwX)f+>R-TU;$cUhdn6}9wy=hk6khuQ{YEiu zD_dnnGvuAJ#JzCS5pg#)_|7-%nP*=Rip4J$?kRfAZZdd%Tm$z5QCS zDT?+oL~P>=JYouu9pMUzgG^x&PUN!icOMlBP4jwSisNM&LEYuIio7-mT(-^3zB1L2 zyNb`p_|BPX0XKtO>)hP1N)o#B7(Pgm`C4%GHMR(!;iWb(ysDw=hpGQv&p33uM#S`+ylI)axx(;tfB~p{krkKBsDx|&T|#^A-Mx0s5qF9pPps2` z3;o)S;}LsqQU5_xf$g{b&ik}en#r-bcgCs_9P2GC&d48K%oB19yJPH#lEf*YWA{eldSqF?yQPgY4? zvHcE+s$&)mgp=e+%!#Rw3%4jvPaUf0JN|*Z|7+{y`km5<*vi*Q=Ppo95yo!lI*Y{gVWM4dC-oKF z`ChS~N;M>nlj@Pp_Q24w$I3yZ2`%)PjM;&pvW3*S#w{nNKwd*d;j`w|J ztUCI-@TA@%JSqEf4=ALOMwijYw7PH}epH#9`_Fa+Fq?{}q)?Yw{)0??N7t4_Macbs zigOs&_42f1uGJ^OcsCV2DBa$pG%BzaoR~D%&lC;ofvxX?=@J`8KeJk?$(w7}gVc3j zLG3bg1H|uOPyx37l`%>=Z3JrQ2ofTBDEwG#;Qk9zpQE2#ct}m?W#U{NEisTzs7JXm{$eHe&gR&BhU3M=4h)Btdm~^Sn zE@TLU&HdOlvHNnK$ggA{efre)OH{-D-0M)Ca!e{|w!0ZNlUj)Jx0CRkk)En%b_P+c zSAVXU5E8eayNf@2M{~_!;I)|YNm+HRX(lp3u*hyI>r9aVuDoP_xK_r`2eFR&rkm}h z-kXWZ=5H=NYtYH*YOlYcM6+n4jDr`>2ySOHxz;T;UuUU_%toQS2Eki(%1a_z-fO!b z>xeDo-F+U_Ktg@hkCq3MUtp$04_)0Qs{g1*I^kLRF2tlLYHxIlbA=e z|G7&(W%hTvnTDAwuj9w!p7iXcts-27Bp^x&WMgD)md^X642m|O45_dFyal4?)|hGV z?dlkNbVZ&y$c|Mu;1FSJq5@25s*K(@8h4OLxkSxTrFqfKV6#&V{r8z&>5tm!UvKLg zZ@(0LeC62WqV&hKJ#3A%OieS(ryyWZVzS~Qv>mI&eT;Sc?0@g#fwH#|xgvZkNia52 z?7)I=W-B%Y2igOKjG=if%iW6g;ik#s)K|UQjR(WC;oYJPsh&$8^d%??>c2L6wtc@r zzyG7o?C~8T+w|ToTxT#(AhIjhkYwiYVv^{tUpcp+9S+Hauj8GOIO>TW!a{?7$cK+YdcEN=Z3`_b-o-nnnBd03a?ebN;|Evx)yv`kUNPDxIO<%_Y^k}dC6wmB zkLNxfyp4D6sdrFMKOmGUq}SN;o6QsX-E}3JZ4OlY&0oJ+akj3BnoF$=xe#2EGxOr>X|-?SNi81+x7FB?V-J@rKW@;0Ymbykz$? zhX7xAuZfA;%_TG9%qwJ9OOtoGLOQOPN{kItr5|>3ZMRwq)i9gVPVHqftA1;T#%J~! z8TGt*zY?5O0-go+32szIG1>7}?_NVi|LQ+!XMhY#Q@yC(!Oi39iA&3a-b}82e=5x- zU+|Zv5z9H}pjzX3>#!O7O!1@sOrg%dlOvBWos=K1MPUhlE(}t>n;GpWuzn|hU zQ(s%RlO_bE^qxZqkSOz-^EbnvhoORi4*-{OY=mx($f2T~qkzKzmKfgK`2}Q}z+37= z-1+fVm4()5-@(F;o07j5p(1~cx-s3iT`x%0kb%DOC5O~ac498`bWmDMl&%0vJ_qdRtr8LBx#+M5XdCNs)ok81aaA71Ae$2Fwd0KTudK z=2>+(_f1!(Bi!_*+cF@^TAR(zv*nIhT`t_^mz# z$(Dy${t3$%mh|&)ipD=jlys>#u0=pMv`l~IhcZsK0ng^bWHgu3-CreYMBy8?ab5#k zTRijKnD#wbxx+Drkyq6b=#(P%^CM;>Vq|zX{j%4uyDykaTs>A4g*VisusoLWccy9; z<=Qobhd;)*m{HuF$*Wg+r4dYa73=jPd;N@vFd#8lNG+-}0d&&O`yw@%-5y!I&o?1B3AO?LD-V^!C1!Jd@_+y zBZZUl%1ORR+0w2YDm?W8QtkaWP}3nrAjK;`l-gBa%T-@vR9<^ODx^@OYK<2r>%aUk zZ4QXzwpHpER%WN$_abLW<15omvQltAJ-ydeUibkc+bboMbA0;OmI=)rZ|_2i7wv1$nNe<7YeAT za}`5sYhj?Ge&iUr+bTZ>sXK6-I-DLuWTMW_zNVu~B}#X9Ij72hrpgWwP4%Kj7)L+pZnwmWq$%W>ewrEgj=(m?x^0Dya5jHrjRKY(rPN)9>4IP-PbF~ z(w7ZKytmMHiP4}rE*$;ft*}uMCPPF*IG?i~he53BXE|R$4(M)rpE)|B>BEg}b{E5C zzW}zlxEa8HEe?JvXS0YX-fknCG~cq)^kLBvJ*P)i!Yz|KSpPKMIsPy<)tU)od3$Nb zd)S+><9>mQS`X^IL$`x?d&==b*zb#%)G2|3Bo(ygVk^)Jm`wjkwOVec!|J|eOvF4^ z8> z#33)L*KXPF>LdNfslYE(bhhD-Go|mtmzs((Z!l?wCkeene2JZlF!|jF&84X3I6qB8 z__Z6Litta@T_0V*Q4P7|ZdjQ^U__X<2=GZ_MOGI(O(~i}qnK;tQvfh6 zBLotz^eFraw$B+4DqclltW7gId>3o2%ewN)hTIhfI>Z&*T%4@?nR|0RvbrtsUAQ&b zcVuK5I~3iXgLCTLRjEZgaCzzAZy&ULQKx<(i*5z$oq>n#EAxIs`b*z`88oOmPf}Bs zHcvjd^|#JRqoJyC){Qr^Vu|2XqHl4S=t`6FHO`iSQJ&x|>q7ORv@*HM6XU%jJfBbHg52(3nQdH7x zQmD{kpf{uVpH4`VNc`ECt6ZXv?}@tdw_>=E3q6BB9XUG7B=LnITX?usg|lSj#~VjW z5c|m(dH?0oZuG5`b1_4AcG*6-i2Onha9l3ydT$@jr#?Hl$Jp3Le{}ZxcsX&~Vn9O# zYv*2eFMj#Un<$-$pqk0H2|~jrmgEk6?UjHU3eFS z!zP;BdY<&hPUxAMMpcoy4rZ9GvdI&?4I}M2I#<5}Bx#GKZ6@GwVmA>nmHv|-A&6EjwfYed^XEBSAbz1w5e`PE1$~l-$3LtdXJaD`z2ZWaBzTg!^5^D*m7vq94ne}2S~JEiyI93tGx*?nTwbsxN@)oXp?3eG`CCo(q#?VXj(mkaq?P?{6kfoer7_hshTFvmgy&~=ReNlHQ@EQKDOQw6Uz%z) zC}1cH_{c|F?%9RShDHDCBy94&#Ctzw?)=TzM+M{*05e5D$FqEoh+Qw?HGi#@yH;%( zKVWzje{K_E8k=RCS%}s5)2Z;ny~ZP=mi0u>-QK9O zr8LeWme+r*Six^HF*L!M<}3@kAZSnxlxu0@{Sc(D?Ucsh0J@>VJRbe7OyW`C!^m5L zX}cX!T~kn~hz%dXPv>vK#VJEmtv-%?Q7^opc-`0f5+c7mYMg|5tfev2LdU>BMuIPg z#=dD^08*U0x9@HBF?d!e@N29eHso~MRZ`WQm%#j!4;d?#2HwgN+_vFSYdS%bFz@L! zg=4sWJS&ODu}_@q_{5_`7UV+ctzA`495YUK0jDuXMdC?Eltn?PZ%rpiyK*KL zH?gEL-V2YQQf~NXFdTJ;k*C1$p243!KAVC&!6NCi&$e&nF3F^5WSGq9s@Juj473=> zsQf|lT&q+mQo_JQh*Qzm>nuV}aLFjZR_fF@OU9_oax2xe_9%_w z{bD@EGY;AnBJT2~Ov1tKvGb9?wS8dN!*gHwU8jBixvk6d-jB%2&Pwx5`sp4Q%3`zdclHHJD>z6(&Jiu)~Fidbf=mm=7+Ao5mFA zhqS86jNZ)@fs*XUGgW*6YWNA%CQ1CBEXeG6sjnPWn`|l?wo7F+ZRd2m>++ZuWm{dx z>_(t?VTw{Hg~97au(x5zzGMCXD}n`I;U(YPLlfoLk@ljMv#BW6?Ite8Z$uG%DYH)r zP_k^Dkv7E81WQoP zUpeBFLi(!jElW>K2FId%@V1JgVS5bJG?X(Ix=4VPrxL=On06mcojXfBc`fj2NBS;;JmZG8P_hgE z{=JvRfH_fnFIcNaYhOHg_*x0TV?29_V;*&yIDSDFJGuOK4|oZf!K%4-jwcS*01tU$UKSuq6g`AYz#VnNul3`I_7)M)b8DxKc zaefX_MlX8cnEXcK8%E@2Q57d%*E!xyI6;luJ$Ht}`71GeSE`Dr=UR##TKO8=?@%8P zy*UKm8)ytydtrtYU-Q-zSaLKxvAL!c2H(YgP@EU>1r1a>VJwW40(!=BD_iG>HVRk5=N37_ zQGk@9FwX_nHXESJxIl6sPVMVu%Wrz^m+4f!brXxItdfWClKa|h4dyKHghiPe?tb3B zoF+8}%PldON1M^{G!9%&i}V5xK-TE*r!K}_P-SiMk>EqKQ>(f_m1}*+8Mbe$5EyUu zd|(QYSlZf#$6}FiP53#CP4ov(C^c(L{{!44&%Cz*-f>Fmu_;+cLA&%aD)2B1=ld`wkzaMkONj={(P3-&aQhIgcO%y|8YVjOT ztm^>r>%fWU;0Q@yGJ5YYb#ZRwnDm2JM@E@rV=ePzHy$fS8-rpOJ2cvp*z$zz*lF4G zjU)oKF&D{8m^(|U+!{&A}s>vhM z@HlI4wkd z<3mcx)wncMN~J+fMAzy~n3tD-bb!w>S>pt%H550%qWiQ1VpuobrDV zP#ON^6a=QQh2`k?emWn^DX>tZI-?jH%-Nuv^McbCF2Mu1O57y*o7$RlCLsI6z1oL) zTu(B;Hd!nNNMRxPOOxh#Ys}DH+UA^lwSICRobLzCqzZVC3*P@D&I~b^G+!S4n~>{!DDll8!U_ZL8#`&y(7=ZI`Z(z+P8=Cvvb>PW{T;;%*i zD_2H;!bYA*jt=v#t&KbpkB#r7d6c#uM}ya^aJaYy60E(O?au-go-4;2$2go#;z@~( zf!YoxeD!#(FL=;Fah&xzO_E}13<7=h79HgXT**5kT|f$vz;`K@M#fK(U|h1O?p^54 z6)O6ugm)8#@?oe29h(c>Vb&R|Q1pjp7b7{UDZZ56RhG!0&fmKt;UcbEtf5uLtk0TL zFH7i{x@=04b&3Gls6}lK@_AnJyDgqy_FL|&k=wTSXI8L(2TZRBr|6Gi^1B+oanl&U zVI-mK;&$M91k8|z5E<(Yg_d;{o-c0L@=b&=4e%5g5k_nd8lOEc9T&U}UtT5Sx9-o^ z05THub+Yh&?q-+z?rn4g(tmPyYiznG4i_t9y5%p20f$JG+jeX-JzAZfcmyu-J)ia@ z!F?1sDkESw(HX|(M!GuT`gD40i_rwnuH2$ZyG78#M%)OrMq(x0%SR5h_k1KDpR&AJ zvYryV>@p>j(3E~@LE^o&AVO>7)#3fvISi4tHn@$HnJkHr?@g)(xMzh%McmAv{ z$r=Co$DIJsnAHfO44hE`9G0sn?~`%A4Mw$J1&TIHi8sDLqeTo?e&8L2J3({mFhoQm zdE7Fa>Z2evQ{LJcj?=PImorKz17LBn1*yz_^F}zt-P(deriZmpg&qs!lfpbQiwRmtV3J0Coc0-TVo%{8m zM^^O$D>%Rb#{zI*dqX_e9+;7FW3z=Pdd=ih4k_jn0_)-NwHarGDa9~w@npHQK==N> zC+*nIz2QfV25npTn}-+86PlSin>VW$^&K$DiRCg>QzB6yu>}{^KPN7>>tO1JZvy3L zLQ~oG5VW5Q&pDDJziTe zW_T$tXVmcn&b!{Q-@yMob)KYNO>7;<@zcgbHNGL4%YGU888k)TwWKeTkX=<9(W-9> zGrv)+rG0hX;QWPXrKdsHj?De)oxJ@#SaS2~YU}dq@(xq3jX{j!+Vji!^ZCpuO3Zo_zCT$dOt#Ayw}@#v!mL>)l&x}W&Qc+&lB0u&Jm{Y zu;X9ZbiQ&Xbv5DJ-Acb7S?YYfWwhH_XEN)x-Bd59;TGC7b|`bz+!;8;Fd6;U7=6dGRt8>`<=ohV*5Kz0o50EzSC;z^q1v=bsV*sD{2{U{1ZPO4S z&jq|m*86;!EGsyif6Rq)n7KF4`$nz@U6AZOkteVZ%5$0Pc`ncyJ`&pE68^?nv$sxX zECqm-7NBT7&s+1l1sMewZTn~_hn6(N(+k;}IG@`&m5r^ONHH~ubL`tTQ#7n7iHQ$`0WPg6DFnojk}+R-l<5R2u-? z`eIN`7V25b=*Cg<;)ho17)xoObk92E>z>k2IYgHIBC zLC{mn{acINc>UY`ORRM30|8f_CO%4%PHhS9@-%sSPuAeNK+|?ydhN*pZSD>(gCbD@ z_PdR$7K*n3H(8u#y33D@YuBntd}`$8CW%$_J9J8^xjghsb4SxR9M&E;ZP&|uzveS; zOE(GD`3ovZ58p)zrP~z1**nKXdoQ3sbP9#qp#%LkbM}>d@#8H$2ru^Wlw0AHS-;z( zy$7p~(Jp2+Xn7@?(2=cKvl3nkPok=Nmbk_i^WLxvcBiA>ZHDGggoQ_#iA zayC8r{sJ`xcsw?rt^Yc=%l*1orne%Fx3}4V>fjL6A+VQQ!Ov3Fc6-a6LLZ1N;~w54 zewD~#t^465`XxB12r9Q^8LwtO@>W>^T|@twYp+5p;c&G%MB<5&=7Y&Zuz!8FRi&Of zI*`LCp)}StV2kS{HRCcWYo%8!W78}W$(?IJmsCo&C-ljE2M|o5noi153=ME*cD&sH z)Xq9zW93afW6Zoe>?^P8B3xuOf5x<=KhgCHrcu3Ayu&aZ^H4mjuw`n)P8Py(jWkwf zpF7MI#O%OtO(y0|gVV$*XS0b&&!Li4L>Pm|=fn6cj*p2zM)B2U-T zY|%;2BIM|tPLFn1!+qeDh75jQ5%+o2wBdz>b6)k#H&hK^2QExMv+=`?aLKUX z$e@#?x@E^NuK>YrRE2d6ak+SGG7yiY7TmWz~&#e3;68ft{!! zMo}sL`nx=VZ+LA}cVwgDP>P&kw6x*PVYyI=y0?yP&}bG+5^*~9(`8_^OB`J8*Tf-_^=Cu(WNJY z@5{J~=6hPGg1`mU0tGlK?DFF}Gs+-Hom*IoBHt7aM>xrzfe5@XXA!jS8I_qaPfPe$ zWKf-|-xlMATD}^zupx%vBVClng$3p(ufSROk`H66bj!g&@=hYT=Pl;7Jkcn`O%3BTiOSjQ{ShQ(=-bSB@jV31wTf3q^9Vf;VZ0tg&?fc}Bfk~w%qi|i^ zud3`L4bv{k??Z?bYR-hej>s&soVNTgTLt&Cx5~pfupx)@9c?G#LH11Xd2|?k+4u#D zgZX6*#$VrrA{J+I--zQO?fNtndCp+JYEG3_Qz!l!>IGf@roi50Naxj{WkS8rP9ED264X%_AiDN66Zv;Zb8g>~olERM_WVQ!gh@(;XIWjY2 z!CKOAq15iY4ejk1wn+Zu2EOgiJ#Muho6avxY7TETS5xqf>o=6Lsw0i~#DAOXj^yO9 zg_y5A1>!lY$SiV)b>o}udcvhmn0N_+c0_*n`-NEMLl@a{;1o2clGcn4-UbcWE*1?% z!!2Zl2+o7`fLUjH+oV?fzj*9_69yUIbjY`{Qaz(#;3ciSfO z3w6DL);m9Z`Xg_A`2#k+CVR4z}NFrZxBQPu;p|oWqju)FFAkhZgkT1K5TB+OR*EVKYO)f3Sr3hHXL_ zIy^d#o=v34f4vqcD@)NkMN~i$RCST`H2XQV*bRK^|ebU^VnaPaIFqcO3oaS&Ovai zpeu!UZtgPdvhLoR9+ro|dU-dnkGWVqVh5p8C%~SMc$CAy9@(H4X9b#4UmyfN!W;0*c&&o}(Nu(~v&>XQB6q&A8b}XXUglnk)D9JR z*)Aevg5ZLx7+@92>5X1#zhJNCV)u>Rq7BhWi6}6Js29BT(J}h>oXtG>0IalpyXPj`O4d{4Y#sO1Fe>f zxPuSq-jNU_8HRg=rQ^YO`~AW~<;eM4#YAFD)ST^^`pGU>2ph4zo)fxKLrvpS zGsG?Mn_B>4v<3UG>fFeF*x}|Nk#UVnH4*(;;!NCFXZbuoe43p_9?~VXP-xUz+ckAi z7S`p!MW{{exVP{lz^fGmb|V<(K$M&I&3FOxw4rAY^miX>i%-`qN7rRZ2o?@9YA;K~ zDH|FbX0fHOM6Nnbfn$dcRgujAmo0Rs({-!%ta?%um$Bu4M3lJyBj#(bk?qvBrB$=pVb|93NHwSVuF$q#d3pNMeXb-S#@q|H3Ru}=c%UC?9CG0#5brfB=EGRuvuSa z!>Yb*TJsFX&GO{NNi)!SHOE3smUOJif?@AscnG6OLXbycr5%qq(#z>P-5~FslqdgS zYDkT$%Mg6kbUF9+u8Ir|OV+(@TFUVYSAE_So?h%hgvIZH*x|&I(jRUgansSq+628{ zdzF5TmO3fNRawP-&vJ5Z-o30oQi(RAy8C{OycE-yl5M&Y zSy^k?=5PWMvJjn_ty&DuWK|=5H#-odn9SE z^4O5kL_EA=;IGoI2J%Q{nydKMM{+i8+oIgd5RwIhEnJsPp|zavBO}jKp%ARi6CuvM z88zlYAFyairTbA5LtoENPQ0XL+=Vqq%3mXoKikrF$i@EDT(q*E27!Ok?WJDNq~kJ~ zJB5gKX5nOqSY=CIh4ecLdp^_GWMicDn>n@`? zT$ADQ)|{f$Up@d#E!r?61vSSS(%yUi)@m%R{3(dwZ-0f8^^)LYDxcZv?ksQI-wK%j z%-O*i=_{oyx116@#Sun55;fR&`pDFmV%LcY_n>~^BnW$BulLNVsBmL4?2Yzi0JPcG zzqxsCqN5GOvx=Gd&M9f1QV0T6^rjiskXl>YsyCR(7H0H7W!j-;8CuE(6Q#R_=^DQ* zQ3(pS-iQAgeyHSg}&$GVPnTg@M_B|LrETZ{K>nPQkA;Zh^q z0V3F^XH5^%j2W@*!?|O;n9FzY)X8ctxmYT`D|*OnxsN}jVlZZBzw#TTWy+bcb?NOLTX~38Mjvq4p5!i<(B>jEoll}CWwcM_ zT{4d@B2iXJ?sf}kP{M-1xcsiwWmxz<5YgcgTWjt1y{v-K@vo)Ilta+lt-Pn&V;Xmc zd#pgTqbav#4QsJ9*9#c}MmM{T_RdtFg~T$aZF=o?FK)e8{|T z-csWC@=hDJ?w%eg5J;plfsA|&Z5!CIEQ*7FGxL~jU~LIO%=8?HYNdlSAt zd@1G;M#kU6Duv+0eB6hr)V#+$20E`}qOe@D!l>@;{l8+b4?NGW)|K zx=lX7JjBMUhuOCN3GLk-TuZ6*Vz)`TrF?~8v@lEYq}b1K$<;z(7Vpd@@6Q6b#e%h5 zZ1-aG)mF4b$vepEQRK$`g_lDy#JFEw5JFVjm=l$A>|Wx6*6P;?%^SYa=MpxaM9xm7 zkfhv!U9DP-*A^jBDkM<9+W&VJKx7;8&tzF^+~3pTPcyGK8d`<0p?Xq#q+u0F2mjD1 z{FVF=9(3Xj61a!$NL4PhxJqu^CC;7)+6$?R^PLvVQ9SR%`H=$X?t*marv=`Nc7wW? zE29rFrf-`dFHLm7vw*mjA@n@zHrf6HJw?%C!%wGs03KQrAY5>06fIKw)ibp%Dwwh- z`@7&9S_tHk+t}~Dq08Et?EeUGdyHx&!xyQ5#CMxEw$mnAIklisBi;D@*#H9*}O zUf15DGPAR-XjRRKl!t`FtDS>L{jjmlkQhxw~MDQ>9dF-G)Dn~@2CfD`b_fDGp=B; zy;%>ab8lbqLR*INi6QFE_1DDJWoRMu$@rbR#9{kZ^Vx#R-661WQzhO@ehqZjtmQf+ zz(_|J|7bh=G)?0rCW>a zLwUO#65OYQx%_luUw58Dx)6==&o)7TVF)gpXxM2ODo$n4PxWIgm-R+gS#Dx}!+L3x zEL`#Wmu^4OYF|t&-grbwIy_CWyX+bA3!|2p$H;3|W(JCD_}n(PmU1IMmm+g_;*%-9 z87)iMT$S%G73?dgrw|)>mwYSb#gTvZ(FNf;V0ZHe5qzS&MeFo*txk9N$<2urU|IiY ziCjU}n=HkJ=G^42rubdK_S?6AJu9)}M=eI5bE-GoN?$m*UnCcqJ;8{Fp6<{3R!WeZ zZ)pVyjDAKj#?NeVd=V!ws{OnU^@L$W?Wunx$-g~3G!bc{URSWKq5ib4g1Iz6lXcDA z=p3#<(I7--Y%TJ1E6l{Y2NPX*!q~`--{IE(fFD~{vE_Tc12GA-X&+c1hyq!C7m?*pD!;`pMZJTm$m*x&(eCqY!w{oKXh3mCP zuRaMSjn{6J#~etG<-Q=25HR4z-pZO?et(oARQen?+R!7)B_8ehggN{u2;;LiMVX7t ztd;#R;3nqJL$Ec8Ng8vLO$7fp%O^tLJg#D`Tx*72^tPz{(#KHYnfGiC2K*&bFtV z8E~t_XcmUp0Q+Cck7_dP!>DRZ){wz{8p%-7W`%BJ?Y#i=gexvgAxN(xW|)0KnAml< z(x_wG2f8N1KQdClk)ERm^$;H~u$jTJp?oz{1*_u=YQHy6t=)2Q)%H~p9LwOyaZ)hi z8Ph15;ndoqHNv(o0vpThB6O1dEuntSp^_UPW5B!LW`_bc?E=WHP)i^zQoZCe z67EF(d;kPSSb{uCRdM^}R1dQwHBuE{=~vT7V>RmWzA!5x=~@sL2I0n;(=BIx_Mz!F zP}|*iVt$3(Dd9TX*gPwFd>!7GB-&n5zmY^ODgb0*%<^jod1xl>lZ|)?`++e% z>P_o>&rsXKQc1VsoUSk?Jx>c9kya8D$6yqR9t8AvG(Qvwhu!6xW~s&xF?Uoil7bn{ z!BavPQqiEb3v+ai@Cf?$+R~r0D$3NV%TwJvu3}c?n8(`X{pW~z=FPAwUrL9$%M1o3 zH@OM>*ZTa5=KzE24&evi4bwkCuDKsP+cpc4icl6KXTXfh{2Ad`7vZp|m|WKTv0^ZC z3PVi*XBzV<#;l-yd$dmBgC46S2xd*%xeba_$@G%dT%uZ}P~B7Z1vgXb54QY&9ofzw z>G|v5B;VcjeQ`FF(D*0?c-h*qz`i&SES2_f9IpQ3+eoMHm1(=hmcZ?g^HLf~<4mqd z5Wz>ond+(y2k}aAX($j2c>B}}fwQll$BdbS6o62Wzm)I!>9SeWcOW9u1I46?TXVkEi+n*H+30ZzuaGi zdTV$M#audhgxvhNukQG>=awlj->v@e%}YmmO-jfOZgxV>;N9!o8VD8b6=RS<3kOC< zN|5ECBgH5WnW6Q>B^MYwestRc!Cv>c@k)%UdAP4)tSA0Hq5OV^eb_QCjBBlzgcFN3 zRgvCePc?~1^3Wen;1-PybgMoaE=i!?=hlaH<+fNRuVG}=zqWQ4bl+tXFNK+IaxH%5 zD}Xvr_8YWH4kfZe7^=6>eRuL?t5sEy+DF5lvpHg4Y_b9%B=^DgnINuEV} z{HPw!{WBUfaNiACkt@lZ@L?J zEzEuZcGsO;LZ-$88yr;+bkY8P@e1RM8L!cAyepTsb80eO(od>8lrpU$b!xN*L*h`j zg}ZxP8y2=j^Q==9BA12{_u~z>wIs$lIaE-j`f0k+BDz6VRd1739hh`)uK(=1sB|dj z_^7oMDP-8+cN6-n7DRNUQl4po0D`0NBjT;Yr&%qOGbRox!z=_&k==erYCo%Ht|}2Pps1a{=1r`HO|BBieX3ww-w-YdG$zt}nk__WBP5R&WO>E95ieFW5 z93h-i^6j9rp^Wqz$+|T2ZNKT)anprM{cEG$lbfC5v-@tGSjSg+iP4%}&J* zbew8<(-Rx*U>zg*E5R@5eemKZyjwNrLRiyWx}>K~~I`T*Aw*tT|n9P!xLn#|ZC=B-M>B%LWPo}Ev@*Q;aR*GEOHVY9|NRNT zN1yMho{|1P-HD9D#&{MA;kyBTr#%XyI~c8)KU z_}tsd@9rdIq3cC-^9+}>BvZYp(jsLsUfSThPjW<9d zeUAmzG@h&emW>Qd$8Y2BUd}#>26r}Na&W!5eCDuA<@MY?K?j4*zUOEk(H$X5YRzpR zMY9M96C)_BqTIIY|vC(9Baa2@k_pOHSQi9%^ydRV6)68!@I#78vAD$1ZA zf&BF_aU*YDreU(!ibQ$n23UCANS%JH9{qudUstzvjWiN>%=-r+37+MZtNsgS&YtjI z8Vv^2AGk2TA&mvu>sVaC&0{97cQ=pRTFuLa1dA}YwN@4^LRH?4a0UpQi=fhLX*;%P zx0?y%53lEW4xv34!rO6S#q$+C{KiJ!Y^Q)J{7RiX5ElgI zEojZ~f5wm}-&gv&vQMm0b1c_*zf-d9T136N^r{;pEcxvE7X(JSVodYtJcl@JjF7Fm zS=$|5)tX)Gmn~!9a7o*IOy_5hwnzJ&;#H|uePVHImAS~He07xo#y!2x#1EHYyZE%0 zw|ff9XwATIJ}V5iG^jv}=pXv>;ozy<*wVoQzcalRzsn(xkhgRKc9eR=Db}SWYVJ?RsuOcya)0;b`x}&0c=9&;q=BTDZftAJ_sbw#rMdtSr zAr`=R*-?u&r_MX!vjIFUhT;;*CmWn=`SmOV#8((pq9^Tnumj@d7$X|-YNvIS z+J@BDh(f2R{@rDf0ir4)lQ9LcA61eOz0+;uqV&d9*IzaDLJhnxhl>?|&F^%VSAOla zE!LF(Q!yFS*Sqf7a@cB#^H%Zik1HEX&^bLJB(TJ-(MFLwX>z*5K%z2H?XdzX? zOj6*CEauEO|499tmD^tb5~V9nv(x(h4F7^Z}vGAM&=$q z)($*QvMVVFc`oWb0r~R&WSj?S1N*Bz{Co5Dk+m61!QenULo4*TOsM@Xjdrr0HL~UC zX}(KU75Rfiv8bT-7_-hffFLiqma$p!t~ZEzyj1KUKV1y}>w(pkFCFzWELq_mL%Q#4 zxO^$k3=-tl>Q*liuH8Q!?U-ofA02d~3-{|-8jMIXdw#??{*scQz)@r?iN_@&5Z=(j zUer9+X>7yEu^8yz-8f^-JnSK{Jv_v5q@Iaj|d$Ulq@bnU=As_s%?SF<4K$A zVqQaRO%CbK8@Wb!g5%24S?$+6tGq#2iFirZGn5!ZPPqN{u{{zBHB$e&75i?x_Gzuzj=S7RrAY6tk9ZhDc%ECi77biH z0YJf;5J=XY)g{rLW|MXNg=;A-_)MxB6^cb+c1BqK&mzNtUIhc9||Gg>=S?sxl;NUwNtMns8Hxir9F2IL|w0M?Y?6j^m3i z!Hy}fBXLTgFpJ~uHfN|7`Nx}@3kLcqw`wzV7H{UDG|h21I@;+?y^1<3ls5pO8e)=i z43CCLoS%tRd}v&P@~;6M?z&{u6V;ZQjpZbN-N0uFwmMEcX3b7 zdJvrJFr~x$Y)f8n6P5s)qp2h4n~#H5|CQT<;j&WsC->pKE=l8_$2pix<>h(_5;B*B z_HnXMhhd@Uwk|$Hst$AkLeHhR2u&lq++U+}iRKP2n^0Tw-JVz=N)~?!73v~YawEvqqfhn55oDnC6qaai~ zD%6Ppt?=KQ#y_0MlmF9mE9JBM(X7f4Ua~E*Q6f*K5e^%-!f>HUJcSn7{dMmvJ)K*9 z6kjmcqJvM-8m|$ahLae(u>r=sg74xyw>m`Df|pX9r{6`fGn#PC4L=vFzSHG@eeEk_9ZWt$cJt!==Gt3 z2ZJ?PkIueqZ?H+JNnR2we7!CwEy*r;$&fS6xjfNijBFU5l^fX!FYd3;J|c>CUtV>A zAQp3Zp-l0|U0J-m15;*0-)C4v2wPo|cj(M_0CAhv1PN(kgu zUm{l>R%qCMbn_fp{rh|Xc&YnjNW#ZdH_tU5|1Qhdzs*@Z>iNW&)t6nkkZ`Fy3rjSy z_(^Fnzqlo3^VwIEMKmjELeuvNc`~cku)Y{HX3iR!cpY|^34gcG_TyK_1|yhs2)4I6 zcr^L6yZ>WJQ$-C)bjtLCGyg^5gm~PYwGi7~P zpRj;0v_-xMTNCTwDwh#Tmx66vUf9)IO3x0H+I6DG85E96o)|Bwv6}ZS#Hg%EJ!i7+ zj1A>aV@dW1z ziH}(M+2HiDSYP2rou2DQr_|5yUqDC6#&*~vUi;qul>V0FiZ^f6}>2R{|5Ex%ScND-blR=#LKxc%R036 zmJ~A9In}+6Xobe!vb*Aq@+@6-lUml;j*ysDIrPqPxRLL{V+$&AUn|)W{c*>)VYWZS zz`p3)(foTqgN_S69Epcd=!1l{h}xW|K@h+QI9odt?5G+ZU9}R{-iIKL5LlOVU9l!E zwf*PVv5g5PjODIB*C@HAQ9Vt99P)kJ3lUV&%9qZR=B&tk?jew3&1;qT&ieI0vRmvdLliY@Df=0rwq9iq(@7ooCE z+}`dwXu3~31ZTEC7p8J(M&en$*MGZK?#%PAD&;T9)=UPk5)lH3*R0&+2d=PaQnf(5 zWHtkC(h9sVhzie*QeAFDyI@Y2xM}XtMVcWg{C-h}MEj7QLgp&7gI5>6kkn^>7q=Wu zh;F4e{$N{dN5M`u$!$&6C68`1B(PDmx%fr=oIGnq-0dem<7q_?$<`RP72?;vXxheNhWTrmf{F+VW1W~QJ z>#ywf+eav*{gHh0xLZ@dzM;b|FRNiZN6;v=Q^;+23_9+b=GOB*rbIo-a0F7SzfoE^ z%sZrosa=Y{sW^4?)PN*}1AO7^(vZ+O-9w%RUWbI1ZFey($E90_V`lBQ@~9xiDF&SN z(8t#=3U&xAQB5TKADwnV|5}n&C!e)Qb4gSSA_;O_G0X-?9u&|ug4#K zwFyGSw?FWnf4V1K37@J{_B*4E!`H%fg}1EfgSOU#7+^~Ac7He$$Ic&%tbG%nymhVf z0k<2x5l$gKu+JE_yJHi+iDH9ksWogaqSs}Eej+vqL!FVG4mQZ}wsrXpjx^tn_vw+~ zP|F2oKgJc(alpj^2Vn&I8|krX!-rbwxmbItCc_c90Ygwb`MrS{rdi^chJI)m)?rzT z^OuEfQ2Vk;_etw^qDW)0S}++$nf)84T9PcM;(rw_Jn_ZB5Tz@ju=%1Bgo&Y4&V8KP zz%>BxvQUMz00#EFz)5*%<*RUzTb4Y7-L3XJu`-2Mk}D=^4H-`9I}0N`JS&N+l$mBX z6b)#|PR&9bqSstZi#Lnc+CGQKbttR6M zb3}nH6bWu_JjvCM5R>zB$wl$-cqn(3`rnpHzF>9N>O;pWdirZfji%5BgB7H?^sKqT z2JEeOzGuInH@!2ck@)v1vX{aCq>*!f!1%>96Yi;5x$SR#rBOOo%~JNDpF%A&Q-fL)p`Hc{S5~ zbqF8VAzUHDm+2{go!!UrWM?>Wg_(LbxuZr?OTyx7Z;9~aVjLLGHR5K}dN((JIun#x zRZ1I8xDVz>Dj8nv$}ItrJ+#x|n9swCw)*^rX_x7K zfvAdK%YA-agq9Aus(vp1qX(4J!z-7E9e(iT3~tt573s&o=4rZ|X55+-&A<4BZk|#X z(Ae7-m5@z`Ix$Dr(?o_C|4ytrpJ3vi0zo?h+rFg05M|~G4VgKsN?2$Q=y ztm8xhQDfZo>PdBoK4H`8iuI}*@P5fn#x%ykLAbOI{V12MlP~k#sb!R^=i=6OXVoPN z$RWFxthI$yKMov*1iq}xU9Gg*dPpOP%E`&l8my=4lm0G3p~F)wU_k-C!O?e{fm=%{ zM4#quc3@0$T(72IR1W!s8i0$XcFW#TgVKt-IjAV)P|bt57!!h>0-XP;$CGLHibVKU zim({S_U8354DnOi{{bIZ%k6}lmBY=V{~Q1$@me3%FfWO#8%QfyrDV@@bU`3u z`#K$L7P_598}S@vN<^+dZkO91eCC5lQ|fUd-yHBc8?RE>XOj!}DjSHswa!W}Z?KnU zj3K1c46xJ1F04)TKO7d&O9QUlz4Oh*TJg;1cf+?I z-_d1+qimjUITUaFMm+K@Lm%@FJ~#eD<%Vz^9+c4#tU?uGZeUO$5{IATrv$O)1sp*) zC-JFBn?4s7>C8PwlOof4oZ1{LIF6`Ak~fq_Bl5y(dw&E;uf)YA5EIq&m_;yQfqlu&4BZD-L2SIB34q zKfHA2nKHjE2=a6@pahe>n`?PNIg6a_A^be-YbQ!{f08}^D$&T5x@39dv3K4t%bqYw z;%rS;d4i+3V=cP!_A3iBL`(O0eN;japKZ|(G>Jok4c1!FpE&<47--nlF|ADU&0Y-j zr7zB#ZWRr-?Y*zCU$s7*vn}GzD)}pV=QZN{Wm8K?QJrAp61JlBwP7k5U|9a=B79*d zkQU%uu%He8U0|E%Z+vOVXJb=x_L4(IX4M>&_R$!Uo_JvV-;D;4p z80ISG^`Y2ZGh#~8j~ax^tRDpDygtnTA(cz&8-pk)L8KMtS!01tO!FcAlL@@e4zlf) zWpA>3HYx)0m%HvS^$V+CLTa^icYv8k#(MtR4}WdxsO(K;$xl9xCTpj%5m+X$)OIh# ze5}+w-}WZm$tS<$zcUPyNI3dTwaP;UOS%xkNduzpERq`ZikO1;3svk** z=t-c?*X!9Xvpp_QWTv`Wn_F(5q({fC!=j~q|GxYU@i|iw6W^*kHKxTy>mp4=w4ub@ z?ars~npxT}WK%M{s(kB95T?sDV<8k)WI#zEnozCmFRW?S4vyLGG2(Km4^(;{gUPD6GfG|KfypGu?%bxMySP>UboK2>(U;6ur;X3kR9wdg6H(@No^k9 z%CLL6en&j)HENsC$K+H-!||8*wJ70lYokR8vG28rvrE-lpM4)s#xEduB+1^aw$A>X zgCc~u8#%y=Oc9MH!zN88<3D>#Hxc#Z-BaeBrH$9vH$2m|IhpfT`SFh=#hQp0_uals zyDTUDW5)U2TiZ#N^P;xPBvq7MQf9Tr?!{vA3-t84Py69bMRXwmbOsoYU?kd zccF%kEbkdW;4&X4QDB?sC&vbnkj zag0F5kL&*0Qe@0JzkdvB1%A%MY&(44@y^tWJqddk0{Q@4^Fc&C>!JkIT5h*#cYQoh zh{n2ksV5;a=L&7-$;wmpFSH*e{m%lOek_eYZBRHK&&1dCdzGW_RXkA|5Uk7L^pL4g5h9g>aMj|RKwb)hQq8{0_vm&d8?3go$q!``ngmA4%nNr z#GLz7`X*Y%hLyP!R*^Ddp)UXGNVvYmV<4y)(Sdc<0p2F3gVg8qaNe@VM4rt9RHqY8 zjw)NXBf7FO_opA>N$D0g-i>oX4AJthgWU`%3FwloS7NfW zklHJM+z`{Sx-yInmDVtn@{tC)u^%Lg)WhTt4vN zMYSjM%T#@wrz%weO;P8+0I_XD`jH0{%9?;bO zI$dnvn{qTirx>cs?2h=WyzCwTd%M3!#kcXNXW^|he72d)9)ymy+{mK#ed>1Te*JRmb@EChM# zz<6o?9cV<_50}%;HJ5QZDua(+Zsp5qFLDn`eSX*_bCI3R(=>7MHy+H-UW$z($2H#K6w8fniI$qdwQ9hAGsu3$R?<2~a0rY>580d|@y0lO;jLLjU`|CFcTU4GQ< z*Gn2hyZeTVPKVB>=!%bGI^R>7s&y-p-7ND~%~P4WqDj=NIeKxC()slGq1D^lX?n>h z;+13i1*>12AT|1*yYjC$xYEN5@s9He`617}Avu6hH~MnaA>ODZeY&clk@E}Sfz`GF zeMLE$i3_h0BU-9_`8wd0`h)^;)K)uSIi#MjnSeR_sxcX&hy~%ZSDDTSWFd7q4LEcP zB~If2AJI~z0{Gnhm!))m%lzo8+deCVab#lLp6g88#--a((C@b%N-b<&@oeJzt>V^Bg{5 z5lr=K&+(TYdRca#8(5uQwn7;D=OQU(L9 ze-5z~aBe@-hMBg2baFgM`c1$KI=|$Tv(=<;cl|#@!r$N;_V4lz&u$3H^Y{ z5}O)!pIM`Tr!AfZxZG1ue^%ev?<8G$t6he~0{ zQwOcWYd?09mVI>F4#7CxlRNrgK{=xl8I{yHA!IK_cZ=c|U* z@0?2&aoEmJc`oU{PL}fm^K0{8{c&s0bkj>8e!4sEsNEXr^WCbTHyzNw+nLXP2Iu$& zMJtgH)pjLZxIuZfU>(R<9ypy#DR4^C^3A?WKAQmH6GQOW#M=u1k>1XO2)5bDMGPVCPL-2VD&~6)kI5--g;DDUA|4=6@Ihs>my>_vBThC)UTCTgQNUA}F zJ&(!-yNw+`yn$F$g*kKVh-Zg<*8>pL&rQ@ve{cUvp}wK%W2U&r&NXpN-T9KmMMU#5 z&dF|zbziDm66I00x3(M|b%tqK+9~E0MCmG=9xx56@+(~oi(ucYrS_4)BO z#QB)XZ*kqNxW<`P)xiA|M_oEv6i8=!dS4;;^0I5o;zM@iZ zLQav$f!`rw%51$Pte_;6qZWTN2GSFo^LD(fc805l%^}`c+l6%5Pf}jn?p4EU@*d9| zal+e@W{AZ@9_fVEq3g_KojT;aDzCz7xWcloNId%=p2p6#c8U_j`lX{QV;DdN)CLLet68yv!y-9KYgNOeGYHI5s8^qWxi9GGHS1?=$X^^yR zFVLE?mux|$Wa9P(n(Vaw?FGzN=1wEj9H(^&7oB5w2OW3MQpy3LaFCUz^zy)GZQ|Zx z^?ChNHr=POwK-Ql(-v)srTI?2`|0^T!<(dC^o?S#`Fh>ry~|mLoGR$@h|2tgo?;C@ zyA;_HWMncjlDSAI0~w+IPxR#9D=%1olW!;uB=K5)OU$wO^yk1afZEUUl{9R3KZ-kN z)dFpu{?Cp^VD-;x$3Zve+bA^D;9ScxfC0w8@nZHk*Ego(6?^NlD~?P*A9lB&>%)wf z-Gw~)>(mYI({AJQP)9EvZQezfR%>GG`>}Yq>X)}22i7?RRGS_RHp|*b9EFimSG6M; zu`RLLfDH+j-FbH*{hsX4X2Rj+P-PdK&U>ko;W}=|Io>_>8q{BE5N6UU+OPD7Jka&F z#%^*|z$V*eieRX}?=U$UQl-Po7ihST-Fzt0j`_tBv>KwKnPDyVYuEj5^_E#HsNB=O z^a%2X_g;qut|Ml!Z&lnDPnTv(7n6N0rl-Q{h@RT8`N0CwMlMw!ZT*g+nWwyUGe>uK z`nsFC$flFJo0()D7r}OyCGYN9(d!TokU{fd$zof-eU*Oj^uluJ+4TXc@$QU6B{eI< ztd~ojT>^CLOw+nsJTI1SXs=x7WMD;$BdfEuaV`9@$aV{Y#Mv2_(t zQEuOt5>OCC8fm0sDCrad0cnsf5fSMangK*X8fhek?v`#rx}=q6=#s9X{@>v9J>UEN z*K)DI<&ycnd+)jD?6c3_$r#Q(nPEdJ3~uTJla-buIecA-4yQ?+XSTpdgu-6tGJc2W zaQvWy$ii*qr$OPP3GGNdba!pJp5X1z`{63tN{;4)n*MN3Y!ShquWRdh!fyh&7W%;^ zpXD^rn~F|iXORC1W%#b#-nHtG@zgkMqu^gi1cCR-vq)Na3JK8cIz5KF73^^`?x)T@ z?Z#kcU528^n#mMJjwDLhVsb*H&B&O?p5C93&agC;H;8=a@`W>}>?)AL+@^OB&{5l9 z5w@SzJ|D{oCbmdP$9>uj>ftyHuG&a-Q;KxRpU(G#(JyjEy<@(%-(Ms-+G`jb$ZmVlhHCZm~|FYR-gNagD+rOmc85~#EyW+gO<}{IcqC^HkzON{?019UwnDM z%y#vh7;~IOnK+AvreuDUzv*UXK~%Plq2>|zcL@E3V*p>D`5_JJnF%J#gD>ozy7nUT&po z*q0Yhirq%BvPP}1p_62U$#z=(JR$UIef8_viLo^mKVE686CJ}hOOy{_`^&vikLTH9 z=17$nFK=1Z3{rbi!etXzonqZOAUO(Xb-gd!5<9Fnid`e_trp_)iBLr$gW-`PQQYHxjr1PL!EZL^{jysA4YQ znYmxu1cI<@TaDB`lLle;QbC%`Tx&vs^1f;~Puv|H?BRiWbsv$u3@4BrWi&Q=N-)Od z%R)x22gvT*fwon zuhKlQU(Sk8GEKRKDN9V4&+hC2W1Q*tKX^cf8goTn&HVJF47_6D2Nwqw;EN6H!3YkZ zou%c+dCC72s0`oCQF`;QmVd0u)c*YM3f>5gwR#q>b;^i5IP~JQs#*r{hQ`|y0iP$X zh4Qa88y_k3DC;W^9m^$DU7L5azxMNie};44?@dy3Y16WB!#7A+fIp@w>ZuYOkTqlM z371#4j!{TJh$|5x90AGzM!-QGU~m624_Dqt4U@cbhU_{$lFI9)u5|)>i5sh*V_{Ru ziN}s@-_#Fhk7veLm`ev5KcUS;C(Y7kCn1UflFWlVC&4duPI+53nqKN81Vt4_`B!{? z%Aea;Uc=Ygvri!WOLYc_GVhQ!$wEC*TqEUAwg1(Jx&c$#8Cp=7aC0X=#3lA$HIZ1$ zXIRl^i8s4tgUoHYHUnF~66sl~{%|EQsX?!$%u*q|oUO@yvigDfUg~w8cz!s*zqEZb zo8HIWb`-;k!Oh6juoX|@|3ad8DbwxMc#qO(=-~bft_D0ifH@-Pppg!#o7Y9^lZ^1e4;aHQ;-|$6sh0N^$)QwH=Lg*WXPu&Al*BV_J(fa{ z$Z|JR@y+=QLC~ko;`-w64!!Inbu!p-=Rk7kST^=tuUpS;`*H!-o9TIp;SIi}U=bjw z9cNr*3{ZEOnISz>h^!OgQT^TPWQKJ!=Cg6*B2eKoQv#)%ts2|eL5MqGyhen$zRO0G zgstz3v*K>j^m?{&vtDoj)8M~xtzX{|1tZ}OF)1u1?v1FCyngHK9?_%|NXG(SIQ|UA zcS@Fx!~8Z;cDTdshZ0Iq{IE9G9zPEBDQ0gjzgR0 zm&*1RT=M>T4Yz)40n{*+NGoVVqmwmUCc7WtA2~}5jPPQc-VNSC)XR&wo>^AwwQgkQ z#)sTafC5)~i9#)5o@4AFO|AE#TB2yW|~77Vdc zfy={@VA136rpzKjDaRE#gwU(QoG3Lz?71Tsl1%_ z0EB|2NKs9e0283ZR}$d}%+sy!YMF`+imi7U{ z(`l!GpR;bn6#z!r*-uT)R=L7QpeeHiuFEl6L|R5(VEZvmt*q6)#^cEX0ic2iMzq$= z+BD|pAQiFUC)Tzedb(J@W~by&0Tc74@7C2bL5MowKPV?S`161e>zFk#d~4vIv4A0d z`sJ@2ar zh=%T2+7GsF=~50lY8=uv?8rFYzGFW0Myz`(r`d2J5Zn{kl`0ch6 zH=nk{px0;6k%Oy1#uROl6|Qi(PynP24wk|Z%yb3Nc5TFv?X9a-pcq$>Kp2hE-)cc1 zhwTQhNbId8g zF(X!XZN-n>k49Z?1OUzOTef&l&TtO^>vs-7gt78AHk9*KzYmfZ+P;Pe>E##j?@%4sETvG3D{l_l~+V`;HI+Z|xPV+K&fzKi8^j zVaXRG0{EO?<@4AHF#6o|ohI}h z72(*RJvF2YAXutgAmFQ}tD_QPM7*{=T@3XEpk##4?k2j3%l=lF=awl|;w*31)U^ut zQdE;U)XNzGNLptjorc37zeBn8Qv zIq(>AJ%3ch8y#nRErU`T+xi@W10TSA&%l01p}pa>UJb8@pc;f8|dUz zHh0`cN8;Oco;t&I*+N3)gS)Mxx0a>Yto{9FaWE65A8p`o5Jv|0y;MDa`;Q#Q@5~&G zxtnK5aCsWNgFrwIf6U-|tx2t`5M1B@&qvU_T{XpVneb70r3tvQSdSae9P7QW4xl5B zX9`B};kS#fiF1G*y+$fEN&mQ>t${v(TQR>bD5{dTNmgpd5-btFp=}NpjeYS&2 zb=smBIc?W=7ii7UEK|9!rL_Z{;$XlbUF`iVyz`*+%CmblV8kXBF%=?abeP3bPZGXlLVCPX|d;CU! zBz}U?Y^A(PQTW*%z0we(lmp{T0Fi8R({hG+zNp1fsbb~PH-f1$Y;wFAku6m_evmoc zsV8y7pu)UdmMvJcNyspYTMuNhqmT{1$MYh^2^-`E&6Xm|o)eo0Q&#~ zI$I8V6*;0su8R-mv=Qwl4hXPBpk+TlbC+&zm!ONOK_I*hzDTsLl7mt}fE>Wc6KGbl zBgUTNA2#lV*8xaC@fM(V-6L;Os z*%z+D)KI#gZ+)-1iPW`sH-Z+X`q>0_6;~`C7>b|m$2B*6vm`_0FvN)VSdKwxM^8o+ zj@pYQ%|y@eb3|vw)J%x(-wC&`&AoXYdJA2ABJ+aU3CIG5<({}{yuJ*iwQeRm~As*6j@q=&C5yGzuff!QtUA>kQc)2wZSv0+2a*i zN0i+@y!>Z<`|JBYe&iHUw>C=Xr07O&#_;C~UC$O|3<70O7=Ez2k|4b(G(9@Nq~C{s zr}6r72XwVSb(J))^ZNUgH?8JwQQSHD#HMd7hVIyWozc6~b{an54L$ZVzf!}Zg0Qh3po9SN| zUii~HK1-}^iMXnETS2vB{dWBM@=7WhB}vEEGD7a*jks)z;g88Ed{}^yYa@Bx1h!)! zrx1Ut8%J#9X)km;X%|Q*ETab6Sc%Wj52T3=qPP4g4^v+NgRsxAp1cNb%yy8Wnt%QL zTdW}9HoayBM5@#L>|4KGJz2n$%>jCdM-w>ot#*iE`^*u|g_8FIq<8KT)t#Cf0=wyr)T;?pf?%U!Ed{Oux zCf*n0zT6V-TU09_rEM7alDXAUK#-fOv}B9- zoJR%DCMO_~>-{X&sa8FrEH2u9@ai$_(A+eoK92_~aCO=#ZiuJ>6rfnoMI$;Jmv&+C zpcEp%H??`JLT_|kBu=a%Bz5jf>DhoCGp@pKg~&ZqzvF|@5XmP1Uf5eo#?z~lX7Aye zC#VMqg2V7VL~a>`xFFXnLmSi_04*102$0`q^G~LAVZ3hQ>Yuv!={`|U1k@&n!kB&U-f619N79 zw*2DoCha;}k{$M3lgxRPQvW8MGLRLc6ipyTr}JeZGiAR-IDcc9yg*pHkNGY_&-%=X zjDY52WM*L}ki@O)eII?`KX-LpJtDnZXfDdi%)7t60bKOrG41VO#Enh`@Gv6Z(n66| z*nRX#e%IE&chbzm7M}ahgP0snRPE_lFu(S{p-j3VRizf7XiiQU!=iuk=_ZejLf2_xm4b= z!nJWOCW`&YM~9<5*ihUR^)H#MJR{GMo)}q?20t{=d3fGuE~f5z+D4^>#m{tYH)f*g(ey=K$?NRNM3BDH+|3-gHn5!!h%Th8*E8<2;@E#e1})rEo%`_ znE4dF{~9FkGjxH#0Dw}zI`cwSfFj~O0l}iH0_9wSjKxnxcLVmG$U>*_+jD8R9RxfM zmi>bBt-(Ir;VkTt?WxB>;#?BU-W!Gwefp^=Q`$5J{yEogC>_UUGIhdu|JCDJ4otd@ z?&^Y%bEh5rk2?5OAOVm8N!rQqg8(#H)ivG*w<{!GO?~i-) zKc2~x*&igA>R8(-KYqJPm7-c9*UuAn5w)~wia@#n*;SGe7@49WM#lzsyYTzfb;cvh zfo~1I(c6njcy6UcxwU?7QjQo4eEoP44n_^DzyP%P8}_v*Z38;ZPRB6t)kmT0r^((A zHw(4*pWIfl#jq;4GCRuBFBBsz1ST@2Kj6G75Xezfc zdg%R?v4kX-Flpy3+^6aRVl9PbM@q(4 z5AHiPOM0hWmmU{DzUngiSE{gCmW(vWFa?9(uz_EY8?%!Hit|_a!s+Dbs?Q-xJ&=EA z95*D>_`Ap{M?<&;+zpt;mf1XW+AwP`rk%Y&gFb*dk~n+VnTP~JNU;GbNlLbP!Uugk zQTO$5BKfcOX5XD#|x6qJ@O>IC{96pqJda{=&E*%1+fL=5%QtZ_HfZ+!P zG_;4i?>RS1Qmjehx~oW@cS8s7A27dQXY*c@qLQ~|^#tz!2r=e9b6=IhH(=&d<8d;xe7T6qWzph|iw-1V~xeboM*9VlcZu)3z!Iuv|1q92E(E zjaonUHur+8ncJ}Wf~I4r1o*#wW|bi}Hj zo)90>&$xWcsUcq9^pb@rSp-bRbq-C8&aGLx3#S&dhtWgG{5?b)KSneu8JxCa8+l1G zm%bs7mzmM`|5qoWLYlgHZs0O1cJqRP?mKU@No9hwo92;Dg*&$URX}^HD+x|;Nz^NC z%C-%zty6{#3CcEjN2&WWGG&TM&Slnc>u6qT<@qG1VYzk}AeDvnu$DYyzA54XEi9wG zx5N$9UafJ;JL%*B!&3+)iT>I2&bK%~1Rbeo%P%yjuM#+0ii(q89{JMH9j?5*vF)kw zkzZIl;%xBxVi4uM6agCZCcCYh!d^sxNTwQdxH8u2(F^>o#JP)Ca2#GY6)K{s8vu_n z$?h==w{tdti&A@W$4eVV6Iai=Z$_g_KfP+~c{`L9XZT4>=b6q8wg70MaaZ~(Geq&D zFG~m@)JED(0w`j`mKA*)=RI{r}!{W>bpwEmut4XBXOjE3$f?d*pbKWO3fT)lK4+IP-8(N_lg%Cso*S#I zm^K*~cL9PlygSRo0sD}5upjAU23VXv3N{~P^I|PO@YYSF?a0eLDxO&B#NB5txgcu+ zNu7Dt`tzfe;{;QjsDJ7C4B}q>bk1lUaF7^cj6jiOi|iq^Tq#zcj0}UT4hDN4tnBD| z?ZfLJf**$fa{=GmSSB`KDS(F{|bG~R%N=>6g*xZ zA*)FNct8oQI%s=R5Gk&xGL|&lJx)YAXeqKWJUnfdV|;g$<)+tm&SKa^ro1N-+x9@9 zZ0#hpI+#CzcR8G658ZvoNPu9tM$uCCcx9TF`SSvuO-1{&<+z*+SFCw}_Zky@t>RF> z^2u`8r-RXXqEIJF3>y5HqQ^P29u~yPmxpoPwPfwZhPm)c6`fv?6j1G@rNEN=N&x^l zSx4!!`q6e-DW1o~W|BniD$g_oJwL_qADBEb!vIceCQxb+;zs zYcdFGgUMQ_5^z-f7!YmZo9T2Tv8{t-RLQPqjWXtiZq}F-RTxAXENHdA{J1&cG?myl z2pppC{cVXa#x%r6OKsgOn!Y{w&TG!;t&w0qmD`X|V5g2wbS>aGnhI}sd+dneFkKh! zp|44?$|W-EH~$AvRlCjiB@C?2CIU7Xrdl7=X1>)!aSm}}g(L~7qi;4B;?wQP zE^g8dmrnha6W2+m{=5XwjodA0Uc9z74zP++PIMc|W!R^R?UZm0L%>tMg0yzi3OJy! zGnfTkHRdJ%VNHcnOPm!(0sxY-O3wjcK|444f)0qO^qT{auvn{^M^9sBKs;%Y-#;Sz za6cK_0?0>5~=r_s~Q2*%G2RgJgQRY#Jh^l5utC?{K#1u z4DZ>Mg8>Z=m(SJAERcx4A9v-2`)fGH3xdJ0q#DD;qrPGXbxVEmk0i>U=3Y;Kf&p(j zzwvKaRkq0P*MXRGtz0`+RZEJXm5$yu{Y*T?-*#|o5Ix^J`X{lMaAI)@F!>gT8QuRt zZJ=kfTRX(M4ck39KDtvwYdnv)S<|{lMY*5wlX!%7;xVS37~n`wGk{0r^26;-0jP4K z?&oqJRY4gEGiv!jpP-N61k#CIsmw-7pg=L*MW*JdzyyZdUZ)M)+3pxoG;&!qLNbq& z3azJO?|XSuX90WLGTA%#)F2U~uWX&4+BS%MmjevxEHe1-+Dju6u~a{_%!qbihGkd8?K|i?LP34s_s?HrqZJ`7V@N`7%RWh`AI@mN# zd!bvr@ctQ!c#sm4xF#t9IqZVDnnaUgVq(|i4#i?eFr95YzVK4%g+o}|a5V*Vg7HLI z?doCY$`R8+l(?5ckXs#PW|FMPNw{{g2kEf=Tsur~759A$H&5=BTT;(MBc0YI8JUd5 z+pCIDCX^24TdZP~_%qp7D23MA$fb#^9hLt`7vAJIOf){+GR3(x z?P!#CaGM^!yh@#9TkMm@qT*&ioBbu4KeIJ<5EX%J5knxA(^hJY$+D6In;Vbp)7z|2 zS#)OjT^1J=ax)wrzcAn*HF2e!(P-JxhMY-RsM-YGQDRPC3Ga!dja+#lw@)}%UaQ@y zMaXZ_uul;iK3quy2BnMnvzZRb{1|YgL!Ob68&a}8ZUZ!5t+5LnMplm)#NdKKK1^s~ zjyCJ~ejytrn>vwj^`8ilByPHiM|XU$G`AIAM*H1f7j4hvhJdUz>_os|csj!kHIPEqB*!ptM6R8da?_=c*dFIga>?cp^lMuG zq$(eg*~0|wsv1kmRr15ek`eSH=1hT;Mgt*luZTJ#7&vt^y$G((aH;P_VI~0pR3FK)*M! znk+kF$P1~enhPXcWq-T)u&-fLBmQAk|Kz^Vs`ZSx3XWN7Ll#6vTW@)JyQ=y7lA-(r zio-a-NEx_$;_Hq@Ph%;CMnSw^|>SlP|G-^7Ps zS;eofKrJ+`7FTxKPAd`jEcw=K_VBl#7AueX%QvcuZ0fQ%W;MhV$LqTrKD{wY_WV>> zy`QhgvHpA73}7kxAwhG^0K=5w`08;>zo@$w(`u(}p_RW5C<)Xjx+wkY%!#?m_$aUW z(>%E##T4O=9gvx9E3xp_wlACOK2a4wj0N2~;e_)R5MK7=R^fMm(+;xEnP-eD)nD*C zW|h1+)8M>RyR@G)rCMa0KqhMYPB4c>GB@*PaOB=ZcXxp*EY%gFD1{nzb<&>;8_L}K-D_$Y@iNu$Q} zB>R4FS}yg&dv`uia}P#HyoyZp@bixGP6^xTtULetbHq!@1^QELr+9Ks7ihTTRG&<5 zZqkJ(fxXU;x3hAL^1WMo>Ds2)!;xf+$sVMK+^uiKLyAfkc8N7acOyZ=xm5o$8YET% z#d?+0rScJkwvSC*E4}D1vRCm#=56pdrv^Mu6qy!G5@b4Kv=E=ae()W`W7;tnC~f;r z`meu7rUPblU2C;xk+$-DWpQDK zl0i&DG0jkXw~M(!T7#$9h9k+dp}9#CB@5-!wN2mYsK{!Oh1%_ySTB@G1q)mHo|-bt z0HpYK=$&yb7B0-poxAI6b8w+~VBWGv^fQDdey_Z5S~kVCLpcT&wV{JydES&{cHlps z|E~}I+P|gB3}>&uyejs;Pw7S5qELA#U1_K-wF+<51z}~FKwX=ewmx?X+5&Ha?>2G~ zBlpj!JfydIbs zYuMd6nv;_!W_HBhEb)-=oq{>7N`FZIp!wn$u|{(evbCmI@DiDDw5+gW*uEyl%IgQW zS5(UT{=d&Jp?W2iX3IB&|M#-oJXXt7yq43qaWA-A?+y0aou(OK;@pmELTU{*cg=Ne z_MzODt6>A4U?oICUrSq0+jgbA!cZ#8+cUx~4u6#3jR7Uf>V~6rq z`|zbMcDP$fHdp$d$c?#GU!o$G z;ym@#&dY0<{=t7X2(XJs710+`=_;f;FG@;Wa*)9-nt4)mt4adl zMw;@IMO9K3pN~D?1(5kI)9*eqeQf$<@#}d}J|jPl;2`6b2B#yxIDVhNTdMqP%)32N zlC<3Lm=NT&%{ArG+k>v+ePrqbu^q-T<%N|UsLN?!e5qhV);9j+md zTBU@0Ld1!mT^_z~nI`55D2Dc!vMuE8(9%hE+%d&0*+VNS+bVmgp&0u6`2Mpb#?LUL zPbB$}$Vcex4gR%{vfmOus_hNZ|QC~D{;0*KN zy!N!k`Qp|a5307EGj!MOIgOH}UUYaZ>z#$;3gOT`5HvzxV)ZxWPs+G*9dH7la{c5z zCTXs~Dk&=z$o*>{{3j;S;C<&iZaaxEzS-Y@JPbA3OO>>^qM;Vt6yhg|Fjrr~yXUk! z2Vb)fQL0BUNqp1p8HavEMzO@(j94QdrzDkq%^ryE%@iBY8)cD#&@s-E#B|1kFdITt>7ul zE@ElCe8=Cv_f{ShcXSJ@vEtJPR-K}UzfB3St41WLi-ozr`@&XB(YtpgWgvyDAE5(L zlMSEpZ6BXmD3f0Frx__NSoCxO0ZF_=dB7qmElx;Ob{nj|v6?{3SHkIWnqPuiyjdt! zTko>NA?l^&l~jV-I@sUNr9Ht{`oSEO+)Vqvn&;2m{`+=N2|zOMv?k}?{^P|X&N}(M zZ&*dAkvxx%6QzXXEc^`bAG{5Xd*+^(Svq*DH+6#-adLd!(OypHfaQRXtmtT0NKr_q zt3aD9b-pr-ek2=mv6it3SNlF~KTsL&6_0&!h)WUzE!HPbtQEkX@l?k675PzfUy5>7g;f& zB;CCP^aVVRZ7Tmb_YCiS4Ap}69k?HD(cv`bDS_NG?2X`&uhq-!>u-*g7cKQ7so}hA z3iu7qrW&u1u>r7!5?^KxnNw{D+meF2yPkM~aV|gGZawceI_C*VBoSBX_*QoptWm;} zAqDq8p4tHM-P5n+51|u1;4_+OTXb#WMt^72*&s1j2Pl{;C?}J!a-+ZOEA(hk$pH(c z+@?(8{8Z>L|Ik$8_0kuOa>dTicf@=D_lT>@_&8=XBMB@suGRgq=Zw)YcWI?S1O;nC zhEGCo>lzNvo4KaN3A%q&KiYSGW^MN?8mP-qleKn{$aFv)buzT|((S24HnC@I$S}VV zuR}>bd=pRGr7BtVsV(1{{UEDKm2PdK$HRz9RQ%ssTRPk>`{BmYd^T#ghy+hoc4kEr^4 zpK;zr6TpXK`HZ+eFV6aNg#wX4w)r9Oh{emdrA(pcFJ04!LEpNR%Ta5FZ?@QNn$)8^ zs6?b?rD0zN+q-ia$xZ?p2W*5eb8UONl)apV*yC*!Nm=2MREx4Q-if^aF#Sl6$f0|I zO{A+@JJ*gMLZvz!$KpGhCZxSeN5K6`w%6y8bhy24T+qtIqU5N2{asWGQkI^#J(Fc42+7W}Q z`jE5b_AeMz4OyrMH{K=f>^4Ii&BL{x>3yiUB6h4GjzNJ(Ez8$%MY8>K zj+$#Sm|Xow!@ZA^M)F7Z&7K{Bf9(ZF83}*2kFA64)CQI~{&Iy}Dx_z4Rf$A^N*g?< z0sCrY*r@Y@mpAt1JY$A253nF;$~%NmEPdC7_w5LZ5)2sanhAmc+*tJ_ zmsz*=V8g?7b{&cXV6eNsW|`eq$GGA&P}wsb;iaLH)1*uuM~`S{2lEVyd=`0{bBBaD z#d*>KUe1rCk~C8uTV%oSGVW*d%5o+)Bny^{iUeM;<{R% zIMwJL>P3!Ike&q6Rwpu`aQncTFe!V)`p%25pys@>F=Lo0byZrVv((p4~wyt1wHFqeM!9{NMky> z-0#!==fD;EOAwo{fmeqPt-fiL;0%>5scLoR3v4unwh0ClfQ50ESE1ezi``>@wpCq- zLu^fgIt$QES#6v@C&nC&O7mfOU?cAU{8K{jeMWCJa*zHAWPmFcM})A>2zoHHCexn$ z(<$dI^i~UXg#XoK$)&DEZjN)hH{e}DXavj>2Q7w+HxV==NHn3RElW|*i2r6QEtDdNW`gKm~E#6K*~bUY@`NcEa`N3uUS0%drwjWee057TA8*#?B8$M|Jkp z9^e#Yh>Y&vYAf+vX1(FV4u9VRj!JkWA>xw-Ay`UTsxSlvwHkC4YJTo7RJcQ?jJWnN2kXRkI?YyA=F;NC>d-`GTp}2qj5Z5B#%b6am&1(-n zIOCqSnCW{*tY|m!eM%|3JcwHg{-EsO2Xh)f3DFFQWvT5?5$`ccapFE<3@@5*i=IY( z{;|6uyJXei{QCe^NK(6Gzh=oGUoThwB1zABtD9<_rg$W{J3VDs zHo@s5vslkpsm%>7;+%?a`aint758%$ysbGB52|s34|RZ<6j-_;MKe>l^xl#|j+DNE zr@>4%k=6`1pHxAj|<}5}WA|LNLcbLr|vv;wOgF*?6DHD@i znjn4nID_E5!K$hot@pk%ij{=~c(EIbJnK?KGgU|A^b0Nr<#B01jM277r|_(5 zl3nZVS9=kYz%H?n82765r)} zf*3>J1AbM}v$i-~W%0{H7(h2vvh8#UJl%Z-Q6tU~j4NkX* zo+{RoJRsSZ49=JvHow$tmNWrjnc~Kg!MUoovO|4RXk-lXM$Vp^(~+iITvE@jrV~`! zv?f6v#EC;BDNjpKUEp637mG6Mx-d6--x1U%!j8j$CUx)Gb>4YC^|A#h>%bG6F1V=x zE=YB=LG|%+h5g3ceLLe9JN2tI%Qo|^Crbj=K(71BWWoQ33%RBd8N#dS~HLqx)R+?LL^@^;f$@G-xfJJw-~svDUza?E3Aq z`gq>GhuT6HE>e!5IvGs^CSe+Q*?;FBEg^WuwhVq4s*Rk!^5}r5R5F)-jN-@_eSgo7 zt@t3_gh0Aqd2gx<+RMI9vN(8ckz=87-JDO*YF5R}Azo_pFmxGM(S;P|bD>b?+>omd zhdDv=`y6s!X*jn#n9P~SlcJ$hb?O09bM_%pBG(PLvweYZA*uziK>uL0qEQ-z1O=v( zv7H_4Gd?fPK3}_N#)PRU5L2tn_SjKqPh^ori&*v(S7&&wjcI4`4qWl_htD8+(lOYY z*;89_t9COR)auLgWqILi*FIDGmN{IQQ{baHvHl;Ty#Z;RX#c>6OEA<l|h{nBQFTlLjn9EZvePFx#do zWql5G#|Cflco&<3BoervW=fB8k)ho;T$*g+waFKM)yOqrJ#-MXXCVyx9`0Vqb$IKH zgaLP{k2xsZLS2lVCsfO|&^>tL(Xs1-B@j6zc+e{NR$nX06!&(;$*F6_p1kQ#DXILm zl>h;^Qk<|2Mjzh<(5WcG+RK~e1RzMBiuwBhGB*9e!01kO&CIrq)-(_(*%5p@8&L`z zW0LuWQ}?-}VwcAKUXP249bkH)YS4Rg7q);G?)qf!VpY8Pdy|(3LOLa89Wh)$j38nH zG53;y%V^&NV3X*I02Ezb;K_!_dT1<7&+Bv=flVMNtTVrSjTvDcDa zxA4Rt^buoS&*NSh_nwAV?T)nZ*Z=760io_6Nj1S;fcY7#v6zx;ldae0e@Ds?T;y?A z4wgeTOtMD9`|I4Mp6nz>p*6g(l!-y(Bb;%Yf8`;Z9B36QlFMd?{MBF;psM;Q_1s*8 zyEtcg64?>iXhTKVgBQA3i}MNru0q*A^>B6oAmSxbT*mI% zxYKV7akRhK>)cVt;Lg5Q&@G%p1Ix)+2d2F$NB-6X&O$s6K!THmc` zqn4lSb^N0paA7B)=lTc8a)0a-Ou*qF4bJOTNYY?vOv&f&wHoQQGw_iDpRo%mAWbvf z+#p}^<{Nt9jEiD>k;)Bj=NeGrs%iSy$uJHS($Qi)vew?`h`Fi5l-WuFHO=KR5BLyjfx>^te z{W&w2kKV;@1K;k*;m!5yh0?WXw@H(^wS(C$qju`XK^uQ;ODhhPX1jj$waMh+ zJ0za_X4gY3M~ak}3VIKC=o2T5pWv1uze;fuZ;j4bl8 zy$qs~i`jk%+Mjpu1yM8|eS*?M&U%duS5yH?1!84xz&=1JRlj0hN3T7Y)>QdkTh~^s zlmzj_)( z%!0|fzlr1*+H~GJd$}ZQ1o)gAUn~22rFW2=Si?5qj@OkA zw`Gi7(c*1iIu%wDv5>B6)LY^q;r|@3-7#;X?A7m2p@Gmuob?Z20)D(db)T&J!O|>h z2I~ZN+-OtJ9u0SmylFN4*nhxWqxsKOoOI81nM$erW-~N^Ah_BOnl_^ zOEtd<4*dbrcTC%DE-fZIqd29gEv1ehdx{n+CE)0S^Q98D|OB? zi(;;B<2?0lbQ0p7#!mW0EfG8H;#qyC?!JKKY@z>bF0)b5sml?^-SM?lqB-9ATQ}fSbp#?=GMK9xepqGN zb9CzKEHueWgT8dM`aD&^kFrIl4nUJ%^v^G5i;QeP>~(&R`UrhJ;x+7UxicKvA^;et zy|B_0Txn!Mm2B^~#z7+{{o#wXy);W6u1dWC791xR0qN&5dfXc$+z_-kP=4juYbH`b zfq$sifQ?R`mr=H24g8i9k%FTb{v4ufM=LgnSqub*FNKlhP zhYRd71X_4WtKy=nN-SG+bFqX{)HiMNADN?y3FZV#8%lgA$#sn@!a$*%(`MRO0po(C0Ht3YS|`HsZPukV^t1p7noxTyLc#G|7K0uTSz z;2QIFLw6Uhu6gRHcc}hbyzlyG+QfSyZ7iKlK4)b;_@IoKe31^eoXe?dTaf6{OHgJB zYWefR0_$i!vwOMFyT=KheWI3*o7uJL%qVpM!=BN@`KQunJpM~VgwWyq8WzEI;`W$3 zSASHsi0#7g0qJ~a!<#P2*iV0X3@=rEeiKpNk-LqzPqY_BR+V^a<*6G?=8aVsvwvN9 zZZUFiy;mL{4!jt86n+10J7r?ixBkIS9H@j;c}~yjWqpW?>Ylt0H(BDE_F5kp`UPnF z%lI6{9K{lMy5A-~bMm6BgLuW{JW{kw4!OW=0|}`gYJyhl#n5dCK_F;kb)cYC8(04bC&%PW@d^A z@dLEt&AggvY0Pi7%R^(NPs;TLLe+)3311mei8`#X9`f1-reo}UMq>K$ufsy(`vy&* zv!owM<-v*m5s~%E$SFyf=O=#qf)3U}UMO!!%FA^52h4~cn}Hwovl!hnH7zo5Jk=X{ zliPkB6~I?_NB}oQp@OO44qf#o6*31XV!7+=hFst86xIzVx!bW`}H_7Gl~&|esDf=V#Y%@ z;CO7-#rUb`h0wuYfkYhb@G&_{o7j+|c7dQrR~IO2PoKM)afgWgu@4tb1_tS+QQJ8B^<(RlHGAF0jvDTJM)Q_R0i()me@)Bn^RUuM#?Ttu{g>mB-s z0I0uiW5pp(N&+36zC_y;RyC$q>^$;}ep*1G(tgVk=9S!bRZRj2j#d;NVOVyG(MU23wM{8!z`8QO$C`RRo}K0*rfJ< zQs~FMkH6#L-{diWdSH&-6;DG3FWadE_De^4h-@lZ#%0{rec&Shu7MWsvJ2+8@_Xkx z{=N&hGGMrY#GB(3Ad%|$n$M8sQwlaP>45{$efO|@Yn@`p=#YwsGAP!x92$?z#4$xN`iyTfK%J0 zP{pi4jO>_}xIg{eA}8c9_3*OJZuG~JefgbPugp+l$GSqjCyzSLbHfjKZ*B@-d%Bij zTGF#X`3_ximrk<0xHA>QXga_9{y^u5ni^Fc`|2yt^DVfnPh&pzMfw-$N$I8O4&pgq zhx;58f|<>7QO%kO+akeUgRM!-i$WVRSA`ayG%A*Q6K3Nk5gn>k*eHSOUI`~>kEuOA z*9y$!+_rr^pb+l73j3HPoqtGGGP1=k7!G{Co2?kr;M7lPKo&ICqEWsJ+a?Qowo<|pLJgu8BTb;>Lfr7j5 zki0%zO?DK_-PMpL{wsvi$x@RYtuKeYoCJk5+-B7b2djc$W}}R6P@7{?ldWHL1Y%#% zCQl{2F2s$nXQmVH$j83!&^4OqMFhoCY?R5w(X6qiyFDrF{`7ig)-S@N$Ttdr+#@BzUog}|& zkBU2GahuvNwEer}0)7oZ?i47nG3CiuT=x9568me}TevaV9J$J#2`Y|c#(lUG3?%W? z8kf?9C14yfw=udNT4m2q454vhuQ^f#>ffNfE8{4mUEYVfzT}qiAs!g373%mus=hm( z>c9OTNeba(9Argh<{wj;WcJgtAT6DC(uS^@L-y`>7qE{mC6!nA`F}7 z4);)1yF+rG7BIMZ7Gh2#lB^n5znJAYrqTI54Ty}NA9Wre$oAG6KP@C)iaY(Vme~T>DPGgpcGFt7?ziF zez){N?Eb(H>aN=G=esIaH)yg%IyNqcgKqXlDGt!`KRc-_A2i?s4&nT5&D}hXz}*mC zV5mIQer_4s-}$1Zeqkz-4%ym%@{}aX?w{l}?t&a@eQ~0mw5bWww(H25D5vB9ly!ch zSK8Y(3?IyQs$E;d)oWLZ?=v{Ue^Q$;g(B9MkgtPilcb42%g=Udo)mqG5a?AF)||X` zjI5l){VkAgST!>xzzk3+t^tlY=tpxjLX<3WPb)=5X89p$}Hy%aF?Q#{Z zd`1s9^jW@i0G)D(Mv7QF8ne*`Pbv%!rTTafoT zac(KN`2yTkbN82M-WuG<0YI>5h+`-b13F6&XXb>Jzh^}XKaXu%;iGVSv!LL!>_1!z z%#R4@UiOgItdIRk2!JNtbVy4wadA zv$+q~CF(5e^XKy0))QQeJ8zWC7>O-cKs>cGf~10Pgg$-z{yw{XU-0M~EpUe{ z`O0m<_-CUPS!eFeBiYkL%{70Fu=zF4e0v+M= zV+CV8%Zxpx`XdTKmvzkdHcUOC7JFS8Hm|cbpl3Lmc&1ffkgslvKyxGY)3tby`njnG zk}zFC;}Y5uZHo2kMFI|*tS$c;?c|(^Cz@Se5KPoQv19VJqaILtP0b2DpvNs7ZBY+o zh9tD2l?)c0cjP;A*HXGgxeq=7dmZ$r~fIrW_ir=8?Yr=A1pH zWJ+VpeYwkdqg(5FkNdArEW-GsiT|s<3s95VSPy=qFyl=`SBwn~l9C-zggp z=t{eC_^gxk_k4$`tf|_?266$?;kI&a&(`r!19-qEa&pdh_pB2w3L|LPc=nQ(y@%%Q4>_Ep9X*QF1dzrjX$=!weCz98GQ1}tJbCKV&Tm)9+zZL?& z&t(AD@PIU6VZS3&a6MUdp*;K4I*s{<-mdd^*96Rk)q6dE>3$w2;#4lMB;-hkc{PV7 z2~3ZjxMdeI?bvYMRky=yB~bu;4|@viTVGn`IW@Yq44A+!^s5;=T@N#(^K;2p3m=Pu zz+LhO{%a=hEYVFmaQdj#BU@r@9FNTN=WT)>>biGL(~Vv($^Nb5X2iO3DIV6}AGaud z|NBPL`np8KlR0;Ec7i&eh~LdR)^brnYOHsae%H}lUBqE@D_ zvN#6aXeo#mKt#yCXPEmJz2jmG(sb=}B^vjs{?*4m)4y1*o>kpc%rZ=oh7iTkQK z<`HG*tb?<#zI>}zj05Xd)l7npknh<#n0RhjIcpJL-j(A_NGP#lflZ*oB@bla|Q zc^Egepw~X;Rw$q#p*GgM=mkejj}aZ2PazcYs99bAUVD&ey%-ur?Uoh?s zaud!j)L(ZS!%vC#iG<5uB!Sp+7I_rg`$!k5%qW_>ODYG(3BRBZY4iP#6xB~7CQMt~ ztBg@AIqH$kO!0+v(6!?1&`@zA3q|%VzIj|u(b1E;ARS{SB~~k>N6w0@(oZJhlL@UB z8N+K%8>L9MYKgJv6=%rj|IN>%q_1`NH3;+;7GfAVQpkcKhM;SA0FXBr{ng4fBlh;0 z!%zT6O7m@y=XUoHPgA?=SWMpzd^%N-fjimCCF{eN_1Yh#*$uyaA18zjp=^J=W;<|6 zQj#N8keAspUQ>eE`3=3gVb!B#QQxtL4KGflUbzL#O7)AN)y&ZD`kg*pQ8l~h{vPv3 z`EFsLjBpTbjVR4O8TLeD zKQ7Qx7D?F5h3>o`t}={T+T|tNzQkYU`bljrf?I7uBVLUb@A_tbofvJ0eWtGFkJG%4P@B~G-eBoGiiAC-CCKJ|**isMpL2G`pF%foA?duY@Bg{zgnA2!R}9n)u| zZUK{17?DN41o#UOst?OO%wPi+tG=;|uD#!oq}1+>e|!@ju{Zo|f@O4qh^EFA-S_U1 zcre+eVpJJPnj!+nTqe!MJpuP~5|8~dfq>S$(HqKfs#+P~>Ktl@heWHa{wFG#=Rr-- z>_g!d^&dGGsh(y2$5%nivAhQANve6Zv|LO$pha!erh0Pfz)WWAXZhaQL!BOZ8X9Jm zaV9)}4?X})JUm2T?_Uk+sj%jR0ti5;@=H3y=+b$fc3*srx|oh!SjxDraT>pJKaFO% zhr+tD9Z%U67NNGfayL1GIyyr6N2a;N@}aH#{hGpxuUqL!HR;0s(LvoAk6v06pD4+J z9899pB&Ebn+pBA!9(qV;nO^EAAtx_lWaM3kS&!fV2e&MObho_IfVLBKt(6nt6t6fMh3C=c{&7Kol!4$V#)&Rf|*5iZ~`Z?npvh z(X%s+T#9t6;2NPh7@}}rM7-_zlaqzSZZJ&M@ISW3eY0>F%Eq?;4N&Y*Z6E2hs{B_d__4voCz9+0Xt|6xWdhjXcp$ZN~i6EkVu7nk;0*jE3!)~_?$g{o{4s$nH!E60=>pE3E3NM0}CYC7+DU^YPw1f-_ZCMUn)5UU{&@~HHxW__Pk9VEaNj$fA z@#kNk>Z7@-$k8wvwqyiBv-Z6^5dVh=kG&!+Lp&8}yXReU5mnk@1CSW#Y z_meCC=owyrbD~{?Brm~=`?L(-kS6UXhyp?hiUGX4nV3=HdP-b#*>Bb6m12;=BG2pa z&_j_Nn7tJXoLG$3w~C99C}f-)#KvQxZ|<6LpYe|n(XCJd3weRvPl~ysWU;OVrk`+4 z!G73un!S!0Y8r60Ym+Ng9Jc@OKVA*np18@lCMREmBT1XmGf*GQ=vsE zZh1M)L~6ySr5$;m5YS`X0;5pdg9ZQ#gaVhMWTQ_BwG}pqh7yWumEXiP-s7YiuZ|3} zV_@LnhiN+vYO6D3xenT8G+&H*$mPgyJ}3jHsFkxYHDjGcxU^(i=3jW31a$YDx8qTh z0RqA$J@3#jgwJ$=E8EPGI9E(%aw9!*B!?eVI5hRmCGfK4QDRq=bGGIEC7kbv3SCPD zg92Y^NhIse#)Mgt zXL}bv`E6+UYvG5ANBSbDd4uf*gUo1nJ|_E^vsrE}RET|pU1sZD*CBZq z*B+!S-ukNO9g855G-rxT%=1mvkLWC3Y?@T&W_3;9EGc=YoUz>y@+mN5NA+}S#qP30 zFP&bO!Gy13KGZQXLP7GO!nA1-RSTErv>LGNS=kHR5Exu{yXq;|BAB;@&|5j<1 zxfU(PPc-THIPa*+`suN`-8#N#2@;)hQ1#!(A}chjLNjYN+EdxOlQ7462Y?T=-u3t7 zghC`OrGcwWzR?$`+^1fK@-6`dWN(r6_RqoVYF7w1@27<8$$m0MbhHjX1_m<0yTmaT zxV15^)-QZALkYE)Wd}n?*t=&@U}}0WfT5X7SKA%xzBd#UMhpCR zF-$}lS;iy9E@N|XDWV>p6N5&ZeQ^({aVoVw5W-EjxyljZk(*X!bx71bKEiX!Nk=VQ zJ-mSMwLI{59fTwnmV}s+-f_GTTEP1teCCp(Z#S-Y|GhP&#kKvyo71AP!t}avg^3iK z)Wm%!!)X057qNh%8{n?~opoh7cQWZIW~h~l{ALjFldUWH9MfbWvw~HW%TzRW6Ig{W zF%5$C>g-$EV@FW#m!AbLhBsB8JN zo$Q6wyj6SOvx6i%$sQKOZ1|w5>H4xU!sT0yGJB=t<9jHTSfin%Ejt|Z6`~W^BJ?l$+*uC}uqzpi$CwO{@nNr0{wE$>`%(B8JtWD91YyC8t8sHXKopfJq;T& z?`Z#{LB@EXl)aJZy#DX{t`8B@RJeoDo6C0p#i21|gokvKMy~-&UWCipf2HJKw?K^eJL^i>!ISSa%N?rkM+iZ- z+r*KOhGB=}1T_H4Vkw19JATOW#D2k!I&=bPM~W&iN=NJ(I&&BCsynT=h?tBGqy9G% z4L|nr+%WVasU9@kI} zz~r>uuls+I{86*E?gSjQ7)qi^_s4mzhdFQ$bul*W%pdgS8!#a9DQc zAs8xFoE1#syHTFG;-YSE__pA<`T&ODQLMDf+WmrEkWmoUY<=3Wej?F;;%tm*hllvK z9Vj}jE@D6Oz)t>Ek6j{{fM~6vUieKt@bPSf5L|;Mtjw0&#BUOrPxogC%jjVe?9dzzk!w5l{${;c+fNxQZerLqB28x zcHf_A$zca0`>j{W2azLcuwG)xb5~}0@NW$*d*pMV?A{n@-D}ICbs~l{eCtNK!bUrk zOi_>5=l};M(ZVn=o9xuAmf17vJQ_pvC4q8z$)(G8T03>XYdRygSRP3~HK@yrk;tpbnZf4qnGT-0gT#_3O|XG3h^S zRllU}&zZqoE}Id&o+_{puEiYhzqrCgaav>R4nndzvr6&RwZM%u1eXk2tOrDSmWqR`C{-0JuJN+;+1u%ytE9*=~cOHdGz66h9IY4w+V2?@%a9 zw`QEuERhQ~Z8iXAH`7->s>6*@^>MqT)t)fkIr`L2VC$ly#BLeZceazlxJEtuM|9&+f z=OiK8__F!_ZRUJFN>bNVzH;DsD28J{`$M(>+j{&cqrHrh#RZG2hxbx2#J zLl33H#c=o|Xd&<874(P!c9T<%ar)6z5=*2;awO8LKS?@6Pja%qQ;^E95*hmNkO3i^ z>K439r-a326cz8Er2PEaBU_d205=I=parjaj@fF2Ew?ZcLs6a2tM_>ffF9}J<92z< z01*!B+Yu6FtM2fAk+;Dcvm^? z0K8MXoaQHDT4a|$z`y^lu9De`s8>>vszVtH%nA2u)aPS|A!{!gpM)hDg;pQL&yRCzPg1GB;{3ZCuAy!yb!V zZ(p9MnErC`{T`s_vhI8C%~qo>@XL@mCU5_-#1PLxk)*6@!2*V6Cp^=s$wEXSX77R1 z6sD1u+H5QAl+qQosYnY(F)0Dy;42OyhQ4`9iJv}SkMq9)#m;k!;dkYv_pjNyZ~5XD zhVBhEv{hXXpuaeCo1ACzR>;RmgSe7hU*I)c zk@p|tqowYL0%-sRMf;v>TFJbZ@Cj%aDz$A^j1M;kjmMReQ_ zj9i@uY|Gi`@hohzdch}5P1w^=ZD>rO$2)(oD$E-Z-gjKZgNX!>dN43{-6qTh`S8<~~)clyaVr?Qb zq~^aQCXw~06{JDevY_ymFZL+t{=o~OjPA0cr>HS_D1oN}7{#gRFiuGn=QuUw;-?9uH=^k}S7dESH0Z!yy{oPr=?hUL|Ml2X0Tn(QQR-~V_>jw{okwrY63 zoD;pK>@G-Z>B!S%Y$bk^I8KHv$)cuh(Xi23SzN~36?Jf{b$Pe=|MN;PbEYSb?|Is_?Tw>ZIv8z%esAetNI(VbRg-?|Qx3Rj4Cx(q6)Hq$%;Dss|Vl z^{HHu6C1?mRie{EA-Pf`=A=h}4OqqzpfPDJAamVa+)b3H36#EQ4i?56&wd6&Nxkto zat(JrY+11G#q5&31M(OsyNU>2WPc$1IUmRd^H?{Ud)#^nDE)f@nGq0IEm~gT%kA!m z#DbMh-35)+N{e1nvQix1iKuO0>fy3BuYQFVAzAr_@c3YQ$FyR)a9C3V&&_!%^1EEq z{8Om-LGqJ%L~2uLu_e?dH_hY4Y))vC=356T`$b8!>1d|2-hu-ADJ}6in;9jpJ|lo3 z5sPO?n88Q|+m+;lMDis%Ef@^;_a3(u;IRls;xfIdYG>=%1W(ccABL;JUw=ESl{r4m zITY6{^wb=#)sh}E1#%uU=73R`u{VLsLQZo@`r8H70E7m#KUU zNW`li!P9?#q4dmdWdDU1@fp0u5a4V^@FErA4ib=`%uQ#BCW%SRLjW!e{$0B)jgM^5 zmBMQ#F3$S@)Sj)e2R>#w5mdu_ots=N{sMfhwV5bB2NfMJ_(gpb{2%QdBlnQlg_+W2^9~Cb*1}Th3>KD?!-QCy38R|MUCG|RxoLT|d)y!kzKdZQ0?f`Ge^O!d( z9?Zjt+UDh^9R9vrTDFzzcumM}QbF;rnH_x7?^Uu+4JDQF53aGF+N{uABXFEG;QR=X zcy;}L@(i;z5zPmSlpd&IL4FB!DhoX4J=H?ZI{w%Uk*g7)9;=B!qj%hw%Py!6bZ4S- zpwr#M$!kqWM0e!orRe^@?bA6&aY)F!3{a&-M?KM;1(I^J+`fF-1K zCD6nXl61X{5ILj$1;iQ-@2-2CDF~iGMpBx6uLWY7spf$k1-I7AYYixUdiW6Ks2#kn zWCt%^z7*zNB>ZCgv%?lGdmS(0s*(VLQ@Z!9lTa@KtyOIr;8MMS&XV4Q`M5Np26RRIo~yiqaw#Z#y7DJlLK*|PjvfiC&& zLX>RHE4;O823klI({)pOdSR$MVj8E*5pkxQ#r0v5Q6!*=v?`Cu zg9p~@QiPlymyFawrLl35g^EG|DVN)SDpK^ViQu55byUS#Ldl0A<-zczrf2`M`Y#b` zjGQ=Waiw8*0})({OT`(VA&xbtbk~Q!dW#RD!7B->Z{G>^i)D^)v6fypXl(}6(%4@Y zTfe%Bu3Yrao%N1IAAg9xz)Si-VXiHl!GOPbo$JplZ2Fff%7&xBlKDs1ShSh0dI7)5 z4G8SSBS}2*%9Cyn2cV?^fl-~R!PY?TGA^1>1JGXYuD^eUade7^kA&b$W^jZ$1fFL+ z+F!;*xZRv35Ct^X@f|fool)arzp`aXWLj(cUo(%RyWqLs1fT`|^mP^4b%r>v4$8>x-SOY#pI%W(gUD^O@bf&$Qttx zTJP$ldJPyi85z7FXYicz+|yN(ey~BQ)2|9IypRolD#K&~L**%@ z%g$b~$@X^l|8eqx;L`^7`ehqNmifH!eZOq0zzm*hD~usI4l)BgS!9pV;d2*2T=T*4 zVe`$0`8^TMS5SKq^@~9=*Ca3E{{ZVG?qPhiH@HE>72mpmb$x@r9)WZ5p1Z9;+3vvl zv7MR4zD#b@HzQUG_PO~Zm({%5>e#rZ&}u{z6 zDA^rg>s`qF+J3?1sS<4ahO$f>Pd)nU)W~qnI7j12+mdk-nRch;NLI0cK?ne9YZHsS zebj1qvs<&*{b%_abGxLl8{?T~fQd7)$T-d>VO=-&SBEm#fz#w}zcJb9wNW-Y-3Bdv zkuK+BgE27CKACZPxSnxhe$UT8yC2-bi@KFp6{FPQ98gq-P$C=A{*@)JY!4lVYV30_ zuZ}83pe5{~COjOTq)}uwV0uX@s;fM+?=RPPwbq)++0$8KjqOFMCcB+#6xvRE)G)FT z3Mfa5a7t!$#4clo0c|3ABj4fwXHxpev3;xG^EqO6YBo9;6mTp$iH|3$J$-bH*SA2O z@zGM4_Rm|OLP_m&aQvZ;F^I&6H@TVMTHU?E5C;>lY7aq@v%xSg?Ta2TKucE&Xki1a z|3lUd_39UZ!=}$hgt3?G^-t|^uC)Zmj+y9pv`#C<>R}unoFaK&V2pP&*l0}=AWd&)MY&1&%DdI zE%zt7dqQ&ea01*DN(F3t$5FG>o_j{HjO}^teYHuNx2uJ+ewz z3a}c2^wyKEND9vR7Lfn`IV_0y#AnhY6m|bbgxF|jXWWViG!T|(_k2tk0%!8qQki+q zXx_QDwk5&YDDl~x0lZ*2($1`=4_;Z(z~GJ|Oc6=wNXXfPM7$3qc-)&`K0W_`s!`%sR`ve1GeoQ=I=6M7k=S=R3@MM zC)1ETm9Pvu;jdA5 zf=q}IHQj5X_Ds4R3eI)Gackf{05m$fIh`MU0EGZTgn;Rjpr?;46S~@6cOp4i>qg)L z?`->?BI5#hu4OrKgw(pkb){3bW~`n>MqRxW$3)2cNJ3?-Te2huJ4Ii&(7j|sM_X*7 zO|QSd_EvJqMaquUnzF!PU?syZ-x0LV+@0zrOcLp9_R4NV)R9<>j1gpfe& z!<3x@9|4J!@UUoRO;(gk=8(4@iER^Oi|Tgf+57nQTrfbscvy zz9J$5Tx}2SrWe~ZF}EJ-#+kOk`YR5G_x!LN0Fe0i9H;=pMO6LeVyx#nA>$q{+j6qB zH)lo*PUig2`WQ#QOx$wo52ZHWj<&7r2Qn9i^!9Ds&q#cQsW5wc6H~YoMqeKG#+bG4 z;*ZB!eRhDELT76)(oxF)0t@*ICpMl@o0(qhc|VNvcAt6tZ;>FQW|+Di=~pWi1)?k< z8*j~k$FF_9S~wpjK)YsR!W>QUX85t$V5_Tum*IUE`w*aow{hTF?s#iDk zC7FupNJ20AdJ>Jf>q;ccQ907qy+!fp18<@IEHJ#S%lUZa$=o`99i`@!JyS+KFex;u zJ8*53QcER1t|L8S*zq5|EGaI^T z&26%v%|1W>dQ1@&dXeYWE-ZPagW(y;J4m~tG+wfk4g zs(<8GgDMAqY)xhuUI?A%QV@^BW=NJK=H&*jj;{Tx-+3s@v<`^-K)_f9$LN$<+WKVbb?Q z$(h(}KAGR;a*V`Xa)O~dIc5C^fEFf2k_fX?-!AEVZ=JtPwMZ(yax4z`Q7`-Q0AQ2g zaLfAyW*G<%RsAavzz<|63P0EKie@1q>nVB7rA<6088Qg(R8?O{?$*awk#N3F4%Lh*c!xrI0$meq7T1C5U==WaHaM$HTWiqF zc{j9pm&V?ZGAe4gA4hi}9yA`8CH+hwzv{yjtzc?Bz86^4F+GW@ugFTisU$|%9`o$n zA`zto6h1^s13d$^FrZv%nxVl2YoU%UjNwsI^yWd*cY9tS>(x)sO;DcSUnpSWR#Y06RNP_b$83O!@VaI zDX;FK5~N{u0~~A=`F7aD$*s%iRAcNh>M#fUP|DXZ8y1*XbV`0YFngm~~cKDEY+w)~5UmeW{h8FcGf)&d7Tq zoUbkyH-bb93TC(BU;fXx3=nTu35Et{GlbT<1fMLOMILCpMUShUU?KZA;#Cbbr`lZ0wo z7O?uX$wS#$RSEo{z@!AMmWl^2XF`Oc3C>6v?%eX&+-7kwTuqx{6!+^tj$td;2*(J< z#5G+v`F_%6m8T{fl1JOUQeF(q`(A$+o@nWvT~2F2t}H3J!nhefwf0Xz_Xr^u6RPub zh7i~_VYfzX#9sgO6h8&w28yi6Nk{c2ELb>-AR$3R*PKkr(hE*hQZQG(Np)kcUm0)3 zzuD}>F{R_8j&`zyQCTqT@KH^bj!TqoOw8r&BE~{^cyvMZGxIl5<$o3fOq`91T2d_v z^>Ai42{v`6W|@bI((c(eS9e|*IblDdgZ1>`gX5|@9h_@gL|Qv_ZDR8s1%f@Clbb5#STkdRTKF%ZmrClvwCs17-?6R_h&!#RpPY9d>uI{(Ts1g(k7F4i&3 z&D$N);5Tzu%GBFL*$l|-di_It2gSt(h3NM)b${swd2o<81_LL3aoD=&69k;F@Fm964TE-^%EOQrm z`ZYaU9jc^DPRrrX4Dm3jTVlwPl+HlfM+rc-b;`mdx?V6(UhWNIEU=LKGD8xkj3uMg z1BOf?M;}{@$7~kg0P3t|Vz0;gp$~2^1fR$E1tRW`xq;7fG$-@-KGidQc!dc?MBOsN zPr^2(2LazxD8?){(1j7y3=fFn+7;T3^+us~@bSpA1uy+$)!Be0mo$Dr+HvWcm`_RA zXPxLlxDE9_|D;=Rbf8!FN>x@}#;EB%JXQYFJvXRJO^@BmZcV;B2!)%RQy5cFKImTk z=Mt%^qB6@@(%5_>F!h?CGclDGJ&OS7NhjE7*F=C9)lG@CSFsZ~K(xv&G zLh_*Pwo!+66d&w3zcP%)T`!+fBerD5=czanHXcO?u`fJt^VMRGj?*gyw)h2yLN`Yo z+T<52BgYzje$fiSu4tFu|5P7D6GFlAxFWlj0W6~jZVeI}d{n7gcMp{Tfmh|KTa4Km zic8-bHm4eJre?_;nO7ZTx~D=xkN^Jid5bP7TlIL#Rz!6FX^NTn9ishidlNN*U!I$M zf%%o)!bBm|A0mqW5e*j|yxpMfs3{SBk4tqAo;rL78sQPQlFdIT60(@u;vRm*A0e5f zv7c)ask5{450_yE2nlFlgos!;Q#{F|hqrkY@1ZbhOgH$#Li3h3J}dYfq;`wSd&-@s zK&yCrKS7uFN}=7L8)ut=yv2laLYwd87-#Up%Iu_Zx8RFBMZFW+bjfXksr~<^Aj)oz1y@UjC zXt#TPYXSuzZ#87EsD1{ggV zlMy)0-ax6n{DNxRz121!KIE9{&=b^;R58q#q{3z=V1YKEdN{np2=q%D+`jGr={Tim|<=S@Raz78Jr~s3wKT4 z${-q2tMh{R*M~xLehXfJS3jk7@4){BNN6b9>Om=A^ z2uSlF-+B)0e0$6{ZVlIQT7*ld^w~3z1Sd~1&07CK%;K>TfMd0)kfr534qQ_}TZJNZ z*#-P`{d(^MJ|Zb(#4hagn;iWDdu9)&TC|opq)j}6Q^BlEdsN`yLf1iUU0&louE4v( zqZ+HuB3(a$=f^?3Vzg`Q@g1#HZJ8?Mpvj0As3@@a<kBWvW_*#91<7Z zS2`juJecrmwCeVdRzTPBkAAO;a@o;@?DJfgu>O>$GdkRy@g>PxtcB8pym|U1>GI^#IgYLK&>VbFL+PFv@LINTO~9 zG?!EZQsbj1%|{~)bfXNqu^l*CE)&f)`56VB{>UZlH(T_Y#I@Y$D?F{m#A=&Mz61+F zLbM^LE$s;Lu(29Cd#axq$tQrp|G21J!ZzFEeOVA#HiEUWJ|TFozD`uwbgSK2-nto`HvM=QS(EsN{H}I zst{Ct?2gQRuIMn})1B)82kRLiMG9vSH?3JUnzhy*Ano5IP}`t=F!Q)H) zyU~7>GF2mZ3o!Jbqp&~RfV-3XW|RX!q`?7^wLr!swz&BwUnpcPyAC@}XVzaGLctTL ziSJ34{Cxz9okXI_;gU2aY7PAT(-uI%b7ACx^clj8_)bhW{GQO!7rjmoFB*z47Rh{y zYSL7SD`f7!ec;Ae^`!8CMLJ_O7Y3k748Az(frRkBTL1{7l1s)XFe)!#&{=krcC6^E zoNh_ej2#xa9~@uHd&(pQ=G*7cp{-78UA(nM5=lBRen5lc-Wq=4 zd^V@>=vRH(c3Df)Ph!z@Y>2^n*^HlM`D&|9C>IvcUi@Aua;nXjz`}aVl#F%=Q^4EY zxc@3=FJma~|1hx3Xs8>vfBgpdE2@4{livG-%Am`&InXTi8YzxPj(i$!Mc zpRev@z2*s`Kegf_R*i}1Wa%-DX=_%zPS{Q0!#~M})we_oTdXJVJ+6-*+s3#C5*d2Or5Bf&(M0B35CisbTghr7;&OE$``>Ei zW8Eo`zQl0HQj&_i?=EGdmCseko*OX1bd^`Q8**RwcpI=p-DcX0RCD=JTC(I{H=kECZEQAvy`t`g#$s#ZQMWcw9GpjFshMr#ekP z(Vy(Uqk+8Jzse^7P0QtlnGbJKQB~wIploLSZMW!pJDm_vBU%cbePKF-#rP$wx>fcR znM{zH+x$9+Z{0nh#?oBMPHlY0Eg-zG=up6*KUk=|qZOqmWfyqFY!aEZfG$AhU8i@0 z@mMiu3O8XM;7?GEvQKrGv~^;*>Fs?eXzuQB%kMSPq!dD^$&G)jlAaiZ_nH}pZArEk~GWr5?9uPhiH#ksc|4ZvtknuAyoMb;r9Wn0@qW8%75;#al_fOXlzbw)XmG2d00JkED-Z{KS*XG#Lm!pDKub z%5xycn`|rd)9)6cdhIG);+TozRz8|XyV>vA<~EGJe()FfdWQCa2?u@E)j7TgNmq2F zNqEgvQ;_d(NZBg>Wu!DH$tCq-z&^nUWfq6!iCs)>G8Kv9 z6Bg(TTnu~^v>Rf+I|lh8lG+`)E&^nK^baa|&B#Pl{SNjRm9XXbRT0o8G0fSJe%jlWdU1{euW5B;}n^(%6*;9w|3S?*T$DO&{lc%ik9*y1G zmclm|uU0Yded((czA;kElSq4|^Zy2K0A9I$vVYF7pDSx|WBq@giyuml*KHxm$oD3G zceF}VnO^spXlB*LJKkQEa{of3AyslT(JrIAdyV#uy?(fyy;0w7rQG0trd8N#`46B9z`6yRP>n1)axycPc@O%Pw~+GnZEx zrOeC9vXiPwk7|v+UU;k(<*i0@Z0v@!0`^RyKCq<6oZtnL5dVtJk=oCNMuyF~*+b>` z`%k)Wv)3Mcl?VHNvB#i4^LavYhAs?8i}Ds#Gyt}O{xzcgM;^DbL?5j`GI9M;o(lEM z%;zJYDtiP@R6HR6qDca{EM*4x940cDH=>1gN!``qhYIDt1`-H*ckGw{yv#5Hd~^UM z<_GJb4oXc*#@!C$lYqk7_m4CVl%s+41r{{W=-Vj&}&(vuCe@uOKT$Ahn zH>Kba2~p{m5@CRJBN755L^>4_28?bPaZr$!9571h4#^RslyoyGL3)ht=DCNS@9+7S zmoj$W*Y(MFn8e5RP#CC}?yCK_dM&2FCEfzulpC809RY92`&d1Fm8?dpZ|UmxzMWwAYN~<)R*J>< zn-iUjrsroZ3sfNW|9jpK5=ONvUM2?X4N)t!nuJC^lcl7HQEdL_-l+^pm|se6sG4}= zU|=*zuAN(bM>BPJTnn9W>l|A%@DNam(De%T*HwkU)fQ~Ho(WOKB)QL z*FlmX_`{3esrnjxqHO!Iy!zVTAc3O^NQ%!01T_R94db*WAg2`6Gtgd-T5(%7MqSpcruR1Aq{r=BvwK(36pdbt}`uO8EUwIu|u3 z#ME@|`8)$)cC&*O^f!tOwMcL32PD60=Gz*vTB(}=Hj>h#u9ZHFyR5Nt-~Tkv-!K&` zypM(bRT8^K1EP*48Mmc3>lVVIeHU48cmps7|C6PbPN_{N%S2!LZL9!%rI|$93L>_; zb?qN)=MqY^N=96=JNM!~hnGpQgMQwS?HNN@W^N=N#<CP-$4uZ11DTKsuq)+KHK*SPfn3uNg#)vDq#faaPLeX_1VA!Zcz&ly*TlLl zFUeR-0v^?m(hrKQ60SzKffD0O9iZ23JbhSG%VY1FMVsKUlZ}$TdC728r?eA5D4BVV zH&=GKR>tzE889|~fJH)w(*k*}UFLE@Cb9Bb=lI(ld(E}I>K1IYwDiMjPj{Ey+8z0N zE&PvlAE*O_!%ema9>#|*5c?iZSW#k&%-XfV8|jwIs^_0B0BKkDEq^_ehyh^QsN0Se zmxumkKg-_7TY7fr4?md~%F}t@RrH7#Jf&M(n4UO5fy*;5mJ%2EU0~O8`0*oEul~KR zIvqEn?e#1@vrvH@huT! zqfk5Y#qUR+I(S!G(m~2XzXcyF=b;LJ+Z4l+JHk3t<0<}dgn*m0v_5R!pm~QFg%J$Z3;kaTB5!6Q~!r4|Yviu2yeyOPsmcn#$`2NDo;&4Y=q4Y~^3* zhDzN$E63t4vqKoXSI1{aynXLz%B+NcV<3^2ia0+0N5Ds@dD>%4M-RGW@W^Cu<=@-9 zoR*6NGQrzpvdzRF&gKYZ^mDzi;?Z=z+uv@OsdiM5e%Idy0P1!-fL4CBCm>GSJRqZ! zdGsH=g>b67KwaD?13*Vtqfb(pqYj}K0Y2Kg;c!+UC@Yhh_kT!VEp#+VU1rEJQG^S+ zV-#d*6|`(vO#%K_cn?}6;UK|5=L8rex5SfuaIrT#c$?L$Kk6Cr>J3C6SueVl?|WKU zwr6q&m-nkzt|!Thm^Fhk-G8-f+XTQb`{yhyob0lcq7l_t+Fe8 z7a^h?s&22F`mUuukct6b%8F0Gd*dmS!NPq$+6Vfy6J>&K-fxSkw+G|l#gO!I3<>Ur z(8S7|Hml71#vxEE=w2AXj9wk=z_blt|0&>}2Gk22waab@y;zVPFA$r7w7ciZa1kpS zYR{Vl5a0OL4`|><%F;Q^D{vV#uCgSrAx5Gh7+`;p*`QxHIR-6;>eOFK6(*?%j}cj>gT&1#Av)~;fW=~q!j!5?^^jES{W>S zg~bz+Ig-}*Axd#Sx|1t1$pw51Daz{U;F?A550gVbbJGv2jkQNQEkZPi^>$o_!Z(U+ zwRw5L;eX7Mm=rQG7i=yXX;#{%_s#CZ$0vS!6C6LMYI<`++Y^AINE2Dp4Y8!}uypFL z{F`y*8??*vt_OCBh!Sn&Jqv|5zgY+E{AgeK9%l5CvGe-jE(ZiG4+uktQl^7zy8K66 zt5+by_EKfSRnMCdg{`K~j#sQ{<@>?kHE&WnePyGDQKVnQ|MhL!55jSIyz&}$pFNQ! zOI8ae_EE^eUftH?^vn0ExC3PmTtJxL3Pq1s;jnbf`^L>{w0%gW9l$XFT?1&sP%18mFOa)*W$sVxJ)`+Ok-CHBB9h2Fq^D{f@Vs3YC=cvgR{L8sq8o+z94*b^=2 z#?Uf$lQX>!65k%e;x*m!NE(Y+#I|kOtQW3#3Ibj;J&s9|E z7Y&k>yNy2rx+&NhOmei4FFu}RHMf7G&f9;)p_E`1(enlxgDon|v&aYOCTbC zJhh|}MO{#C6ncyGUlOJ2FT8_$t_rAo02Tt4T;RoqM7IQyliuoqLE-D=47M%7JPQy? z%pfRr-nkv5BQD=vuw_V$GttX~SsU)S^Rv(!wwO>9_vVqtZ_911No3>Xs%D7J-V2HN z=yKIeN ztWxkFi)r3nIw3mUPLryXy3IWDxG(x{b=GvB=zZATBVs68dEO} zh$(;yGNR?mY6~1tT3?8qty$|rgc1ml2S3Flc&yEgC4VP+3#OSjf5HdA_FwjC&SVTd5MUr4{Vls6 zF8w6zVDU}DY+FJeS5sMYw&cda62`g@*n3ce4+?!*M0hNKTEp}3c?c~^1EO|#c<`D(oR)KVCTiXs%I z7S1h+950w7Z?sQXIOtvCsVK&I7QV ze*3=I{0JfC)b+FoV4U`#c_ngs7>A5`($_*4%94=eWWh6uW9h9H#g^V$rMCG8V2sIg zuc2Uf#~GsIjM)1}2h))cjuqG6`xLU;Uu%*96D%cJjD@ed_Gb&JpKb7&%#x zo0nH1Wsz2h^bucTND7%u`{c8D{1`?9QdqkS`lpi7L}msz`coMLNvG%;Ga8-oW+zo0 zGLLCffX@APKW#xYpDgWe7R5!KNTY~}zX=4Vs-t1uHCT-AlLn^>+k#TQ3??HC#>)*<;^>?9THi@G}%W+WTJIW%woLJ#{4t<7Og+`ak;h zmnBL`-b5=VCWFOoo75+ zo(8UdLE-So2$bfQ#M6Ne+Fyh}P9Na{Hx~S=UqvhVTGLxfZD**OE&o*C)OmUf(#vmO zufAPVz6vuKW}o^K{=x#rMV7zU_*-k9?Qp#CSMj?gncF3zWx8iWDs6~*Vft^(czv!A zU@2;NrqZ370;YrSbKE&>%5`s_W&jR!Vjzw!?>(=$#|^pr`HZc)OInE#5j^pqiI=J6 zn|E?ez_9pPQqn~z8$zN-r9#21K+2QcBqdJ354psL@2mT1IK zXI+0neCH{fjFPPZjRsVuo8Cs=#JdZWzSh`JqJTO9{;1WbvM!wNV$>#Tv@~V6Atxic zN9@I+JSMZqhb3+RJbSWeD+2fY#tq*aZ%V&*%$QOO^WEiFZs0IeIR4GuW+k`^{{O!< z4c@V4mNn zuKLR2|3j}vv?LL&tF_-c0j%T3)&S$ULVKrhqtS~qV7{s=wHp&%3~)1RUt$?FMOcTU z(;rIGFC<=c(t8-Q-sH&eb#_}$w$cR(ZR@N%x8jdt;f>ki0Ntr*z<2q{7}j=fZQ$yB zz-X_SWe028cK|qtlp%rPotZ=uByINU@~8+q(;0itkrvM`_E)Ll&+GmO#4kryRQLqkjYa@iks%NVn9^ z#TZ$0j!5hf@01IoLyLdg`661^BICx|+ElZo`K6yo?KbY_j^N<;m#7Dz3e+Xku*t(V zqGg}l9Q@vy5&R9LHTX%E{@!Bx%!A?cHp#d%uUC%B8RQU|(v@dz|Nj}LqQE^5r>BeO z5~cZiX8-oVCTO8_*^F|5@i+y(1Er@m6osy+_|++Efpp7iGfEK;QnFY*g4 z$8CO)5h}C$gS99G81;>Xa-^@~2l$j`-}vu-DVo!g5B7{65Auw~1d|NoCUK(rx1-M$ zb3&C6!pVXojbwvD43$^RAP}NURnuVgBXgu&-M<~5I_4Y6Tx>;f14T82>GPG(3Y>UL zE28Na=$l;UA=FXV^*8`G?u%r*upn7hI1xe_YZInl@l&?4^;msq%w7k78Jhg;cB6DK zSd8lSfUw%j;JWcA^Cr$l_3D5EwSufP%LY7kAv{-`eepu`PS6!qpaE;9n;7cWrjsuN zJy8VV4I$FSU1(vHbdbUbgK={RUpr&Wy3R;HQcgW*nkwSd+lDAm^VMfQgOUc||mAJ-u{$sRgTb zB~7I)22eb&nozj1#AN`sV-09p^P5k{%EouemFR}_H?b?IQ;smKP(bbE^dh@R0d_5- zo0saWa)51Gv7{JGt2Z6ZV&*K?J8TVfwF@fZklqO$^IA#MWBdo0Mw#hLE%QegkmC3O z{OrQ7p2}x{!(HCk26BOZOP^8vyoFF&b82t4gpNfLU?=iD`Tx?6Ih1I-B@E|zRAaO$ zh<1NBPPB4gQZiohK4Z;h-IW~yUVTW3_v)i&YbBv{Kw-U&vSjKcQWkJ}R6JY!E&W?M z?C35g&{a!OY5KZK0NAw0xjQ#%--r%?k~Lhi{muJxK8;8|_SfHG|Mjx}-u@)*kBl(P zyK$o7MDTZeWvPlZlS))AGm%8}VLcmNm9sn8+s=aK=-=!7#V3qs$mk-&T`WX3tK1yB z2u+JiLcCWPsm2*ey?U4ii(+3a{tp9Oo<=lhw0@bmM{qjuO^jp?2z(hujLtjNQuA2) z>l)?FA6Wl48>ekI+g9Y5gf#ne}mCg75mk>WU$ZDBWl z2i_;U@UM;1*`28KV?$uj%Ah}kR`)1$UYfUV+D#}c#@bAdk#48hVR>z*+yUn+X#We8 z{}^H?L~@bYZHgwiGh1M(O>>6zA16x?;oF++JAHgbQ< z2)+^7LPn4@a0~Xq=QEFES=+wObRKNd!3enO!i<5(&bpvNq1kHsZKHeCeXyLR{Ir4$ zGT1v#c%qRFC9IHhB%0Yz1JYDfN#Sn1cQ*_JKi4$T-g&ttR7x;$@YB&u=thU4wqa=c zIXgCXwB)j8R!6V!aq;WrHYfP<_EkXAp`krjb1;{vZF*-vxa-`X>be*&unr!&JBXy@ zu^2k9@);6%&^KFqr#LUh`5;eth(QdTVM7g(81aj^uYq~KqdYR=-7u_u#y?+tfjX+` z;PjHErt@3i_T2J2YQRt${_y>z@vgOVz;X`I8eK{lvy$FRUc05?`4ncpH7HM7GV*_DzBGZCoV7qr`@`eNldT6N zJcVHFn7%N}&((Yjuv{uFYR1~3`A2Rq#%YwMU#wf9#Mg2i;x%Xwh8L3ueBU0RHT;qD zEfx>OdbF%2{#Ub5MFlW?_z@VSlpL!)lY}<0yt6go>^xsLU$)Jp()ipoB(5RVv|*Hs zwMD9!Fcs|Ey#azuvH0Lf2?o1pBzC~A5{;>+9{(HA@ft#144+;=GCssKql|GxuJ5`E z*+(;4GfCA4ATFyJ;oj(a_!@X)_`K*+qyb!vyKzXhFFG7{+sAiS_KGBAgUiSedPaCBC7wtpRvI>+_Vf>(K7HEilv-~Ro` zdO}(-%kG~Qy_Lj0i|hjqVcZ^nNe|;ch`YQI-3pUgT50`dCQiF<6K(sPq$N*yEY^0K zsk)>$;M)Qscwz)pAc^xXFhMzL&&R}8Qnh#DzQs7=VfTM1!gH*;!I5{!Re3uXa`kh#O|vbkRNTj?KK@qq#OgOH z_qlG$8&{WGSIyp7+fEa}^Xg#WvU%3=LUKW~tANp!^v6{&e zLVPsk@HH%he>pE)jK~W}(b{jofbglcHE^qVXg=|C$IZg&Ml~b9^R(Piu6Gu| z$L`mFs&$H~eKwnI>4&|L&G+$nUi4V1{1-CwUoEvd$_t3Gsmfp03~ed!jg%*dy2G3E z$GBi;gWGlq^0F3!@bTOQWS7-=_4_PVp0zhqy1zaOj$7Y2i4F@t8snb*cqsZ;vmFqf z5zo%`sH@O36#0GZQT$|gPhY+!twokf=v81e)0KvEH)2;^N8?PP-h&7XS+&gQhWKhV z@bAHEc*c{LneAzU1C~=hrFz{~*;7Sj>zxXSTBcwh0|#C)I!+wt18b+8$OEh9?2xPn zXh7e<;{ZrSF(9pL`vF^~Z3L`Z{i_rGcQppk zKqkv=g2p?hhGngky^Ye1Q(O^VLdzrEz9i>ixnv^~&um!5G8m9Kh<7-9!3@IVwDF61 zUPH;_K>?-U*~w%8-p9BPM#C@_P2|?{nHUA7-tezu_mjfQcT(@ge6b_Q4Gb{dcb-+M zACTmWOn;nrzt=?4<-Qd|I4+E`D}ANfnx_2wt}5HlZoK2ECvGQc6UdwKL0p4C(^Ao{yxjynl1Lt1EsiOmWsdS{vpB6SC^2Y8Ns? zNz^W@)^?#%FFRY@!kxOznd{qbPAMo4fT}J-vp6_;dDC`zUexaWhf1Of!@=CmK2WBk zC!EpO$kHhA)J{)gq~%c2^8Ljs~sg0KsvAkDi$=wW-FrLQf)Pt%E(W zFR+bHfi`Cru-4317e{ALqR#;(9;2h^+iU;^E$dOUE|rsJ3igs9-UqO|9XzV72ZGCp zr&{=Kl6!@5*L+2~u!)hOHZ{=qn@16?wkL=-hb+>e*uA1)ZU_-Z+~uufO;C5EBRi1d zbBN~7vM)8B|C*krcI(aJk6N#xrq%#@`A1QIaH?-ji!V|!t>6GhbyPn^F?mjSx9zQ z1$iRCRdAEESFim6FYT- zBINH8j$j}F%8P+w?DBDpaA$w79(MZkrTycqHug#Q54v@Uoi>W2^M{Uxoj{Gs$au+( zJ1opCCri5g(m*_;%`?6tdgpFE&%j`n{uY&Gd?MAak!Y3$=Zv zn_N^QWp%#Bk7#u4nDfGRdV{~eAsCNr`54k#e6a>w-D9eBJKcndP4;Td~rv}kc`;pR3{FUy7Q@rU+X^=1iV@g}I+{m_0b zB81SXkw6{2gm<`XAcF2}08T*I^7R%j^zeY}DV@x7XPZ`e`Z%oCk9R(`moGYXPj&g1 z6+nOqz_Bn|XeMQG0&~V3Xx3@a*UXp_2I)?7!R?6aoG+8m*Tn4XawTh%gJj0|l^z<3PMKju|96u&wRp)9eA zu`RKEY_3Z2!i=2i1^uPos1xbkI$p6-*E*-!;lL&P>yo2}0C&8;%MbVCd#bw*4eV^f z+B`+s8*IWk7u?mfMg3K&g6Xyzp?!nCJ&%FL{5o1Hn!|{!7s}v#^Q&y(N@=?#WlN2Uu1(-{YQd-W~x_pzxkh+Aw@uZ1{unC;3D5ub>PL`hNd$xNP zhI=N)LlMpdEi}W%Iw{(=8RK#$KBC#8?pnHyRSW+TW&dvzxJ2Wvk`U#Y?I;@$51dcb(al2xanqa{?W?ZOR zz^Fr>i$RebCEw0jmIcMm76T>mkPdqVUQlC<%98K%ahkvxU>f*>l3^Vz^j5pnjbqFs z0lklA$#nWu7RNW68Hd#-($lEj;kHTc@Yyua@GZYDWT_!!oIzD*&mT3d6jB1=ClFLW zO~gAy>c+OafkR!)x=S7CYqk_|ikTf5o))K}PpRD?G;YN7Pvj?8Bmosq+kvD2LG=fb zd6sSodU~5!w~XTZxFz@&{tA4jq^_DfKbEwuFSHFuRXLh<%M}ZaVV^SC+KdY^1f56G zNI}yVxzImnpoQo$Z6I)N94F{KP87}U2pk!hcPe@^RJ?X@@t`yk>4^oq+80Sq-2%W> z;^W10lJCnk#BF-Blds+EWVA4h9iI*2KWtJS56Tf_K((Wfx_JP)sW~T}_m63guj6CV zjT$F4Gb9At?XH}~Rh@=k1nmtPUKk*U1)w4fc)hpiwnvy{x=^M*8=eWhRoSCguTz}| zqj!q%{b9bK4a>y#Km*Hxd(I*far^r1aNVb>&G{RrCdH9>K30I?>n9x z@7`D>rP}^YrZQ+`KI9U=+0nm{5efEljt2OKk;-^~9qC@Md&GL_|>xK~y_QQA#ZOphy-6eQAR z7&QJ%Uj*AaO|WDV_QJ$7g&HHjl%8d3GQO-G&|S#OUWw2yd9S68Qr^f>>7I6{-A4VE zZpsYX0Z%u;O8xJk41n`QU%39P`Nq6LRCV&l7b$uQ&y(NZ(l)><~MfMFonE&ic-NyX7s(zX0E|> zW5ZeZ$zRxSI7k>M3W{PrD)1?+Nm)RU5ZT;8NwaREpR?N2Fb0ZBtEyOm$4IsKB4V!}0EWi~pGCbXmURH_BEB2HX4^`A6-q1nrqrd|1!)~* z2W%6~sqlJ5yz4&^S|0Zn6ngUi8P#(-d6VMN<^klW zl^gmaZU;Tx>~Dav*%Fa83C}>qRIHEDQSBS2#}C78)mM!M*e9OMR^K=++~>?vT_`Z& zZR&ry9b((+tMb9@oi2R@`6Auc@cow#6u=9w0mS2}$MH*Uh>5Lzh81%3jgsP)97ss? z$xspX|5Hq;rE82TGux=)RlxDBW`Hk;+E$l7p$l_*Q?jJ;f5jJ{6oI?GVA9Y3P1-x2 zna?1kp^hwh{`|aNkENChX1#G@7R2aJvX!uz9SBF$8Qyi@dKO+riOZ~0QTkjPg}Ae{ zH1w0vNmp$zXkqay%8;Ga%It2klO6slusTVayD2pe3Vy8 zwW?}#;VT51462>i+6)g}7t#oj2sxeO|A-w#nfno0r*BL-_n~{A#I|#(R@NgeoD z3jZufi>Yy;jw&gi`to)^C`k#7#s^E^rTttJz=NKL)_~pJ-U8DZqX?X%Uk8p5fKh26!ATHIf&X(1iCI~Ut+e|xLByxNIO1{eQaac-j1C^P^PrWA@O(Z! zsOAw#9x4B?he{F&Lt|SNBW^N#3go`RGgI@~i0e}gNZv@a|L&#WUvzb9woreUEk7>F zm1ZYMsE{7V)hZ~BXG}Og@G0BrTbfN0Wc7+BNgtEfeIwXD;d$l9gapF#k{8pQ%nAE_v{fktAS3d3#rE3c=)0E|!} zKxkA3SCTiEx9?9B(hUd@4zyFuq>2_m|5K1Ink}&QqTbj;U4hr z>7o|j#|AEy^YvAhH9eMaW-`=ZB^|1E_%v-nEUm+#yUeUo2p}gBq%7x%$~=Fd$1uaa zQ1c^)I`9<{U+r3e`4oe6lto7+$xR#RxL<^>xve1xFU6`(8unpluCH*X@cZq4n}KgD5*?7Sl%kAB~+?+~4<<!G^Q&su}?6DE%tDQQzvTRh!INr-ywcz^+rBpz7uUMBcZHNo>g*cPP(z zs+2Hrx*3n2@a*{3EBb9!uMyMtru#eJmPo%X9Vx4yc#|RhN~(&`j*PJDUbV%S?1PW< z=B!{vT#HHPqVyaK*d#zD?9o#kAbw_wgVe0q!$g=>a z2P^np^G=)A!{9Gg_6dNXKVEObx(Ja;n`D0z#nD>5sQ5?~*<1=)5@8700)dU84`SSS ziF%E2JmFgGM$!B;QG_$jcG=ZwnUCAn{)mP{704u@v&zWLfbAgRC(i3olTa|kppu)1 z!Uvml_T&S6Qkgo9D);%-xArqx*0MIv$sEI7h#bMpJE?u2#f2@MEOI>;M?cjGdxV86 zA@Yr?VEz8|c;9tE<@FC@+p{*2sfmArKG3J(+UG0I2oLWNg}70vm!VvfELDQO(b4RS?FA$+c)mEW-0~Ls{fHS8J6C zk>iF%<`EEOj;g14U(QScB|m@fvX5G98NQh!)t3?sZ#bBTRDBLyIgFaoMg2!VF@Ho< zE_`>=uR|Fc33~83;)-G+T(V8q#KQ5!YCfLJi@4dE_S^HioZX+&OP`@UeWayps(K8* zMki1%j!37j9=;BCh9Sl%_Ly(vlIDCis zKy1Kx^kIqE!OyZr}Y%F`@<2Qv!3KH3yjSd3?SJF+~WCDBcwyLOWv<^3a5T7^{4a8yB}YZ-XO@vJQu1flSi$6rQMa=|n^oD|WYxJ!!1?YXOiPG8Tx zcZg7`Z+L0+r?SPuZo-)cSLW_FhnqfLwPce7Mp8Yq&yVi&tDjZa{`0dt_rdsn$YH_P zHM4HiXjRwCBHR3%3Rj>gp^ILAzu7T8vzgV~vFg-|Lezm`%w$JKTH{Gk$WObe+m1#e z6u28RA^YmKg>~n6p79C1rGoRhUQ8`QLEFh4pIv{ct&9ss!3tFue(p0zDtJEfXg?dZ zxQ=BwMEet@6^a5Y9o0`LQO1_<^4`lL8MvA`HGB$oJ~>mf#k3*h2#Sp%Nj&(@4OSegWR|YB$uLSm~ zc3?mhMPq`_<3IrXQp(Zv@g#C&K#OcO$gfH=7L#j8LJY_iy=w>uuQ19w}$a5p%@zLJx+8Sn-Fp>8K zbz#E|{h$CeM}%`rM5_J(uFomaOAxz)7&?(-ful z1!Qt}iVv4an!`e>vUdclQrcp?$V5T0HK?8|(scx&xmZ~A8oI&YeVuk+0ObS(Pk6gb->g1gkT`tzSuGP-Ru>PsT zfD6AbCu?ii9})p6P@^v><>|+d1GAq~ld$j4ykCv${Hn>ZmCP{~RfI9l)cjt(1g6*J41muY)ABj< zLz(Q*FAs7c1@_8Kh*Twv$Nq%pC5X-fS)MA+P$hL}6>(VE{M0E;K^--j3Qplj%sF@a zja5f#>0tTTkMCObjB+L1I-@6J(uP-U#i`xJcfH=l0+cUNLx>ysMJkN#AHydLO@nMB zsr$dIM-+@Ea>V!9S^0DbsLCYNwkJKo)t~I)8`!*^R5f^aU#*PL@M}nHlR!6Lt1re`#f^46d%Tga%&^WfM7TD_)d29 zQCxan+3n|x$=qETL$9RE2VV)3=-(AF0nFBJSY*9hyizcB5pc&mj11xzy~vDS(>7cd zKSLw0?9Y|@<~81*q9kF;%fn!%dPldSHlXhf>OcgZcJ;15R?>8f+lln#PUec=Z+?XD#2=Oi`q!MBSKloF>^6?NP zjdZe{aP6ASu{raa*=Zv*?!mJ4)q5Sh`~Z9((~jjnPR5Pct>F&Dy7MVJy>}b{c+7;- zx#wuP!@+7>HsK>43thiL2SciaA$MLjMuJ?LF(d9%G=a>(a&|>|Hz=9*Q~Zi+L7)DP zvB@e+L1})tvz~a3iLeQp%`$${yw>iuJ^d0#fdA4h|Iwn7mB!-%qYYK|i@UUsxGK0S zxOj?sc=hyvEC{v~o>`b7)ulnF}_ih7E@>dHcr3jI(NA+A%YdX2Ry@8nb zw8nBMC8vo@xIGEXg|<^JfWIGqSAjFCKr{;I;EN%t&q@D_W=y3sv0il#>)+K*M31S% zpTGb~yy%^P-k(FS9yJ-8T7mRs4xB$>%9#L>3bsJM#Jfx zX1!5n`FcULvc{hEX1pm%FSRlD_w}|ATR}jH%I#gXLTCTI<44_`k`B!rK`PJ}o3vH; zlPf7wLPHS|vLv{XcYqpNU%Z3kVFq`ng?GG5&G6FtU~-4NtiFhojg1QTSf@aTZM~`? zurpR-vwc0OSQ6vJGmSa0a>{52fOkDO)0KX7!lZv)$N6R>27cJ6ya*rnfWy z<_S1;Ea%3IPnP-(dA!2mV0<|HqOq#6^t6UYYCc5rt-f&L^pl*+^bSHv2I6?>wZs{F z_}}Ukql9zy6r zHd=lP?fdtN5(B<+yI<}(%)9I~b?Do!a*O5>#7H&@bMx*TJbrw zhy@Y(7$q}9$Z8#Bd(6m3)j0L$H-wEo_VyKrf|B_ z2z-zoBc6o&y_&7h4V`~J7MRfiahNii^!|$}uFtu5C}~S-G;Q)CQ$_5x8z&Jn>>a5j zZu=WtM>gJHJACR@*?~6`@m_Yy*f>^E0=b?v0<1STiTI*!rJNni8^~OAz|1EjaWMBt z5AN%uJ}zh+lC+0a9^|B9mpRo`rQ|3DR3VpiM|GA-eWp;FAQ6LGcaS=Im`fvVP9}>r z(xD?D@up0Vj}T*Pz`vOMD(04DL^U$JcA_dl zW~IvVl#dNjQxj>Z);cRd9cal+*10w)UI(9CChL=_4EME8>#qf9CW`r5yDX{B!yzvp zMPILC8n(zcDVheKiycJ(K>1>RNxOj?B_i4_Dj)(^b4}>*X+iaiJ!TXmy;EPZqq5Vm zi<~9GfdS9SH~uh+|NHUC1(TR_*Yyq#d&DUTwm<2M+|ufA-!KE86?U(@lA;Z-DiwA* zG0s6(^(Xi0Ez9ZQN_J|p$CKyWy=&n08LCb>k~M9K zz*c^NmMkrREDe1TLy3XYop1p(Y{8|?=25A3aBV04Bou>0B7M5UQRYCEZ5g#D1a z?fC~=x^LQEL&#MBW+J`}SD1lj(}DlAgG$Kav2H{ zEbFKjTKilq_FzVH_~Pj$uK{}7n(-iZYWO*Y^bc%~0c3y(p>-a~D5B}D*!vku8oDDY z%Wvt0!{}hA)s49i_y@Q@YV0O87Hqn$z1Kqxkd7#Jq<64Mb_4Bl8q9eHAZxmeRijL& zCdoFxP`Qd~A_adLCQ$(O*iQfgB7SeFP8Sk6%wUU=dx8_Eh{G6P?P!p_atkI34LzLJ z=8t-eFr@meu@>H=L7DeY*H4vTc`ZF|pkXY=#obI0(`=lzE}M^S2b^QJhTU?CQh#I> zpKHN^eh8RhVJORf|142MeYR}xw66siwulcn(n&yvFrVO-aecvzJ_Mxd*D(kyiwHKq z=R@{90f1YnZ=(;)1*B!;r`slag>w3cRYW|V5zMn^ybcYI2L{C18VmdxZ#+-=s7>rI z;XIJ*M2rKhA*)Eb%A=kK+cF3kqL^D!!!tg$yb{6-IU}k+5jmaJO5BRw1qK$Q?_}G) zb}62U_ERsQMfH&DeYK>xk>SJXPcG}=LX1^^Z&@I)#1wmqQ8G%szP}e_A3@U~z541y z!BKn#h9&*ig&h4C^cg7Ss1u;BW91{kLpb_ZD3Rs3{BsH(FPNSBQD2Iz=~@4}8Pl99 z9HS)J4Rby8vL(8(nkV9G_Bj3$hH#$Ouk-pFj-L`Nk}38t%kFSyafgMFzAaf*IYdi``;N6Y=TI+3~6z7kcBy?A8cQbiY-W z_@Nr`xwy`=pM8u=oq;n^@6lGj*G%pUr#*Xucm5f0R6Gqza$M?0NqCsjuNfNbXo&9~ z?y~#8o0d)G!7kexN2i0kddbmV1jCXe9hRNOBX2kmNl_&8OpLb9+Dy@qGqO446JZsu zrYl!auCMyhE66Mq1stga>H5#0_r1Oe&VU5Dtr1oYan&zeT1y<%^LC`d@d``y`Y6LY zYhp`xnr|B(Cv5qGQEx&~O2T%&Du;__4F0pU(xDV9sL>~f^R;J`G;lg4wloM6HIl_Y z#iy9jps7C9W5^jh-F`Nix5OnQ307UGOfYrwTvz3lnq*nAdrb`%u1J-(T(q+=C`$Bj z6#ctg;?p2leE1kKKtW%Nt@F58newPiSw`TgC3GGqRr3WUrrxWTL`!J4TQxtnt@8v1 zpR;E`e8>jftcc_9TEse0$;U-ePSCXGECGxlyGM8_ald1Qthk%knlYE}Fw%*1VjC|~ zeaxH}hZIlwPe)wyHM~p0Ek>6FfNg1stx3pD0|~c*FpO!i617nrISy`QKP>(}C$Mu)M78Mf_R2%#SmUfIQ##)51U)eH$4doMYi6_430~9}SZT z(Deomk$qe+_^xVat>4eP_WAKyeot59PXr!+^iva1xeGMMY#*P&vZ*Rw8m{QI3PwcY zppq!v-rXHW_KP#(pHjgY=in1;`IB#Lh}z(&0=340>3lKcBXtVGpchl@4Q>m{4RtG1 z&2zPzfaxjgN0yd4o`Z@-&~GJ({YQ<2@H{uM<}@NcVIpAt59Poq03~#|ebDqSAxyh5 z)j~up%Ovs@QmT3qFrNnK*#U8NfIl{t@r!H)T+*&@yZRAfs}#>?^B%-&i1@n^`gO~bOnTu~Ag%p1B6N-DG} zagIX6a3vJ*dSM|m(`W7E@1wiP|G+}F6Zew|z>Le$fWD^l8u{Q;y%E5O2Bs8UM2==I zq-203L`HvUx8A?Dn=!l{LX>U~tb-zzWEZlkvS{LVa|AU++Esh1XWp`Ap5%r5fNCce zTAFAnaX0Xj{9_g*jqftn3IU6+Vg}q9{Gu|kt5joK{Y5r`jn2}p8VB*_|B6Q-z5HMe z7+444ELT*F15YrJsGL8rO0KKj1t~c2S=f#Bdgz(ML?*KSlJBi=Z)M74u6QC|)NLU= zdLnCB8#$wj6|^XFZHnrB)JB06K}V6f$r`~WPXNX_{=}7d|(1WnfW7?V0c^J z1Zypj9Hj9k^TIT$JB!B%I1_*sy2StE>Z{|L?Eb$2r(jT`f)WD(5s3-XF%YCnMd?yR zKtLL)fr!%GQbSU@yFrF@H_|yeMuX28-rnE)_q<-$U!t;|>zq%WcQ8~d(F{n?{M`j^ zFX`%zW??uvSIma=ArYT(DBltoY7sfirNdL_w~I%btM~;|h6uNz^m!Kve)z}Eg{;@W{}Z{2#l>ZpMY9i;V zstN$08jYN_3eq5C+^nN|gk0OQ)n1Y4{s-q^Hfnrm&~09`m?fk2r_I5a748}?kgzb!|u z)W}hT*z%ucX$N?XORl&U6GPqEP+Yfu89yR@Z+WuUx6{YNati2`=DBhT?qcuP`HxAx zpHn-?nkiQTqGI26%xd&*%Awqv#~|6xv&i!X<)JEM6J}pN9J%``^zdl`pMlJ+=1yxv z-LN$MuHwF{-8m{8sRUI^J#A}TIgz5`C>_<)HbvFnKhl+~bag^L1fMOP&5kb9LIt$t z_$-wL@w8M??=lhp}Y*$UN6m4lHR}=DMLUdLJ_N`lH@G4Vv)H`(C>d%;~b8XAV=xDTVF&yC# z=WER-zXtg2w*~!jQ$_!b%}Gf0bZ9rpCKwrN?;Q#Mv|JC^3gI*>q$YT1Cws!&-CcGg zu;zxq;d5C-!dR9jC#G=Fg@X>wOiXm&F2Dp%(u`f%^QJC=5ckjr00v^BZXwfUZl%=% z3$u#h`24Xv@1a2iC*Ig6oSnxb%9+8WJq>gdDko|@X0|_yLXPsF?;;nJV}0rP*>sQk z@0AF)S|$b_-H#mBgRUixmu96_wcTej`;bN7iWq(InzF6r?-b7?o*TJu&5 zaU#^u_6vm+s`Cwm=eApk)6@OLJs}@+PMY|b>&P^dE32I zDgH>_w3XIJnB8v(*<%U*F)>wb+n1;8dVt}lE*ptu|Dca2f4kWBKa=8)a0NI&RV;rm z6G)|5!YD^Ql+J+EVU56QUSkGUvCt_cm!)C6xbadnDm#ShX_KmY>B^sB-@k-qNxe~m zyl1zqq&-i+=#Vt%(<&@KBfcI}1M3lIvA?tA>djtEHxX8I)av&{Mu*4P`V z&%v&aSYoxoy)c?hKC*@5X>-T->Wh3^^>P=pR|8$VCF^))ji6*?y@=akqD@><{QgM} zh7rwRUJ+Or=9@qLg;VF~Iy}@l>$xHg>ea3TgGsy3zF8oc1@)8aoiray9tV~j+uPAOy# z!1+j(+HvXIfMqkaY8(*J8c0uzcNC7XWwtx9aMtdw)cFIG`JZ$E>d8lc0hsF1br{pg zFzIfCmX^TQ_YGxYvp#o7OR%yEdpz186n&FC&~B;Ziw#27MS$3Em^ckK2Ut0z;{Kio zcQs-8DM@U@Ut3A096y$R%2BujW9d*iFSk3Mh3$y`+*WXq2ppK(SRRNJY00$dkJ9dC1K(?=_nJeaYP(qRt*S4YmQed#pE6J4|jT ztCv07moKcu>6&&CD{s-3NT@%+tNG+)-d|*(4%U(9CdpveT7o z3p`z`-wEylf;K+-cozWlo#OBbe(#9%V$ zDEr-@qm4VN`~lhjjL=FhxGl{}-@?ajnc>&g9HrPR=d`M*SxdNfo)7h?2{x@OfKBRY z2G;R*0Bg}*4TQgPbTApJYsObZ2AsBAm{f`F>n>zqIhg}k?lc+@11&L3uG>r)+5f<(vg+LwST=k4T*x`8P#cbaRd+ii@&zEs#5}0M z`H9;ga+D;J;j;L_B~5X0(HZ&sWO2xmh?>Tx`!Pl*46(TZIBKd`@(qG-6JS_!b#@cU z2k@^b@lAh5v*;eSG7#YuAh&SqgvFab{S(dp4JW>L>+XreW~R)nH1Dh}wd%erG(w_U zyBc|^AN2Y5-uJ5~;kNP>tNa#os$LW9XM?zfh| zo;)0VCOZoIIY|;JlFFwgP-a{RP+eagXZ*Wqw{gTEL`XUKaCcglX<{n-^~tmpK3jLntp6y6_4V-=+`rQqAa#mFu< zK_WsIU~8Mbegk5zyj}R1@x4^=h=!&I-!zmm7pX(O5u{_B&>O#cB}k_#R_b0Ft2-{UrO3O@X6VhZdRFJmdHS9_U|(nChPo> zknyYAqpPbBTe_R=Ynj~)m7ZP7`*kTqgn^y)F8kQ$W{RPzK;l~~jZ7iG89K`X3Q&&k zDBDI<)Q9Wlol*;CrT}tVQtz0g`$WF0xHDIoe2mm!Q2TH5az0Nx$%2VwAD5I?XCI+e zYc;SL5JMxEvV*AwW~h@{#3z=U8$c~wyJ=dO=(rlokHnJC(RCMyv*LG?nO>R$&SU1I zE}=rW^iQSZce-0x3Es1JQy@qwpMqRrjRC2*>gZ^wHs`_KOxDia>{D9ZGk0vTKPOV! zwlb%^I9{(b#{hAHw6hVt-n?>PEAGp8IeHgY)MfS%XhaxcB|*%Y^|p*ri$J*&OT+C` zgS#&|bUEn=PgyDc@;7&$MSCkbzdm*9HP!lO0P~MN%NhX~v#40rDQk!7I7Ig*xsYmZ zNBh-Lk#_qQ_m=EDj6*0of0Y*^&+8FC=-pPXEE+G;X(G>N%Kor!x!LW?T7vn){}iU^ zknngIrz?RHb-m|9?9oh91-E0$Hnqz~xKr_#L!mv^C@awzk z1APce3;J5CO{!5Tl$HIrwsCJ`n*6&o=dR&E`#Pm%<#pe&w-8{J)1kZjh$%@q`XIw5 zBkeu7F#I>dYI@cIL5M|kqAz>K!Da1uT_}j#POs{e@P4n#PZfyFky#wM39xu=gksqNHkv9{_iHD}+Ygfp-zER@7J8CIiP zfE}AK<8Dj!^aKcv^r6YYfm^xo5vcaE8K@xU=%MMv(Di3o&l0*swvq@}ewQx2-Ohp` z(o}iRfN$<(mT+zT2~cDHCa^cGzf!j$BmxvMGtTC9&r(W2t`X5CUAHT(e92qAtYAXT z^hgA_rVj0$Tza$%lz1wSfWo#8Fo2raiN`dZVhc6hDJ3A*>}0yk>#$+pEU1Dt&ds<$ z#FxO1?2_aBnDC)`MH^!-Wc@R+12HPeQwVeH@y$SLJWN+Q&NLGCrsft|J~?*;I;JCa5nuR(q~qog zpc>KEwrsjNKu@#fRbfUY^!dO+ub1CxuL6n)-UPMBRy^`*4x!7*V!RT<4l}@8Es)_e zcNJEZw+>YkT|~ONnABhkI>oeXLA(s^<}25`z=1z1jZp$@&U3W`OmAdO(HXVD%To{D zs&9_Xf)me>89x@V>Mx-*$HUDkfvzXBnOFzxfkMDuJW8~B7upvW*5W2+df)>k-vnX` zx~G+0v3T>P$U`EpVZm;d>lV zoc<<2@c*+39=?KDKrjg!u?w=dx|YLu?&%g~T^3(U1>n6rTo4-^$cNf4R7v6#Gu#wZ@e55Kw+VgkN;hk?wSkG(Q35UgjXEl(h>&H2ZLW}$( zjjriy4`zxCJrxz*Xm+BS`;jIm3%s@J(9u~rr_$;*vwbdqB>EJq;9~xqKl>0X3N(P# z#;10e3%-d&%yrXt@{pfm#Y`}#8?LqRn~v+{i-Y($%S^Su+nE>+)u*wfcMKEVfGZCW znBqeERkdlaKaaB=R4@J&S$i^n^puvcr8xM)$5?{XR@Itjh?6|DK z(llR563NOANFW8-z;c!L_wzGVzqZL6frs3elqUmD3ZQqJ1z5qpa9n!SJ-tw3Hncp|-V~S8JcT{`B9`f2RBChSR<6oxVo?3Lx{M8`&a)ujn@1 zymrRg2+`O939;sk6RJBuyUR$d@dT%BS&nFPde~Ym5V<$s=Hz_30mM5T5f665bDa%r zLEAs%iMwHwCioI7SgVu9W^Ug|Iau4*zIeBGuQ)gTKsZ)O%0-Rm~>A&uT1T7d!&__pgCldg*$!@)-n_pn0_!UJTR=U zwZ}M`T;kg|aX)P*))$K z`u$8tNSP=3B!8!SqJ;{_8Pw+*aSjV#Z|e5X$~l|jNH}i=!uGcctFh(|8N`Ss@d2Qy zoq>%)13(?=Sv63|N+-tOKJIX?IK!3_m}Dm5bb-ZKYcz7_CBJigaPIGFAWD{tl}wVY z_B3&CKuKVJ_bZ*gK$HdLlJXR9fuhmQ7FdC7FrRlxW8OfrVIjb)uD~{BzAyFp*!4;y zviYL;s6htUBVvPpv6bzFxO)s$&ia9gB(u*3;cwp~w2G>$Zm==FqTr|aRA}R~B5}*b zSEq`{Qs+(d5nUaw?*Xx^@q3G*tLt8ZR(?{ooNSGje})vpIuNxX^*-AKV*X3LX`)&B zN6q;0hNZ#IHx60Nv|Qvh9TYX$@U%=0dABXS(p(5FcdZ#vaVKcHwZMaG!Jhr)twIe_ zu@#K_(aQOp+YaZtS|Py1}JPNx&%wc8r9d= zzq%`oqQLZcc_FAM%FvG!bstv#Cg86s0{HL=WNrrq5#;4Qi*Ixw01Ng}ICrnwacdNX zxvMyo>K!;F7Rdmqqp7-VmTVGDbIlLdmoJ0(UBdad!s`Q`ZJ+IhmWJ%=oR5S}-qV?A zLh&dEl@FuM#^Hw0H~S?`&0c3-Z^}+cm-jlC)qF(Le@8CRciMUlB|Zr_{4RK@_jT2a z_ifmL&HswJf3k-u#AQZ`GV04DYaAwJWB1+SW&HJ1WQ1ueRv*T(QQrDs28xuc0xBkG zBBqDwI;Q+n&4!dc+Ey)$wt_*}X`z5WKFvw;CJ4~KePEF7 zdC}H-l|vX%r}05vTuAdd_)L$n?hcxCL!IFg+lw|$i=h*<=jh{jp6|6VA+;j4x4osd zKRgRn=$9(buDVzGJ#agy;Aax&kK>5Y1-aSs?P$~Ih&wPgmJYyxpXXY>)k4`ul}Xh~ zRhTs_+4eWmg3@^S9%NT693d6i4sR%;f^3>$KD~r#O+OoK8|KhVSc$hniS-|j26{Wz z7TEl%EZdhlwJ0ve4j1%p6BwPoA?&!=Tb&vZD5U0YvTLBPb!1itbrp3ZoZv&=0mGzrg*&K6HGD-Hd-oWk}fS&6oar18D{V|3@Ye1nD|EdW)_!IIuv9(fLLB z8C%J~V^A~@f!`-Pnr(b%2eT_SokQ6GLpkzY;%-i(lg+0_MB-KUvV>d4tZp5)CvgS> zf$V2?kL2&aHsI&+N|Y#L(r0p6F#4&d4fj*{#%+WbO3iCU=z= zyK2J?peI8HZQOotc3?eYN1Hgl2^F~Ix*L)G80Y#-As2*FYn9PAm8hPJ@&F_5KZA$= z71@CP>J(5+g=1=x#iOafQ?8(<7#U%WEkHaEa>^H?QxgjvTS3dGkM2NjqY46rwtxCW z;xOv|Zd1`qAsxnn%rr(5E(KIRB!%>P=4{cah06B<-1Kf{>!Y`)`B^1coc9kC2K~CxnAEDL*ymXN~pAxntq9aX5dwKmw^L}05^-W_?;y6Nb zE&?GOEqrEIlGAEN?XzuUu-g-(++T?0aMzjR3get+y%TC_ghvvL# z@4h9bg&cq{PkmNk+VpWdLzU$PRKR&+0w(XCUQm8@xl`Ob|0;(LcMiQ&faQoxZqYXk zU?2V`Tw?3D9f^;IFhNtrVG2`$rra~k+Df>p&Pdc3PYp)c6cn|qu5S67Lue^jY3czSLF1*(m!9s_RDVeQLqm` zyWm5yISc49pN-}_YO=#DK~gpPb0Nh%k0jF+$C=s)cE|Ro`g2`A9NQ3*Q@2%o*yIgX zjz`Z$HZXoD9lZ6QqkjIC~&j(>JC1wO8@7EM~w8k+<>luugv7Hj}h z+!a2iBB`k{S&)YX#L?BAB$L}QcTq3xIG1Fx^ysi#UwMolWzU=}F3%je1IRdv+Bb0X z_EayDOIY$BawChWuP49FL9VCmp7mDGJH6u{`k8`d5%A(|S5BqNkN-ZY;?)Ttl=^6^ zV4~Yvyys1vo?_YMK2>CpO%;G z8aF2z%%PPm)m5mV&YlW5f0PB!&onj(1BXLx@_OOO--J8S9Nzsj;HauN?-(apl z?vTRF>h_;k{?HZjyfy?Q*3S?+5s`Xgk?mQaDl{ssa;GHZr6jxSyELl`K7&l5@X0Wy z68d6h1z+1H&M?=J<^Hdk;+RJ3<@{`Zb%z&|(}DY_mbu|u5g?pV2mH4$HtLpbuM zlE0eKp65dx5+uX4M>b?4TieY2O#mad}fK zawkYN9n;LC3#!t62JYaVrsOg%stP51><%=Vgt>U@mDMX=z+CyzX>vK}w21N3ktx6aSctElt^>z^p)KP3+|mnf~o(iiG+aVakHV3P)I?u}>qT9CY?U z+sP4u&2-VKjdt!zscET4{%dfO0T^tv_(P7b#-A?fpECz`8V;n$;U-e>!InX&j-5V5|8c(B%@ccSL$miQR z1PBPaE-ujYy(SZrmwx0d#^6m*p5sPnBKt~zZ$e2Dsh~) zc(wh*PvdT(s6Hv`HQF4jTV|0vlx}8#BzKE~)|VkoXtar|gS-_MKvVfM{8qVXg^et% zxND@TMD|vib4w4>r(jg?pIi?_GitsA*yBEh#m3(W&3Pbt`x%G$gZ+|tcN^{2J>g${ z?uoZ;i_6l5_}r(S1yp67BNe6~?YeWov7=c^qlfbv(c69QRMr zeKuth_!_+j_qq%+%idDmxyu#2aBm@*YZFJt`lYf9b|ysOAsV}4RiL|0*l0^>Z7lrd zkvYr5T9yR&ANhL)Sn@<=z%aWMvEntY6Y+&~ep`4~Xt1kn1;>^WJ}C_3fr@!8473(vfDa?d!1l;ncmo z7?T4m&<48&N^al|Hc_Ab6D5f=Zw$#r{|uMBfYCEa@sG1N zXn5kR-C>}V5}75y;Qk)mJf~v>Wz5ut-RzSTX&fUB*y5LTQ7i+ZA)KgKPcQg zcxyqNI-sScw>W&d=q-c`3Ia33ha<+v3*6cw9j+b4d`$$Vg z_UdhYeJbrck!+5KVHVOm54Y?$X*B!Z($c!7o14#+dC&UflQygQNg*bWOtFkX+phOz ziwXHe6sSd~_y8Nd;#ZQ{#~2gfEHrYQyLvYg zG51X6aK<=#!8RpLsF>-muKJIBhalb628y z@hYJFYGab)hV9iAlK|0_eJzcfz8k~%*ospGmEgdIjyAFIA&nx&%UT;Ji#DI)=&aHo zWrdV2XsH4r85yNOeT{c=RlZ#2R#C+K!p^_KCzmBf34R6|QW4Vmn{CSus*w-bt5%>+ zf{tRb=2&lmJhd0E=APWEcoO4I62tYUjv2mPPu`H5C6lS%u}v``8c6?TVezcIfgp!A z1{*J-&}>dZ^EK}&x}WQ zdg}Mja{AR#00oK5;?nzy>pH!s5Y-@RTRH$>ETJdaPf8!ZMgD&HenMMIer%M|-pufF z&1b3GG4{31Mfdgb(>r;}+1M5$sRb_D{7WPPTfmeOPbhbr9H(?Gqq6otbfO0~~47 z@%NV)-HELFnW5ucu8`r#CB}3G-*)nB_)I9 z=fc^hvTe&Tt3B4gY}c^gee?nGtPF!TxDRVjVXk#>h1`ulwI*&04v_N~mJOwHP(=0> zhx-E70ru#BfN=4mT#VL3-}l%n^Y=={58mtxB3FQB`h{5K87JnraLxN3DLim@QAiOX zovq>Q$~(PSSMwd@vh$exXCd`qkllyM#O>a49xL^R8Om_ssIknaQg8inVi>_(4XAl| za=msQq_&&&uk#`4t}0JN@4ZNJxq^}Vg8`1NYeB~K@HBCBtag^w?)#=PDhBt$f z%M42SuFB?Q!&YU!-A}ZVN-lg#{<`2P+E)*!-iy%^+H5}5VdJPfksqq$|u(r7#+CF&yIr$l6ppdi** znKGuO76gdf?#FQdQ?8nd2>GOY9nH*!&QjEt1lsbxoW^5Mwp<3QbB^Qq>d%8C30~HI zJE`we0sLlr0Vp_iTDE-=TC~FGK0af!il~TITa<^$F8!r;0O~{QGlZm zBqh546mH9}(ZLor!Gqsq)55Ikvi!1S*M~PH2@rTicMZS9@c|XK7@KF4_lOnR29(sF zDrG|F^<|4)4l|beu?EGJ zrhCMEK;`?`E@15!C5y%!M1Ras#U>JlPaoLX)Z(g8Jsjx(u1-a|O;m^kP7 zg-Dm&AMJIu8@PwY+OdumrdG?w@EHyKAAPnLCNIX74F*!E>hOK@#WwxI_(%O7y;a%< zfF;3+Q@^!(@Y%mtPfvgJ=P z=htab)OgBjAHKHHbsU-%)Np!b6BV_jN$XxpYG#ihpXEoGRW-S8_7N!PUWeStQ5E`p zgZQ4ztGU0ry^!lzU|1atmZab#M*#PA{>{DJD#Z@FbUlFenGwerh)0*CXBSz+;9nqNnw4?$sOT! zc`MP&Q_mfPWGoN*Il2#8fKt)Wm-+`eFtcQ8i;LROlpdmyxn7l?PO}{Goe-2u6)%JN zBlvnHX%q{drN5HF`q~R#2KWuiZ2aKnLlLxRtN2)npR-;YRpqtbqpveJFh7G#d%1RH zb1wpH8ky8bCs_vm?fd#tLTQ>WZ)Y?+7ZbD_yf?WMV?B_&_n;C>Gw{QH{q>3Zk++Ti ztz7C-X-S~>debNjcc^C>mq+u=ufW%v?FpuyuFPN5qDzbHtl76esATw46suixJF0(r zASR!~b&^mb{wbNQPc_k>(XC{0wMdagn`-Pu&P|phcM;k9ham|>lJf_gF_|VvnoZ@{ zJy>XBm!c*?Z~e%2yk@|0HsWZI;mw?g%4+M8XUfNl_qIhxDcYTaqvDSe8+UqJA#T|*tlA`C89IZ>r zkw@=ALsaKXI()nDF%ov|{XbgMAW`M91FcDcQIc6lBPQzQr5on5&0v-XZp9=iYSkWF zTJDB4tx3)2#a#TW1+f2kxa7~~0x)c~{h6+x^G9w)nCnMS>Qz@f!v)fa- zj$}VF`0_4C@3u0r3tH}`Yq2yy^B9V#ulK)tz*i%4M3%l3BaMGprg&w$u*Di&XYm@C z4Xi{zyY)m15zHDxq54bKB>RVVsOU!y^5%p$%iNtzJ!-rS1nt*wv!UZkSTer?yBX_! zF5PB+or<4*huX#pj=c&%W)roqCLB2H$LZb7L2fXT%JWP_r7~5GXxMkg_Ge3hzww*` z##oC-`|u6^AFjH`-8w3BjcaZ;HK|-3YY!a8W|ZqhhXGme9>mT&sW!Ul01_51@ zUL~m)af9ex6y~Wsad$;;KInh)N$QXTzW!lp#ZTf3uh=JziG2wkM?ii}>4v zj{>Hbnm;f|>!bzVceT?VBB7!y69eu?VeC217DpFKTIne*iA>mHR@bPF^Y4R`5&XHy3xYQ>52h9#T z#%tVy0|Z&*xWBB&J@tc7!uxp1_H$cs8M9ul-k$V0i|@=iMWf%F*x8sCOyrcyvGP6V zHH)a|N=y&vMJ(gYO^R1<<*UEEZDtlLKKk2wrt2l_bW&GGU3tq1fn#A{QF-_8g0xA9 z5|#56%P5x92+W&CYdCap4{~^9&Ja~7eHzpT~n&o&Aj~-y|qn)f`IgovFL*W6xhQm6Cd`yU- z*P)gE>|F%y)BuJZQ)5hI6k@y4#_r$jtYAF+r1jUV*}jwU9U#a3cQB_XNYY&atD&y8 zgVEy&&F(1SNIk)_4gU<|uj6gi`^Dyh(!4MB`;c$N_t1`z==InBqD**>3F)5SEn1Rg z6dmMOvVL6pyR?3FDzqP6yX{1u(oT`HSWGE!;zMq}5q!yx)PdLV9SuXN*L}4RG)d+e znVGCP{wKWDH^L_IrKB9m)2S?1u_`U>!deX4MKXPf{=Fq%`aV$4^S%@sU ze_U5D+Y3yOMUAcIY2Ryd(6s&_P#TFcC~e=XFr|OU4JnhdMat@Y?h{#n+*ehMdsr`0 z6wd8v+FMm%3b(7&j{mp5f2a*%#{$ehULd;fXvZrF!(JVuTUm0)GrwakWk$^bil~=) z8ldxXx+BzD&i zutjK091ULEB_NQfEb$TFpM)q=C*CXc9)Ie}68W z$GKs&?!WVOE@zhuwg=c(>A>4|DPEBup=c{U^T1Gs6@p?2Yy7vr7Y+oEn8uC`c# z`;?b3*Tc3=G062>_4tUL#zbr#|N&9A6^QLNz3rko1w_9ObBF`NkW_w0Ti!=W&d zA>Y)K^tFmr+2fFsTsE4%=s7hQk2nq9$?w&;%fl)!2hLPgVN*EwA88P{U5Id1=o4jp zMta&#$9BWByQ|%*J-vOz^yxEBg=09pr{M*a1q~-z5eeDi?H;Q8Qa55rp5|rN?w1^! zpGp$#|WYIdrV?~AmV(m{+=2>Kz`JVdR zZnc7iU>20Ncc@gI)w=8=G*o^QFzX^PVnk~A~@PMT0D?J4NWutze% z=GWls&9dp15$W6tRBb}$p))c9!^*_o3XdVtUq6b7oE8hqQOyRd`7B3&4L_Qoz!O5f z8xe55n!$J|IK7f-@cU&cf!9auQ-tT{v|irKGhwd}?jbYBsBtEpcNAhoTx&XFc)JEQ zOxT?Ee|=a3sQkj;`@heSW~IOEUcoQ0%)fHqnYv?=_3D11Uj3)1MEezup2Ve28`Zp7 zgQd%-Q`ui1yU7=xDNp{I-EBs*ULgnwXaW3PSL#DuLEI|*6v za|$~Jcr0grOg-^B_DGS7^sowwidpVW@g<}d=wE36N`{)q)jhq{p-B`ZHx-{n$jj?> zAN#FE3$II8crY$U+d5;`OI?=?s1SPNZ4AQ4D{Ia|+1E^a@%9Y4_tMSj<%HIiG;WU` zo@gu;40e@N`?aBrhu`)%P(lOSj)kCsYeDNqZq1xb7R=wIsOSa{g;45BQnP7WKIcmd zew7U?URk7{Za0TQ%W(&T=F&;T7aHm;BVW6+dP`zUft~sm!cbQo&pnNS`at;y!Q)W9 zhy@0(qw;9#b$`-D#KYe42g_1Pt*=k2A{U46^S_mb{=9}nGBf1~OTRZFb2Dybpx$x5 zZBE`Cta2ZT!>3QbFLpiqMyp|bZth9E0gLzS%|+foO%$_SOV?mx){_Jdh`mer-P7`@ z6fJ$pR22zYSBQO0e}UlvLTqY7H9D(A!RUqS6Hy5ENw`d)<$8+Q1JJ4;0ABk7Y?DRP zU$#Sp0SI6v@_}$H;jz2l&aJED?AN-Al~d*OW8POV^IZtI{peoS!v3g9?g)Ueyuzop z2|Ar7b|s*P3Og3ndv5?RP!aH)Nr}5ewU;+tSot$X!1i5~r@J^E)%9r{FT)NH7CLF{ zr*JC)zTi{nT{x~E&SI_QENVpEFwS73dO!XfvEW8kVSNuV!o6_jn$Sx3+npRmXFSXeQdyv zb1p(B!v0$Y!eQ7lkvY?^jn>~=2QsMeHCa3!@X;|+=j(UAiqT#;8-@=_J#l34No394 z`7mMrRP09|+3b#z0`}h>KsaWK^vl(HYl8LPrIyr_X=+R%_}BA8dFJ{oA7%$=phuW? z-6(BcUbbrNx!LXvCneXYgdAVOJX$x%G8CyIoXL+HYBDfdVKjTkR<29eBPA;muSTAx zo->H(&=AoXowBdDy~fEFABHR@9Pt*-zWihHb}xa@M8iv7{-q)ICuut z8jBmI(N9Erx4**%Qg!r|1;Nk2lP!+Jax4t;Y)OwKDO87Rkx+0GE2s<4u#syDz``nXg79MY^>x3Nx2=_Yxj2~J9j!;CIzp;ixYjzf` zJ47EDyt-B!ILxQ&-0HW8Fk46_2N<#DQob0pAv3q;U6Q1K0WiUA%`2cqqte`a*y9QF z(36zrRispqt@LTLk`WY*!yM@`pd=WASQABfo0H*W7TpG`HMKCR+Cot_nsdfpxjl8? z>!gEsGB(3Q#=>Y;T&dCVn}-B;f1@CqB%gyASOyNaErk>diMy?@QujegN!lyVO2 z)9vk(q`q=Zw+Xef@}%q~-P${=XgDfdGIHniQ+_*evmn zLiQ2Y%(&6)PP%)$lPEO5r^nAy){n)c4q?EN#@@de2btv>WlQPELYTZ1%xk#xz6o!A zKmV0gl%?!a-f!8fBm;eq2jo!YlldDbXMPwV#rwfsryYE# z7J=6-RT}wI{G#4FZbKnzy&izBTa1oD-*kHonb~PV2Et(q#4$xxugzF=u{&~G?Bh_S zGG>5S%P$s+MOqIk-L$A}3V(*B_B z#jaVx%E#0&V&0wbcnzU#UX5AJE}gi7B!th0DSu*9x%wj#-mNMPan7MYLOh;`QZKRK zA*I9;>s`CkN-pg`?(hYlj2RaWNjR?h9+9ZeuUm3QKo@_-L-8@>2`v)BSC88glPp#- zd{42^6fULdkoyo0X>Bzz4Y*@39LVU1l!$D)m(H%++1yq<03rX${3qx2N8?b#H^-Xz z7-svAxol6WGi{+OFGWyGamP`qS^M^u#Htygv`4LHVu@Xt^$*p1s%&F)Qi@NS&8Ev6 z%q>(Q{+Rk&7lZc0pvG**RNgH z6S@}?#G^}s{F2O2)$tX{H8MXWL-5GVg|ckwyNh@vXCjal%eQNOGus- zjmpW`n^1*SDUqY#`jPbsrwwjriCJ(2Op#+V7#agatnGAPEikquUIS6OxXG7X*-Y4J z@7ig76*TrjXv4H8lm`CDZ{U)7&EeZ);QnW-W@eEVXLp6=s3VGl-uVs%^5~28lmxl< z8**J|OZ!2ZhkBxFBg7aYkaklQV%{>v*@W0G_cpDU%@PR$!$q-B3y&X*zB?ojrD*Z% zgjO$Oz8Mf4_)^XqUw2n*OR9lO``rFk_mgPx)Cs2S zA1Aawzqi42lHGN;a9FP@kEMwxU0%ymX)!MShzYwa9g4Jet5e*=E2L7QG?;u-JED|b zWahF%3txHIb6px!%Ub1j84~@*TY&>`l~;*ZoJ&OF~h;)v-=JWo=R&YljR4%JiJ-)NeM$l$2n&ei7=Bk4TIXsND!7`K$tO%@I~ zn)=D>a$vioES65eX#OymXDXh zbJud=8FBhkYCIuD`|)oosJ?BZMw%OiJBO@tak=glYM^zyzGg979v0zI6Ggav^Pf6M z6rzE|*^uFQJ`sr$$W7B$pyDgup%h7slOAKoT3W1_mx@TrJ7h_qrhi}ADAy&Oprbd6KO z8)QG4UhVWWG65;@;qCx-IvG^gwhra(IIsW7ZEh;Gb(hDV zzY)sU%ra%M=%qNX4&o5g ztRainx-{dZC0*i(;yE_yHQ7R{2z z>4KjL)mvX*IEiMib0a}sM!oZp!DS(P{pIm6;}=IkoCfu0NF2pcvR>~$O%p4wNN}q? z-DQ$5+p|>+#}QxB-LO9qB_o}}2;^$-!IJsEhDj`%ag>zFGas_Hah~A{=Z^R14l+#z z%0biU@fmlbJl@1M+pP4pP7+kRLuYL^8_%8@QQ@Ajzuvy7sZm$nXq_G;&!U?Xk4V}j zdgK?W`5~$8&?J20>W1?Dt;xCJvOtP7n(HM~^_*Ud!kbZdJ^9vh$J&zuI5+fP{N5J- zxG33Mjz=x#OmxWU7wPzZS}psn-tp?A<1ah~C=+ANj{FSvzYF6wcyDUMP{T6W-s5q? z?v||0Kn9Zuw43HrX;+A1q4$bgoY)GP7t%MnbY3-5N8vEBT$Pz(4%^uA8L`vrzf*yYc3z?Zmp4mU1GRrJ5Hh`^ zP?CK_tgjc#Ihc}C9dG&5z5z$zi!<3&Oljids}K{j6!=F(Egfm0GZdji5o0`Rr`k8D^(a>9?>Y z>G>l|S2ua`ytThKg6xvc5#Q*ZF%%ZIxJCkFjN2zKlsR_Smk*gxzi=qsTp`99IvSdD zNT4_AWC-U!;GTqMNd<0EK2Cd7B1Ii*wUmGol6KXe=D4I?jpsVlEdoisDl^>E;$Y>A0?kJ=z%r+()?2q zlq0wfv>O6xsvG8@N5`Od&;S9|6wG$t>@$MJ%6Nh^qz7zaXDoGMC3)3y!m7B&9uUe6 zU_Z*AN3?_B;5aIp`}yN0=gU{T>xu%ImS>!isXNrvoKp|o>uf^aPzJFoQLhk%FDZH6 z`>5|tq~`a?e^&A_B!pl}K)x9+@cR|JzokxEXlv^7%u199k%MWXj;YPkM|oK%K~%N2 z1~mF{*GgG!&LbK**HbB-{Ew&jisPkIZKty8Sw!=^__*=pLZ0?6Pd6CN?sJHlPg2pV zzyu$-u@2R|)oZlAj*t+_e8aDs8r3=W){MzU`fuy~cPreH#pzlzvnLQ^fK7YR2M~}c zkTEG<)7B`n__hW*F*{IX52W~pGAWhB3pG-1li}&Vq$TcIGLX$o+DeL)K=y zC=WJUW6M!T!O`bORjig_%Ahl9-~WGPT?aVT`~SDfCS>odL`KA+;}Df>smyF8B4lKo zVIa$2?~CUjNVO-n#ex{`cwW`93|j`?%+PKcDye{aSCcRG$07 zMziVe$3bohv1=Ww0%Ee;lZK5DluZ$Birk_CG`8pAx6SxDr%+3lAr4s*#nrr<28Vi2 zt|dLG@mUsdohNh&=*`K{JwBsMq@FJAhmwSL7>rAcsCtn)qtv`HFu*oW#MpLs!l(AS+hYIM3YP`it@u69nK#OI zCrA5%vbtdA(k8Y`@;&Ss>1s~V>IK&G+HZDBmw_z>a>bkdceTDd zcgtBX<<)u*L%uK!1;9S6uGXGE||NS^u#Is&AH zajT&;4&X{;^IzF<%YQ~yI?MxO~objnJ4S=;OOM@XuN-as%f;C zc1h6%_;uqfo9K46pw+0&9?@qvP2mYPs&jlQBij7mwUt?0b;+1G=Y$>M7M5e}x-5Au zI>rR$QjV}!nh15J4G+%G7rLz7S_Ba=*JxoSgHW2}r*_Q*U4%(IK?3Q)1CyIhVcctQ zOE!B2^pXZ!2D?=@QAkOIKUd4wY4XFiYvj*P@05~7>7eV^l{Suo(z`dE3NM#vdYl44@?5~UZuira0@HS2DJy;qz zPxq8@`*jZY zpc5QTQ;AvzFCf(lZX0ds;-_rGi5i6e&z(^RYS-i>U|h16LEnJ%b7Lgx;z|4^E;P{V zJuug2cLdFPKET9Sb*0!OA8WTZQq{diIzGB_4Z!%K%K&|?0+YDsFK|z4*o<)0eL8dn zD}VZ>UJ$1VEl?A!mJB%tJ8nyX-J}l83W@ZK1au*kDDf6+uWs2T;mVVPr4?|mkZ6V& zJ^hmY1HRc=|CO4`Hyn15S#M4=IcynDX*v8jHPjC5(k%tpp}@Ry7@Lh~&E5w#Ji^c! zd3c>_)Ww)sWrV~qZ@7u!zpfQo*qS6-4!y1 zcO)kY%MQC60T|Y6R6Dan9`b0R{3WC>+AgDG@qP~BIp1mexqx{L(&v10i%;>k=ItP! zwWBmUU3Df2f|l2H&OunA-uTK!nNqv&g*yMdUR`oWo46?`;Pc>NOTa_2fRd+`fGu7rL}L?9_Nr{($5=dux; z^UsV}05*@=Dzo}Z`~K<^-H5%amCQS-1X|J8Hd=Y?!DvXCy>4Ct%V{GPs0Uz}xcahr ziTq1F6A!+e? z9U$(3q|E?XJ!5?f@}u7G+_f3thfO!9ud7_Gxdx%I*scclSkb_NUCew?%z+8m=a^51 zS^i$sc*!?DVGPi44GTnpk1wRI|G@-uuHg$URWt@4sQ|h8g4s^b$iAc}V*>bj z>p8l-9TS=aLMMZQ1{i3NsTY(|-v2ZlO+GO!F@5@jttjnx(f(DsU2YJ#hc(V6@@J+B zOXu-y^v7pw;dMu>J6mlV8Pwe3IE-$+nGmMz61h&>y_cVIdD>;BVeG7H1dN01xH43~ zThg9)Q^i+P5u!_@ye7No1no(oMiWWT(>OHyTE^G%M0hK4Y+5438O&6@i*Fi3MelG< znOw-#hT(ThcdDE#nI6UlTCyHqQejvkZCf9k+`plMh)m1XKW+nc1vgFj>2lY3ebQyt zN^0>tB(tNiahi#X!nQ{nDW^QEIT2o0UAlCS!{eVTq@At~@qqPYk8^QWV=>P2WFNZ% zfNL_Ur;q7T!vTHly7@|VRz1X=a5Fl7brL$y`1nRuH~kfdIAb6uKH17RORNY1jEfQ* zY_oDQkTtM-%i9+ejg^*km0?q zVnGDyOrauDXV_PW?Mdz4J1lVz zO}zJK-2l1wox#YC<^Xx{O?vRE=hnP7L)Q!ao99sLH}^o^(z%4){1I;5voHO*7LA^K zE@`k1pA>r)L!NSN@+Dn^Kv3<$Kwe;>`?{|dPpZ#jayJ#fW39IAfZXvFbBfvGx9RF~ z_Jv`ooxozthzqb>q!w;c{(xEJ^L<(&{DPRY!0`Y z@a<*gl+t*{_OzN7soU5eB3a$PCP)Pd8WLyj^2t`u`p>hN{CAuQhxq{X{(?<Lgv-? zQG*;s{j)b+Z7*6(cf<(rHugBGhU&Q^LlgG-x(A>q>Se2mw1|e2VbLDT1!zUY-jSFt ze$z|(#g;bT+tkcgnreV&JK&k)q7rP$jBVTECJe5_q!~IBeUA}*%LkkZ=htP-9FG8w z#K}erx<##PU@P`ApmQj?Ud7}2`*S6n(KEE+!_hf=aB;TdKx$5o7Qa^|ai-qiF=j#K z0Pa$Nkz(G(y2y1GiF6|xO}ct(g{4$nV~j*l$|z4h++1`)TM%E9$m^9g8A^nYBkpst z&m~cU3u~MN33PRv5b1gNk^JC(@FYxf!K9c&ZhJ_)7%2&6>5sx`qRnSZ172WXe4+g) zn4)CH(bt+0RvIvJonv`^j8Uz>xgvPrZursOZc_&EoxQSAU0nRueXH0Hfy7fTD#l|N zc0?o-F{V|k3r#E^?2Tb$XqTdTnY?7M1al$S36magLVCLyN@4 zpjQ`-Mf{}HpGwMN(9Eco=8UI+cKVe>_F&64ZF|LdP)+2ccDjEsP1Yoq1FZX0YmmUj;XCTr$$ZORA8Q0 z4q>86pg!DP`iy>;`gihQqG?Wa2O3DjTuQ#;i#{>R>t?$nzc;=iuk8U-+iX zNm;dSFcm5_axs66?jeAwQS5HR=FJtEaByp$rR{0dB%GAaw>g?ydyamd8Y^pB7M_Yg zX$D)DY|a+Neznzm`Xkd6p!>}( z%6X2|3=_zhKIcItEPKBnWc#*+CMi2XuZ`c3m8khr9~eQdf1s3QYVO@-E-j&eCVjhA zFv0K84x0=ZKana4o%FOK$aqx_THR|nDsP6fuHt&nXtlk#LE|;S%k!4IK z@&w~tzegc<5F{H(02^wDadFPxsm&uizJ4mkpHndei1i-p{!l4x`Ghm;1{O_|Ysi74 z{3&k!cWbGQ-T?enajj=Sw7lfr5na5AC+b*NUNFK%Fg1-1rxp_XAOhEA-u5SjDIe`k zhmZb^Es`FS25kLiJ7ppGM>YIElm-m-mJSMxrlsP;(#sy2M22}pt4P_%&ePxE;;&O- zQk{`Vx*w~<|J2k5yb*J}zDtBjZzk~w-kwK>D%z7`ynR{OZ6a>5;YZqtBHoVdS3LN1sPqd{mN}Mlk2>GoKgU(G{Mz@- zCT#O-lXi2_XpAf#!?L|1KfktE)_q|-68?rulmnR=EUu1h13Bcb4W#ScJ|{(hk9=hQ z>{2c#>K={|alB~7$@>S?xshx_wAaA7!u>NnT<=}ny||%)YbKd;GqZv{-|E!1x-*KJ zf@88;su4-$zaXLkKHn!qQb%PxiI!YnCS}vKnek8f?^>MTIjM3YyO!SS8Bj0YWjSsc zi+aAc=@i2Eq|OOB9T|zXiFL_B3ojqHz&wo&*-C9ZdeHmVQ{=A-V7j?m1U)qGD2Dr5 zhFqy(*wCjqdvMp8U`)C^$?SCR^58&~gzazK;jbq5@86T35-qOa1qF424;@ zaCr3t?x)Q$TR`Gxwr9z!^@Grk*9#FStEG1Fu7Z4uK9AU8YH00|40d}=&jcynykHnY zC+sA=$&If%SgqRKXNGAv3b606*&&Zujq@@ULfa{Eq9z<{JvviPi)s zi}8rB{|6yyBVC)l$}8Rf?tDg<+D_dPd~E29T%YieU9Wt$7>8Pj;9&CtFKc14eI(`P z#VEU87n9X#6<2O>1Ik{(mqqn32WVb|bv`jvo__fxaAN)*5&INLh=gct>*`~C3Meru zkdQL$GTU8fOYCK%!KAs>OB?6O=r9Ea!=F6Aj7e8<@T!a?-)|Aj3VxUnK7Gb-dJ`KV zpPOtwf>xSjJH2i?iDx8?VZ+Eox)^s&_3P$P&YmLiJQ}~2Yh$)4hOXGbIw5JlXsUk? z_rK;DG`H{m=Z_EY^QV_Dgx>o{@Ngsf9g!np$hq42hpbMpl`1O_AdSkiOt8QGNC3}` z0;Xv!q$11c{+0n<{sz1GwPC4u%Ag(XW~Zb>q{^*;JPUd4*tp*T*)Uo;n%gHO6`=UY zOf27G`3h=EwGhVUeuQCKcM{_k=sxF}W)objxO5Kl<(}P*li|8`j5G>e9v8$&4&gXH zd#GqL(=4S@p<|~QaP>s#X+XjmkKF4AOBGtqIahx^MP?;+1j~?plp*Q0Ii~~4&HsP% zlU+&fJv>uKiZ#)ueIwBY%S%S^@CPpbc$8g+WKW0_AFSj&L;dvO-!Hd9FkRiMh>w@$ zs}%HpfrcThUFXqd0lv9(% zzdWHo_-s~7K9Ew?OWMO-q^)9N?eV$!XCBV+!kxm8#KPSy zmyuICUxmy?%uk7P=YED};t0EDL`KgTkvr^QGrWRVC9zET0ry*l>RjWKs_^M~gPiGo zj-{gEjpqT8L$tT&FGjt(Z6t!fEOb=EXryrt&6Y^O?UAd4bdERrg195|ACTg&DzZit zD{MeJM?sEwTQ?fVimqw|xr3H~7xC*%k{cZa61O#qBhVIo?|3U=-=BG>{r)lF*8{CF z)+k`g*j{`HlDX!d?T4yj45}&VF1jn=B!mF}NU&>GWO#4kdeM}KMa4JjxP}@~T8*XO zT~v;@Ej3y1%19Y3wb{m{uQo4kxbC?>=)&$pmK|5W<@j0p&47(IE;?+oL@36Lw~V6) zlI;q=*i~PWdq}*zqRdWo_FXTJlC}Nh2;pd??%Qq-u29M-7A67js6Hvvq}i&l^>WJ6 z7+!NVWlmW#N7xI`(wkr59|fZp<9^;H7%Phiv6Pqixm^(E5tH8pdvTXUdRZxV3<=q_ z;j_=Q1L@tIcQz7~Sk@&MHf@Kl4Lr~y7~9>pi?Uxgyg@2A<#I0VC0IA+a9g4VpfB2? zqU{?6FDU_FJV)F}bwF(QT85N>GMxE=z&slX`*c;VwwrbWOX+|6E*zqAmH^lNB4&k5 z3%;c)Tjz^RSOGxQ?O-8ev_I?aN{2*uu|yZ7@hk>Wzw63~Yrty(tt303btneF z?04L_1;%at`j2;BOF-9CSNz?3yu6iYgIk0~ zkn5PTb_WyU_d1I=D&wW=9L`CjZo#(TZ@ZyY%NcT}M|}*QElM*{^79TkqOm)!O{F<; z2g0SoQN*thO8vonk& zyAVN40M(tYF)Qg2#vjs@${1C+*yyLP0z4Ec zx86io3gE!e2yP`s@5ra$KHQXgGMJ-{4x-^{keCC~FTF#66_`=WA31vMpBx{Kf;4NE zx!y4O6L(gCO~xz7L!3U~q$M3+KCB$B*2-_;JJZdN_M|A;cO}S3IgCS>hVo2flub=s z?cK%Djm3XTl0prMV?*t`!gsEIx`v4OweBYv1T~+DSg{4BrhXU>)kJ$(O>Wz)E4rq~ zsPq*Xw@FsoE|700A|i0}yC0OM&LMlE>MrM&i&fn!tMnG9)6t;$8L89JIIBi@ta$}3 zRrym!XM)VM6Wf?s{c2&pFz=qE+WXP1o4dC{8k3xcFKE7k4k9`I;LIUZ72V{mEf?4z^uLwPc>EX zbfCjZu*W@ox1ZyGrAxVDtWLJRY#2fCc6yg&NKW?5m98beyjQb3DzpE;R&6L_0Gdkk=fBSoNAHvJ96S)P(I@EqzNxlU2A584tCDu|K)CQXC}*VusuR zO)v_gOD%D*Fa?zl&~RoYy7a2J9d8dIx9%vjKkUoUD#J-KMQV>W#85q-jmp+2UVHe1 zl*guntp&?!uia9WDE{wYvxoiOjrG;cOEtg`3_=#!TLx=z)dDq-D4 zLFI1pyqR8axP{fh4i`J?F7h&U9Led;G__|X3P75oSwI2VgG%wJ$Db&Pd%bE8WQzn2 zoLQ%SB3gS?c=qo&H&-o+`X&nQyvB=yr)dwSA}uV%Bj;ud~$U= z&xPrZ_Ux`*p)mA=vTkmN{)Z}q7%K@TYGEEe`s0qD=d9BW8n^!M~C&h?R`o6 zUMmJ8hv`x62Y?z~J3`L1#C=sqNA71*Q;$Py5Mz0erfKG#O!6oSPjO0?@BY^0Nddi1 z9J^XOgNqKYVAiBiZ-_y_bdb|+I|?T0aHYkzn8%FuYMj;_E@PRgA=~PAd&91%gjcB& z#-}{pc(G-}x$Dt`yO4`H@DoOlj74THw!Yb{AbZ7lrri6`eQ-%;sl8~%HT()SV!L)B zh)4TM*M$7mwVE)A#Avi9*L3I5%4YJv#Q1-qw)iv`!|MVVO_<20oNL^rkg4ptr-y(l zWr9@)cW?18nD`vKRs!I3^wf_+2!8|hbiiJ*$^j_AR1pFg2eM*hlHDYVH!Hj#qZO)D{~{go$aD) zawQXo8clw;o9|xXnf|;Zb$5b%d*BoXC}dMuS8!i`sH@F_;x@yljDF~ZrV!f0PSIkT zuw2XX2-fM&{kDlhbaR~+CA#x=C-P7(tIDOF4o`WlVQ$)K1iQbtQBv~gHTq)M6WJZ} zD`Yl{W_>|*mWhQl2!;1d^6ajy&1%n5yGD7ao*JK$(U#=c@&Ls~r3_BMQ4+ zf#X&Ik;Pd#6RGxMnp2l%88BZ=H9jy^FK@QL#D!-9sn~6Q!JHI0nDa3f*n>!1&9`eo z?MJ$EoZM^1U#mSiY>jMrws{j9Z8c%r0w)ob$`y7quhX z)tbRd6dQY>_xi62tju3dhiP&^hL-4iw1~E@#%}k% ziHb{+IN$-d5+mST>5ViZYV*o<*S&cKP2x1O-zfMV@{&`TSQgY;Vx99gJq#8<2Zq+} zDgr0MfKm6(Cvn)oFUjaXb-YKGNX(&Zcid|fc&5w%vQx$r25G=KGeAA)dtr&WI`n`J zM#;8xmYTy3ckzQZiCLF=Cd0QVa$)eZQRZp&?l?IW5@-ot;NZox#C3jz#HD)9fR*}g zj$|K9dTPZBMS5>w4n%KUBGrS*=A}*X94^!{sS)s7{%`l3NrzvPAHrv%+fq^N!aXY7 z7-XvWK|N$8L_o;^m&C@l~8-7r<1f2NOe?6ZGv=G z`F&{ruD;3MM4)-6hc)74eM>(opSx4NP~w37IIJh;WCE+a0i`RJ!M+4LT2oT;fbwAW zNq0B%jV(j3He|rpJtlC({EVnc6{J7B`|RNn;CwqH%FrBM!$2JK=tvoMG`$nar?w2L z`js4E_bwT1v5&}bpN#V;5b-&itbI~nmmYU-1au5d63rd6!5o7#?UbA_=uB#!J)2`a zZQCDq;?H~CdIlew|LBJWGOYO7xCphsm3+}wfM}^8@cpXQoU=-VziK0XzNi+H!`O>P z$fgW*(bk|)2Qy#h1 z^DS)s5mV|xygJ@c}LQzif@t;{G)sA#CP{xM3vr$ zE+EA%4eKZV$vNIg&w4G_erz|QYV~&Cx}ko~_$_P^DmwJ1R?1F6_qo?9BktFlg3}RS zEJ{1N8y#d0aPrH4{TkrP2R5V~+qrJk1P9WeEy)i<)Ebe7`|S4*tswj2RiLhhuxv-S z?tye+8;M69BvOvh5W*xQTyp#kZ0&Fy11TZ zUQ_q0wl{_+NTOAa>v#zd7NmzgP{aNprdjLK88K+ZbC&Ys{H%vTSNOaOQZIV3Pa`;|&+s%qq>}Imv zSq7+H^_EjeeFg&BB_wTZo7m~%fXDxzt9hF{VoNwWEqliM+22yWKrIr1P+J^ex4EaU zAx6y&-NmNxEc*ZN%msAw9KN4QzP;hk%^S3QymPSLKukTg9z31q!h?S8-NJe>@9MSBZPT%@C&WscxF15gop1Md9*{R9Mh&JP)76)uy%nBq z?gt%S$>F((6WS%u5goZE&xOW(!^@+o@=kvJY`Ur@+9=6BPMm_?MhvHrY%i?f`sp?0 zRmIEhqs9s?rW%5lTb}Y(pHuVqTe_WPWN6$A9I3TQ*i_~8-G)=J4f(oL=9lwzk3ky01o5nhM#COxr$4Lws%6D6)M=T?6E+k$y$r3-ecTqHp%3*bryqSD<{{UX4{Z`Q2YgXk<XzZj~ebLXpKcx6D|2NB54#Ufe_5-ohKAdyp~j5^)}THA7#(z1m;+YEnt4 z1tQZL)-71fjE-{27yKw1%Q;l&Cy19+oOCN%8V7gxS$DnQSqC94W z!OORohPrdhr^tYXZ`1B`$sn1lMZ@G9WsZXVO)L&B!wpD7Oc;QB+ZfK5d2%pW!ZdCk=sN;XJOzQu(E zn#s%{+0wCR?PB7Et2KusAbr>i*XT4b{UNPz<(zAR2=ayeZK*ktjmK02c8Hn}@ zC1c%xrcgQ6R5fni89DpD7&_m#@kCr?;JYW0nCY9DF?5Wjm}%XU3wAUa!f9HCHgcW; z$~RyEpBGWArdmDz;KCR5ba;P$S9Xvx@z?p|tMf{JbisE*ipU3{oFJ>3x9K!@mw%RT zp{+pqtuVgjRQc=gmMHVh-NKN&y|;!AEF_oW!V!MQ{+9u}(`v~RTcsSbZLk!Z2oAF| z4Q7?rb7i{o>g%JO!Bdk*uPk{DmJ5d^D@1v}wLjWh$K7+6_2k(|XmCwG+a_Rri5tmYf~F=12-lQ=~Ey>Nyehmj)0u38JE<9c^o8#LGRfU3`+NIUtuV7F5Qb-tpa5n5t;=vTky!P59_vtq8y?RLqoQ5a6t|mzi2!3{7fo z<&OWNR}erZ&-nSk9pUv&+^EsPn5}TWa5#CKw^H#y81c$JUh_%tespNQ+7k&C)P3Z- zRf&C?vCiliBkn8jRKx3*{_+g33i!Ekw%5r95@hKxK5mH?b=PxucQl#TUdPHWsoC>Z zZY0r<+{&LU30Ry3yz!Dd(+t3H8f|?XqbotnWdGj2lr(!A_E}7ututD9k|6v+VZgPhpC*$7o!{1&dr2#$c=b!M4^~;C6BBwL3je8Txy|~Oa|@uE#C7ySX2$Bt-GPN zY)8y4J@f4qb#I1c2CSx-=#mDecpg8h$b3?Ub!ciYU9jRS6td=lgh3}RP<|aScqS$6 zs!TLOx_21m_K@nrQ7EQlJVWgpUyK^7N#tdhJ!2i*ZR+=oRI>dl(Xo{&w}-~o?nJ** zOZO*CwuZcU_5qxqB@}xpXYeM;_*%&$uo|cz1l&^X*4q>E6AmjLlugiBNq> zqLsN8p7W~cnXFvmBS@#IJ9a40?)dm^f7z$NyC*lILy6sEo{;H&Z@6k3!-zb)9Ci0K zd*Am+PD;oSID1OwSFbI{gKYZ|%o@e@tb+6m z5){M7Rb4c%8{W?+dju41j{56)0Qg$5Y*4zK!26GQMpg=6U4aMV9d4B3G6)^Hv*CRu zi*m&M70?OIB;qB-v(AY3z+zj&UDX-e)!MZBuA8p-B3>Q` ze2(^AUms3U?``%-QWISHs{2pxV0)9R{aA(Jz2Uaj_G^VC(g!#d$n(6*q5{0SoBqxd zpu{nAB>)|)`AxYTq98o$Qd~95i2x$pr2zT^+2=pmHuA~1mVso`s?qvo!NCOBU1J&d z;F*deK?2mHaI&cO_+YyTC#Djqn7>fDHx)d3>Lq$_<0}eyq%RF)=o%j!Q;W`nvs{5` zLU!X(gLyE1I^Zlgr!@fE9jRE#W!0r>YV+F5S@p#yd_iZK0-u>Is@dxo(tW0OX|)dn|9_#{Gj&=l@ng0-{PBe@t(E{E{FC zw%Nude0HkJVj->YDo5@oM!!JXy7qWg>9Hq_pLp8w_6fp8x5!kA)42%J&#l#x22JyE zy9V(z$&PS`JGJKEpE?H0Ry&&}bQ#7igQ6^Z7T+XpL(=eMj{j?pYzTU^#VM_5QJWFF zcMb>ZJ=P#dKcXYb<%Dz8OWZ8Ll9)KC@?qO?7_y|l+^UD*H|rL)m>OZUi7k7vwJ|L5 zp1_6^#pHNp&)I7z)IHIqxc`}u=b4DsLV5y;)o!pbuVG{!ww*r&+wm-L4%qoN-0r-! ztnpQ_{)d}V84BlM659P-*mkM@I(64m9JjQ+IMBXEYL?T6?A|?Dww5ni=*t+*H*ajn zaNZX*o8bg&JySl{`xB3XvKPy`gHu!Y?&XLOn854mpN^F~;fkXHunACT`jUXV)NNj9 zv2wYvefcGwWoG0zMKjN^h#xsMZrNdP|*Dvl|6WTzp81P4eG?+Efe)G*TROTf=)D) z*VpSvnAOztVL_<9tvIa4UCAhD^WlSpN3~DB5f6MH1AVG5*6u`8tjpSAzB1jbwQxqa(6i|rB%Se|wC=PS2PjauA=ohYuOy}ZXq`%=!)?nf7E zRGeD>z@XiQqc$R4gyW*)KKlQ`P=CWQE(ECk8S~7khZl3SPpYE#aPp2?xg5k zYgB*iud1owOuM$Lh&zCJT5VdrF?acM1srzc=TYGLeT#=eIG%hHKUV?KFit{$s2NDY zLpfOxUoQ@U&fT42ZbE1OQ0nij!wRX+GiRvx#=OTa0n{-oSMfm6!^ZXDWx{L!5y1@d ztpR3(NM3ZDuKV59Y>e4Hz0uyg*#`W4GY7gW>*rn|2QBawj%Ip*|KvF01Z)YJsEIT! zkd5aYiutHjq*hqF)<8~~P5HEBK(~E6sx*LCm*S-am+pldYb!508>!{l76Ya~{58(k zWe|zgM@oH-{HXjj4Z7I&W;S~L__omL!o_VpfGq$sd9&IerDdBf$a?wmXi_H!b#jwo zy8N`c{Ow2HVBEmFPl72E8(&jDzZ5SLt^6LUqfpz&b`VeeM*zhB%C{D+1W;BFM8T0o zsX_ZR$32j{6XWX>XpumRC7E*J-NG6bg@JvM9QaTzZp;@V*4}9NplW-n@wR7Qfh~bN zXSryyS}D6>=WOmr$(a%W|0zHKW8IoqWp9Wr6v06aziI!V>Ko^#Oc1y(yD9 zBPTs9I%;HNrDYwGm<}Jj^6NOB13?i-Ua)p=%djMT*9g$!xhg zT4~6@PjZzkrqwt&(RFay(wjA+NEZp>u$V4HG!w7_{xjQms`z#j&m-8hdh0*NdiIBU z*@H^WfDdcDvtb~zupcJNfgM=%U$kK-IN1;HMnuEiUi00eX0;=`0E+Uucz|ViJvwC* zXLXDl$kk>K*s#4LKcW=0q}lthK;;rKZi4#NG6ne>e4+9UX1H7^1$E&&W-7a&5ATxJ z!Ve3Y#YF%y2)|Cu1}(5E-Vm>k@rF~#*9=6Jc2e2qy5d8!*?n22jICMQETwC|e(;`4 zM2=c*l$I6=g%6B%Wcrzx{1tsJ@z=${%sSlPZ5@_Q1>V1YpA>V#H-6L}2ccfD*uTGG z)+IzpF1t?u8VhGyO#1y(DV!m4uf2Dj{;Cj*JatM6;CUDeElo<6*C|pW90(vSYR5== zHqPO)=o15R4YcK@3CaFSRZ=tgO{R+(A|}rJc5T?Zw~21&1+TmT2qbNf!#Nd6-3@4{ z*Bh4_-Mf076+OGpmW0Y4@Y#C~s;nHsIlFD&Kfm_(J(0{zirLRmlEIUj*0i04Z6cEj zhxLnw2l~z)YuJhgiv(-v92>MXOF31Hc>;*C8kWc)s!ROa;0_Wj=sqXti}Gn0ZM|Dh z^Xzr$AM)w9EjUlGhow7x#xIzQFd+NZX75pZhS>pLKr`x&c+ip|v}(LGsDOsuUQ<_% z@>Zo(K~)(QuZgC8WuW5hDAt>j-gc1)gpiN+f@Q(r*c>~k4Y zPA}#eo#XdKbRC;?4b{5Ky_|8D;By(P>Ov#Doulrp-sF-eI`T+FnPdsD$Kd5|l0+Lo ztV%wY8Veo^54xpp7KNnia~3TNeJ`fMA0xI%DgODRQtzQQCt~Z(F$jGt#YvC99IVCY zLWeql8pm_@r#)`Xa9q2!1(`t zRq-lj$Hw(>#u&W8EB8dg#I)4%^bJ}5J&PLB)>X3*JiI*msrV@@E;VUSrY53mv{z*C z&T4u%dPiO0JMV(oYI_|&wcqYy`qr80Ae2{D@U}YE^(Z~WsB{*H{JN*pj^63QpJFBC z_!b`}(4u9gV0p~B9x07DdoV%H9Lkb0W&abE*+^Xjo@dvI0_Xol5 zP@{6k$!FUd36O7|JIgkWfs{qKU9(c=4KHUmSY!5p*f+a~0K+-kGn9$Qbo--c)gjqK zUg1Q7ASLXy%DxHKe%d`nl0OEF&9nrNn*8-MXVStY!ae8K%X89J!n7HpmTXuBX6x%m zwO3v9*FDqdS(NY&@r~X|aBlw#pZYagsgkaJN$&OB&`Ii!EVyxMVVodhb@yUkw5Ns9 z$KZk06|;6cIaF%(jD%Z)*8AcE{X8r51>fluLo`vIU_^p-vU=O58;sNGUlr)FD?#G5 z=cV8uO#Ut~5|7|4ErHPIKOY#C+P(b@Qp9%tFA6S|iJf)wx$-yXh0tIH5h%TT~`S>3-@KzfqrZsV+_q7WZKV=u1>(08qAiM zD&*eY`n3J&@ot$iC!!xDGU~8OJPKh41~Rw$GmOJP>$40f*nH#XKL&6zHtQshk3WFR zbn+!GL@?Aaa;re( zt4K9*n+-hyglesqJJ^}i?w=X+cXy~maPqwYE%aGh&7*^I&x5dwZ%>9OZ}UKL+quhc zTOt(1@RU%E6xazPs#QFk_APdt*wqDMHf2kBzFfe+l9to-Hlkz`?l;45jX_7tK$z3* zpSp)Ltc=|aVU_OsS#$Ulhn_Cu&`exH+GNs0Fh`aAa^?{P4d4}A;#&3r?%F`FSzrhq z!ujbicqCYboO62s+(O`@rs}u6SWE?9Mw{AM-WzgfX-ZWkLOKd7E zz0gETqKYw3xRRZF?=WYZ|9i^i(m38MB`_VI^MljYc`OR_%G8_O4zGH2tyX5Tjr9J< zD1Ni7)U?YxT+`9}W7&)MO-L}1dZ&Aj(_wSi^zKFE??U$br3G9MDr+5j&lXb^5SP%n zw#nLm=R)3qKPuZC8k>Z_ncPm>Z5yW>=TNC_9I(;5xO1R5I}J@!qqGYy?BB52fr?gz zU_Zh+rpM2^q8EyH(*DehkL3u;vFz8}auT1l37R`Po^8tT0!mF=oEg0_!*8;Q7z=xe zPk441xY`ttoKc4flPJD5LG$SA3LI=_Y$P7{B(Eh=cvN8(s=ndxKD6KSXXX9tu=^d* ztx>mz-2EUOa(7NePZK5Ez>!Tq8(MJat9iRh`b=co=!Z zG;`N!F_QO?6MIKgkYM0~uO$U_nd7q1+?d{Hsr&!JnEq}+>M}%G)}gwZmf>852>;`QRG1|KmlbH6hkxfRm1C=Mp4-<#x%rgR zV?AMDUD*wJnK4t0G^eJY{yw$&d{2s${fOkXg|w&~wKx$sH2d?H-Di2IrBao{Z%Hy< z7s8vfkj|vL(33!Pr1U!6QWEq5d4a|pi?>-!-*qj0d;a&8DYTe|d?FwvkKYN|yFWd$ zD6n}>wAcwnUuez&u9B4msund) zY=TB)n6qr>&8+F2SZ94M%rtvk>jn>6{3juH|Dw}V@g!cguUhX`b8aBLtp!od{7;Mx zRz7`y^abY)TVk?_>!;fI`=AOQnU8r5(aT;2!mw~6Kh_S2*dfo z&+n@NcDV!6CJaz8#IG?LB$kLk@Tk}z`PIXS-5=JtwbNb|Z)lEM;B!&(r?yrC{W}>% zh0B4$Qkj>7jTzqC4#uQc+kT%(nNYEO7nAO9wgorW!aOl08a|pgeM}S1Lv*5tf_gXKg z*)_Tt8g*pYy)-TvZPNc_Y+bU{-isEtXDG_j^Dc06hpV`;5&Ys)R>yzLb-rgjTaAnO zR2BMsUYFn{frq=m3rpfsN3sXSkKb(n{4omjPutpOyZmsGjvheX@g5ht2_glkfsc0} z2b8<|vnb+Q-@&riR(_zP1#3t(53w)&>uXP!t2VoE?Ueidj}TpvWuVD9lwaupv!2

p z^(rrDi{N)P#~Rh>%g8@(_u!Ad4ju2^DsGRYW(}|-s=G?1QZalB=U^HE3aKv~DI6=s zs;tADDwg%foVv@FzCQ6BDph4Wq!4;vKMSH+^nGIj{^1fb^%}TPVSWVFGrO zre*>T=#zj%648z{M-YtiGVTe8p-^dy)%*vSyh3yPJ=1mbxplv!g*2#C;Cln2@p(mt z*I?Nzgy!`xQ>G;;2Hri@q5QsONB8qTt?%zj08DP@>flV+dM*U?iOFV&t{lA|oF)ic zA%k7v^IYz8s!BM5-_dsXyV!EE`n}f0YYM&WeS?)9q+M7fUh`RG+8cB34&b9LP@RxJ zS*ON@z1ji#u8dnL@a0qfg?ve#SlA7Ao?9U#mhU92!|IndfS#1nBo5b){uT_Ix; zIOflP2;20E&K}o|edKiIrV;@vdslsi4Wo$zU43?KD&J6;PJhkc-?0@Bo&Yn^IM%T@ znWdsUw>-31a-^kN4_csq8+mx%p&PP@L-bpv@gI52Y40Iymd zc2m_;E8}`sr^j0ctxJSJJ2`)w$i3t4;FINbHo1|2u=iSr?w8vNyl?+*KQqtZS6SOO zhF|t+wf%NG%XRA7SAit=55{2m&Q1fl=FHh85x`+$>8f?vR& z@RNS@lSiYWP1y<9Mm++IhW0zl-|HuT0#$J_4h4zXr2^5wYlLzjtA*Z0?}1Z*J!qQ( z9mFi5VC7{o5SfcXc5v~P*TcKXrzL7n!J=P!yB@uoOfTS_Y+QV?rz=r|4T&L&6E?8G zwPT8-`9Rs8MR{OVQa5DkQ;pzL1YX6z3_)Dv_^`D4LHVZp?{ck;4O-q3vqF0gzZTf0 zZHkF(gKj+efXLmx#Tg$($rR3)c<^b0d;l~XzdzZ39(vx8n02KXa(9YHc&f^qzDrZW z$xjsQN!N^T+8cc}@#;NHhb|R{tH$KdOs2MWFgTtf@R-!-xivNxQ-^e|E@LsqJ z9?#IZ%%xpc-wm58%)JBF}&!f?SgjruhMwWUm=$b4|Ih8 zqZ}Q~(BiBFZ?hdpDRMhpDMQBWTl~PO6%Ln+!g0HX>({BXNgCFv@ud?;_9izNr<0F% zQMg7DB*L=k?7b9QtOeN#%jt0@g^+RN2IB z#3thmx(;Z>`Qa>(K}rtj@JB7wQ*ePwu2qR090MCuBiDR5$-Ost0?LJLX)4hc53`4j?*k8i4?c+2c@yRtVWSD}*sM)wKNlDHOMm{) zVfcMKG&Jy>ua(44OhVmy*`6~PM^Y~SGU@u=qQSU+zO#b;{9|v|zm_k0-XUwfTs_u> zfpsF>b2ybUu-tx8FnG2xH(n!s-(bYdhp%10_erbu2>^Xgz@>VL)CVO~mNu1m_PI%Y zjb5I~q(l{HYKj43kMyU|e2#MfQN#AQAXD`!)Y4XRo?H5wWiAW7$x0klFbax?Kt-~R z$sx|D1L7}1_}M*>%1B)b(rb0~7pPsQoa?23xJ2PZ4fXgm+o?xDv&mqhEux=ahx158 zv|s~hUbD}znTc0g>>Dg)Jwh!S>s2#b*@y;fuw%Hk?yn28HJ^ruN1r z>YR=14C;j^xIB9PkFoEL$9jMNM>@8bJ z8Kq>e+a`PO&F^}v&Z$1_Am&bbeV+x>dKUa#xAp4anwK25Q$twfE5KE7@c+?>=) zY7fE)1X|qxPAI}byN`-q^!V>;n`Yj0L|X!pn4=ln}F6-Mxf7gt)SC<28#SO>dg1sS92(64xPq4 zzQM20H+F%3@;MHkuLyQqcY#?aQ5_8*DcuSP1YGOjhZa_jKkez`$5SLvmm=oApxR~; zEtoy=7MJozWj1!k)7R?3Yu^RUi1wh-mYKK%sN3r;@w-2G^_KP}6t^K&^44D*Xu8?z+y#da@df7~QXd?1fmvAuUm0S25Wxj1*8foLL!)=%O z1*rQw{2#n9vBY}$A$^rO?=2h4?fb$f@~%{<^I#wT8SJ4kvqSwEGbeW&b&yD*=5XnS zzQlz}ZPDwhU$^o_(k=Kf(4&|L!(kGlNp979L0$OIgZtkjLZ`?nDfV=2J|K2^5<;+R zA(W`!|1YXY4EOb(!NZRt9{ut`rqWqR`MZG;5Q zj|cMm$3k$o@1O4k*7H{L>bFTAMJFn80+!Pv2hMFv?0#>SPVix)zwLC|I22xG#iH`@ ztn{jT-rDjdU^$hO(TyM2xmq+9cL>YBe@2u$9zFkTV&?nt)xhTq^_&u`vj4t^zi;d` znb?pZ-;;O)n`8mxpT`BoIAsFsENFkQe48%d4)Th{V9cG|)=nVxN;|FziRIlO%apDG zPaf6=VO5LRv}@lBG9%n3uizHJ$<2G(?w>uIQ2%;v{{A8>qi2aBcD+Aa#@)lNR#QgW zr{&Hzjx$X@U3!DAFaiH+;Y6~K zRgd|In#aG!-@iYec&rYVo$Tv#MX#L0Mz$@l(Ma~Jw&Z4tD$jRck&H_qx;K@YY>o(p z-Dyqg1`S)9hi-`C4`YI1J&-wLgX|+tWo|m%e@Kcw?+U$z%RJ8dwAfnoMR+5w`&XV|OfyO~z*5H{!dT zVTO-d@xq}!g)KD*MC)5?Lu#_Ld8FJ-?D}R@ltrE-|BMrKPYAK>Y7@@Yco7;cIb^>e zOq=rg&r1S1U~s5;F~9Ds@O(CzNKmx!TTJ*`tT$!GqEzIK>-i2Q7j7fg6Mewhoh9&u zsCU5}7wITzy}9kvzKL)fq3@rxJ3_OcgG{CjKH>o}Hjt6uI2(M~V0pZea5WQzSgKO3 z56Dq?6+muWgK^WYg`p2(=YoVxTUGd>W}ee{h_;ICXztxXf$5A}h~%UE1M5ABXcKZ`Hd^lwf6PU? z>{Fh)lV0nyvhZO)`SpdD_l7MQx<70Fa@t~Tu&paK(^{!#w2~;+qXH?*wNLy!@qZuv zKVNw56Aii$*6)A2Wc66y_v97j%TnVOX3_f6*ZWV{U=LK@ZMC6_8$PkUyjVxmv)ZXo zBkX}cF3dL!(&9%!T(AN$_JE}1AIczz(cw_3*L|pW0mx6<3sB|!p^HHMTPr}wgh2mu z0P4#0(I8JC02(oG0~eHi9OIG0!e$?2<$Ze2lZiz0t;xYvTGn5lUnm4sddkzgNwSf| z>Vty7g%XRhJ|Zpxs(2y}+h7U9=n$VtRR!kpCNz z7!A@63T2c#a;wVN|A#i60mo^KMft*G{r=ybLViY`{1ce-#p5G(UjmGt9W6f#X=QOV{NwQ_xI)p!63-`(oW_6wuqUT*>c|-M1<|kJjCXU87FX!FS|Z< z1bpWW5No;fi#EridH`#k=^5B7+#Tig$b|#KlkYg07=i9{9I#*(gxd@8hOuas<1GL< zi7Twm*y{p8SY5r(NncJvrmHKqAD^6w8_0gUXq}Jct5L2=Ws!@Kvi(ton zEGfv3=cw$j4Gd`RO+2ro{3y!B5?s3T_?pl^jfnr+onCfWQeUqXbTG6BQ!`F3XQfM2 zj+BR2zw2oeh`DvHBWON!7-)2?F1R3ut+`=02lGs~vMaVs`IrVj;`s9Nbq}eO;38fW zACm2P(I#hIfb#r7+VkW6L)7XE+mi|&6f=X>osi!QUcOg-ToPLn)y4}Y!Ut1l;T!E$ zFiOkAH-)f%JW>!#gQ`iwLQw5{RSsSg2POHPe;eY2*bdy9$JB=3lyBIb z{|xljo-4si8?Kz{7ayq5Zh05xY|-|&#-Fyc-uS@eb>?^z4)(W2H;&;z0?kxwC*Jz2 z>(^%gL1R#%#Gb8Gy!{I`AgCkNGW~+otCaoUD$TzyOwy;fpzHyekZhq#EMu>zQFyzH zAwq}vB}T3RwQ%Kjy8bTf#QsXMh)?+1PhOcB4C1?6@7zB-1jMqVL8b#+9Yx;v2fT2o zi@ld=&{f8das_A%$G=94wd(IJMIeks{21z5e^=n13C{);4u#!omLTtj3jg%!WyL{Q zx}y680?d5O!xwArj1E1=l@!&d-W89=74s!1Kth!iQ#9%GUO+n>fT;Z7a@S*OA%SYj zU7Y#XqZnNi<*{pjheZm}cY?iV^v=l{S6sUOGZ*^fp#JlpvB$CB-d9t;oKE1TkT?3g zWl6f7+>}X1cl6o1y{+sII`wTBj3Z>5x><9X?~ptATO9)hzgL(s?GnyiIX4SUn(02I zxqz;50b*T3?Y?nyyOy`&_{;5u$pI31zA7ZTAdu{kZHe`>`-cXpU1n|~osU__$qfrW ze?n|*tN@l6^n9dnI6=M*6ZmGtC+)#~qZEu9C{N=h%SJBkcMB4pPxa3@7NdI=MbCtJ z=;RfN8r-Ym&1x6w$do!f18#g}P9h_#d7G7&p8lN3o{S_IkAzjrE|7K%!<$}7m9#fz zexvi=c#D&Bsmtwt=T|-UR_&>2D)X~mlRX13Z%t>FS(d~q4opjbOfErim=`-zm-L!( zw%2CO5a)@_dwy8t?AqUsY8RM}ZchLg1s{Vf_k0)-PrHC3_`yiYYh&(ZVe)HFtDNfkl z#cVy=^zWZfI92p4aTkrT_=MeGx{r16d${kTU@kN>BO~K-U#3HQo{>!OO0q*@joM22 z33ly`{m897^?h=d*{oVtQ+?o23bDqtOtX$=i#ro{v&0VqJ*;nUG(z8DaR4;bbY4mG z5w%@DMsaCDW|tz`e6Z_%JA^rHfc8zJ8yFZ+v*W7906gHQmD zRos~qA8F^PB*Qx&23`!s#fkbfeek4pur+YH5^Uik)F_YxfAHh(`y2|u(Jqr9$MAJS3;kcar$rqXn zt@Y9|^0bgu{B@!JS-!*cMB7&%D2{Y1>IGe`j*e<*|kU-jXK@14CDRCWvcN znOS2A$naL|KJq+T1*#hl`990=t7^AJIvV25GlF=yBNENK-|CPS6m*nUEhWyk#SyzU zImIuUD&+Trgz2|C58Ja%9xcYt=NBZ2ha@W3W54+RScp7PvuBC0o#O}$%B-W)g3wnR zKRicEhEo0agWiZnsmhWIMfha379>t9TvFh;<-Y#>;@={e3%&+JdsbYJ6e#U+l>&6bF&9Q?m zL&VI3={KRO%0%3GI7HWI+F(9(_00EO`7i5fC_@|lksVDV)G1IJR4CC%vdUyC!_ByL zxTse7$6iK$%AG0Ju440?G`9|!L}jbbwTFIV8ufk3tSzKrheNKg@IU1I{=2=D8HvdH z)xuHPVe7$G|MGqZPZiZb|3tbsslt3~*@AK`J@Q*O+0VUz{K(z2gIK8DyvMlbc|9K= zJYuh5;zvgo-|rhO7ETnr4b-b^yg17_m!(DZA6tYd&8~&j%+R z`iM{P2B{s)8?{Q=j?*FWFf*ex693s@+>n=dqA zadS?5`Jr%o_-td{>G0LKv^sKwVC}PdEwgNvF%vKNS(QKe)Ga9Or|?OU=}+2fX^~}D ztsqaa-UAaC#QD;FSkdG{geONVYDx+A6#x5xK$pVJD}k7-%V*heHnrBUbsT^B@6#ql z+~k-rw5`PdeCD#6zMX8Zd?%|-)POcg5mR160eh!S_-YB)G~;!_R{lfRO_7IDLfllz zYgD`Tr!#<-|Vd|zp5|dtf@x90Q2aXCusRk2OC>-5wo;GjfmMpv#zgF_N(MXoV zthTe6`^N)&@))JHBGO}cHNW#5*4E*(dom9lPFe-%Gv$5CXBXDk5$JSU;^&|d_TeA4 zZ)+5Hn@av=r~dg8HHJ;>H@4J$B1T43=*ahTLwm|=Vtvl9g78bB+_a$}V^=a|l$=MK z`Z%j!hhI_InIESU6PNS>DpL5})&2V%)oC}&73(@57fLpcICSe7b7|z@+2?V$toqvc z&9Jtf*sW1AGZfHRNKTs)cUNuKj{JIEO?c9nQG2+vxk7wgZB!$^An#5(Yvqy<=dzKl zhDKonZ)Lbf>>qpd$EJ$qA$}AvI^1bMZ1s6XK;Xo2JquUITw`2RY!VUffbGM1R_V+W z42ga`4CS>F=g5H*oTDo;>{hru0PR z;yoB&Y!*;Lcbj-5jpHu)yE$snU=gLrVXvIxJdJkT_(E53|+vDvfY}O$0%LYlI zz}nK$3nP8!Y>hhjvKLy0c5ffGQ@Zx_=J+pHpC9f6&NSlHRCZmDJR7yXApLHVhguF_ zr2RB1+-_FP^yAx1E01cVxgU0~S>Ytfjoy)EK&w=a`JrDEkB_=kYKw#_XV_m{%oKnD zO_81S-m;XoO*Ko$L_zJ|L)#`)LJ{jWW%X;ug2t1?)&Im33@-x`X10fiwtPE{EBBpJ)Z>Mc@OiL?Ux*=x6eSuS+zRYaHsE*B2*WA5EPh{a;Flu;xAl2Q9)t>nF`G^I77Mg<=<&FOzN+2-MqUE zwR$FKI^2h1_C44HfV#j-z$~KO=Fcwy2sjm6cAdg!4coTX0(C2n* zYSTdc@i+ooiR_W_W}C%jYpQH+NtS1{WBB$gTaOfe*%s|l@7VYG>L2RtxmNMS4U~Ug z!MBX_z*AU}Sw*H}5Q9JjS;2IL4!Q$4DxiT2ZXTiMoQh0G0qTU^s*eY{U)(IO;Tyd$ zrX);Lgt!>O*i&SiIu%Io`5jKB(}T8*h0P_GIGkbb%U1dKpitoJ z`6`eUwgB!TJW?wKk==Y`u(mA5Y3vjty`5pvQ|JISRU}=HE?D|;V4N1PO99C86KobE z#H$lB<1PZhnL(O+q(Tb54?O*=nVS0mle|OxCiO8OHy|yc#9^XREkxB!T+m$Vk1B={ z;4%ywuz|bV`YDvDCn?I(6e^rFx5XOrim!z|me=><*o5m=CR^Qtj`4K8t7BV=byW%b z0G-GFvqT-~8jbYzt(pz#LaEq@l|(m3+03vTQYOLNZ&dt|sVDmI0oKcXalMIU3#|tF z(FJD4pwf^7egYp4KRknuxhGlk5NTz)EmYw!4!tI5I6DB3u_Rwq+A)m$kJRGtEfkAB zj!SnrF-y%)&v)5`s=%+1Nnjn%3Au|DeXzYfIg~Pu5@u7g65JM-H|(FQ%>J3A)X>K^ zGO=Mz?dCog4$F2M^ulq*e%E9Tw8$uQ#s1#GIPqoC=m!qcQf21dT;yp~*6%PG&-5Zg z=8pYN#wVu=;xVK#n1-Uhs?onM17q9_m}qv7o{><(VUmA2(FapH#gen2rk0AdFh|7z zFZ_#qUaGnix}Dq5p+meMIOa7}%O_3?cmNVK1G(+pD3F8dLp)Mlpbv^z0@#Pd2*5W4 zR8~H)Wfi{%D&pMT9!YAa>Rm8%E1!iKL1+QlR3EQ+NVg(Ie~NuLPHfVxVE@Yfze*Gr zb3PE*UP&;o5j&2vdPBh$8B*iOpBbpEM8>}7T49cR+@ZOI-}j&7=l7$3L;-~*DD=6N z=IHdP)SwuTz3!D6<;J5x){xu=%F0gU6d2kW3#`-uFWGqa%o5x-JHh^|zO~L-X}3V& zO>cTLX>U-|cg8T=h9cW(<)s5Kyo4(|nzXf5%s%7n-h8tYVzs1H0m-%(T5@o|^4=2d z(K9#hd!L^;VsJY6rxt+Asl!(kX%8Jn0YUJ}W-dUePEQ_O4W!V5n^CA(45<+T;7u}@ zJc1RMVD$N!d%kIV{e5BPE-!+^o{CXhM#&c_()sO}H6M(%?+8%vn44%{+@SUN7D2Gd ztZ)BuJ_o10KJ4{;o1IC_pSvUxg^E$Am##X#*_0$lx$nuS^n#j5syK6g@O2FkM=bCUU1_R1aYJTcM!ZeQZ>bpzgCG@)nJNw z&j4}MkQ@wMkjMJ45XbSjwV*>0ZLrFw{3@Pc4ro`FEk#!P3YqE5?k`4w+@35s*ospP z_fkv6))bq|=4galHnG4IW~Bd)>ePs?y7*QZ=>34Bvn`;gmhe*`K3$rfIpG$OLBuds z)dSf7@W?8i2MlpfZMWa&*JV?V7p`V~I)o^@0dQmMK@VNuN^Ok_mLsw8n3f=VyyvwE zkfZ~p-XAr<-&vVgI9~PHflqY(&n5&&%5IQk+8<<*Py9VAFpl`nym!hEf)wk%Ts+UJ z&zhp-Y-a+3Rqw-Dag3b_$6 z>lA05Xg6R=v#&$tS!2Xf?=keNH0!q_0De+szFNpV&W-F!(N>+U&ld|mt%WKG=R)i# z&g^(HsN~_4UTNB@KYUf?N=y4-OrGX4cbg%eQ^JnVY} zCz}*t|1B|i&7Z08e40{!v1edDJk#uqTz^mqW0rAe4@C1#fHO}1CH6Nz7i z=sRP8_?$Fbgr_-zU_>5VCPlqG6wQi5b7Z>;laIoaG*=;VO|rNZ%ys475;_vw@dQ`N zy9$Xy38buQIn_^-Upc3V+<4NEq4fr#oprMOlRW>uNe|LpHEG)D4Gs=;=tT+=tKdeJ zMEQ&S5GpM2CS-`iiw8NBuD7WVs8!cJm=?d)e)X437r7A?&9Pbz>)O6 zs<{B%lHMj{B(GMhLWVa3c+fZBOLBhxc~Pt*>0I@Uti+_OBWahecas|3q!Mv|8||^T zDdmeK6WrQ1_hIsrFF8xM3=N`<#+^F0a=oujI*k(P_DC?xOG@ph$GpWQ3If$KL+?uqYgThvpZJ(Keq7qFi)qyS*yCRy1;3EJxJR|A)&{b~HHwx_Jp zZs~NnFD+NVpGWSFoyhKriiWrJu{2v2fpB6Ct4n>C@j6Ho6_i%7^>=~Y5{^F;*3a`o zU2}Q*7$x=jDQyYfYt4J4Rh;5S+jgMa{Yuxm>hKghOs{wr?0Nx4^DZRlriZqO0Q?A)P}$i$5Yxi`EnJ8~o#j^_r4E zd2>&uzs7v3&Wwxkf#Y?9a;>5fN-e(v@mfO+y9oIh*(37erH?XW4o)V=Sk}L0z1iQc zp>Lrev>GM^&HpT3H0a#TK+^4)8Rq@YClBP`M56g^lltJJ@YoECck-~1qZB#)LMym5cx{EWV> zE8L)IZ%u+JIKwTNiT`3foj4XIttP>vT$yGfazyKt(HzyE=kG_psEX%BTiUuz9d4Z| z!1^A4HczUJ8vXus8Vg{>4NOGh4r)rBuFf-m0CK1XhFDrd#14(UsJG?pfRsoZSfVT) zlL>PKVZ3_}gbr&Wp-0GN3chau4bv>iZI!EPe*L7oR*>WD$x1hGk!H+rdTa)mWzu7~ zLY=5%s42X;A9GDHc_tm2ftB^En$U+{#W2-(-%OOm3?o_u`?6K@j6*mCmlAOiFD2cV z$tfO3Oq0sNMG>>_azAxc15Pl&uO=Zg50e zm~>J)gB>%+g8<2+M|l>t&!?BZ?lCJ}9T~CD%n4i4H9&=@1in|D3Jk_LPo=f>FTiz* z1-3T^r&{Ltxj21SJ8BDjx46v03x<_MDS_DU-)t;Mh;hZe>Jr~3?yj?V-&+vE=JN3O zvxedI4f{2|GFlhHIy5)s3-xl8;fjZw$F-zO#&R zUGg)&-R}QFCFF3pw&wA5(jkS$(hkr%2*=7}$r}XipP52pbZRd|59(-zX{j5@Gr27+ zlO>VfS)b%za31DUiL#R=KJ_sEo#m&k@OnP%Th}2v?7juO;${j2e(lj+M3-t_VHxLG zcrB~0^>eXhpT_ovv`4I z<$g9EdI1u=(xZ{_=dN5I6~Um0}bsh9#P@!sG1U zH1I5wR&VWO(4Bd8NzQ*GImKtG=vG}&(ls~x=?;6!a{B2QLC36e*rgnds_Lqnocm$5 zd`ivBBAJ=xCi|>M(ipFNJ4~)~15_&Bz>^lY zT1+oswj+8!%%RNkMhSsDdB~IpsNbtwvJdq`#bOLmu;=n{ACMEKL-wFx8)b0^yG#%A zbr3^PLHPa%uU*$QS+$t>y%fehU`IG#5=s0LNS?U63#28X8tZ$O$%ViT?}m=E36MP2 z8YleQlIe2$VG{3-$Nu&}A5%21Nz*%(rGBt(9BpOGiF6HGH_K+%&d-6gyyt!l%kCol zrWjR>lZ>k3A0z@!L@Lu4L3y~2hWnZv0{{u(=iLwRNMu?3WfFI89&VG~FIkde>U-Mm z4suAI88a9DJV4(w8DzM@L~1Fg%`Y5bH|rS)f7@48KzV!tL*?g zDXI;m>;Z&MHe}y0fuHOY=VK2j{)FlWAr&*1ZlGi*Ze=0X&%b7pe9Z%pphaW9oAL!R zkCIEm2{hd=-D#=b)!2s0yte{MV-CZ%;(L{@PC%HS1x10SMQx2!H%H6Hbl={~^;w)E zIlI{W3Z^ecCVd%9+TK>p6NOtthm*&k8W$*EnogxR8jjch1n!Ez6hA{D)T&Xh9nz8u zYM>(<<;W&zsm1ysZ3>_80avt0wA)*K!VodPb*TQ5r!@5Kvd%Mh3h3L8no-Z5J;46U zxd{GcfrgJajc-q!QLv*T8Ek*3`)t2Utn2^Y1yo;@t^>$^nny< z?Mo34JmRiBCMP>rPl$K;K4D(K+?jsP>r{~J+ls8ts7M&_Q?cHB`!(S)B798AYx(kp z5OHjbxZcX;Z}9;f(#(4S&&g&~XZ-0f+6dk_S-%Xlh?|(a&B#`=cBjJNF?;QcPiqu& z_N*~a1J%m>Yacqfn{DoLM>u`ZDS3R1%{TXER#TS1ibM|_JmH4$G!Zp@*S=!?$=bj6 zxl%`Ljk%JZ@N8)CqZiV(cG9-o$6VaU*aFvS*5fYCi7n8%iwt*G1yLNVroRGe|EeUR z6KU3n=-Vdh0z8Yfgte-xNNJf;fxTkR7$HXJnt)_e>TZ66Pze1jHvc~6KD0y+?nY*i z6$rsFm^l)(h7n@`#t&>=;^S#e$4eCH->EX7E1{x5?1?9vu`Wgm3S?tHrl29Cyt*yu zI5*R12u07hiEZ=O)yKuMoSUTEq!zq~^)!2*hO4@Ls%;|hni8jNuRN8ym0zbk$7wXc z`Z+6Y?ZntI`@D4b`rRM`T}}q56W%RmL}K%0n8g?$xPk9h^cZ zTTV{~6-Vzp3ij)Hy)0KLmb@(_0P|Bi$;)gsX5YAQ+!K?Z;UyHy70at9>#sTnoUfZ_ zPqbIp+JE18Yoq&dC)(mId%JzFuyf|JGF$klM*DW^++KL;zM9+qbLfWret6SFFcUl3 z6N%$@Mu$URjWUw#-w{q^;7E+W=8KCVGZ{39kDZ3+rAo-4vZ@w3JH&{6JzH7u^ zTid(4OgV%GKWQ{q9v2|>3V;)!D&)0rk=%VQzO#ARMD?vK+DHtF)gxl4ee#IARh9L9 zR2ymv?wb`>UoGd6yNF4|sG)`HbwR+C0&%LJ>M8A{vOh0uwvv=+kw9QEa zP^SoE1AJcDe{U}plHgVj`j!?h-`FQgnZte`@jgzzC}zqs;_LoYZ}9x5fpWD5#?AT_ zRdTGYa>mnsbp5HES}rFeQobhBW|!tvE8w=-VfkU-^FyzA1P94(n+ou*;dr4@w0-k0 zYi=t}?U6+hPXB^>x0z!(J7{#jKV3$%;pb`|BUowj5cVv2AosNXD&SNl{_DrOqi# z$T-sE<+82lL}rn%JxTj&m?dkTf0fZ8vg`FyLFaM8rCi ze5|uga{Z!I&NA#K$8`SN(l~ypgit6OTP@+U3RLVLg(++st}{g9CXrno*iktH=BBfP zT{u_tii0}1y$D^HFMFBCP#GD-_ysz=@Y7}FiTo0t?Zl-e!^0YxwHdWqpRH7-n`fID zcXlN`=vCeJ`_=mN-Hv$3czfmdPq6D(xF*Kgl!YJNtBlih__P!EUsM2qFB`Y&R~Fb? zaZC~iFA4weA9+NImNmB3iFt8NyfZJbur8Zq)n_yH3^aOHgd36!&0-pQ+uPL}`-!Qp zJR*P9kQ!9TKtW>hIm~xXif&OiX(U%hmezTI_`0h3u{E;m8WpTYWy))E^Z`Tq7dp<> z>b1_kY1`J=lS=sCFsOJDccNQgi4LRVsOt>Yp9jrf0!3qz;Jfokk4Z2XEzzjb{W%B; z00dZ9-o*=?j>KJR>24!%F0hK&suW{rD+tCAe27}5CEBx4_-2tN+#PY(gJoNNY9sY-+8%p-yQZpyZDf%NycMoE!83Yhq2{+7MjJ_p~PF3Kco2C_`lEMqKZ z$3=c|maasS$-`x6xn~E&CdevRTnzJ`rg3(kM4wS?#hY-9H)K$YsSP{$1GF5tovgx* zR30sA{~$FX7mB}}C$48r#i0zs-(iv1BY4Mbia}071a)mi?{H$`gs@f#bOJIn~W*L6`4j9>vt1pt{Zi$ zkC&?%4}|WXZu*YHgrAKaX#LE}dpP0G5`&h?i#813K8LU~PThqsxTCvEavXOg1^M4z z)5UTU#!C_bUi04RtwJYj^PVcmi2d;ic!-*8slpXI9}2yZ#8rKWx|j9Dl%V@H>xu?d zjGei?eoMD%VI`5->*ZWfK+1b)&P?=(xXGnx2AIIBm9;?*)_)*Hv5d!Y4FqOcoT;1( zftMbWYjkC*47!!INpDyjJcL~{zMR&GcFR9@N?2QkG$YKX5?0#(9QJP?uZ;q0s7fxS zV9`2Dz@2Iye>W~DSgJG29eEU9qC~ia1@Z3%3Tg;9pZdwg+xT?hnEmg8`w?o}gHapq ziCa!Vb(^$`NxZ0g*P?^*A`ktx7nzS?eX71&Rn|n471|CEIJ<%du`sIEEKsiPnYla~ zc_^eWP&$ecGurOJ=y+c)&g0WhIQ?fm^LI=~w%$sw04LTvaOsI8YhcU1%o^vG#!0KP zdM8hHH+75kVqw}?=3|AM<^ltTzj9(yE@STYlew?^zcL#WpH+%79AC^G5(wQiOeZ$? zC>th2_Tc34Ym9jTMW1a8d<6Pw$_aSlz|Qme|KR=q#M6SXchKJaCYTLQrTktn;P@5n z?zy)?^cNBT?T8ZtR^s}1R zhrL=Di1w&7povS(WExx~Q3u?e5iyT6eyF8wA(pvvk1bIaYpxqkChE|;R- zzu^4zGitAeL^--fs!DYtngc=&};~wm47! zN92(Ib_Rc0J5@6*sFMqz>MXOi0Ia$qA}^1N{E#JdHw&x&qTHZy@@4h_f}rgg1fxLg z>R(CXApLSF)gnM~dSG{1$viuE0S{~SixXnVZDLoo@I@=WBn4vua@xg$0{K$=(HJ@s z`RC`WU}`~Vx-B&{IGWq+nOr1SO~S$~^j85Zjfsq3_HoC&kbE*rVQmqzs!4Umn4WVZO8JEC{hY0jI{l6Af^OwP>I1X(=H;e{451ZY zfsp7e)p$3*j$=H{0@2atYTk{Nx(x0O)r`$LLj(1SOs|BT%(A8#qb|fOkJ@I5&m771 zouG)EV-k313#Gge=I%B*ftK&GxP9rK_^q}-8z8?w0z=Wkj5lBNKyOQ=BSDgI8tOj6 zW=C-Z-zj@(B+O{ z3C|fT2M#@3adWA?8F_CxOn@YrcmjuG#l}=aswm5x)XBQXqNo6$qxhZMlP9^LJk*h` zFX94xfmztxIl%5HyS4~ceB5~C$ap~Txs*j#X$Du2K<)xI$%trW$XUtKCpPaqF*#OL zUJN7r?JXcYTcGoE0hK5@seBqVs4;czUjO%xXgW{s>>J^Wt3{l}hH zJc3QCq`+bi6e-qz#Ww1|9Y=uVwZO&bz!c91pZuBRUt!FOA!};@vt%PQ1Oq<2ZRqnC zo!9lzrW3?lrN>asT=mx8ci%0(JE+Gx@d^MDr zTT5}^ggvmgR`kGN#7q;2#@jad%^EXQ=PG@AaPBPt$ zrNd`^+7B}%x5@QgZAviCccsV|X3^~d3iS~IbXLK&FwfUiQsj@ zTN7u=*I1&hn2ERTLFC6(1k{uAVMMK#037uY?7L6a08nzJcvCV7P;B}}w5e$OV4fmu znD1qo&2jy2l3E{2bi_q8zwa7WDca@;i^#$|ld={g4$adLr8h&n8O$2(T$3D1vPh_s z3x43GUN2-(G$%gbbXrqSEu7-g(J0LNbb7>vBpos$kF?yCE3pE^*H#4`(u8LVxOUaJ z3%Qn)dQ^!T8He7*6!?z_+TL*rPeCVR~h`WG$_-k+B@r~;-- z9n4&fzOO+fRiH1fo6l@B2;&%>3i{_3hqcWzM?>~66@umpkHd_nf{0GGo)Gy-YTGXt zgfrv!rRcSeNDQ!tg?pdf!0>t@PuneWquDLuMWCz9mrUfX8)Ni#WE5ru2dj^C$$7;g^0vE6>%Yn z>Cks_Se1(OxGw=D4%mcMA`sCy(yOMfRa0YE#nI;9&xENz-(NoaRE8rLZon=3iQtaZ zjzBbvTdp!uFa>vg!Ya2}$CokyvdjU;$x5amIKK0~7!iK+ag|jM~qwYE=w`-hw zijQw&r{k37uI1gi*kJd{!7q4*cPY7Yn++qoj0{2>OSrT+^?B5W92qO_=>A&IG%ygo zFTqjrpAz@Qr8jLf(rHNi^!3+EQBerba4Vyjv8(-o(1h-@Pr_WepX{<;+=vL0OkGJ{ z7F+lJrQlnj^h%P8+HxcZQsv5De#_d0=9719$hsA>wy8lCk1pxIuuV-b7o)Lxl27dm1}NBEKb$LdQq z!_}%|;M&DIPaZ0gfw53|{`_bPY4qoaHR1%25%oEYl?@l zUtbJ;F3;escUXoYnyWMmB%)o`R!Bb>zqENMxg;ZS{yu>paX4c!ROBX@qwypQKXT_D0%FRl2qfU1njPrwL$mD7_PrdscIvmpzE=c6A03dL5?LY zs{ZPuUP-^?eW=(%lBV&8#e5vD&mI*f3y^$m2O z*G%c=0J!VR-nkFTGEc2pC%@#oUv9S+9l^RZHISZ%l@VIR0Nl_){ z%F$fc;&Mew-3J*ieA!0 zxZZ`bK6cuyDm@E1Y~|9CCw0Lv zp8>)zqJpT<&lxv}w6R!R3UVRHH$Oaj$^d}zpg!2OYtY)8$KcDBGXReP7~M>`O;X>%HH_# zzir}QJ}B}sWfJ@S`h(Q9NI$u>MYVQb!;Oiw7Xp)SjS{gEtOhC=kH?BT1z%ozv$bPV*ocEnA&@g1S4fDeT@oFRY}%{ z5pf~s8_>pKdpwG7A$b5DK_(b$h&!=AwygG2?QsBeCq*OQFDhaz@`pOQIF+;6N+5otOw?g&Khfwl`{0D%!Z-rPKmDrBM&~W)PJAL9~X(*MzKLu?+gJgSY(pe43a8T2 z5gzL^UPnQP853k87jfB!`c(w+7D6n}U$GX$A65}WcL)*Bce)xA;=@v6dEN_yuRA@& zXr#mYg2%#iz;!ur7X(!;6G$rsG`95kGwj|;4_t@T?K-qCQ$1jU0pKAjv%b=Tn9o^q zQ9RUD*+7?nMOUnpaP<{>9XjL7PTYH8D)NhMl~Lw-e%zM5U2Vy$3|E2h#W@Qqhxstz z*#+6?lO}x%=qt3Gp|w$^^CGn%tGPQAG#V;nmt{Dv&8wxKFSH@~M~e2xCjRX+o8o%` z85d2O&w(xT0xZ$$GaaWp8nzE~$}~f`);+?>^uNhqJpW>5aqjho-0e`AvIlO>mnxg> z!S!(dedXG0kH#mo-_Ei1Gi`q6@fML?qo1!%kC|nOepS)k<)BZf&j>d$PqO$lC&x3w zjq|@)`|ly~_hZ-|iORxy7vsM5sh$+$e8?apB`vwW00X6M*~HUlc(WH*&ytkMS|IR; z3@Nr!$RRaR{iJ1v-jjj<-wzhvuiYQj^!T-HX<>Kba0p94+RFq|*QPzz5lN%kzY4z` zzot5;_dttl)MITI4rhb<(n3nP~0+&yw&liVFr0xUKqoOC4rSL6;(0*S3M1t(!FA zN_;gJT`$=oAv4i`U)|W-L?1`-dqe||4~b@m&kA&-eH~>uMvibNDZVvByWbssHeGDK zX=E0Yu3r+w_RymwBX#%ByQz&H3;)Eq98fddeiK3Dlbw-W9j{@~Dzn(R{3l_%=M8I-DC9eke)D z;}pyB;IZB-yp>l#-Yoj~lS5!yA!l2KtyXf{`m5QUl9@9R7ZQgx7S8Hs;za#dY0bG(+tX_j@;% zHtmv(Q2cqF4wkm(I%_H)FO+1cF+0H5KD#enVARkr;Q2pqHX_{13-1mwD!qIq2s~sf z2FyVa1T;moOITP3f$TD+wDnbJU8g8CN8SR#9g*A(u{i7kGSTFFLBDDGd}LY&nR{A8 z#_Q-Sq4Mj3feYeiML?EsaDb^F@)-+|``{qzg>2Wg95Ma@(&(7Cu*64|QauUGwtEtP zg*D=ww)Xraze^$FvSyC7FoXo+s(rMtd!zh_j6}d|W~qRI`4=cmJ&BwSZgXW^^Bragt5i=BRyjcSyZ{8}F!Xw}zeU%H=Oo|35*9 z)Yp@4RSxu?wpJ!b2(GEF!#>c0=Eshit21Zg}CtzQdk=PH2 zyQJNVj3+oX7kx5!o%La?~!l_aUu`e__8a6MQNDJ5j~0Wt7IgRtJH1wvK=iq&R` zURZ@jCo~t<^D97Asfd6|;JGq+y4_4yb6b*)^m=;yG{WIG)BP-LGDA;sbbN}<;SAP| z+<6pPX3S^uQJ<44re>t?Y5IoEJ3n<)q;-C*jghr5LzYBrr95=f{lA#R{}Sg=v-Asq zLzfoMgA;%~26F8Mp1N3j7BlTi^DbCO${_4MU7%wfaPF>4V1|{)&$9&P01EVF9gOOF zB-}y0HU~IZQ4hSOo3~*=%gI@dv#b0?}*y)mao3YF85LUAw95s znt>L=1;1n5x4{OA$8JJV!8qfiJ8QIneIDq)I-_r7p1Ba;LZ1HdHl*0=%LbJu*+!ez z!=Agr^`Iu@&v0;PRkbX@ZL!{057i+8Bfs8H@PW|xdT&`U4gv)CeRg$6x!ce zJ}{I3Jv1lQ8-BQ48T*gC0J{Vy5C_jMt|y_TOuJar^mibJ}pkV{xbFa`uE(zUf6=jc$ zCRmk`P3EU34SOUve&&AK6x`P?mr#kr;rBS}Wjh5(bGTEr;8p+s1iPqXwXTy z@LdDqm*ng}UMiO3imx7SUY2!KO1ZF*? z9x@K0z*4(H+llK*%=OG?!B1`Ir0svCFTW%;v0XrDYKi;W5u;N}X8L$d`BA}<+^fZ$ zKH6*%_Jg}Uy7fM~9JlTcMSm86%83h-N&&TAmELrTjWgtHH+ zMDX2hq~b+pqY=G5zlk8|z)^CWhC%A!Jjyd9Zg6FD6ZEhjBzde%7Q!iFF)i)s#rgnMEI?jOg8&k$i$LmLdSX!tga^5J+1a)?qZj8(7V(Rbdq1%@&y6hffuMpV3!S?NY-S)_qHyE$5S&Z2*am#E6z<50qS{{~y#$QiY z2F-;)>-V3d@hud!60kVolw?L~`gjMDvF@zUOt%|RpNmBUz1FQ9QoI^ZRv0OXdYtYgGQt`bU)PEitK_q(TE6p+uY!IDI#tx_0=#;=TLr8a@^( zI$cg8)lork_7dVi+&3n&zZtv>a&38lzO~d?i5fAxZ%45M9TEYy(;pM!u)- zK0=5Qh*aKnUjnKb{6jNBq9B~YH_w31-FqD_&gJ=j_}(>E|BQbtBZkjutZMM-Td+`ACR$SnyuuAt z(ZzVf8BHiJHT=Kk@6)xn`2N5{{d;fmkJklm0!ChjO~$%Xi@}<GrXNe9 z4_C#eXe1Xv3;N79jeb#Ov+kXKT4Gk_eHzAn6ck*&in3SG0u_tTt?oa`IbNs-?`u4w z*C=>@ef=XD<3}PscpQ6#Kl5$mjf}*W?7FB8`@3l-9KKL=*WWm)4qSeZxQX~Yb;!XH zBh+-7XCaz*5|+U;z;C97iSHp9Ee+-(4oOcRBB+1U=>#K$JcXKkCyxY_oTO}`xUl@; zUFCB)M!D72IjCtK`+lI*#AYX#Lw|Q%X%}12I=psf2ao38GBp$3F@a5IqkAy zGPmh>q|zgaHV8i94wx$u{Pal1!AB9W6|wIp!2;2AjMniI+s?)sUxbNBrIi9w*HDGK zxnKNjBAwEAZfCq5(y*%U=!LVqKh$?sYJ0=A2ZlD(b%_IBDSzHG{Kv-GhK15PCizYI z_kjJ9fz~M!Xqk)DMyT_}-t^X$4&`Hd~FI2PP%fOzH+EGJ*7#8kus(ngnd|93AwpPjKPzLv_`%E ztZ;e=LeRI)eX@5Wfs#i8j@E1deptrTJfBAJuEO*!N;)hzi^7=%k3MIp7$aaY*a1-R zH6rD5<;qylP!iiAg!UhLwkZ_4(+Ba&-%nX&$&CPV$l{NjC$d(E=nN42I&4Y<<{%&`1JVjg)LDs4JXP!pkZ%*j?AE4P49=-uN+#)A*IpvL z_JKd&>Ho7%_RStJ5So>zD=yn)Eg+Im8=c9n>HjfMmdU&kNd84Z&Q6U zWIiEi>7sqDq|2-!Vlqa=e!O{wy^zjZES1}xDwipPDPb8=WE$&xFlDJ;`@v} zlfkVYFQa@aZ4Nid#dO~s{X${yTw978`zsXhjiRv!atyIp)GhmLg6D^#KB4(53-{e> zF029pehoOPE94l=Aj8}Ytm-1Zn;DWy2_2X=sVUS1E>PBYEkVm*RxInZv2gBWwY;Mg zL!9l6bmgHyUwjz6a1ns)c_a_}1_)F~Q-`@{dLRS+XV{dgv3``*tPf|Hu z_AVFRJ0GC~m!aMBpeR6UzF?NpLbAVYolSgRN=*x>~yn>kl1O#f63JQpd z^BjJ;436RDm(;s;I8KC5E6RzV?Y4t-6Bc&W#P=XAAOgo`kDSGbGYf(iF!J)AkG`0$ z#GNMN_zTE`73KJpI}mOO9tNcv15(sxVY?<_Yxw+>BK91$L70N(RcVHs7VSjZHAOr@ z#<6a?{0nD{hjup{W_7}Ph`0;E>56ps>8ASFR<#&Ul@5mR%P#IcBTy3CekPUv9bgwl z3tzQsxu*2>iUZlUhL_N6MH=j*`{i&g8thxBXM^HDy+)%H)aALMO`320CeSl_f_7hq zYCyS0fX8{m%*ew6^m`Wq4%zINQYcKLoeEt~^4+_~?UrA25l2UBLhO2C}g}FW~xPyM?XHG5bA^;+l2<7&;d3*(;$Z()8J?;k(qY zhbv;zQnk+*QolbZd7Bnji_kv6tEnGQl4-KM~Ww;61vca^J;}4H+YS_f9 zQ2RIM#r|R4=1;rFKU`bFl~YmgZd?)KK0<;P&l=F?4LXmA;?H5AQj^p>SYC>I$gf>RBI^`eqyGA0e}E)e}Z{i;@z^gCDA1 zNA3*xL34umONsrz{ICCZ0q#btpBfDT@Jfs|NL!tVGB=GWV@HVk<@ITjHX1~Jj#4n5UX z>TJ$fheYx2-4Vr;{d%Weh1ZJvl>xw)jrgU`;U(SK@ZpiViPvtL`M_{jf9BXHl})YV z1`Hgpfo}%qFabzdO-ntfeeVOf#bc{KoEo-mOM52IfH>Hxj^TFGoUlMtMb|nL6lBUk zK=`0OvC~i2`6F*O#YvfBh=XLMDbnO3fAzV(q8ubL87?F@i zg+?mmLe=BE5^F|GvijI#x?R{@xumAzKy35*eSbg0wd zWg?8yyi`GeRmLMq_Zaz_g$o_P97>WcVfb4wEq9&HO-W3Q7RBzWq6a%Yo$*N`N;+8& z{oqF8X1Sz=qgsw;-tv_bL$klrKo)N`w+~)k)G$7id`%l`(je?Mj%EVV4u;ut32AeY z1=y~q&w}qiH+>@eH-F&*^)`H#s??@wjVc-MuZ2^7MNjQd7)3eH&}RPDJO9dQXZWY$ zHf1(e4ysF0UnE1FVh1@+ z2D5KI!)3bv&b!}zt}}MFSU=3Tn#I_Wg)38zaLwD)8aCePPhGnIM?+jLVg9l)TNNOG ziEh?`zxUAa(ta*Urf0_$? zrTm>zM!CmA{uI-cI=}vF*@cmd>bG@F8|xbB;=&eiJ7Ebskm5Dl^E%`Eq7FALGbT?) zFdHB5KfeF>egJ)_jQEsBO2FGy$(JFIW+cCcO{}hZ8|tuYq)5ICucO=J*k))|=jNR= zNK1a%R7TsN&h&f7$y15D-Va9;PbAn4*Si^Z+25=U@=WvNdi`sMb1US`O2KH1V2@t@ z@t7y}vuC0%n4bGf6zShFbpVTmujtB+*CJkO9lGW;1~(%xZ&d{l zI1SO=cu9&6Ac}V2Kb?DKIonP~dxfcr@ZitHeuW(k%cgjHIH&ijGJMj$rQ&>~L&;Pt z;jF>>zCgjZ+|?(Jmk0dblWKF|uHBMhiv*I$30q9sci%%SpdGR*0@jo^%b-d#fFh`B z-=Y)&A3b%~}AX@FnLb}reA@%-KEZhCgVk55$T?ljp(DgEn_gF`xy}j zUy{GN7 zDhemPgpQco5In)piw@GYs*~u_j_`33DpwO3F_rwy$K|OsRrf6q3kla@l|vfKu!W=T zbhVrS(QzI<7#*XbE6vE9>ji-rb#5vrqBg_Aw2krg0nKzd~;SJE&NHcqYW@ zg2`(PA749kGDfFQ859~1-hl{|PJ1vRZAbhXXDY!>g(P5ft3d-<$6Apv)Nuph!7W4u zrlW6J{qP7f07UN$1@erfv{_?wIPRt#z_ER1c5yy5 z{JjoX*h0d)Vy6BtUY5D;Pt9)+ceglj?Nw3s7Xg(3GI z;7(owm|4ej8Wx;#lp?;(B9N#wz_OL@mVQqU%0%_i=YmyV6?POq{i8OGykj>$vljs7g2IX5e$mS5@ zONyungIIk_<*Cq%GSt4U#C;u{yO5nQ+uEN&8`>2R2;^ZR+MTAbn&y)QFHTBU9!{eR zpj{W%YO{gr;7au=)wPS%xJMzKL}Di-0vSC@;l3a33arVHCS8r%fn-ONWtWAQO5sd~ zO;|{#1sjFbTs7S<(2o9zy66(9D+&Rb?m~8R)RQtbB0zsok~+mK=92s|a#YRiGN}vX zwyR86XAv1Os=NBaqs5OpuSOv6MSXuP;msAGSmgnQND7(W+XBl2?6&Pfi_rw&!-s- zR4sNprvAT33I5PU|HR7_V_67U5fmCcb{M2}DbG9~dha%E?|rt-R-V*YGu!N<{XqgB%FbTkt-U8=XDi{t0-NSMds5@4{ zVzss=5PX1n@zPYH%CiX=V)Dz$X*V8>qyf)vm!tC5JOLL+S(weP&ne$ zxJkd3zM}cN5v5~1m7cc{K5^ArdTtKX4RO{E)P&ZpWMew{DKflY%SH6euOYT6$9CZX ztq0G`J^osOG+K~UMn3!pca*;>`3fqh zVz~Vw{m=D1J|LdwbF>h;&I$XGMs)QjZ4tF;3{`gecGdP~u6$1zOByuc?q4{fN;#Fw z`67}+#OuO`R{+v*v5%DXs+;89L>KP7{476j8i124o2d9`ZUb-HW>%yJQ- z5D_L_Uoam-2D~=@yK&Y2`o>a03V|*66b(}L_#z-2BA4FO);FcLMXG?2?J`tjqTkj6 zD#rrO8WmmcCn%lJv8c_4&!@P<8Fl5>VctDEd8(!472G4l{ zd&GGS(HlxFvT|3i-8c7!b3tRmwJzj8tvUTkYa0HoZOFNTvGI2iDG!)Lw{GDN_Ou&I#VHrqe2(D1lo{8#EacZ zd$>1W;HrJ}5j)46-KfK*8qmW`r9X`r&>=vArbFg|B8ZquC<%`Fq`d#=Mu%?%t!|ML zFlu#z-*BqvL&Lk|u+La;D!n|Ls+4d~^xn7=%Z65;MY;k*%G@?l;?3cTkc%Pe?xGj$ z%|3F*aC7rwXmzt$Ded|07bm|D%{)vzn-D>-e7a~4ZQfBOxS_UVPZ&iA$EbHiM4o!F z*m&aFSq>hInH^1@+!IBW7S%A3G}V0PZL9SrDTzk08Bfs|3u)gS>y3gpeOlS5DcBHeC%2H7(4S^Ct#Vt>7JljY#FE0w z%JX?yi&G-k-u+87rE6VdL)#!}{UG3pZRG)<3lZDP`Gen08Am;$98aA`P3q z1FHGs1dA%Dei?KIgRzYqRplw971&ge%86H&UqCznG0t%!_fUCw+~J5OTPOU;6IYuB z(W#aaC~3(J{pixvGrpnB657mx-td;Ae-Jm--|T~lzLg=A&!89`(*VC_WgKyRJKW~{ zxh_3rgQz+O)?`^CuUk`G#A&ourJyg&`I$c3EQ-NxyBs5zqi>eOp8X=Ir7*L2kAdJq zm!Z03!K5O(?;!cfOHkVhs$exg)yF~qsMyFLf z3~bNjmuAO?$G_T=78%ZJqpcKd*A*F#tWRpX=nD}Jfq3+}RJYN#HWAfI-p?TK?P+-=Q2FBa$=5xwpNkhFSpT$qRSMMa_4_ zYz?N*)=}4LY$uubsK-m{*(;?oQ@g>`X#Ok$#q=rLxa!70_$}A#`??z;7RX*_HE#WS z$$h(t?F!Z`vUDehawJ;xh85R@ip}L)&oK(AL3JiePPZiFNwS+X2M#0yjZc970KllBSs5vT{4vySK{A5C^t$ z5Mi`_|89a|9v99$vHbmgAfr1Cy0L3RZY(*h;-|zBpV&+usg;-+Hc5AlFU5<921Ip zDHAKukWI|hl}I6;??&%F+L=v5`f1w;doKz_cY&HfwJzOyqMvE=;dbn6o8oW-VP#xv zilg|ruIiF`v~p@qxqW|S1+LlBSwC<-E!|SZmITT);>G;**nM0+q!7$@(eBE)t<{2& z#Egh6;WSJBexY2{>zq4{3JFBA!;XdOdTCEoj?Kxc`JfS|Db?$whi@NdCEM1S8nBGK zE$iDNKw9qjT>vwp%_EetKF$`d4l5UQa_VcB8ET`O~x>JV2hH|!5D7}Jki2XZ3IeY5qTe~0J6T;bv11vwu@Uy$y! z00XaokpSCR7`b4gN=}dwSyJZ+CmQ`!_S3b9O;7vZ;?Ls23JW`bk@@yaLAx_Evi#^W2qXGX+6&Q+9@dnst zP-hg3U>RF$49gr6&6_oE)W&El>Woah=<2MI$9J=%D7tD$A&0OBY?B1YvPN-Po{8BF zr9aQa>BrlmCDytv9{GO$&68)5?bX${x~$=$Zo*|b*5_iPT|edB>=`Uo>r9T~6KQxk z@_4zM?a>w1^Vc0qSgRczeT@v;NyQ@1wEkJyNb!G%bZYU(-I{`EdG8m>r<9gO%12#!EV7IE9UT_jRu&(#HHvdA z?EK;=wU-V!l8}|?*0Yj<@s5-Bq|Y~$4RzwOiezItJG8B~yZat> zLMAt=AHVH#DtqHAPVdXL!{zpb7ipPc!lCxlN&IyjUW$z7E@ch6QVtiY-125DLMwms zag@2U(-1#sRz9655O*i_>bvKp&R4d)U-az@g|jQa^OZLXvGAyuYT zw8M9kYV@U(bC??qRz|#NR8X_qd0II2=1Sb{W_~@sPm!M@Q16@8QeKP(9{69&z~4|+ zm4kP67X}B)v@FRHlWuET2Wz}4#k-w3-nfmoOWuhWBn~mwMw-91&=;h<;0riROY9JS ztC!rlQi-Lsaqaq=zjh__O-65@P)M*&s}3aIb4Wx8=qKg-hKQ( zYX1K5oczlHRVumDEWuF9{L4`{x(H?Gj-iTuq3L z+xq+QSS(hb&&k8%lJ)L)@87>~Tj|`)CG<~q$2e>aSQ$_^9Zh{pkz;+*o})~$^OGg( zO8M?clDtvaxfp}TMZaDIm11v>;n+#rQ>?$yNxqvr<5Y%gGBid-s~11-yK{z^WapK7 zql`JVvQ0@!GxgZWE#xBDTyRGVE_YinEkuhwFP$bXq%H!q#yM5t;XA7=np#n{tva#q z!%P3f=__qT5q4#b<}04DOVdfD^5A8Hp=Kw0eadi|ynGU#!>CZcjH&-v&~M)1@V#lT zGgUJe>X+b6DkfEQn0hAq@W{6be<{lrisTo`6&z0q?1eby^2VnALh>a!L9f7}lBk z#n*BdKLr<9H|RK6XNAWeJkMWqV!^D$J4-C7(WOtms<%U^IGaa`82z(1B~L%r94)wo zT3!|HtK0LN_gj}LPpa01$0|;B?TZb6%0zMKLT61KSBYGlYgg^yb9DoQ#Tc7;7yx5( zU@dsRig@;nLBHRnsopKS=$_6n?CwbZp4G-LbYp;)jV*mlZ6W6Nwea{7dtz7?;UC#( zPi0OTnpSp|kFyZYwY@DQ-srX**5|jJ`kMzTKkMC;wKy%3x}jd7yx5vb@nm8}am8?n zBY(;9`dYr>Jco>xa)DGp3^5dG69y%wC}Qza>R8px0m5B^a%*g;&j#Iydh9W!RQZfO zxh$&+Yon>VT0{L7KNgPH8y~wf_3`6JhS9M-JU0|t0}b=#jJUd&AB_occ65%NJkbgF zi{92KRjRT13*&DNwC9yf*ot%sFAN!L<1BjKjJ=s_k8qz4l~T>OvuP9+Ke3+G8tSQX zc?_Hu#z0G)UR5Q%w7AILIi}dr59$XkV`F1tnU|yfHfeW@^zw}AHj8-d(Q~>ljtc0- zV%Hg093}5e_daxWTx*ixYU!yxm})TBnKZX+GYAn&I`<~}cAaSo2P?V8b8=93L=!K%t|U;_Hcy`LPHuAjIVc%P zu8mI@*PdksL1NR>PjlqZW$~2dALK#ug5W@3(*v(cKkw~)U5t(%03x zgH{A)jHS;^Z`zfPY}T(wt<1!ZPkdWses;~Fsdh`PEQ%yGZSUH&5<0`$FJ0+}WfiJ8 zqqHpUS7l&P66)2{yRh|R^W%EZiu%|lZT}GtFOo!!^IJcU?X~01GgcojS4nSd&8~l( zQ6F33sD!^*eg`+Wy7^_gm zC3%D3?Y5xo*aL886zr?wp|qTh-+jwQIZpR3O5a~s4xn*i6sLd>ML9LuoU0}Q{|A5(Wl8`5 literal 0 HcmV?d00001 From 66c0af1a1c8e7a5c0c7c2344d643ee8331b39928 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 12:46:57 +0000 Subject: [PATCH 383/526] Move test --- .../0_stateless/02456_datetime_schema_inference.reference | 1 + tests/queries/0_stateless/02456_datetime_schema_inference.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02456_datetime_schema_inference.reference b/tests/queries/0_stateless/02456_datetime_schema_inference.reference index 2b88f2783dc..0719f0eb44e 100644 --- a/tests/queries/0_stateless/02456_datetime_schema_inference.reference +++ b/tests/queries/0_stateless/02456_datetime_schema_inference.reference @@ -9,3 +9,4 @@ 2022-04-22 03:45:06.381000000 2022-04-22 03:45:06.381000000 1925-01-12 00:00:00.000000000 + diff --git a/tests/queries/0_stateless/02456_datetime_schema_inference.sql b/tests/queries/0_stateless/02456_datetime_schema_inference.sql index 34749dbd412..123bb324f87 100644 --- a/tests/queries/0_stateless/02456_datetime_schema_inference.sql +++ b/tests/queries/0_stateless/02456_datetime_schema_inference.sql @@ -12,4 +12,4 @@ set date_time_input_format = 'best_effort_us'; select * from format('TSV', '2022-04-22T03:45:06.381'); select toTimeZone(c1, 'UTC') from format('TSV', '2022-04-22T03:45:06.381Z'); select * from format('TSV', '01/12/1925'); - +select * from format(CSV, '""'); From dee865ce78aee73912370a12131a9e4cb2d16d8f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 10 Nov 2022 11:05:35 +0000 Subject: [PATCH 384/526] Fix trailing \n from SQLancer status --- tests/ci/sqlancer_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/sqlancer_check.py b/tests/ci/sqlancer_check.py index 51c95e50746..63c7d18fe46 100644 --- a/tests/ci/sqlancer_check.py +++ b/tests/ci/sqlancer_check.py @@ -150,7 +150,7 @@ if __name__ == "__main__": os.path.join(workspace_path, "summary.tsv"), "r", encoding="utf-8" ) as summary_f: for line in summary_f: - l = line.split("\t") + l = line.rstrip("\n").split("\t") test_results.append((l[0], l[1])) with open( From e83878493c9d0ea7492288b5fdf401532539bd20 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 12:50:17 +0000 Subject: [PATCH 385/526] Update settings history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index b78b812da86..7635e121f8e 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -78,6 +78,7 @@ namespace SettingsChangesHistory /// It's used to implement `compatibility` setting (see https://github.com/ClickHouse/ClickHouse/issues/35972) static std::map settings_changes_history = { + {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, From ad377b357f0dbd3a9b9671d5828b6d6e34963057 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 11 Nov 2022 13:24:43 +0000 Subject: [PATCH 386/526] fix backup tests --- src/Backups/BackupIO_S3.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 5a4e804a778..8342749e230 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -89,6 +89,7 @@ BackupReaderS3::BackupReaderS3( , read_settings(context_->getReadSettings()) , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) { + request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } DataSourceDescription BackupReaderS3::getDataSourceDescription() const @@ -128,6 +129,7 @@ BackupWriterS3::BackupWriterS3( , log(&Poco::Logger::get("BackupWriterS3")) { request_settings.updateFromSettingsIfEmpty(context_->getSettingsRef()); + request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } DataSourceDescription BackupWriterS3::getDataSourceDescription() const From b010645c91e4c83032abda913ad6b9bc27b56451 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 11 Nov 2022 13:40:10 +0000 Subject: [PATCH 387/526] play links --- .../example-datasets/github.md | 128 ++++++++++++++---- 1 file changed, 105 insertions(+), 23 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index fdc0212bfe5..efe58a03341 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -48,6 +48,7 @@ As of November 8th, 2022, each TSV is approximately the following size and numbe - [What weekday does the code have the highest chance to stay in the repository?](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - [Files sorted by average code age](#files-sorted-by-average-code-age) - [Who tends to write more tests / CPP code / comments?](#who-tends-to-write-more-tests--cpp-code--comments) + - [How does an authors commits change over time with respect to code/comments percentage?](#how-does-an-authors-commits-change-over-time-with-respect-to-codecomments-percentage) - [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - [What is the worst time to write code in sense that the code has highest chance to be re-written?](#what-is-the-worst-time-to-write-code-in-sense-that-the-code-has-highest-chance-to-be-re-written) - [Which authors code is the most sticky?](#which-authors-code-is-the-most-sticky) @@ -58,6 +59,8 @@ As of November 8th, 2022, each TSV is approximately the following size and numbe # Generating the data +This is optional. We distribute the data freely - see [Downloading and inserting the data](#downloading-and-inserting-the-data). + ```bash git clone git@github.com:ClickHouse/ClickHouse.git cd ClickHouse @@ -99,6 +102,8 @@ CREATE TABLE git.commits # Downloading and inserting the data +The following data can be used to reproduce a working environment. Alternatively, this dataset is available in play.clickhouse.com - see [Queries](#queries) for further details. + Generated files for the following repositories can be found below: - ClickHouse (Nov 8th 2022) @@ -238,10 +243,14 @@ FROM s3('https://datasets-documentation.s3.amazonaws.com/github/commits/clickhou The tool suggests several queries via its help output. We have answered these in addition to some additional supplementary questions of interest. These queries are of approximately increasing complexity vs. the tool's arbitrary order. +This dataset is available in [play.clickhouse.com](https://play.clickhouse.com/play?user=play#U0hPVyBUQUJMRVMgSU4gZ2l0X2NsaWNraG91c2U=) in the `git_clickhouse` databases. We provide a link to this environment for all queries, adapting the database name as required. Note that play results may vary from the those presented here due to differences in time of data collection. + ## History of a single file The simplest of queries. Here we look at all commit messages for the `StorageReplicatedMergeTree.cpp`. Since these are likely more interesting, we sort by the most recent messages first. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB0aW1lLAogICAgc3Vic3RyaW5nKGNvbW1pdF9oYXNoLCAxLCAxMSkgQVMgY29tbWl0LAogICAgY2hhbmdlX3R5cGUsCiAgICBhdXRob3IsCiAgICBwYXRoLAogICAgb2xkX3BhdGgsCiAgICBsaW5lc19hZGRlZCwKICAgIGxpbmVzX2RlbGV0ZWQsCiAgICBjb21taXRfbWVzc2FnZQpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSBwYXRoID0gJ3NyYy9TdG9yYWdlcy9TdG9yYWdlUmVwbGljYXRlZE1lcmdlVHJlZS5jcHAnCk9SREVSIEJZIHRpbWUgREVTQwpMSU1JVCAxMA==) + ```sql SELECT time, @@ -274,8 +283,11 @@ LIMIT 10 10 rows in set. Elapsed: 0.006 sec. Processed 12.10 thousand rows, 1.60 MB (1.93 million rows/s., 255.40 MB/s.) ``` + We can also review the line changes, excluding renames i.e. we won't show changes before a rename event when the file existed under a different name: +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB0aW1lLAogICAgc3Vic3RyaW5nKGNvbW1pdF9oYXNoLCAxLCAxMSkgQVMgY29tbWl0LAogICAgc2lnbiwKICAgIGxpbmVfbnVtYmVyX29sZCwKICAgIGxpbmVfbnVtYmVyX25ldywKICAgIGF1dGhvciwKICAgIGxpbmUKRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKV0hFUkUgcGF0aCA9ICdzcmMvU3RvcmFnZXMvU3RvcmFnZVJlcGxpY2F0ZWRNZXJnZVRyZWUuY3BwJwpPUkRFUiBCWSBsaW5lX251bWJlcl9uZXcgQVNDCkxJTUlUIDEw) + ```sql SELECT time, @@ -314,6 +326,8 @@ This is important for later analysis when we only want to consider the current f **Note there appears to have been a broken commit history in relation to files under the `dbms`, `libs`, `tests/testflows/` directories during their renames. We also thus exclude these.** +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUIHBhdGgKRlJPTQooCiAgICBTRUxFQ1QKICAgICAgICBvbGRfcGF0aCBBUyBwYXRoLAogICAgICAgIG1heCh0aW1lKSBBUyBsYXN0X3RpbWUsCiAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgIEdST1VQIEJZIG9sZF9wYXRoCiAgICBVTklPTiBBTEwKICAgIFNFTEVDVAogICAgICAgIHBhdGgsCiAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICBhcmdNYXgoY2hhbmdlX3R5cGUsIHRpbWUpIEFTIGNoYW5nZV90eXBlCiAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgR1JPVVAgQlkgcGF0aAopCkdST1VQIEJZIHBhdGgKSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIE5PVCBtYXRjaChwYXRoLCAnKF5kYm1zLyl8KF5saWJzLyl8KF50ZXN0cy90ZXN0Zmxvd3MvKXwoXnByb2dyYW1zL3NlcnZlci9zdG9yZS8pJykgT1JERVIgQlkgcGF0aApMSU1JVCAxMA==) + ```sql SELECT path FROM @@ -354,6 +368,8 @@ LIMIT 10 Note that this allows for files to be renamed and then re-renamed to their original values. First we aggregate `old_path` for a list of deleted files as a result of renaming. We union this with the last operation for every `path`. Finally, we filter this list to those where the final event is not a `Delete`. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUIHVuaXEocGF0aCkKRlJPTQooCiAgICBTRUxFQ1QgcGF0aAogICAgRlJPTQogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBvbGRfcGF0aCBBUyBwYXRoLAogICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAyIEFTIGNoYW5nZV90eXBlCiAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgIFVOSU9OIEFMTAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICBhcmdNYXgoY2hhbmdlX3R5cGUsIHRpbWUpIEFTIGNoYW5nZV90eXBlCiAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICApCiAgICBHUk9VUCBCWSBwYXRoCiAgICBIQVZJTkcgKGFyZ01heChjaGFuZ2VfdHlwZSwgbGFzdF90aW1lKSAhPSAyKSBBTkQgTk9UIG1hdGNoKHBhdGgsICcoXmRibXMvKXwoXmxpYnMvKXwoXnRlc3RzL3Rlc3RmbG93cy8pfChecHJvZ3JhbXMvc2VydmVyL3N0b3JlLyknKSBPUkRFUiBCWSBwYXRoCikK) + ```sql SELECT uniq(path) FROM @@ -402,7 +418,9 @@ The difference here is caused by a few factors: - A rename can occur alongside other modifications to the file. These are listed as separate events in file_changes but with the same time. The `argMax` function has no way of distinguishing these - it picks the first value. The natural ordering of the inserts (the only means of knowing the correct order) is not maintained across the union so modified events can be selected. For example, below the `src/Functions/geometryFromColumn.h` file has several modifications before being renamed to `src/Functions/geometryConverters.h`. Our current solution may pick a Modify event as the latest change causing `src/Functions/geometryFromColumn.h` to be retained. - ```sql +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICAgIGNoYW5nZV90eXBlLAogICAgICBwYXRoLAogICAgICBvbGRfcGF0aCwKICAgICAgdGltZSwKICAgICAgY29tbWl0X2hhc2gKICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogIFdIRVJFIChwYXRoID0gJ3NyYy9GdW5jdGlvbnMvZ2VvbWV0cnlGcm9tQ29sdW1uLmgnKSBPUiAob2xkX3BhdGggPSAnc3JjL0Z1bmN0aW9ucy9nZW9tZXRyeUZyb21Db2x1bW4uaCcpCg==) + +```sql SELECT change_type, path, @@ -426,7 +444,7 @@ The difference here is caused by a few factors: │ Rename │ src/Functions/geometryConverters.h │ src/Functions/geometryFromColumn.h │ 2021-03-11 12:08:16 │ 125945769586baf6ffd15919b29565b1b2a63218 │ └─────────────┴────────────────────────────────────┴────────────────────────────────────┴─────────────────────┴──────────────────────────────────────────┘ 11 rows in set. Elapsed: 0.030 sec. Processed 266.05 thousand rows, 6.61 MB (8.89 million rows/s., 220.82 MB/s.) - ``` +``` - Broken commit history - missing delete events. Source and cause TBD. These differences shouldn't meaningfully impact our analysis. **We welcome improved versions of this query**. @@ -435,6 +453,8 @@ These differences shouldn't meaningfully impact our analysis. **We welcome impro Limiting to current files, we consider the number of modifications to be the sum of deletes and additions. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIHN1bShsaW5lc19hZGRlZCkgKyBzdW0obGluZXNfZGVsZXRlZCkgQVMgbW9kaWZpY2F0aW9ucwpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSAocGF0aCBJTiAoY3VycmVudF9maWxlcykpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgcGF0aApPUkRFUiBCWSBtb2RpZmljYXRpb25zIERFU0MKTElNSVQgMTA=) + ```sql WITH current_files AS ( @@ -486,17 +506,23 @@ LIMIT 10 ## What day of the week do commits usually occur? +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXlfb2Zfd2VlaywKICAgIGNvdW50KCkgQVMgYwpGUk9NIGdpdF9jbGlja2hvdXNlLmNvbW1pdHMKR1JPVVAgQlkgZGF5T2ZXZWVrKHRpbWUpIEFTIGRheV9vZl93ZWVrCg==) + ```sql -SELECT day_of_week, count() as c FROM git.commits GROUP BY dayOfWeek(time) as day_of_week +SELECT + day_of_week, + count() AS c +FROM git.commits +GROUP BY dayOfWeek(time) AS day_of_week ┌─day_of_week─┬─────c─┐ -│ 1 │ 10291 │ -│ 2 │ 10519 │ -│ 3 │ 10598 │ -│ 4 │ 10812 │ -│ 5 │ 9964 │ -│ 6 │ 4583 │ -│ 7 │ 5134 │ +│ 1 │ 10575 │ +│ 2 │ 10645 │ +│ 3 │ 10748 │ +│ 4 │ 10944 │ +│ 5 │ 10090 │ +│ 6 │ 4617 │ +│ 7 │ 5166 │ └─────────────┴───────┘ 7 rows in set. Elapsed: 0.262 sec. Processed 62.78 thousand rows, 251.14 KB (239.73 thousand rows/s., 958.93 KB/s.) ``` @@ -507,6 +533,8 @@ This makes sense with some productivity drop-off on Fridays. Great to see people This would produce a large query result that is unrealistic to show or visualize if unfiltered. We, therefore, allow a file or subdirectory to be filtered in the following example. Here we group by week using the `toStartOfWeek` function - adapt as required. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB3ZWVrLAogICAgc3VtKGxpbmVzX2FkZGVkKSBBUyBsaW5lc19hZGRlZCwKICAgIHN1bShsaW5lc19kZWxldGVkKSBBUyBsaW5lc19kZWxldGVkLAogICAgdW5pcShjb21taXRfaGFzaCkgQVMgbnVtX2NvbW1pdHMsCiAgICB1bmlxKGF1dGhvcikgQVMgYXV0aG9ycwpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSBwYXRoIExJS0UgJ3NyYy9TdG9yYWdlcyUnCkdST1VQIEJZIHRvU3RhcnRPZldlZWsodGltZSkgQVMgd2VlawpPUkRFUiBCWSB3ZWVrIEFTQwpMSU1JVCAxMAo=) + ```sql SELECT week, @@ -549,6 +577,8 @@ This data visualizes well. Below we use Superset. Limit to current files only. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIHVuaXEoYXV0aG9yKSBBUyBudW1fYXV0aG9ycwpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSBwYXRoIElOIChjdXJyZW50X2ZpbGVzKQpHUk9VUCBCWSBwYXRoCk9SREVSIEJZIG51bV9hdXRob3JzIERFU0MKTElNSVQgMTA=) + ```sql WITH current_files AS ( @@ -602,6 +632,8 @@ LIMIT 10 Limited to current files only. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgYW55KHBhdGgpIEFTIGZpbGVfcGF0aCwKICAgIGxpbmUsCiAgICBtYXgodGltZSkgQVMgbGF0ZXN0X2NoYW5nZSwKICAgIGFueShmaWxlX2NoYW5nZV90eXBlKQpGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwpXSEVSRSBwYXRoIElOIChjdXJyZW50X2ZpbGVzKQpHUk9VUCBCWSBsaW5lCk9SREVSIEJZIGxhdGVzdF9jaGFuZ2UgQVNDCkxJTUlUIDEw) + ```sql WITH current_files AS ( @@ -657,6 +689,8 @@ LIMIT 10 Limited to current files only. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgY291bnQoKSBBUyBjLAogICAgcGF0aCwKICAgIG1heCh0aW1lKSBBUyBsYXRlc3RfY2hhbmdlCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIHBhdGggSU4gKGN1cnJlbnRfZmlsZXMpCkdST1VQIEJZIHBhdGgKT1JERVIgQlkgYyBERVNDCkxJTUlUIDEw) + ```sql WITH current_files AS ( @@ -715,6 +749,8 @@ Our core data structure, the Merge Tree, is obviously under constant evolution w Do we write more docs at certain times of the month e.g., around release dates? We can use the `countIf` function to compute a simple ratio, visualizing the result using the `bar` function. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXksCiAgICBiYXIoZG9jc19yYXRpbyAqIDEwMDAsIDAsIDEwMCwgMTAwKSBBUyBiYXIKRlJPTQooCiAgICBTRUxFQ1QKICAgICAgICBkYXksCiAgICAgICAgY291bnRJZihmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NxbCcpKSBBUyBjb2RlLAogICAgICAgIGNvdW50SWYoZmlsZV9leHRlbnNpb24gPSAnbWQnKSBBUyBkb2NzLAogICAgICAgIGRvY3MgLyAoY29kZSArIGRvY3MpIEFTIGRvY3NfcmF0aW8KICAgIEZST00gZ2l0X2NsaWNraG91c2UubGluZV9jaGFuZ2VzCiAgICBXSEVSRSAoc2lnbiA9IDEpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnLCAnbWQnKSkKICAgIEdST1VQIEJZIGRheU9mTW9udGgodGltZSkgQVMgZGF5CikK) + ```sql SELECT day, @@ -774,6 +810,8 @@ Maybe a little more near the end of the month, but overall we keep a good even d We consider diversity here to be the number of unique files an author has contributed to. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICB1bmlxKHBhdGgpIEFTIG51bV9maWxlcwpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSAoY2hhbmdlX3R5cGUgSU4gKCdBZGQnLCAnTW9kaWZ5JykpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgYXV0aG9yCk9SREVSIEJZIG51bV9maWxlcyBERVNDCkxJTUlUIDEw) + ```sql SELECT author, @@ -802,6 +840,8 @@ LIMIT 10 Let's see who has the most diverse commits in their recent work. Rather than limit by date, we'll restrict to an author's last N commits (in this case, we've used 3 but feel free to modify): +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBzdW0obnVtX2ZpbGVzX2NvbW1pdCkgQVMgbnVtX2ZpbGVzCkZST00KKAogICAgU0VMRUNUCiAgICAgICAgYXV0aG9yLAogICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgIHVuaXEocGF0aCkgQVMgbnVtX2ZpbGVzX2NvbW1pdCwKICAgICAgICBtYXgodGltZSkgQVMgY29tbWl0X3RpbWUKICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICBXSEVSRSAoY2hhbmdlX3R5cGUgSU4gKCdBZGQnLCAnTW9kaWZ5JykpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKICAgIEdST1VQIEJZCiAgICAgICAgYXV0aG9yLAogICAgICAgIGNvbW1pdF9oYXNoCiAgICBPUkRFUiBCWQogICAgICAgIGF1dGhvciBBU0MsCiAgICAgICAgY29tbWl0X3RpbWUgREVTQwogICAgTElNSVQgMyBCWSBhdXRob3IKKQpHUk9VUCBCWSBhdXRob3IKT1JERVIgQlkgbnVtX2ZpbGVzIERFU0MKTElNSVQgMTA=) + ```sql SELECT author, @@ -847,6 +887,8 @@ LIMIT 10 Here we select our founder [Alexey Milovidov](https://github.com/alexey-milovidov) and limit our analysis to current files. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIGNvdW50KCkgQVMgYwpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSAoYXV0aG9yID0gJ0FsZXhleSBNaWxvdmlkb3YnKSBBTkQgKHBhdGggSU4gKGN1cnJlbnRfZmlsZXMpKQpHUk9VUCBCWSBwYXRoCk9SREVSIEJZIGMgREVTQwpMSU1JVCAxMA==) + ```sql WITH current_files AS ( @@ -898,6 +940,7 @@ LIMIT 10 This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the basename of the file to identify his popular files - this allows for renames and should focus on code contributions. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBiYXNlLAogICAgY291bnQoKSBBUyBjCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIChhdXRob3IgPSAnQWxleGV5IE1pbG92aWRvdicpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgYmFzZW5hbWUocGF0aCkgQVMgYmFzZQpPUkRFUiBCWSBjIERFU0MKTElNSVQgMTA=) ```sql SELECT @@ -932,6 +975,8 @@ For this, we first need to identify the largest files. Estimating this via a ful To estimate, assuming we restrict to current files, we sum line additions and subtract deletions. We can then compute a ratio of length to the number of authors. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIHN1bShsaW5lc19hZGRlZCkgLSBzdW0obGluZXNfZGVsZXRlZCkgQVMgbnVtX2xpbmVzLAogICAgdW5pcUV4YWN0KGF1dGhvcikgQVMgbnVtX2F1dGhvcnMsCiAgICBudW1fbGluZXMgLyBudW1fYXV0aG9ycyBBUyBsaW5lc19hdXRob3JfcmF0aW8KRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKV0hFUkUgcGF0aCBJTiAoY3VycmVudF9maWxlcykKR1JPVVAgQlkgcGF0aApPUkRFUiBCWSBsaW5lc19hdXRob3JfcmF0aW8gREVTQwpMSU1JVCAxMA==) + ```sql WITH current_files AS ( @@ -985,6 +1030,8 @@ LIMIT 10 Text dictionaries aren't maybe realistic, so lets restrict to code only via a file extension filter! +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIHN1bShsaW5lc19hZGRlZCkgLSBzdW0obGluZXNfZGVsZXRlZCkgQVMgbnVtX2xpbmVzLAogICAgdW5pcUV4YWN0KGF1dGhvcikgQVMgbnVtX2F1dGhvcnMsCiAgICBudW1fbGluZXMgLyBudW1fYXV0aG9ycyBBUyBsaW5lc19hdXRob3JfcmF0aW8KRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKV0hFUkUgKHBhdGggSU4gKGN1cnJlbnRfZmlsZXMpKSBBTkQgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpCkdST1VQIEJZIHBhdGgKT1JERVIgQlkgbGluZXNfYXV0aG9yX3JhdGlvIERFU0MKTElNSVQgMTA=) + ```sql WITH current_files AS ( @@ -1037,6 +1084,8 @@ LIMIT 10 There is some recency bias in this - newer files have fewer opportunities for commits. What about if we restrict to files at least 1 yr old? +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgbWluKHRpbWUpIEFTIG1pbl9kYXRlLAogICAgcGF0aCwKICAgIHN1bShsaW5lc19hZGRlZCkgLSBzdW0obGluZXNfZGVsZXRlZCkgQVMgbnVtX2xpbmVzLAogICAgdW5pcUV4YWN0KGF1dGhvcikgQVMgbnVtX2F1dGhvcnMsCiAgICBudW1fbGluZXMgLyBudW1fYXV0aG9ycyBBUyBsaW5lc19hdXRob3JfcmF0aW8KRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKV0hFUkUgKHBhdGggSU4gKGN1cnJlbnRfZmlsZXMpKSBBTkQgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpCkdST1VQIEJZIHBhdGgKSEFWSU5HIG1pbl9kYXRlIDw9IChub3coKSAtIHRvSW50ZXJ2YWxZZWFyKDEpKQpPUkRFUiBCWSBsaW5lc19hdXRob3JfcmF0aW8gREVTQwpMSU1JVCAxMA==) + ```sql WITH current_files AS ( @@ -1094,6 +1143,8 @@ LIMIT 10 We interpret this as the number of lines added and removed by the day of the week. In this case, we focus on the [Functions directory](https://github.com/ClickHouse/ClickHouse/tree/master/src/Functions) +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXlPZldlZWssCiAgICB1bmlxKGNvbW1pdF9oYXNoKSBBUyBjb21taXRzLAogICAgc3VtKGxpbmVzX2FkZGVkKSBBUyBsaW5lc19hZGRlZCwKICAgIHN1bShsaW5lc19kZWxldGVkKSBBUyBsaW5lc19kZWxldGVkCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIHBhdGggTElLRSAnc3JjL0Z1bmN0aW9ucyUnCkdST1VQIEJZIHRvRGF5T2ZXZWVrKHRpbWUpIEFTIGRheU9mV2Vlaw==) + ```sql SELECT dayOfWeek, @@ -1119,6 +1170,8 @@ GROUP BY toDayOfWeek(time) AS dayOfWeek And by time of day, +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBob3VyT2ZEYXksCiAgICB1bmlxKGNvbW1pdF9oYXNoKSBBUyBjb21taXRzLAogICAgc3VtKGxpbmVzX2FkZGVkKSBBUyBsaW5lc19hZGRlZCwKICAgIHN1bShsaW5lc19kZWxldGVkKSBBUyBsaW5lc19kZWxldGVkCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIHBhdGggTElLRSAnc3JjL0Z1bmN0aW9ucyUnCkdST1VQIEJZIHRvSG91cih0aW1lKSBBUyBob3VyT2ZEYXk=) + ```sql SELECT hourOfDay, @@ -1161,6 +1214,8 @@ GROUP BY toHour(time) AS hourOfDay This distribution makes sense given most of our development team is in Amsterdam. The `bar` functions helps us visualize these distributions: +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBob3VyT2ZEYXksCiAgICBiYXIoY29tbWl0cywgMCwgNDAwLCA1MCkgQVMgY29tbWl0cywKICAgIGJhcihsaW5lc19hZGRlZCwgMCwgMzAwMDAsIDUwKSBBUyBsaW5lc19hZGRlZCwKICAgIGJhcihsaW5lc19kZWxldGVkLCAwLCAxNTAwMCwgNTApIEFTIGxpbmVzX2RlbGV0ZWQKRlJPTQooCiAgICBTRUxFQ1QKICAgICAgICBob3VyT2ZEYXksCiAgICAgICAgdW5pcShjb21taXRfaGFzaCkgQVMgY29tbWl0cywKICAgICAgICBzdW0obGluZXNfYWRkZWQpIEFTIGxpbmVzX2FkZGVkLAogICAgICAgIHN1bShsaW5lc19kZWxldGVkKSBBUyBsaW5lc19kZWxldGVkCiAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgV0hFUkUgcGF0aCBMSUtFICdzcmMvRnVuY3Rpb25zJScKICAgIEdST1VQIEJZIHRvSG91cih0aW1lKSBBUyBob3VyT2ZEYXkKKQ==) + ```sql SELECT hourOfDay, @@ -1213,6 +1268,8 @@ FROM The `sign = -1` indicates a code deletion. We exclude punctuation and the insertion of empty lines. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBwcmV2X2F1dGhvciB8fCAnKGEpJyBhcyBhZGRfYXV0aG9yLAogICAgYXV0aG9yICB8fCAnKGQpJyBhcyBkZWxldGVfYXV0aG9yLAogICAgY291bnQoKSBBUyBjCkZST00gZ2l0X2NsaWNraG91c2UubGluZV9jaGFuZ2VzCldIRVJFIChzaWduID0gLTEpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcpKSBBTkQgKGxpbmVfdHlwZSBOT1QgSU4gKCdQdW5jdCcsICdFbXB0eScpKSBBTkQgKGF1dGhvciAhPSBwcmV2X2F1dGhvcikgQU5EIChwcmV2X2F1dGhvciAhPSAnJykKR1JPVVAgQlkKICAgIHByZXZfYXV0aG9yLAogICAgYXV0aG9yCk9SREVSIEJZIGMgREVTQwpMSU1JVCAxIEJZIHByZXZfYXV0aG9yCkxJTUlUIDEwMA==) + ```sql SELECT prev_author || '(a)' as add_author, @@ -1267,6 +1324,8 @@ Alexey clearly likes removing other peoples code. Lets exclude him for a more ba If we consider by just number of commits: +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXlfb2Zfd2VlaywKICAgIGF1dGhvciwKICAgIGNvdW50KCkgQVMgYwpGUk9NIGdpdF9jbGlja2hvdXNlLmNvbW1pdHMKR1JPVVAgQlkKICAgIGRheU9mV2Vlayh0aW1lKSBBUyBkYXlfb2Zfd2VlaywKICAgIGF1dGhvcgpPUkRFUiBCWQogICAgZGF5X29mX3dlZWsgQVNDLAogICAgYyBERVNDCkxJTUlUIDEgQlkgZGF5X29mX3dlZWs=) + ```sql SELECT day_of_week, @@ -1296,6 +1355,7 @@ LIMIT 1 BY day_of_week OK, some possible advantages here to the longest contributor - our founder Alexey. Lets limit our analysis to the last year. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXlfb2Zfd2VlaywKICAgIGF1dGhvciwKICAgIGNvdW50KCkgQVMgYwpGUk9NIGdpdF9jbGlja2hvdXNlLmNvbW1pdHMKV0hFUkUgdGltZSA+IChub3coKSAtIHRvSW50ZXJ2YWxZZWFyKDEpKQpHUk9VUCBCWQogICAgZGF5T2ZXZWVrKHRpbWUpIEFTIGRheV9vZl93ZWVrLAogICAgYXV0aG9yCk9SREVSIEJZCiAgICBkYXlfb2Zfd2VlayBBU0MsCiAgICBjIERFU0MKTElNSVQgMSBCWSBkYXlfb2Zfd2Vlaw==) ```sql SELECT @@ -1329,6 +1389,8 @@ This is still a little simple and doesn't reflect people's work. A better metric might be who is the top contributor each day as a fraction of the total work performed in the last year. Note that we treat the deletion and adding code equally. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB0b3BfYXV0aG9yLmRheV9vZl93ZWVrLAogICAgdG9wX2F1dGhvci5hdXRob3IsCiAgICB0b3BfYXV0aG9yLmF1dGhvcl93b3JrIC8gYWxsX3dvcmsudG90YWxfd29yayBBUyB0b3BfYXV0aG9yX3BlcmNlbnQKRlJPTQooCiAgICBTRUxFQ1QKICAgICAgICBkYXlfb2Zfd2VlaywKICAgICAgICBhdXRob3IsCiAgICAgICAgc3VtKGxpbmVzX2FkZGVkKSArIHN1bShsaW5lc19kZWxldGVkKSBBUyBhdXRob3Jfd29yawogICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgIFdIRVJFIHRpbWUgPiAobm93KCkgLSB0b0ludGVydmFsWWVhcigxKSkKICAgIEdST1VQIEJZCiAgICAgICAgYXV0aG9yLAogICAgICAgIGRheU9mV2Vlayh0aW1lKSBBUyBkYXlfb2Zfd2VlawogICAgT1JERVIgQlkKICAgICAgICBkYXlfb2Zfd2VlayBBU0MsCiAgICAgICAgYXV0aG9yX3dvcmsgREVTQwogICAgTElNSVQgMSBCWSBkYXlfb2Zfd2VlawopIEFTIHRvcF9hdXRob3IKSU5ORVIgSk9JTgooCiAgICBTRUxFQ1QKICAgICAgICBkYXlfb2Zfd2VlaywKICAgICAgICBzdW0obGluZXNfYWRkZWQpICsgc3VtKGxpbmVzX2RlbGV0ZWQpIEFTIHRvdGFsX3dvcmsKICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICBXSEVSRSB0aW1lID4gKG5vdygpIC0gdG9JbnRlcnZhbFllYXIoMSkpCiAgICBHUk9VUCBCWSBkYXlPZldlZWsodGltZSkgQVMgZGF5X29mX3dlZWsKKSBBUyBhbGxfd29yayBVU0lORyAoZGF5X29mX3dlZWsp) + ```sql SELECT top_author.day_of_week, @@ -1377,6 +1439,8 @@ INNER JOIN We limit the analysis to the current files. For brevity, we restrict the results to a depth of 2 with 5 files per root folder. Adjust as required. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjdXJyZW50X2ZpbGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUIHBhdGgKICAgICAgICBGUk9NCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIG9sZF9wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgbGFzdF90aW1lLAogICAgICAgICAgICAgICAgMiBBUyBjaGFuZ2VfdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBvbGRfcGF0aAogICAgICAgICAgICBVTklPTiBBTEwKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIGFyZ01heChjaGFuZ2VfdHlwZSwgdGltZSkgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgICkKICAgICAgICBHUk9VUCBCWSBwYXRoCiAgICAgICAgSEFWSU5HIChhcmdNYXgoY2hhbmdlX3R5cGUsIGxhc3RfdGltZSkgIT0gMikgQU5EIChOT1QgbWF0Y2gocGF0aCwgJyheZGJtcy8pfChebGlicy8pfChedGVzdHMvdGVzdGZsb3dzLyl8KF5wcm9ncmFtcy9zZXJ2ZXIvc3RvcmUvKScpKQogICAgICAgIE9SREVSIEJZIHBhdGggQVNDCiAgICApClNFTEVDVAogICAgY29uY2F0KHJvb3QsICcvJywgc3ViX2ZvbGRlcikgQVMgZm9sZGVyLAogICAgcm91bmQoYXZnKGRheXNfcHJlc2VudCkpIEFTIGF2Z19hZ2Vfb2ZfZmlsZXMsCiAgICBtaW4oZGF5c19wcmVzZW50KSBBUyBtaW5fYWdlX2ZpbGVzLAogICAgbWF4KGRheXNfcHJlc2VudCkgQVMgbWF4X2FnZV9maWxlcywKICAgIGNvdW50KCkgQVMgYwpGUk9NCigKICAgIFNFTEVDVAogICAgICAgIHBhdGgsCiAgICAgICAgZGF0ZURpZmYoJ2RheScsIG1pbih0aW1lKSwgdG9EYXRlKCcyMDIyLTExLTAzJykpIEFTIGRheXNfcHJlc2VudAogICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgIFdIRVJFIChwYXRoIElOIChjdXJyZW50X2ZpbGVzKSkgQU5EIChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NxbCcpKQogICAgR1JPVVAgQlkgcGF0aAopCkdST1VQIEJZCiAgICBzcGxpdEJ5Q2hhcignLycsIHBhdGgpWzFdIEFTIHJvb3QsCiAgICBzcGxpdEJ5Q2hhcignLycsIHBhdGgpWzJdIEFTIHN1Yl9mb2xkZXIKT1JERVIgQlkKICAgIHJvb3QgQVNDLAogICAgYyBERVNDCkxJTUlUIDUgQlkgcm9vdAo=) + ```sql WITH current_files AS ( @@ -1458,6 +1522,8 @@ LIMIT 5 BY root For this question, we need the number of lines written by an author divided by the total number of lines they have had removed by another contributor. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBrLAogICAgd3JpdHRlbl9jb2RlLmMsCiAgICByZW1vdmVkX2NvZGUuYywKICAgIHJlbW92ZWRfY29kZS5jIC8gd3JpdHRlbl9jb2RlLmMgQVMgcmVtb3ZlX3JhdGlvCkZST00KKAogICAgU0VMRUNUCiAgICAgICAgYXV0aG9yIEFTIGssCiAgICAgICAgY291bnQoKSBBUyBjCiAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgV0hFUkUgKHNpZ24gPSAxKSBBTkQgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnKSkgQU5EIChsaW5lX3R5cGUgTk9UIElOICgnUHVuY3QnLCAnRW1wdHknKSkKICAgIEdST1VQIEJZIGsKKSBBUyB3cml0dGVuX2NvZGUKSU5ORVIgSk9JTgooCiAgICBTRUxFQ1QKICAgICAgICBwcmV2X2F1dGhvciBBUyBrLAogICAgICAgIGNvdW50KCkgQVMgYwogICAgRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKICAgIFdIRVJFIChzaWduID0gLTEpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcpKSBBTkQgKGxpbmVfdHlwZSBOT1QgSU4gKCdQdW5jdCcsICdFbXB0eScpKSBBTkQgKGF1dGhvciAhPSBwcmV2X2F1dGhvcikKICAgIEdST1VQIEJZIGsKKSBBUyByZW1vdmVkX2NvZGUgVVNJTkcgKGspCldIRVJFIHdyaXR0ZW5fY29kZS5jID4gMTAwMApPUkRFUiBCWSByZW1vdmVfcmF0aW8gREVTQwpMSU1JVCAxMAo=) + ```sql SELECT k, @@ -1508,6 +1574,7 @@ We consider a rewrite to be when over 50% of the file are deleted, and 50% added The query is limited to the current files only. We list all file changes by grouping by `path` and `commit_hash`, returning the number of lines added and removed. Using a window function, we estimate the file's total size at any moment in time by performing a cumulative sum and estimating the impact of any change on file size as `lines added - lines removed`. Using this statistic, we can calculate the percentage of the file that has been added or removed for each change. Finally, we count the number of file changes that constitute a rewrite per file i.e. `(percent_add >= 0.5) AND (percent_delete >= 0.5) AND current_size > 50`. Note we require files to be more than 50 lines to avoid early contributions to a file being counted as a rewrite. This also avoids a bias to very small files, which may be more likely to be rewritten. +[play](https://play.clickhouse.com/play?user=play#V0lUSAogICAgY3VycmVudF9maWxlcyBBUwogICAgKAogICAgICAgIFNFTEVDVCBwYXRoCiAgICAgICAgRlJPTQogICAgICAgICgKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBvbGRfcGF0aCBBUyBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIDIgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgb2xkX3BhdGgKICAgICAgICAgICAgVU5JT04gQUxMCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIG1heCh0aW1lKSBBUyBsYXN0X3RpbWUsCiAgICAgICAgICAgICAgICBhcmdNYXgoY2hhbmdlX3R5cGUsIHRpbWUpIEFTIGNoYW5nZV90eXBlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIHBhdGgKICAgICAgICApCiAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgIEhBVklORyAoYXJnTWF4KGNoYW5nZV90eXBlLCBsYXN0X3RpbWUpICE9IDIpIEFORCAoTk9UIG1hdGNoKHBhdGgsICcoXmRibXMvKXwoXmxpYnMvKXwoXnRlc3RzL3Rlc3RmbG93cy8pfChecHJvZ3JhbXMvc2VydmVyL3N0b3JlLyknKSkKICAgICAgICBPUkRFUiBCWSBwYXRoIEFTQwogICAgKSwKICAgIGNoYW5nZXMgQVMKICAgICgKICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgbWF4KHRpbWUpIEFTIG1heF90aW1lLAogICAgICAgICAgICBjb21taXRfaGFzaCwKICAgICAgICAgICAgYW55KGxpbmVzX2FkZGVkKSBBUyBudW1fYWRkZWQsCiAgICAgICAgICAgIGFueShsaW5lc19kZWxldGVkKSBBUyBudW1fZGVsZXRlZCwKICAgICAgICAgICAgYW55KGNoYW5nZV90eXBlKSBBUyB0eXBlCiAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICBXSEVSRSAoY2hhbmdlX3R5cGUgSU4gKCdBZGQnLCAnTW9kaWZ5JykpIEFORCAocGF0aCBJTiAoY3VycmVudF9maWxlcykpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKICAgICAgICBHUk9VUCBCWQogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBjb21taXRfaGFzaAogICAgICAgIE9SREVSIEJZCiAgICAgICAgICAgIHBhdGggQVNDLAogICAgICAgICAgICBtYXhfdGltZSBBU0MKICAgICksCiAgICByZXdyaXRlcyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBjb21taXRfaGFzaCwKICAgICAgICAgICAgbWF4X3RpbWUsCiAgICAgICAgICAgIHR5cGUsCiAgICAgICAgICAgIG51bV9hZGRlZCwKICAgICAgICAgICAgbnVtX2RlbGV0ZWQsCiAgICAgICAgICAgIHN1bShudW1fYWRkZWQgLSBudW1fZGVsZXRlZCkgT1ZFUiAoUEFSVElUSU9OIEJZIHBhdGggT1JERVIgQlkgbWF4X3RpbWUgQVNDKSBBUyBjdXJyZW50X3NpemUsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9hZGRlZCAvIGN1cnJlbnRfc2l6ZSwgMCkgQVMgcGVyY2VudF9hZGQsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9kZWxldGVkIC8gY3VycmVudF9zaXplLCAwKSBBUyBwZXJjZW50X2RlbGV0ZQogICAgICAgIEZST00gY2hhbmdlcwogICAgKQpTRUxFQ1QKICAgIHBhdGgsCiAgICBjb3VudCgpIEFTIG51bV9yZXdyaXRlcwpGUk9NIHJld3JpdGVzCldIRVJFICh0eXBlID0gJ01vZGlmeScpIEFORCAocGVyY2VudF9hZGQgPj0gMC41KSBBTkQgKHBlcmNlbnRfZGVsZXRlID49IDAuNSkgQU5EIChjdXJyZW50X3NpemUgPiA1MCkKR1JPVVAgQlkgcGF0aApPUkRFUiBCWSBudW1fcmV3cml0ZXMgREVTQwpMSU1JVCAxMA==) ```sql WITH @@ -1599,6 +1666,8 @@ We query for lines added, joining this with the lines removed - filtering to cas Finally, we aggregate across this dataset to compute the average number of days lines stay in the repository by the day of the week. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBkYXlfb2Zfd2Vla19hZGRlZCwKICAgIGNvdW50KCkgQVMgbnVtLAogICAgYXZnKGRheXNfcHJlc2VudCkgQVMgYXZnX2RheXNfcHJlc2VudApGUk9NCigKICAgIFNFTEVDVAogICAgICAgIGFkZGVkX2NvZGUubGluZSwKICAgICAgICBhZGRlZF9jb2RlLnRpbWUgQVMgYWRkZWRfZGF5LAogICAgICAgIGRhdGVEaWZmKCdkYXknLCBhZGRlZF9jb2RlLnRpbWUsIHJlbW92ZWRfY29kZS50aW1lKSBBUyBkYXlzX3ByZXNlbnQKICAgIEZST00KICAgICgKICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgbGluZSwKICAgICAgICAgICAgbWF4KHRpbWUpIEFTIHRpbWUKICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgICAgIFdIRVJFIChzaWduID0gMSkgQU5EIChsaW5lX3R5cGUgTk9UIElOICgnUHVuY3QnLCAnRW1wdHknKSkKICAgICAgICBHUk9VUCBCWQogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBsaW5lCiAgICApIEFTIGFkZGVkX2NvZGUKICAgIElOTkVSIEpPSU4KICAgICgKICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgbGluZSwKICAgICAgICAgICAgbWF4KHRpbWUpIEFTIHRpbWUKICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgICAgIFdIRVJFIChzaWduID0gLTEpIEFORCAobGluZV90eXBlIE5PVCBJTiAoJ1B1bmN0JywgJ0VtcHR5JykpCiAgICAgICAgR1JPVVAgQlkKICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgbGluZQogICAgKSBBUyByZW1vdmVkX2NvZGUgVVNJTkcgKHBhdGgsIGxpbmUpCiAgICBXSEVSRSByZW1vdmVkX2NvZGUudGltZSA+IGFkZGVkX2NvZGUudGltZQopCkdST1VQIEJZIGRheU9mV2VlayhhZGRlZF9kYXkpIEFTIGRheV9vZl93ZWVrX2FkZGVk) + ```sql SELECT day_of_week_added, @@ -1656,6 +1725,8 @@ GROUP BY dayOfWeek(added_day) AS day_of_week_added This query uses the same principle as [What weekday does the code have the highest chance to stay in the repository](#what-weekday-does-the-code-have-the-highest-chance-to-stay-in-the-repository) - by aiming to uniquely identify a line of code using the path and line contents. This allows us to identify the time between when a line was added and removed. We filter to current files and code only, however, and average the time for each file across lines. +[play](https://play.clickhouse.com/play?user=play#V0lUSAogICAgY3VycmVudF9maWxlcyBBUwogICAgKAogICAgICAgIFNFTEVDVCBwYXRoCiAgICAgICAgRlJPTQogICAgICAgICgKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBvbGRfcGF0aCBBUyBwYXRoLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIGxhc3RfdGltZSwKICAgICAgICAgICAgICAgIDIgQVMgY2hhbmdlX3R5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgR1JPVVAgQlkgb2xkX3BhdGgKICAgICAgICAgICAgVU5JT04gQUxMCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIG1heCh0aW1lKSBBUyBsYXN0X3RpbWUsCiAgICAgICAgICAgICAgICBhcmdNYXgoY2hhbmdlX3R5cGUsIHRpbWUpIEFTIGNoYW5nZV90eXBlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIHBhdGgKICAgICAgICApCiAgICAgICAgR1JPVVAgQlkgcGF0aAogICAgICAgIEhBVklORyAoYXJnTWF4KGNoYW5nZV90eXBlLCBsYXN0X3RpbWUpICE9IDIpIEFORCAoTk9UIG1hdGNoKHBhdGgsICcoXmRibXMvKXwoXmxpYnMvKXwoXnRlc3RzL3Rlc3RmbG93cy8pfChecHJvZ3JhbXMvc2VydmVyL3N0b3JlLyknKSkKICAgICAgICBPUkRFUiBCWSBwYXRoIEFTQwogICAgKSwKICAgIGxpbmVzX3JlbW92ZWQgQVMKICAgICgKICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgYWRkZWRfY29kZS5wYXRoIEFTIHBhdGgsCiAgICAgICAgICAgIGFkZGVkX2NvZGUubGluZSwKICAgICAgICAgICAgYWRkZWRfY29kZS50aW1lIEFTIGFkZGVkX2RheSwKICAgICAgICAgICAgZGF0ZURpZmYoJ2RheScsIGFkZGVkX2NvZGUudGltZSwgcmVtb3ZlZF9jb2RlLnRpbWUpIEFTIGRheXNfcHJlc2VudAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIGxpbmUsCiAgICAgICAgICAgICAgICBtYXgodGltZSkgQVMgdGltZSwKICAgICAgICAgICAgICAgIGFueShmaWxlX2V4dGVuc2lvbikgQVMgZmlsZV9leHRlbnNpb24KICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKICAgICAgICAgICAgV0hFUkUgKHNpZ24gPSAxKSBBTkQgKGxpbmVfdHlwZSBOT1QgSU4gKCdQdW5jdCcsICdFbXB0eScpKQogICAgICAgICAgICBHUk9VUCBCWQogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIGxpbmUKICAgICAgICApIEFTIGFkZGVkX2NvZGUKICAgICAgICBJTk5FUiBKT0lOCiAgICAgICAgKAogICAgICAgICAgICBTRUxFQ1QKICAgICAgICAgICAgICAgIHBhdGgsCiAgICAgICAgICAgICAgICBsaW5lLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIHRpbWUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKICAgICAgICAgICAgV0hFUkUgKHNpZ24gPSAtMSkgQU5EIChsaW5lX3R5cGUgTk9UIElOICgnUHVuY3QnLCAnRW1wdHknKSkKICAgICAgICAgICAgR1JPVVAgQlkKICAgICAgICAgICAgICAgIHBhdGgsCiAgICAgICAgICAgICAgICBsaW5lCiAgICAgICAgKSBBUyByZW1vdmVkX2NvZGUgVVNJTkcgKHBhdGgsIGxpbmUpCiAgICAgICAgV0hFUkUgKHJlbW92ZWRfY29kZS50aW1lID4gYWRkZWRfY29kZS50aW1lKSBBTkQgKHBhdGggSU4gKGN1cnJlbnRfZmlsZXMpKSBBTkQgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpCiAgICApClNFTEVDVAogICAgcGF0aCwKICAgIGF2ZyhkYXlzX3ByZXNlbnQpIEFTIGF2Z19jb2RlX2FnZQpGUk9NIGxpbmVzX3JlbW92ZWQKR1JPVVAgQlkgcGF0aApPUkRFUiBCWSBhdmdfY29kZV9hZ2UgREVTQwpMSU1JVCAxMA==) + ```sql WITH current_files AS @@ -1745,6 +1816,8 @@ There are a few ways we can address this question. Focusing on the code to test Note we limit to users with more than 20 changes to focus on regular committers and avoid a bias to one-off contributions. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBOT1QgSU4gKCdoJywgJ2NwcCcsICdzaCcsICdweScsICdleHBlY3QnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCkdST1VQIEJZIGF1dGhvcgpIQVZJTkcgY29kZSA+IDIwCk9SREVSIEJZIGNvZGUgREVTQwpMSU1JVCAyMA==) + ```sql SELECT author, @@ -1785,6 +1858,8 @@ LIMIT 20 We can plot this distribution as a histogram. +[play](https://play.clickhouse.com/play?user=play#V0lUSCAoCiAgICAgICAgU0VMRUNUIGhpc3RvZ3JhbSgxMCkocmF0aW9fY29kZSkgQVMgaGlzdAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gTk9UIElOICgnaCcsICdjcHAnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgICAgICAgICAgICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICAgICAgICAgICAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIGF1dGhvcgogICAgICAgICAgICBIQVZJTkcgY29kZSA+IDIwCiAgICAgICAgKQogICAgKSBBUyBoaXN0ClNFTEVDVAogICAgYXJyYXlKb2luKGhpc3QpLjEgQVMgbG93ZXIsCiAgICBhcnJheUpvaW4oaGlzdCkuMiBBUyB1cHBlciwKICAgIGJhcihhcnJheUpvaW4oaGlzdCkuMywgMCwgMTAwLCAyMCkgQVMgYmFy) + ```sql WITH ( SELECT histogram(10)(ratio_code) AS hist @@ -1825,6 +1900,8 @@ Most contributors write more code than tests, as you'd expect. What about who adds the most comments when contributing code? +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb21tZW50JykgQVMgY29tbWVudHMsCiAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZSwKICAgIGNvbW1lbnRzIC8gKGNvbW1lbnRzICsgY29kZSkgQVMgcmF0aW9fY29tbWVudHMKRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKV0hFUkUgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoc2lnbiA9IDEpCkdST1VQIEJZIGF1dGhvcgpIQVZJTkcgY29kZSA+IDIwCk9SREVSIEJZIGNvZGUgREVTQwpMSU1JVCAxMA==) + ```sql SELECT author, @@ -1853,23 +1930,14 @@ LIMIT 10 10 rows in set. Elapsed: 0.047 sec. Processed 7.54 million rows, 31.57 MB (161.75 million rows/s., 677.77 MB/s.) ``` - Surprisingly high % for all our contributors and part of what makes our code so readable. - -``sql - -SELECT author, avg(ratio_comments) as avg_ratio_comments, sum(code) as code FROM ( -SELECT author, commit_hash, countIf(line_type = 'Comment') AS comments, countIf(line_type = 'Code') AS code, if(comments > 0, comments / (comments + code), 0) AS ratio_comments FROM git.line_changes GROUP BY author, commit_hash -) GROUP BY author ORDER BY code DESC LIMIT 10 -``` - - - ## How does an authors commits change over time with respect to code/comments percentage? To compute this by author is trivial, +[play](#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZV9saW5lcywKICAgIGNvdW50SWYoKGxpbmVfdHlwZSA9ICdDb21tZW50JykgT1IgKGxpbmVfdHlwZSA9ICdQdW5jdCcpKSBBUyBjb21tZW50cywKICAgIGNvZGVfbGluZXMgLyAoY29tbWVudHMgKyBjb2RlX2xpbmVzKSBBUyByYXRpb19jb2RlLAogICAgdG9TdGFydE9mV2Vlayh0aW1lKSBBUyB3ZWVrCkZST00gZ2l0X2NsaWNraG91c2UubGluZV9jaGFuZ2VzCkdST1VQIEJZCiAgICB0aW1lLAogICAgYXV0aG9yCk9SREVSIEJZCiAgICBhdXRob3IgQVNDLAogICAgdGltZSBBU0MKTElNSVQgMTA=) + ```sql SELECT author, @@ -1908,6 +1976,8 @@ To compute this, we first work out each author's comments ratio over time - simi After calculating the average by-week offset across all authors, we sample these results by selecting every 10th week. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBhdXRob3JfcmF0aW9zX2J5X29mZnNldCBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBhdXRob3IsCiAgICAgICAgICAgIGRhdGVEaWZmKCd3ZWVrJywgc3RhcnRfZGF0ZXMuc3RhcnRfZGF0ZSwgY29udHJpYnV0aW9ucy53ZWVrKSBBUyB3ZWVrX29mZnNldCwKICAgICAgICAgICAgcmF0aW9fY29kZQogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgdG9TdGFydE9mV2VlayhtaW4odGltZSkpIEFTIHN0YXJ0X2RhdGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKICAgICAgICAgICAgV0hFUkUgZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKQogICAgICAgICAgICBHUk9VUCBCWSBhdXRob3IgQVMgc3RhcnRfZGF0ZXMKICAgICAgICApIEFTIHN0YXJ0X2RhdGVzCiAgICAgICAgSU5ORVIgSk9JTgogICAgICAgICgKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBhdXRob3IsCiAgICAgICAgICAgICAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZSwKICAgICAgICAgICAgICAgIGNvdW50SWYoKGxpbmVfdHlwZSA9ICdDb21tZW50JykgT1IgKGxpbmVfdHlwZSA9ICdQdW5jdCcpKSBBUyBjb21tZW50cywKICAgICAgICAgICAgICAgIGNvbW1lbnRzIC8gKGNvbW1lbnRzICsgY29kZSkgQVMgcmF0aW9fY29kZSwKICAgICAgICAgICAgICAgIHRvU3RhcnRPZldlZWsodGltZSkgQVMgd2VlawogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgICAgICAgICBXSEVSRSAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkgQU5EIChzaWduID0gMSkKICAgICAgICAgICAgR1JPVVAgQlkKICAgICAgICAgICAgICAgIHRpbWUsCiAgICAgICAgICAgICAgICBhdXRob3IKICAgICAgICAgICAgSEFWSU5HIGNvZGUgPiAyMAogICAgICAgICAgICBPUkRFUiBCWQogICAgICAgICAgICAgICAgYXV0aG9yIEFTQywKICAgICAgICAgICAgICAgIHRpbWUgQVNDCiAgICAgICAgKSBBUyBjb250cmlidXRpb25zIFVTSU5HIChhdXRob3IpCiAgICApClNFTEVDVAogICAgd2Vla19vZmZzZXQsCiAgICBhdmcocmF0aW9fY29kZSkgQVMgYXZnX2NvZGVfcmF0aW8KRlJPTSBhdXRob3JfcmF0aW9zX2J5X29mZnNldApHUk9VUCBCWSB3ZWVrX29mZnNldApIQVZJTkcgKHdlZWtfb2Zmc2V0ICUgMTApID0gMApPUkRFUiBCWSB3ZWVrX29mZnNldCBBU0MKTElNSVQgMjAK) + ```sql WITH author_ratios_by_offset AS ( @@ -1984,8 +2054,9 @@ Encouragingly, our comment % is pretty constant and doesn't degrade the longer a We can use the same principle as [List files that were rewritten most number of time or by most of authors](#list-files-that-were-rewritten-most-number-of-time-or-by-most-of-authors) to identify rewrites but consider all files. A window function is used to compute the time between rewrites for each file. From this, we can calculate an average and median across all files. -```sql +[play](https://play.clickhouse.com/play?user=play#V0lUSAogICAgY2hhbmdlcyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBjb21taXRfaGFzaCwKICAgICAgICAgICAgbWF4X3RpbWUsCiAgICAgICAgICAgIHR5cGUsCiAgICAgICAgICAgIG51bV9hZGRlZCwKICAgICAgICAgICAgbnVtX2RlbGV0ZWQsCiAgICAgICAgICAgIHN1bShudW1fYWRkZWQgLSBudW1fZGVsZXRlZCkgT1ZFUiAoUEFSVElUSU9OIEJZIHBhdGggT1JERVIgQlkgbWF4X3RpbWUgQVNDKSBBUyBjdXJyZW50X3NpemUsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9hZGRlZCAvIGN1cnJlbnRfc2l6ZSwgMCkgQVMgcGVyY2VudF9hZGQsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9kZWxldGVkIC8gY3VycmVudF9zaXplLCAwKSBBUyBwZXJjZW50X2RlbGV0ZQogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIG1heCh0aW1lKSBBUyBtYXhfdGltZSwKICAgICAgICAgICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgICAgICAgICAgYW55KGxpbmVzX2FkZGVkKSBBUyBudW1fYWRkZWQsCiAgICAgICAgICAgICAgICBhbnkobGluZXNfZGVsZXRlZCkgQVMgbnVtX2RlbGV0ZWQsCiAgICAgICAgICAgICAgICBhbnkoY2hhbmdlX3R5cGUpIEFTIHR5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKICAgICAgICAgICAgV0hFUkUgKGNoYW5nZV90eXBlIElOICgnQWRkJywgJ01vZGlmeScpKSBBTkQgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpCiAgICAgICAgICAgIEdST1VQIEJZCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgY29tbWl0X2hhc2gKICAgICAgICAgICAgT1JERVIgQlkKICAgICAgICAgICAgICAgIHBhdGggQVNDLAogICAgICAgICAgICAgICAgbWF4X3RpbWUgQVNDCiAgICAgICAgKQogICAgKSwKICAgIHJld3JpdGVzIEFTCiAgICAoCiAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICosCiAgICAgICAgICAgIGFueShtYXhfdGltZSkgT1ZFUiAoUEFSVElUSU9OIEJZIHBhdGggT1JERVIgQlkgbWF4X3RpbWUgQVNDIFJPV1MgQkVUV0VFTiAxIFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIHByZXZpb3VzX3Jld3JpdGUsCiAgICAgICAgICAgIGRhdGVEaWZmKCdkYXknLCBwcmV2aW91c19yZXdyaXRlLCBtYXhfdGltZSkgQVMgcmV3cml0ZV9kYXlzCiAgICAgICAgRlJPTSBjaGFuZ2VzCiAgICAgICAgV0hFUkUgKHR5cGUgPSAnTW9kaWZ5JykgQU5EIChwZXJjZW50X2FkZCA+PSAwLjUpIEFORCAocGVyY2VudF9kZWxldGUgPj0gMC41KSBBTkQgKGN1cnJlbnRfc2l6ZSA+IDUwKQogICAgKQpTRUxFQ1QKICAgIGF2Z0lmKHJld3JpdGVfZGF5cywgcmV3cml0ZV9kYXlzID4gMCkgQVMgYXZnX3Jld3JpdGVfdGltZSwKICAgIHF1YW50aWxlc1RpbWluZ0lmKDAuNSkocmV3cml0ZV9kYXlzLCByZXdyaXRlX2RheXMgPiAwKSBBUyBoYWxmX2xpZmUKRlJPTSByZXdyaXRlcw==) +```sql WITH changes AS ( @@ -2043,6 +2114,8 @@ FROM rewrites Similar to [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) and [List files that were rewritten most number of time or by most of authors](#list-files-that-were-rewritten-most-number-of-time-or-by-most-of-authors), except we aggregate by day of week. Adjust as required e.g. month of year. +[play](https://play.clickhouse.com/play?user=play#V0lUSAogICAgY2hhbmdlcyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBjb21taXRfaGFzaCwKICAgICAgICAgICAgbWF4X3RpbWUsCiAgICAgICAgICAgIHR5cGUsCiAgICAgICAgICAgIG51bV9hZGRlZCwKICAgICAgICAgICAgbnVtX2RlbGV0ZWQsCiAgICAgICAgICAgIHN1bShudW1fYWRkZWQgLSBudW1fZGVsZXRlZCkgT1ZFUiAoUEFSVElUSU9OIEJZIHBhdGggT1JERVIgQlkgbWF4X3RpbWUgQVNDKSBBUyBjdXJyZW50X3NpemUsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9hZGRlZCAvIGN1cnJlbnRfc2l6ZSwgMCkgQVMgcGVyY2VudF9hZGQsCiAgICAgICAgICAgIGlmKGN1cnJlbnRfc2l6ZSA+IDAsIG51bV9kZWxldGVkIC8gY3VycmVudF9zaXplLCAwKSBBUyBwZXJjZW50X2RlbGV0ZQogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIG1heCh0aW1lKSBBUyBtYXhfdGltZSwKICAgICAgICAgICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgICAgICAgICAgYW55KGZpbGVfbGluZXNfYWRkZWQpIEFTIG51bV9hZGRlZCwKICAgICAgICAgICAgICAgIGFueShmaWxlX2xpbmVzX2RlbGV0ZWQpIEFTIG51bV9kZWxldGVkLAogICAgICAgICAgICAgICAgYW55KGZpbGVfY2hhbmdlX3R5cGUpIEFTIHR5cGUKICAgICAgICAgICAgRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKICAgICAgICAgICAgV0hFUkUgKGZpbGVfY2hhbmdlX3R5cGUgSU4gKCdBZGQnLCAnTW9kaWZ5JykpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKICAgICAgICAgICAgR1JPVVAgQlkKICAgICAgICAgICAgICAgIHBhdGgsCiAgICAgICAgICAgICAgICBjb21taXRfaGFzaAogICAgICAgICAgICBPUkRFUiBCWQogICAgICAgICAgICAgICAgcGF0aCBBU0MsCiAgICAgICAgICAgICAgICBtYXhfdGltZSBBU0MKICAgICAgICApCiAgICApLAogICAgcmV3cml0ZXMgQVMKICAgICgKICAgICAgICBTRUxFQ1QgYW55KG1heF90aW1lKSBPVkVSIChQQVJUSVRJT04gQlkgcGF0aCBPUkRFUiBCWSBtYXhfdGltZSBBU0MgUk9XUyBCRVRXRUVOIDEgUFJFQ0VESU5HIEFORCBDVVJSRU5UIFJPVykgQVMgcHJldmlvdXNfcmV3cml0ZQogICAgICAgIEZST00gY2hhbmdlcwogICAgICAgIFdIRVJFICh0eXBlID0gJ01vZGlmeScpIEFORCAocGVyY2VudF9hZGQgPj0gMC41KSBBTkQgKHBlcmNlbnRfZGVsZXRlID49IDAuNSkgQU5EIChjdXJyZW50X3NpemUgPiA1MCkKICAgICkKU0VMRUNUCiAgICBkYXlPZldlZWsocHJldmlvdXNfcmV3cml0ZSkgQVMgZGF5T2ZXZWVrLAogICAgY291bnQoKSBBUyBudW1fcmVfd3JpdGVzCkZST00gcmV3cml0ZXMKR1JPVVAgQlkgZGF5T2ZXZWVr) + ```sql WITH changes AS @@ -2105,6 +2178,8 @@ GROUP BY dayOfWeek We define "sticky" as how long does an author's code stay before its rewritten. Similar to the previous question [What is the average time before code will be rewritten and the median (half-life of code decay)?](#what-is-the-average-time-before-code-will-be-rewritten-and-the-median-half-life-of-code-decay) - using the same metric for rewrites i.e. 50% additions and 50% deletions to the file. We compute the average rewrite time per author and only consider contributors with more than two files. +[play](https://play.clickhouse.com/play?user=play#V0lUSAogICAgY2hhbmdlcyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBwYXRoLAogICAgICAgICAgICBhdXRob3IsCiAgICAgICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgICAgICBtYXhfdGltZSwKICAgICAgICAgICAgdHlwZSwKICAgICAgICAgICAgbnVtX2FkZGVkLAogICAgICAgICAgICBudW1fZGVsZXRlZCwKICAgICAgICAgICAgc3VtKG51bV9hZGRlZCAtIG51bV9kZWxldGVkKSBPVkVSIChQQVJUSVRJT04gQlkgcGF0aCBPUkRFUiBCWSBtYXhfdGltZSBBU0MpIEFTIGN1cnJlbnRfc2l6ZSwKICAgICAgICAgICAgaWYoY3VycmVudF9zaXplID4gMCwgbnVtX2FkZGVkIC8gY3VycmVudF9zaXplLCAwKSBBUyBwZXJjZW50X2FkZCwKICAgICAgICAgICAgaWYoY3VycmVudF9zaXplID4gMCwgbnVtX2RlbGV0ZWQgLyBjdXJyZW50X3NpemUsIDApIEFTIHBlcmNlbnRfZGVsZXRlCiAgICAgICAgRlJPTQogICAgICAgICgKICAgICAgICAgICAgU0VMRUNUCiAgICAgICAgICAgICAgICBwYXRoLAogICAgICAgICAgICAgICAgYW55KGF1dGhvcikgQVMgYXV0aG9yLAogICAgICAgICAgICAgICAgbWF4KHRpbWUpIEFTIG1heF90aW1lLAogICAgICAgICAgICAgICAgY29tbWl0X2hhc2gsCiAgICAgICAgICAgICAgICBhbnkoZmlsZV9saW5lc19hZGRlZCkgQVMgbnVtX2FkZGVkLAogICAgICAgICAgICAgICAgYW55KGZpbGVfbGluZXNfZGVsZXRlZCkgQVMgbnVtX2RlbGV0ZWQsCiAgICAgICAgICAgICAgICBhbnkoZmlsZV9jaGFuZ2VfdHlwZSkgQVMgdHlwZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgICAgICAgICBXSEVSRSAoZmlsZV9jaGFuZ2VfdHlwZSBJTiAoJ0FkZCcsICdNb2RpZnknKSkgQU5EIChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NxbCcpKQogICAgICAgICAgICBHUk9VUCBCWQogICAgICAgICAgICAgICAgcGF0aCwKICAgICAgICAgICAgICAgIGNvbW1pdF9oYXNoCiAgICAgICAgICAgIE9SREVSIEJZCiAgICAgICAgICAgICAgICBwYXRoIEFTQywKICAgICAgICAgICAgICAgIG1heF90aW1lIEFTQwogICAgICAgICkKICAgICksCiAgICByZXdyaXRlcyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICAqLAogICAgICAgICAgICBhbnkobWF4X3RpbWUpIE9WRVIgKFBBUlRJVElPTiBCWSBwYXRoIE9SREVSIEJZIG1heF90aW1lIEFTQyBST1dTIEJFVFdFRU4gMSBQUkVDRURJTkcgQU5EIENVUlJFTlQgUk9XKSBBUyBwcmV2aW91c19yZXdyaXRlLAogICAgICAgICAgICBkYXRlRGlmZignZGF5JywgcHJldmlvdXNfcmV3cml0ZSwgbWF4X3RpbWUpIEFTIHJld3JpdGVfZGF5cywKICAgICAgICAgICAgYW55KGF1dGhvcikgT1ZFUiAoUEFSVElUSU9OIEJZIHBhdGggT1JERVIgQlkgbWF4X3RpbWUgQVNDIFJPV1MgQkVUV0VFTiAxIFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIHByZXZfYXV0aG9yCiAgICAgICAgRlJPTSBjaGFuZ2VzCiAgICAgICAgV0hFUkUgKHR5cGUgPSAnTW9kaWZ5JykgQU5EIChwZXJjZW50X2FkZCA+PSAwLjUpIEFORCAocGVyY2VudF9kZWxldGUgPj0gMC41KSBBTkQgKGN1cnJlbnRfc2l6ZSA+IDUwKQogICAgKQpTRUxFQ1QKICAgIHByZXZfYXV0aG9yLAogICAgYXZnKHJld3JpdGVfZGF5cykgQVMgYywKICAgIHVuaXEocGF0aCkgQVMgbnVtX2ZpbGVzCkZST00gcmV3cml0ZXMKR1JPVVAgQlkgcHJldl9hdXRob3IKSEFWSU5HIG51bV9maWxlcyA+IDIKT1JERVIgQlkgYyBERVNDCkxJTUlUIDEwCg==) + ```sql WITH changes AS @@ -2182,6 +2257,8 @@ This query first requires us to calculate the days when an author has committed. Our subsequent array functions compute each author's longest sequence of consecutive ones. First, the `groupArray` function is used to collate all `consecutive_day` values for an author. This array of 1s and 0s, is then split on 0 values into subarrays. Finally, we calculate the longest subarray. +[play](https://play.clickhouse.com/play?user=play#V0lUSCBjb21taXRfZGF5cyBBUwogICAgKAogICAgICAgIFNFTEVDVAogICAgICAgICAgICBhdXRob3IsCiAgICAgICAgICAgIGRheSwKICAgICAgICAgICAgYW55KGRheSkgT1ZFUiAoUEFSVElUSU9OIEJZIGF1dGhvciBPUkRFUiBCWSBkYXkgQVNDIFJPV1MgQkVUV0VFTiAxIFBSRUNFRElORyBBTkQgQ1VSUkVOVCBST1cpIEFTIHByZXZpb3VzX2NvbW1pdCwKICAgICAgICAgICAgZGF0ZURpZmYoJ2RheScsIHByZXZpb3VzX2NvbW1pdCwgZGF5KSBBUyBkYXlzX3NpbmNlX2xhc3QsCiAgICAgICAgICAgIGlmKGRheXNfc2luY2VfbGFzdCA9IDEsIDEsIDApIEFTIGNvbnNlY3V0aXZlX2RheQogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgdG9TdGFydE9mRGF5KHRpbWUpIEFTIGRheQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmNvbW1pdHMKICAgICAgICAgICAgR1JPVVAgQlkKICAgICAgICAgICAgICAgIGF1dGhvciwKICAgICAgICAgICAgICAgIGRheQogICAgICAgICAgICBPUkRFUiBCWQogICAgICAgICAgICAgICAgYXV0aG9yIEFTQywKICAgICAgICAgICAgICAgIGRheSBBU0MKICAgICAgICApCiAgICApClNFTEVDVAogICAgYXV0aG9yLAogICAgYXJyYXlNYXgoYXJyYXlNYXAoeCAtPiBsZW5ndGgoeCksIGFycmF5U3BsaXQoeCAtPiAoeCA9IDApLCBncm91cEFycmF5KGNvbnNlY3V0aXZlX2RheSkpKSkgQVMgbWF4X2NvbnNlY3V0aXZlX2RheXMKRlJPTSBjb21taXRfZGF5cwpHUk9VUCBCWSBhdXRob3IKT1JERVIgQlkgbWF4X2NvbnNlY3V0aXZlX2RheXMgREVTQwpMSU1JVCAxMA==) + ```sql WITH commit_days AS ( @@ -2233,6 +2310,7 @@ LIMIT 10 Files can be renamed. When this occurs, we get a rename event, where the `path` column is set to the new path of the file and the `old_path` represents the previous location e.g. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB0aW1lLAogICAgcGF0aCwKICAgIG9sZF9wYXRoLAogICAgY29tbWl0X2hhc2gsCiAgICBjb21taXRfbWVzc2FnZQpGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwpXSEVSRSAocGF0aCA9ICdzcmMvU3RvcmFnZXMvU3RvcmFnZVJlcGxpY2F0ZWRNZXJnZVRyZWUuY3BwJykgQU5EIChjaGFuZ2VfdHlwZSA9ICdSZW5hbWUnKQ==) ```sql SELECT @@ -2258,7 +2336,7 @@ To address this, we can use User Defined Functions (UDFs). These cannot, current This means we can only track renames to a maximum depth - the below example is 5 deep. It is unlikely a file will be renamed more times than this, so for now, this is sufficient. ```sql -CREATE FUNCTION file_path_history AS (n) -> if(empty(n), [], arrayConcat([n], file_history_01((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); +CREATE FUNCTION file_path_history AS (n) -> if(empty(n), [], arrayConcat([n], file_path_history_01((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); CREATE FUNCTION file_path_history_01 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_02((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); CREATE FUNCTION file_path_history_02 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_03((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); CREATE FUNCTION file_path_history_03 AS (n) -> if(isNull(n), [], arrayConcat([n], file_path_history_04((SELECT if(empty(old_path), Null, old_path) FROM git.file_changes WHERE path = n AND (change_type = 'Rename' OR change_type = 'Add') LIMIT 1)))); @@ -2270,6 +2348,8 @@ By calling `file_path_history('src/Storages/StorageReplicatedMergeTree.cpp')` we For example, +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUIGZpbGVfcGF0aF9oaXN0b3J5KCdzcmMvU3RvcmFnZXMvU3RvcmFnZVJlcGxpY2F0ZWRNZXJnZVRyZWUuY3BwJykgQVMgcGF0aHMK) + ```sql SELECT file_path_history('src/Storages/StorageReplicatedMergeTree.cpp') AS paths @@ -2282,6 +2362,8 @@ SELECT file_path_history('src/Storages/StorageReplicatedMergeTree.cpp') AS paths We can use this capability to now assemble the commits for the entire history of a file. In this example, we show one commit for each of the `path` values. +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICB0aW1lLAogICAgc3Vic3RyaW5nKGNvbW1pdF9oYXNoLCAxLCAxMSkgQVMgY29tbWl0LAogICAgY2hhbmdlX3R5cGUsCiAgICBhdXRob3IsCiAgICBwYXRoLAogICAgY29tbWl0X21lc3NhZ2UKRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKV0hFUkUgcGF0aCBJTiBmaWxlX3BhdGhfaGlzdG9yeSgnc3JjL1N0b3JhZ2VzL1N0b3JhZ2VSZXBsaWNhdGVkTWVyZ2VUcmVlLmNwcCcpCk9SREVSIEJZIHRpbWUgREVTQwpMSU1JVCAxIEJZIHBhdGgKRk9STUFUIFByZXR0eUNvbXBhY3RNb25vQmxvY2s=) + ```sql SELECT time, From 5ec748c664b1fdae5acca024753e92fe7e610f99 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 14:49:49 +0000 Subject: [PATCH 388/526] Fix function if in case of NULL and const Nullable arguments --- src/Functions/if.cpp | 2 ++ .../02479_if_with_null_and_cullable_const.reference | 4 ++++ .../0_stateless/02479_if_with_null_and_cullable_const.sql | 3 +++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference create mode 100644 tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql diff --git a/src/Functions/if.cpp b/src/Functions/if.cpp index 86707fc62d6..f3401713834 100644 --- a/src/Functions/if.cpp +++ b/src/Functions/if.cpp @@ -904,6 +904,7 @@ private: if (cond_col) { + arg_else_column = arg_else_column->convertToFullColumnIfConst(); auto result_column = IColumn::mutate(std::move(arg_else_column)); if (else_is_short) result_column->expand(cond_col->getData(), true); @@ -941,6 +942,7 @@ private: if (cond_col) { + arg_then_column = arg_then_column->convertToFullColumnIfConst(); auto result_column = IColumn::mutate(std::move(arg_then_column)); if (then_is_short) result_column->expand(cond_col->getData(), false); diff --git a/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference new file mode 100644 index 00000000000..376364af7b4 --- /dev/null +++ b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.reference @@ -0,0 +1,4 @@ +1 +\N +\N +1 diff --git a/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql new file mode 100644 index 00000000000..b684de88cb2 --- /dev/null +++ b/tests/queries/0_stateless/02479_if_with_null_and_cullable_const.sql @@ -0,0 +1,3 @@ +SELECT if(number % 2, NULL, toNullable(1)) FROM numbers(2); +SELECT if(number % 2, toNullable(1), NULL) FROM numbers(2); + From 28c090f7a553d7d5161c3d2c8b8bbf23498d82d4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 11 Nov 2022 15:42:53 +0000 Subject: [PATCH 389/526] Fix decimal math overflow in parsing datetime with 'best effort' algorithm --- src/IO/ReadHelpers.h | 5 +++-- src/IO/parseDateTimeBestEffort.cpp | 3 +++ ...02480_parse_date_time_best_effort_math_overflow.reference | 3 +++ .../02480_parse_date_time_best_effort_math_overflow.sql | 3 +++ 4 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference create mode 100644 tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 27a24eef804..7156ac0ceb3 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -964,15 +964,16 @@ inline ReturnType readDateTimeTextImpl(DateTime64 & datetime64, UInt32 scale, Re components.whole = components.whole / common::exp10_i32(scale); } + bool is_ok = true; if constexpr (std::is_same_v) datetime64 = DecimalUtils::decimalFromComponents(components, scale); else - DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); + is_ok = DecimalUtils::tryGetDecimalFromComponents(components, scale, datetime64); datetime64 *= negative_multiplier; - return ReturnType(true); + return ReturnType(is_ok); } inline void readDateTimeText(time_t & datetime, ReadBuffer & buf, const DateLUTImpl & time_zone = DateLUT::instance()) diff --git a/src/IO/parseDateTimeBestEffort.cpp b/src/IO/parseDateTimeBestEffort.cpp index 1dd06401bf1..e0cba169e81 100644 --- a/src/IO/parseDateTimeBestEffort.cpp +++ b/src/IO/parseDateTimeBestEffort.cpp @@ -659,6 +659,9 @@ ReturnType parseDateTime64BestEffortImpl(DateTime64 & res, UInt32 scale, ReadBuf fractional *= common::exp10_i64(scale - subsecond.digits); } + if constexpr (std::is_same_v) + return DecimalUtils::tryGetDecimalFromComponents(whole, fractional, scale, res); + res = DecimalUtils::decimalFromComponents(whole, fractional, scale); return ReturnType(true); } diff --git a/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference new file mode 100644 index 00000000000..1515932af18 --- /dev/null +++ b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.reference @@ -0,0 +1,3 @@ +9279104479c7da1114861274de32208ead91b60e +\N +\N diff --git a/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql new file mode 100644 index 00000000000..5102fb47204 --- /dev/null +++ b/tests/queries/0_stateless/02480_parse_date_time_best_effort_math_overflow.sql @@ -0,0 +1,3 @@ +select * from format(TSV, '9279104479c7da1114861274de32208ead91b60e') settings date_time_input_format='best_effort'; +select parseDateTime64BestEffortOrNull('9279104477', 9); +select toDateTime64OrNull('9279104477', 9); From 616da1f69553a21e101fd8297d9b2ff451547a6d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 11 Nov 2022 13:04:35 +0100 Subject: [PATCH 390/526] Review fixed, style check fix --- src/Common/ErrorCodes.cpp | 2 +- src/Storages/NamedCollections.cpp | 24 +++++++++---------- .../01271_show_privileges.reference | 1 + .../02117_show_create_table_system.reference | 6 ++--- utils/check-style/check-style | 2 +- 5 files changed, 18 insertions(+), 17 deletions(-) diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index ce96b1a9917..2bc5d70421a 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -638,7 +638,7 @@ M(667, NOT_INITIALIZED) \ M(668, INVALID_STATE) \ M(669, UNKNOWN_NAMED_COLLECTION) \ - M(669, NAMED_COLLECTION_ALREADY_EXISTS) \ + M(670, NAMED_COLLECTION_ALREADY_EXISTS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Storages/NamedCollections.cpp b/src/Storages/NamedCollections.cpp index 60c11c90b0a..67847635f3f 100644 --- a/src/Storages/NamedCollections.cpp +++ b/src/Storages/NamedCollections.cpp @@ -18,18 +18,19 @@ namespace DB { -static constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; - namespace ErrorCodes { extern const int UNKNOWN_NAMED_COLLECTION; extern const int NAMED_COLLECTION_ALREADY_EXISTS; extern const int BAD_ARGUMENTS; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; } namespace { + constexpr auto NAMED_COLLECTIONS_CONFIG_PREFIX = "named_collections"; + std::string getCollectionPrefix(const std::string & collection_name) { return fmt::format("{}.{}", NAMED_COLLECTIONS_CONFIG_PREFIX, collection_name); @@ -49,7 +50,7 @@ namespace /// the `result` will contain two strings: "root.key1.key2" and "root.key1.key3.key4" void collectKeys( const Poco::Util::AbstractConfiguration & config, - std::queue & enumerate_paths, + std::queue enumerate_paths, std::set & result) { if (enumerate_paths.empty()) @@ -241,8 +242,7 @@ NamedCollectionFactory::NamedCollections NamedCollectionFactory::getAll() const class NamedCollection::Impl { private: - using IConfigurationPtr = Poco::AutoPtr; - using ConfigurationPtr = Poco::AutoPtr; + using ConfigurationPtr = Poco::AutoPtr; /// Named collection configuration /// @@ -270,7 +270,7 @@ public: template T getOrDefault(const Key & key, const T & default_value) const { - return getConfigValueOrDefault(*config, key, default_value); + return getConfigValueOrDefault(*config, key, &default_value); } template void set(const Key & key, const T & value, bool update_if_exists) @@ -312,9 +312,9 @@ public: /// to a string: /// "key0: value0 /// key1: - /// key2: value2 - /// key3: - /// key4: value3" + /// key2: value2 + /// key3: + /// key4: value3" WriteBufferFromOwnString wb; Strings prev_key_parts; for (const auto & key : keys) @@ -359,7 +359,7 @@ private: template static T getConfigValueOrDefault( const Poco::Util::AbstractConfiguration & config, const std::string & path, - const std::optional & default_value = std::nullopt) + const T * default_value = nullptr) { if (!config.has(path)) { @@ -477,7 +477,7 @@ NamedCollectionPtr NamedCollection::create( std::set enumerate_result; enumerate_input.push(collection_prefix); - collectKeys(config, enumerate_input, enumerate_result); + collectKeys(config, std::move(enumerate_input), enumerate_result); /// Collection does not have any keys. /// (`enumerate_result` == ). @@ -487,7 +487,7 @@ NamedCollectionPtr NamedCollection::create( { /// Skip collection prefix and add +1 to avoid '.' in the beginning. for (const auto & path : enumerate_result) - keys.emplace(path.substr(std::strlen(collection_prefix.data()) + 1)); + keys.emplace(path.substr(collection_prefix.size() + 1)); } return std::make_unique(config, collection_name, keys); } diff --git a/tests/queries/0_stateless/01271_show_privileges.reference b/tests/queries/0_stateless/01271_show_privileges.reference index b332a7d71f7..59c7d978493 100644 --- a/tests/queries/0_stateless/01271_show_privileges.reference +++ b/tests/queries/0_stateless/01271_show_privileges.reference @@ -85,6 +85,7 @@ SHOW ROLES ['SHOW CREATE ROLE'] GLOBAL SHOW ACCESS SHOW ROW POLICIES ['SHOW POLICIES','SHOW CREATE ROW POLICY','SHOW CREATE POLICY'] TABLE SHOW ACCESS SHOW QUOTAS ['SHOW CREATE QUOTA'] GLOBAL SHOW ACCESS SHOW SETTINGS PROFILES ['SHOW PROFILES','SHOW CREATE SETTINGS PROFILE','SHOW CREATE PROFILE'] GLOBAL SHOW ACCESS +SHOW NAMED COLLECTIONS ['SHOW NAMED COLLECTIONS'] GLOBAL SHOW ACCESS SHOW ACCESS [] \N ACCESS MANAGEMENT ACCESS MANAGEMENT [] \N ALL SYSTEM SHUTDOWN ['SYSTEM KILL','SHUTDOWN'] GLOBAL SYSTEM diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4d705e4b0d1..093e72a092a 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -281,7 +281,7 @@ CREATE TABLE system.grants ( `user_name` Nullable(String), `role_name` Nullable(String), - `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149), + `access_type` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150), `database` Nullable(String), `table` Nullable(String), `column` Nullable(String), @@ -544,10 +544,10 @@ ENGINE = SystemPartsColumns COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.privileges ( - `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149), + `privilege` Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150), `aliases` Array(String), `level` Nullable(Enum8('GLOBAL' = 0, 'DATABASE' = 1, 'TABLE' = 2, 'DICTIONARY' = 3, 'VIEW' = 4, 'COLUMN' = 5)), - `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW ACCESS' = 87, 'ACCESS MANAGEMENT' = 88, 'SYSTEM SHUTDOWN' = 89, 'SYSTEM DROP DNS CACHE' = 90, 'SYSTEM DROP MARK CACHE' = 91, 'SYSTEM DROP UNCOMPRESSED CACHE' = 92, 'SYSTEM DROP MMAP CACHE' = 93, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 94, 'SYSTEM DROP FILESYSTEM CACHE' = 95, 'SYSTEM DROP SCHEMA CACHE' = 96, 'SYSTEM DROP CACHE' = 97, 'SYSTEM RELOAD CONFIG' = 98, 'SYSTEM RELOAD USERS' = 99, 'SYSTEM RELOAD SYMBOLS' = 100, 'SYSTEM RELOAD DICTIONARY' = 101, 'SYSTEM RELOAD MODEL' = 102, 'SYSTEM RELOAD FUNCTION' = 103, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 104, 'SYSTEM RELOAD' = 105, 'SYSTEM RESTART DISK' = 106, 'SYSTEM MERGES' = 107, 'SYSTEM TTL MERGES' = 108, 'SYSTEM FETCHES' = 109, 'SYSTEM MOVES' = 110, 'SYSTEM DISTRIBUTED SENDS' = 111, 'SYSTEM REPLICATED SENDS' = 112, 'SYSTEM SENDS' = 113, 'SYSTEM REPLICATION QUEUES' = 114, 'SYSTEM DROP REPLICA' = 115, 'SYSTEM SYNC REPLICA' = 116, 'SYSTEM RESTART REPLICA' = 117, 'SYSTEM RESTORE REPLICA' = 118, 'SYSTEM SYNC DATABASE REPLICA' = 119, 'SYSTEM SYNC TRANSACTION LOG' = 120, 'SYSTEM FLUSH DISTRIBUTED' = 121, 'SYSTEM FLUSH LOGS' = 122, 'SYSTEM FLUSH' = 123, 'SYSTEM THREAD FUZZER' = 124, 'SYSTEM UNFREEZE' = 125, 'SYSTEM' = 126, 'dictGet' = 127, 'addressToLine' = 128, 'addressToLineWithInlines' = 129, 'addressToSymbol' = 130, 'demangle' = 131, 'INTROSPECTION' = 132, 'FILE' = 133, 'URL' = 134, 'REMOTE' = 135, 'MONGO' = 136, 'MEILISEARCH' = 137, 'MYSQL' = 138, 'POSTGRES' = 139, 'SQLITE' = 140, 'ODBC' = 141, 'JDBC' = 142, 'HDFS' = 143, 'S3' = 144, 'HIVE' = 145, 'SOURCES' = 146, 'CLUSTER' = 147, 'ALL' = 148, 'NONE' = 149)) + `parent_group` Nullable(Enum16('SHOW DATABASES' = 0, 'SHOW TABLES' = 1, 'SHOW COLUMNS' = 2, 'SHOW DICTIONARIES' = 3, 'SHOW' = 4, 'SHOW FILESYSTEM CACHES' = 5, 'SELECT' = 6, 'INSERT' = 7, 'ALTER UPDATE' = 8, 'ALTER DELETE' = 9, 'ALTER ADD COLUMN' = 10, 'ALTER MODIFY COLUMN' = 11, 'ALTER DROP COLUMN' = 12, 'ALTER COMMENT COLUMN' = 13, 'ALTER CLEAR COLUMN' = 14, 'ALTER RENAME COLUMN' = 15, 'ALTER MATERIALIZE COLUMN' = 16, 'ALTER COLUMN' = 17, 'ALTER MODIFY COMMENT' = 18, 'ALTER ORDER BY' = 19, 'ALTER SAMPLE BY' = 20, 'ALTER ADD INDEX' = 21, 'ALTER DROP INDEX' = 22, 'ALTER MATERIALIZE INDEX' = 23, 'ALTER CLEAR INDEX' = 24, 'ALTER INDEX' = 25, 'ALTER ADD PROJECTION' = 26, 'ALTER DROP PROJECTION' = 27, 'ALTER MATERIALIZE PROJECTION' = 28, 'ALTER CLEAR PROJECTION' = 29, 'ALTER PROJECTION' = 30, 'ALTER ADD CONSTRAINT' = 31, 'ALTER DROP CONSTRAINT' = 32, 'ALTER CONSTRAINT' = 33, 'ALTER TTL' = 34, 'ALTER MATERIALIZE TTL' = 35, 'ALTER SETTINGS' = 36, 'ALTER MOVE PARTITION' = 37, 'ALTER FETCH PARTITION' = 38, 'ALTER FREEZE PARTITION' = 39, 'ALTER DATABASE SETTINGS' = 40, 'ALTER TABLE' = 41, 'ALTER DATABASE' = 42, 'ALTER VIEW REFRESH' = 43, 'ALTER VIEW MODIFY QUERY' = 44, 'ALTER VIEW' = 45, 'ALTER' = 46, 'CREATE DATABASE' = 47, 'CREATE TABLE' = 48, 'CREATE VIEW' = 49, 'CREATE DICTIONARY' = 50, 'CREATE TEMPORARY TABLE' = 51, 'CREATE FUNCTION' = 52, 'CREATE' = 53, 'DROP DATABASE' = 54, 'DROP TABLE' = 55, 'DROP VIEW' = 56, 'DROP DICTIONARY' = 57, 'DROP FUNCTION' = 58, 'DROP' = 59, 'TRUNCATE' = 60, 'OPTIMIZE' = 61, 'BACKUP' = 62, 'KILL QUERY' = 63, 'KILL TRANSACTION' = 64, 'MOVE PARTITION BETWEEN SHARDS' = 65, 'CREATE USER' = 66, 'ALTER USER' = 67, 'DROP USER' = 68, 'CREATE ROLE' = 69, 'ALTER ROLE' = 70, 'DROP ROLE' = 71, 'ROLE ADMIN' = 72, 'CREATE ROW POLICY' = 73, 'ALTER ROW POLICY' = 74, 'DROP ROW POLICY' = 75, 'CREATE QUOTA' = 76, 'ALTER QUOTA' = 77, 'DROP QUOTA' = 78, 'CREATE SETTINGS PROFILE' = 79, 'ALTER SETTINGS PROFILE' = 80, 'DROP SETTINGS PROFILE' = 81, 'SHOW USERS' = 82, 'SHOW ROLES' = 83, 'SHOW ROW POLICIES' = 84, 'SHOW QUOTAS' = 85, 'SHOW SETTINGS PROFILES' = 86, 'SHOW NAMED COLLECTIONS' = 87, 'SHOW ACCESS' = 88, 'ACCESS MANAGEMENT' = 89, 'SYSTEM SHUTDOWN' = 90, 'SYSTEM DROP DNS CACHE' = 91, 'SYSTEM DROP MARK CACHE' = 92, 'SYSTEM DROP UNCOMPRESSED CACHE' = 93, 'SYSTEM DROP MMAP CACHE' = 94, 'SYSTEM DROP COMPILED EXPRESSION CACHE' = 95, 'SYSTEM DROP FILESYSTEM CACHE' = 96, 'SYSTEM DROP SCHEMA CACHE' = 97, 'SYSTEM DROP CACHE' = 98, 'SYSTEM RELOAD CONFIG' = 99, 'SYSTEM RELOAD USERS' = 100, 'SYSTEM RELOAD SYMBOLS' = 101, 'SYSTEM RELOAD DICTIONARY' = 102, 'SYSTEM RELOAD MODEL' = 103, 'SYSTEM RELOAD FUNCTION' = 104, 'SYSTEM RELOAD EMBEDDED DICTIONARIES' = 105, 'SYSTEM RELOAD' = 106, 'SYSTEM RESTART DISK' = 107, 'SYSTEM MERGES' = 108, 'SYSTEM TTL MERGES' = 109, 'SYSTEM FETCHES' = 110, 'SYSTEM MOVES' = 111, 'SYSTEM DISTRIBUTED SENDS' = 112, 'SYSTEM REPLICATED SENDS' = 113, 'SYSTEM SENDS' = 114, 'SYSTEM REPLICATION QUEUES' = 115, 'SYSTEM DROP REPLICA' = 116, 'SYSTEM SYNC REPLICA' = 117, 'SYSTEM RESTART REPLICA' = 118, 'SYSTEM RESTORE REPLICA' = 119, 'SYSTEM SYNC DATABASE REPLICA' = 120, 'SYSTEM SYNC TRANSACTION LOG' = 121, 'SYSTEM FLUSH DISTRIBUTED' = 122, 'SYSTEM FLUSH LOGS' = 123, 'SYSTEM FLUSH' = 124, 'SYSTEM THREAD FUZZER' = 125, 'SYSTEM UNFREEZE' = 126, 'SYSTEM' = 127, 'dictGet' = 128, 'addressToLine' = 129, 'addressToLineWithInlines' = 130, 'addressToSymbol' = 131, 'demangle' = 132, 'INTROSPECTION' = 133, 'FILE' = 134, 'URL' = 135, 'REMOTE' = 136, 'MONGO' = 137, 'MEILISEARCH' = 138, 'MYSQL' = 139, 'POSTGRES' = 140, 'SQLITE' = 141, 'ODBC' = 142, 'JDBC' = 143, 'HDFS' = 144, 'S3' = 145, 'HIVE' = 146, 'SOURCES' = 147, 'CLUSTER' = 148, 'ALL' = 149, 'NONE' = 150)) ) ENGINE = SystemPrivileges COMMENT 'SYSTEM TABLE is built on the fly.' diff --git a/utils/check-style/check-style b/utils/check-style/check-style index a0556d971e8..ed397a4a162 100755 --- a/utils/check-style/check-style +++ b/utils/check-style/check-style @@ -13,7 +13,7 @@ # and then to run formatter only for the specified files. ROOT_PATH=$(git rev-parse --show-toplevel) -EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing|benchmark' +EXCLUDE_DIRS='build/|integration/|widechar_width/|glibc-compatibility/|memcpy/|consistent-hashing|benchmark|tests/' # From [1]: # But since array_to_string_internal() in array.c still loops over array From 5504f3af9b7624beadce5c4a97b099066a10dfcb Mon Sep 17 00:00:00 2001 From: xiedeyantu Date: Sat, 12 Nov 2022 00:03:36 +0800 Subject: [PATCH 391/526] fix skip_unavailable_shards does not work using s3Cluster table function --- src/Storages/StorageS3Cluster.cpp | 38 ++++++++----------- .../test_s3_cluster/configs/cluster.xml | 15 ++++++++ tests/integration/test_s3_cluster/test.py | 29 ++++++++++++++ 3 files changed, 59 insertions(+), 23 deletions(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 3b8c8b1cb92..350e942f972 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -117,32 +117,24 @@ Pipe StorageS3Cluster::read( addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); - for (const auto & replicas : cluster->getShardsAddresses()) + const auto & current_settings = context->getSettingsRef(); + auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); + for (const auto & shard_info : cluster->getShardsInfo()) { - /// There will be only one replica, because we consider each replica as a shard - for (const auto & node : replicas) + auto try_results = shard_info.pool->getMany(timeouts, ¤t_settings, PoolMode::GET_MANY); + for (auto & try_result : try_results) { - auto connection = std::make_shared( - node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(), - node.user, node.password, node.quota_key, node.cluster, node.cluster_secret, - "S3ClusterInititiator", - node.compression, - node.secure - ); - - - /// For unknown reason global context is passed to IStorage::read() method - /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( - connection, - queryToString(query_to_send), - header, - context, - /*throttler=*/nullptr, - scalars, - Tables(), - processed_stage, - RemoteQueryExecutor::Extension{.task_iterator = callback}); + shard_info.pool, + std::vector{try_result}, + queryToString(query_to_send), + header, + context, + /*throttler=*/nullptr, + scalars, + Tables(), + processed_stage, + RemoteQueryExecutor::Extension{.task_iterator = callback}); pipes.emplace_back(std::make_shared(remote_query_executor, add_agg_info, false)); } diff --git a/tests/integration/test_s3_cluster/configs/cluster.xml b/tests/integration/test_s3_cluster/configs/cluster.xml index 18f15763633..3059340cfe4 100644 --- a/tests/integration/test_s3_cluster/configs/cluster.xml +++ b/tests/integration/test_s3_cluster/configs/cluster.xml @@ -20,6 +20,21 @@ + + + + s0_0_0 + 9000 + + + + + s0_0_0 + 19000 + + + + cluster_simple diff --git a/tests/integration/test_s3_cluster/test.py b/tests/integration/test_s3_cluster/test.py index 2cbb36fcf06..8e082f7d86a 100644 --- a/tests/integration/test_s3_cluster/test.py +++ b/tests/integration/test_s3_cluster/test.py @@ -195,3 +195,32 @@ def test_ambiguous_join(started_cluster): """ ) assert "AMBIGUOUS_COLUMN_NAME" not in result + + +def test_skip_unavailable_shards(started_cluster): + node = started_cluster.instances["s0_0_0"] + result = node.query( + """ + SELECT count(*) from s3Cluster( + 'cluster_non_existent_port', + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + SETTINGS skip_unavailable_shards = 1 + """ + ) + + assert result == "10\n" + + +def test_unskip_unavailable_shards(started_cluster): + node = started_cluster.instances["s0_0_0"] + error = node.query_and_get_error( + """ + SELECT count(*) from s3Cluster( + 'cluster_non_existent_port', + 'http://minio1:9001/root/data/clickhouse/part1.csv', + 'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') + """ + ) + + assert "NETWORK_ERROR" in error From dbf64e27125bcd4e4882d22957695d466dde7152 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 11 Nov 2022 17:23:40 +0100 Subject: [PATCH 392/526] fix --- tests/queries/0_stateless/02448_clone_replica_lost_part.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql index 371f7389837..7029d6bf4ea 100644 --- a/tests/queries/0_stateless/02448_clone_replica_lost_part.sql +++ b/tests/queries/0_stateless/02448_clone_replica_lost_part.sql @@ -3,9 +3,9 @@ drop table if exists rmt1; drop table if exists rmt2; create table rmt1 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '1') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=5; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; create table rmt2 (n int) engine=ReplicatedMergeTree('/test/02448/{database}/rmt', '2') order by tuple() - settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=5; + settings min_replicated_logs_to_keep=1, max_replicated_logs_to_keep=2, cleanup_delay_period=0, cleanup_delay_period_random_add=1, old_parts_lifetime=0, max_parts_to_merge_at_once=4; -- insert part only on one replica system stop replicated sends rmt1; From b12ebab65c5e1b084a61900b61094ab0b515a886 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 11 Nov 2022 18:07:15 +0000 Subject: [PATCH 393/526] Enable keeper fault injection and retries for insert queries in funcntional tests --- docker/test/stress/run.sh | 3 +++ tests/config/users.d/insert_keeper_retries.xml | 8 ++++++++ 2 files changed, 11 insertions(+) create mode 100644 tests/config/users.d/insert_keeper_retries.xml diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 78f627bf45e..36b1cd1ef0d 100644 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -388,6 +388,9 @@ else rm -f /etc/clickhouse-server/config.d/storage_conf.xml ||: rm -f /etc/clickhouse-server/config.d/azure_storage_conf.xml ||: + # it uses recently introduced settings which previous versions may not have + rm -f /etc/clickhouse-server/users.d/insert_keeper_retries.xml ||: + start clickhouse-client --query="SELECT 'Server version: ', version()" diff --git a/tests/config/users.d/insert_keeper_retries.xml b/tests/config/users.d/insert_keeper_retries.xml new file mode 100644 index 00000000000..462c9df5248 --- /dev/null +++ b/tests/config/users.d/insert_keeper_retries.xml @@ -0,0 +1,8 @@ + + + + 20 + 0.01 + + + From 0208fff321bd4090f3f873411a2ee5cbc3e31e99 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 11 Nov 2022 18:43:33 +0000 Subject: [PATCH 394/526] Randomize keeper fault injection settings in stress tests --- tests/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 20e63412d91..c054e66898e 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,6 +456,8 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), + "insert_keeper_max_retries": lambda: random.randint(20, 20), + "insert_keeper_fault_injection_probability": lambda: random.uniform(0, 0.01), } @staticmethod From 86f0194964a19231b00d543174918affee4a3b1f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Wed, 9 Nov 2022 20:27:33 +0100 Subject: [PATCH 395/526] A test to update columns that are used in projection --- ...8_projection_with_group_by_alter.reference | 60 +++++++++++++++++++ .../02478_projection_with_group_by_alter.sql | 56 +++++++++++++++++ 2 files changed, 116 insertions(+) create mode 100644 tests/queries/0_stateless/02478_projection_with_group_by_alter.reference create mode 100644 tests/queries/0_stateless/02478_projection_with_group_by_alter.sql diff --git a/tests/queries/0_stateless/02478_projection_with_group_by_alter.reference b/tests/queries/0_stateless/02478_projection_with_group_by_alter.reference new file mode 100644 index 00000000000..6ad2c8ec8db --- /dev/null +++ b/tests/queries/0_stateless/02478_projection_with_group_by_alter.reference @@ -0,0 +1,60 @@ +-- { echoOn } + +OPTIMIZE TABLE testing FINAL; +SELECT c FROM testing ORDER BY d; +0 +1 +2 +3 +4 +SELECT c FROM testing ORDER BY e, d; +0 +2 +4 +1 +3 +-- update all colums used by proj_1 +ALTER TABLE testing UPDATE c = c+1, d = d+2 WHERE True SETTINGS mutations_sync=2; +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; +SELECT c FROM testing ORDER BY d; +1 +2 +3 +4 +5 +SELECT c FROM testing ORDER BY e, d; +1 +3 +5 +2 +4 +-- update only one column +ALTER TABLE testing UPDATE d = d-1 WHERE True SETTINGS mutations_sync=2; +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; +SELECT c FROM testing ORDER BY d; +1 +2 +3 +4 +5 +SELECT c FROM testing ORDER BY e, d; +1 +3 +5 +2 +4 +-- update only another one column +ALTER TABLE testing UPDATE c = c-1 WHERE True SETTINGS mutations_sync=2; +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; +SELECT c FROM testing ORDER BY d; +0 +1 +2 +3 +4 +SELECT c FROM testing ORDER BY e, d; +0 +2 +4 +1 +3 diff --git a/tests/queries/0_stateless/02478_projection_with_group_by_alter.sql b/tests/queries/0_stateless/02478_projection_with_group_by_alter.sql new file mode 100644 index 00000000000..9ed644fd7da --- /dev/null +++ b/tests/queries/0_stateless/02478_projection_with_group_by_alter.sql @@ -0,0 +1,56 @@ +CREATE TABLE testing +( + a String, + b String, + c Int32, + d Int32, + e Int32, + PROJECTION proj_1 + ( + SELECT c ORDER BY d + ), + PROJECTION proj_2 + ( + SELECT c ORDER BY e, d + ) +) +ENGINE = MergeTree() PRIMARY KEY (a) SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO testing SELECT number, number, number, number, number%2 FROM numbers(5); + +-- { echoOn } + +OPTIMIZE TABLE testing FINAL; + +SELECT c FROM testing ORDER BY d; +SELECT c FROM testing ORDER BY e, d; + +-- update all colums used by proj_1 +ALTER TABLE testing UPDATE c = c+1, d = d+2 WHERE True SETTINGS mutations_sync=2; + +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; + +SELECT c FROM testing ORDER BY d; +SELECT c FROM testing ORDER BY e, d; + + +-- update only one column +ALTER TABLE testing UPDATE d = d-1 WHERE True SETTINGS mutations_sync=2; + +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; + +SELECT c FROM testing ORDER BY d; +SELECT c FROM testing ORDER BY e, d; + + +-- update only another one column +ALTER TABLE testing UPDATE c = c-1 WHERE True SETTINGS mutations_sync=2; + +SELECT * FROM system.mutations WHERE database = currentDatabase() AND table = 'testing' AND not is_done; + +SELECT c FROM testing ORDER BY d; +SELECT c FROM testing ORDER BY e, d; + +-- { echoOff } + +DROP TABLE testing; From 0e031f93e9bbae011e38ec93d7498425d8c4de97 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 11 Nov 2022 23:19:29 +0100 Subject: [PATCH 396/526] Properly fill ORDER BY ast to find all required columns during mutation --- src/Parsers/ASTProjectionSelectQuery.cpp | 62 +++++++++++++++++++++++- 1 file changed, 61 insertions(+), 1 deletion(-) diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 7df0b53da6f..16c4635424b 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -118,6 +119,64 @@ ASTPtr & ASTProjectionSelectQuery::getExpression(Expression expr) return children[positions[expr]]; } +namespace +{ + +ASTPtr wrapAsOrderByElement(const ASTPtr & expr) +{ + auto elem = std::make_shared(); + + elem->direction = 1; /// Currently it cannot be specified in projection definition so it is ASC default + elem->nulls_direction = elem->direction; + elem->nulls_direction_was_explicitly_specified = false; + elem->with_fill = false; + elem->children.push_back(expr); + + return elem; +} + +/// Currently the ORDER BY clause in projection definition is parsed differently from ORDER BY of a standalone SELECT query. +/// This function converts this simplified AST into AST compatible with SELECT query. +/// If ORDER BY clause has single element then simple_order_by represents this element, +/// otherwise simple_order_by is a function "tuple" of all elements. +ASTPtr cloneToOrderByASTForSelect(const ASTPtr & simple_order_by) +{ + ASTPtr expression_list = std::make_shared(); + if (simple_order_by->children.empty()) + { + expression_list->children.emplace_back(wrapAsOrderByElement(simple_order_by->clone())); + } + else + { + auto * func = simple_order_by->as(); + if (!func || func->name != "tuple") + { + expression_list->children.emplace_back(wrapAsOrderByElement(func->clone())); + } + else + { + if (func->children.size() != 1) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected structure of ORDER BY clause in projection definition {}", + func->dumpTree(0)); + + auto * params = func->children[0]->as(); + if (!params) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected structure of ORDER BY clause in projection definition {}; Expression list expected", + func->dumpTree(0)); + + for (auto & child : params->children) + { + expression_list->children.emplace_back(wrapAsOrderByElement(child->clone())); + } + } + } + return expression_list; +} + +} + ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const { auto select_query = std::make_shared(); @@ -128,7 +187,8 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const select_query->setExpression(ASTSelectQuery::Expression::SELECT, select()->clone()); if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); - // Get rid of orderBy. It's used for projection definition only + if (orderBy()) + select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, cloneToOrderByASTForSelect(orderBy())); return node; } From 60d116729cf645cae40cab76c938e89c5cbf4e46 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 12 Nov 2022 02:55:26 +0000 Subject: [PATCH 397/526] adjustments for progress option --- programs/client/Client.cpp | 3 +- programs/local/LocalServer.cpp | 4 +- src/Client/ClientBase.cpp | 119 ++++++++++++++++++--------------- src/Client/ClientBase.h | 5 +- 4 files changed, 71 insertions(+), 60 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index b262c5a0db1..6e289b57845 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -243,6 +243,7 @@ try registerAggregateFunctions(); processConfig(); + initTtyBuffer(toProgressOption(config().getString("progress", "default"))); /// Includes delayed_interactive. if (is_interactive) @@ -1088,8 +1089,6 @@ void Client::processConfig() } else { - std::string progress = config().getString("progress", "off"); - need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().getBool("echo", false); ignore_error = config().getBool("ignore-error", false); diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index fea82b456cd..1d8b7edb20e 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -414,6 +414,8 @@ try registerFormats(); processConfig(); + initTtyBuffer(toProgressOption(config().getString("progress", "default"))); + applyCmdSettings(global_context); if (is_interactive) @@ -489,8 +491,6 @@ void LocalServer::processConfig() } else { - std::string progress = config().getString("progress", "off"); - need_render_progress = (Poco::icompare(progress, "off") && Poco::icompare(progress, "no") && Poco::icompare(progress, "false") && Poco::icompare(progress, "0")); echo_queries = config().hasOption("echo") || config().hasOption("verbose"); ignore_error = config().getBool("ignore-error", false); is_multiquery = true; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 5bd65ac2334..c69a3ff81ca 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -119,22 +119,27 @@ namespace ProfileEvents namespace DB { +ProgressOption toProgressOption(std::string progress) +{ + boost::to_upper(progress); + + if (progress == "OFF" || progress == "FALSE" || progress == "0" || progress == "NO") + return ProgressOption::OFF; + if (progress == "TTY" || progress == "ON" || progress == "TRUE" || progress == "1" || progress == "YES") + return ProgressOption::TTY; + if (progress == "ERR") + return ProgressOption::ERR; + if (progress == "DEFAULT") + return ProgressOption::DEFAULT; + + throw boost::program_options::validation_error(boost::program_options::validation_error::invalid_option_value); +} + std::istream& operator>> (std::istream & in, ProgressOption & progress) { std::string token; in >> token; - - boost::to_upper(token); - - if (token == "OFF" || token == "FALSE" || token == "0" || token == "NO") - progress = ProgressOption::OFF; - else if (token == "TTY" || token == "ON" || token == "TRUE" || token == "1" || token == "YES") - progress = ProgressOption::TTY; - else if (token == "ERR") - progress = ProgressOption::ERR; - else - throw boost::program_options::validation_error(boost::program_options::validation_error::invalid_option_value); - + progress = toProgressOption(token); return in; } @@ -662,56 +667,62 @@ void ClientBase::initLogsOutputStream() } } -void ClientBase::initTtyBuffer(bool to_err) +void ClientBase::initTtyBuffer(ProgressOption progress) { - if (!tty_buf) + if (tty_buf) + return; + + if (progress == ProgressOption::OFF || (!is_interactive && progress == ProgressOption::DEFAULT)) { - static constexpr auto tty_file_name = "/dev/tty"; + need_render_progress = false; + return; + } - /// Output all progress bar commands to terminal at once to avoid flicker. - /// This size is usually greater than the window size. - static constexpr size_t buf_size = 1024; + static constexpr auto tty_file_name = "/dev/tty"; - if (!to_err) + /// Output all progress bar commands to terminal at once to avoid flicker. + /// This size is usually greater than the window size. + static constexpr size_t buf_size = 1024; + + if (is_interactive || progress == ProgressOption::TTY) + { + std::error_code ec; + std::filesystem::file_status tty = std::filesystem::status(tty_file_name, ec); + + if (!ec && exists(tty) && is_character_file(tty) + && (tty.permissions() & std::filesystem::perms::others_write) != std::filesystem::perms::none) { - std::error_code ec; - std::filesystem::file_status tty = std::filesystem::status(tty_file_name, ec); - - if (!ec && exists(tty) && is_character_file(tty) - && (tty.permissions() & std::filesystem::perms::others_write) != std::filesystem::perms::none) + try { - try - { - tty_buf = std::make_unique(tty_file_name, buf_size); + tty_buf = std::make_unique(tty_file_name, buf_size); - /// It is possible that the terminal file has writeable permissions - /// but we cannot write anything there. Check it with invisible character. - tty_buf->write('\0'); - tty_buf->next(); + /// It is possible that the terminal file has writeable permissions + /// but we cannot write anything there. Check it with invisible character. + tty_buf->write('\0'); + tty_buf->next(); - return; - } - catch (const Exception & e) - { - if (tty_buf) - tty_buf.reset(); + return; + } + catch (const Exception & e) + { + if (tty_buf) + tty_buf.reset(); - if (e.code() != ErrorCodes::CANNOT_OPEN_FILE) - throw; + if (e.code() != ErrorCodes::CANNOT_OPEN_FILE) + throw; - /// It is normal if file exists, indicated as writeable but still cannot be opened. - /// Fallback to other options. - } + /// It is normal if file exists, indicated as writeable but still cannot be opened. + /// Fallback to other options. } } - - if (stderr_is_a_tty) - { - tty_buf = std::make_unique(STDERR_FILENO, buf_size); - } - else - need_render_progress = false; } + + if (stderr_is_a_tty || progress == ProgressOption::ERR) + { + tty_buf = std::make_unique(STDERR_FILENO, buf_size); + } + else + need_render_progress = false; } void ClientBase::updateSuggest(const ASTPtr & ast) @@ -2324,7 +2335,7 @@ void ClientBase::init(int argc, char ** argv) ("stage", po::value()->default_value("complete"), "Request query processing up to specified stage: complete,fetch_columns,with_mergeable_state,with_mergeable_state_after_aggregation,with_mergeable_state_after_aggregation_and_limit") ("query_kind", po::value()->default_value("initial_query"), "One of initial_query/secondary_query/no_query") ("query_id", po::value(), "query_id") - ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::OFF, "off"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR: err; OFF (default): off|0|false|no") + ("progress", po::value()->implicit_value(ProgressOption::TTY, "tty")->default_value(ProgressOption::DEFAULT, "default"), "Print progress of queries execution - to TTY: tty|on|1|true|yes; to STDERR non-interactive mode: err; OFF: off|0|false|no; DEFAULT - interactive to TTY, non-interactive is off") ("disable_suggestion,A", "Disable loading suggestion data. Note that suggestion data is loaded asynchronously through a second connection to ClickHouse server. Also it is reasonable to disable suggestion if you want to paste a query with TAB characters. Shorthand option -A is for those who get used to mysql client.") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") @@ -2379,11 +2390,6 @@ void ClientBase::init(int argc, char ** argv) parseAndCheckOptions(options_description, options, common_arguments); po::notify(options); - if (options["progress"].as() == ProgressOption::OFF) - need_render_progress = false; - else - initTtyBuffer(options["progress"].as() == ProgressOption::ERR); - if (options.count("version") || options.count("V")) { showClientVersion(); @@ -2437,6 +2443,9 @@ void ClientBase::init(int argc, char ** argv) { switch (options["progress"].as()) { + case DEFAULT: + config().setString("progress", "default"); + break; case OFF: config().setString("progress", "off"); break; diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 9ea66881cf6..6c85d6a5f2b 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -38,10 +38,12 @@ enum MultiQueryProcessingStage enum ProgressOption { + DEFAULT, OFF, TTY, ERR, }; +ProgressOption toProgressOption(std::string progress); std::istream& operator>> (std::istream & in, ProgressOption & progress); void interruptSignalHandler(int signum); @@ -153,7 +155,6 @@ private: void initOutputFormat(const Block & block, ASTPtr parsed_query); void initLogsOutputStream(); - void initTtyBuffer(bool to_err = false); String prompt() const; @@ -168,6 +169,8 @@ protected: static bool isSyncInsertWithData(const ASTInsertQuery & insert_query, const ContextPtr & context); bool processMultiQueryFromFile(const String & file_name); + void initTtyBuffer(ProgressOption progress); + bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool is_multiquery = false; bool delayed_interactive = false; From 4ad8e2974d3e42e5e1dd7c3261fc6591dd2426d4 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 12 Nov 2022 12:58:53 +0100 Subject: [PATCH 398/526] Simplify the logic by just adding ORDER BY list to SELECT --- src/Parsers/ASTProjectionSelectQuery.cpp | 75 +++++------------------- 1 file changed, 14 insertions(+), 61 deletions(-) diff --git a/src/Parsers/ASTProjectionSelectQuery.cpp b/src/Parsers/ASTProjectionSelectQuery.cpp index 16c4635424b..9b85fcb2dac 100644 --- a/src/Parsers/ASTProjectionSelectQuery.cpp +++ b/src/Parsers/ASTProjectionSelectQuery.cpp @@ -119,64 +119,6 @@ ASTPtr & ASTProjectionSelectQuery::getExpression(Expression expr) return children[positions[expr]]; } -namespace -{ - -ASTPtr wrapAsOrderByElement(const ASTPtr & expr) -{ - auto elem = std::make_shared(); - - elem->direction = 1; /// Currently it cannot be specified in projection definition so it is ASC default - elem->nulls_direction = elem->direction; - elem->nulls_direction_was_explicitly_specified = false; - elem->with_fill = false; - elem->children.push_back(expr); - - return elem; -} - -/// Currently the ORDER BY clause in projection definition is parsed differently from ORDER BY of a standalone SELECT query. -/// This function converts this simplified AST into AST compatible with SELECT query. -/// If ORDER BY clause has single element then simple_order_by represents this element, -/// otherwise simple_order_by is a function "tuple" of all elements. -ASTPtr cloneToOrderByASTForSelect(const ASTPtr & simple_order_by) -{ - ASTPtr expression_list = std::make_shared(); - if (simple_order_by->children.empty()) - { - expression_list->children.emplace_back(wrapAsOrderByElement(simple_order_by->clone())); - } - else - { - auto * func = simple_order_by->as(); - if (!func || func->name != "tuple") - { - expression_list->children.emplace_back(wrapAsOrderByElement(func->clone())); - } - else - { - if (func->children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected structure of ORDER BY clause in projection definition {}", - func->dumpTree(0)); - - auto * params = func->children[0]->as(); - if (!params) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected structure of ORDER BY clause in projection definition {}; Expression list expected", - func->dumpTree(0)); - - for (auto & child : params->children) - { - expression_list->children.emplace_back(wrapAsOrderByElement(child->clone())); - } - } - } - return expression_list; -} - -} - ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const { auto select_query = std::make_shared(); @@ -184,11 +126,22 @@ ASTPtr ASTProjectionSelectQuery::cloneToASTSelect() const if (with()) select_query->setExpression(ASTSelectQuery::Expression::WITH, with()->clone()); if (select()) - select_query->setExpression(ASTSelectQuery::Expression::SELECT, select()->clone()); + { + ASTPtr select_list = select()->clone(); + if (orderBy()) + { + /// Add ORDER BY list to SELECT for simplicity. It is Ok because we only uses this to find all required columns. + auto * expressions = select_list->as(); + if (!expressions) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unexpected structure of SELECT clause in projection definition {}; Expression list expected", + select_list->dumpTree(0)); + expressions->children.emplace_back(orderBy()->clone()); + } + select_query->setExpression(ASTSelectQuery::Expression::SELECT, std::move(select_list)); + } if (groupBy()) select_query->setExpression(ASTSelectQuery::Expression::GROUP_BY, groupBy()->clone()); - if (orderBy()) - select_query->setExpression(ASTSelectQuery::Expression::ORDER_BY, cloneToOrderByASTForSelect(orderBy())); return node; } From e4a6f2a8e1ce0820303f83de2f423be69680fdb4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 18:54:48 +0100 Subject: [PATCH 399/526] Fix the `indent` field in the `git-import` tool --- programs/git-import/git-import.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/git-import/git-import.cpp b/programs/git-import/git-import.cpp index 030ddd263fa..9e464164da6 100644 --- a/programs/git-import/git-import.cpp +++ b/programs/git-import/git-import.cpp @@ -351,7 +351,7 @@ struct LineChange ++pos; } - indent = std::max(255U, num_spaces); + indent = std::min(255U, num_spaces); line.assign(pos, end); if (pos == end) From b8c877db2988f89ccf9da848a53adf5f7eb22a86 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:07:28 +0100 Subject: [PATCH 400/526] Fix annoying messages in AsynchronousMetrics --- src/Common/Exception.h | 1 + src/Interpreters/AsynchronousMetrics.cpp | 30 +++++++++++++++++------- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/src/Common/Exception.h b/src/Common/Exception.h index c5259d157b2..84687581e52 100644 --- a/src/Common/Exception.h +++ b/src/Common/Exception.h @@ -12,6 +12,7 @@ #include + namespace Poco { class Logger; } diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 488ac77e956..a6cb6ee1f68 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -24,9 +24,9 @@ #include #include #include +#include #include - #include "config.h" #if USE_JEMALLOC @@ -123,9 +123,9 @@ void AsynchronousMetrics::openSensors() { LOG_WARNING( &Poco::Logger::get("AsynchronousMetrics"), - "Thermal monitor '{}' exists but could not be read, error {}.", + "Thermal monitor '{}' exists but could not be read: {}.", thermal_device_index, - e.getErrno()); + errnoToString(e.getErrno())); continue; } @@ -252,10 +252,10 @@ void AsynchronousMetrics::openSensorsChips() { LOG_WARNING( &Poco::Logger::get("AsynchronousMetrics"), - "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", + "Hardware monitor '{}', sensor '{}' exists but could not be read: {}.", hwmon_name, sensor_name, - e.getErrno()); + errnoToString(e.getErrno())); continue; } @@ -1083,7 +1083,17 @@ void AsynchronousMetrics::update(TimePoint update_time) BlockDeviceStatValues current_values{}; BlockDeviceStatValues & prev_values = block_device_stats[name]; - current_values.read(*device); + + try + { + current_values.read(*device); + } + catch (const ErrnoException & e) + { + LOG_DEBUG(log, "Cannot read statistics about the block device '{}': {}.", + name, errnoToString(e.getErrno())); + continue; + } BlockDeviceStatValues delta_values = current_values - prev_values; prev_values = current_values; @@ -1129,10 +1139,10 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - tryLogCurrentException(__PRETTY_FUNCTION__); + LOG_DEBUG(log, getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error - /// (i.e. ENOENT means that some disk had been replaced, and it may apperas with a new name) + /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) try { openBlockDevices(); @@ -1271,7 +1281,9 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (const ErrnoException & e) { - LOG_DEBUG(&Poco::Logger::get("AsynchronousMetrics"), "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", hwmon_name, sensor_name, e.getErrno()); + LOG_DEBUG(log, "Hardware monitor '{}', sensor '{}' exists but could not be read: {}.", + hwmon_name, sensor_name, errnoToString(e.getErrno())); + continue; } if (sensor_name.empty()) From c9d4598962309b9f26eacf08f74695189e19780e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:09:37 +0100 Subject: [PATCH 401/526] Fix annoying messages in AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index a6cb6ee1f68..f11684e1f4a 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1139,7 +1139,7 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - LOG_DEBUG(log, getCurrentExceptionMessage(false)); + LOG_DEBUG(log, "Cannot read the statistics from block devices: {}", getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) From 6599016aec6998be11decfb69c5c64802298028c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 12 Nov 2022 19:09:45 +0100 Subject: [PATCH 402/526] Fix annoying messages in AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f11684e1f4a..574d67b3ef4 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1139,7 +1139,7 @@ void AsynchronousMetrics::update(TimePoint update_time) } catch (...) { - LOG_DEBUG(log, "Cannot read the statistics from block devices: {}", getCurrentExceptionMessage(false)); + LOG_DEBUG(log, "Cannot read statistics from block devices: {}", getCurrentExceptionMessage(false)); /// Try to reopen block devices in case of error /// (i.e. ENOENT or ENODEV means that some disk had been replaced, and it may appear with a new name) From 42de4686651774c97ef4bd701fcddde0ee5626f0 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Sat, 12 Nov 2022 19:47:06 +0000 Subject: [PATCH 403/526] Fixed unexpected behaviour of Interval types with subquery and casting --- src/Functions/FunctionsConversion.h | 27 +++++++++++++++++++ ...80_interval_casting_and_subquery.reference | 25 +++++++++++++++++ .../02480_interval_casting_and_subquery.sql | 25 +++++++++++++++++ 3 files changed, 77 insertions(+) create mode 100644 tests/queries/0_stateless/02480_interval_casting_and_subquery.reference create mode 100644 tests/queries/0_stateless/02480_interval_casting_and_subquery.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index dd494d821bf..57c191e7e6c 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2828,6 +2828,31 @@ private: }; } +#define GENERATE_INTERVAL_CASE(INTEVAL_KIND) \ + case IntervalKind::INTEVAL_KIND: \ + return createFunctionAdaptor(FunctionConvert::create(), from_type); + + static WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) + { + switch (kind) + { + GENERATE_INTERVAL_CASE(Nanosecond) + GENERATE_INTERVAL_CASE(Microsecond) + GENERATE_INTERVAL_CASE(Millisecond) + GENERATE_INTERVAL_CASE(Second) + GENERATE_INTERVAL_CASE(Minute) + GENERATE_INTERVAL_CASE(Hour) + GENERATE_INTERVAL_CASE(Day) + GENERATE_INTERVAL_CASE(Week) + GENERATE_INTERVAL_CASE(Month) + GENERATE_INTERVAL_CASE(Quarter) + GENERATE_INTERVAL_CASE(Year) + } + throw Exception{ErrorCodes::CANNOT_CONVERT_TYPE, "Conversion to unexpected IntervalKind: {}", kind.toString()}; + } + +#undef GENERATE_INTERVAL_CASE + template requires IsDataTypeDecimal WrapperType createDecimalWrapper(const DataTypePtr & from_type, const ToDataType * to_type, bool requested_result_is_nullable) const @@ -3853,6 +3878,8 @@ private: return createObjectWrapper(from_type, checkAndGetDataType(to_type.get())); case TypeIndex::AggregateFunction: return createAggregateFunctionWrapper(from_type, checkAndGetDataType(to_type.get())); + case TypeIndex::Interval: + return createIntervalWrapper(from_type, checkAndGetDataType(to_type.get())->getKind()); default: break; } diff --git a/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference b/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference new file mode 100644 index 00000000000..74df309720b --- /dev/null +++ b/tests/queries/0_stateless/02480_interval_casting_and_subquery.reference @@ -0,0 +1,25 @@ +5 2017-01-01 00:00:05 +5 2017-01-01 00:05:00 +5 2017-01-01 05:00:00 +5 2017-01-06 00:00:00 +5 2017-06-01 00:00:00 +5 2018-04-01 00:00:00 +5 2022-01-01 00:00:00 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 +5 2017-01-01 00:00:05 +5 2017-01-01 00:05:00 +5 2017-01-01 05:00:00 +5 2017-01-06 00:00:00 +5 2017-06-01 00:00:00 +5 2018-04-01 00:00:00 +5 2022-01-01 00:00:00 diff --git a/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql b/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql new file mode 100644 index 00000000000..cb6eccb06c0 --- /dev/null +++ b/tests/queries/0_stateless/02480_interval_casting_and_subquery.sql @@ -0,0 +1,25 @@ +SELECT toIntervalSecond(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalMinute(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalHour(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalDay(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalMonth(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalQuarter(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT toIntervalYear(5) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT CAST(5 AS IntervalNanosecond); +SELECT CAST(5 AS IntervalMicrosecond); +SELECT CAST(5 AS IntervalMillisecond); +SELECT CAST(5 AS IntervalSecond); +SELECT CAST(5 AS IntervalMinute); +SELECT CAST(5 AS IntervalHour); +SELECT CAST(5 AS IntervalDay); +SELECT CAST(5 AS IntervalWeek); +SELECT CAST(5 AS IntervalMonth); +SELECT CAST(5 AS IntervalQuarter); +SELECT CAST(5 AS IntervalYear); +SELECT (SELECT toIntervalSecond(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalMinute(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalHour(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalDay(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalMonth(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalQuarter(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; +SELECT (SELECT toIntervalYear(5)) AS interval, toDateTime('2017-01-01 00:00:00') + interval AS res; From b5428f535e9ce3c5a6f506b62fdab46705f74b24 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Sat, 12 Nov 2022 20:19:29 +0000 Subject: [PATCH 404/526] fix --- .../example-datasets/github.md | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index efe58a03341..c16053b3ea0 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -1821,7 +1821,7 @@ Note we limit to users with more than 20 changes to focus on regular committers ```sql SELECT author, - countIf((file_extension NOT IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, code / (code + test) AS ratio_code FROM git.file_changes @@ -1831,26 +1831,26 @@ ORDER BY code DESC LIMIT 20 ┌─author───────────────┬─test─┬──code─┬─────────ratio_code─┐ -│ Alexey Milovidov │ 7346 │ 41799 │ 0.8505239597110591 │ -│ Nikolai Kochetov │ 1106 │ 13361 │ 0.9235501486140872 │ -│ alesapin │ 1981 │ 8796 │ 0.8161826111162661 │ -│ kssenii │ 660 │ 6769 │ 0.9111589715977925 │ -│ Maksim Kita │ 1148 │ 5862 │ 0.8362339514978602 │ -│ Alexander Tokmakov │ 1135 │ 5727 │ 0.8345963276012824 │ -│ Vitaly Baranov │ 1283 │ 5521 │ 0.8114344503233392 │ -│ Ivan Lezhankin │ 726 │ 4698 │ 0.8661504424778761 │ -│ Anton Popov │ 831 │ 4346 │ 0.8394823256712381 │ -│ Ivan │ 4257 │ 4269 │ 0.5007037297677692 │ -│ Azat Khuzhin │ 1756 │ 3697 │ 0.6779754263708051 │ -│ Amos Bird │ 624 │ 2901 │ 0.8229787234042554 │ -│ proller │ 1226 │ 2377 │ 0.6597280044407439 │ -│ chertus │ 696 │ 2359 │ 0.772176759410802 │ -│ alexey-milovidov │ 254 │ 2321 │ 0.9013592233009708 │ -│ Alexey Arno │ 207 │ 2310 │ 0.9177592371871275 │ -│ Vitaliy Lyudvichenko │ 294 │ 2283 │ 0.8859138533178114 │ -│ Robert Schulze │ 251 │ 2196 │ 0.8974254188802615 │ -│ CurtizJ │ 705 │ 2158 │ 0.7537548026545582 │ -│ Alexander Kuzmenkov │ 1094 │ 2092 │ 0.6566227244193346 │ +│ Alexey Milovidov │ 4208 │ 41799 │ 0.9085356576173191 │ +│ Nikolai Kochetov │ 446 │ 13361 │ 0.9676975447236909 │ +│ alesapin │ 1893 │ 8796 │ 0.8229020488352512 │ +│ kssenii │ 649 │ 6769 │ 0.9125101105419251 │ +│ Maksim Kita │ 272 │ 5862 │ 0.9556569938050212 │ +│ Alexander Tokmakov │ 949 │ 5727 │ 0.8578490113840623 │ +│ Vitaly Baranov │ 988 │ 5521 │ 0.848210170533108 │ +│ Ivan Lezhankin │ 270 │ 4698 │ 0.9456521739130435 │ +│ Anton Popov │ 264 │ 4346 │ 0.9427331887201735 │ +│ Ivan │ 862 │ 4269 │ 0.8320015591502631 │ +│ Azat Khuzhin │ 957 │ 3697 │ 0.7943704340352385 │ +│ Amos Bird │ 121 │ 2901 │ 0.9599602911978822 │ +│ proller │ 549 │ 2377 │ 0.8123718386876282 │ +│ chertus │ 17 │ 2359 │ 0.9928451178451179 │ +│ alexey-milovidov │ 167 │ 2321 │ 0.9328778135048231 │ +│ Alexey Arno │ 67 │ 2310 │ 0.9718132099284813 │ +│ Vitaliy Lyudvichenko │ 247 │ 2283 │ 0.9023715415019763 │ +│ Robert Schulze │ 111 │ 2196 │ 0.951885565669701 │ +│ CurtizJ │ 144 │ 2158 │ 0.9374456993918332 │ +│ Alexander Kuzmenkov │ 134 │ 2092 │ 0.9398023360287511 │ └──────────────────────┴──────┴───────┴────────────────────┘ 20 rows in set. Elapsed: 0.034 sec. Processed 266.05 thousand rows, 4.65 MB (7.93 million rows/s., 138.76 MB/s.) From c2a3d83197d19b1ca484b372373c29a4eea08239 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Sat, 12 Nov 2022 20:20:35 +0000 Subject: [PATCH 405/526] fix link --- docs/en/getting-started/example-datasets/github.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index c16053b3ea0..1faa63884e4 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -1816,7 +1816,7 @@ There are a few ways we can address this question. Focusing on the code to test Note we limit to users with more than 20 changes to focus on regular committers and avoid a bias to one-off contributions. -[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBOT1QgSU4gKCdoJywgJ2NwcCcsICdzaCcsICdweScsICdleHBlY3QnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCkdST1VQIEJZIGF1dGhvcgpIQVZJTkcgY29kZSA+IDIwCk9SREVSIEJZIGNvZGUgREVTQwpMSU1JVCAyMA==) +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NoJywgJ3B5JywgJ2V4cGVjdCcpKSBBTkQgKHBhdGggTElLRSAnJXRlc3RzJScpKSBBUyB0ZXN0LAogICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkgQU5EIChOT1QgKHBhdGggTElLRSAnJXRlc3RzJScpKSkgQVMgY29kZSwKICAgIGNvZGUgLyAoY29kZSArIHRlc3QpIEFTIHJhdGlvX2NvZGUKRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKR1JPVVAgQlkgYXV0aG9yCkhBVklORyBjb2RlID4gMjAKT1JERVIgQlkgY29kZSBERVNDCkxJTUlUIDIw) ```sql SELECT From 1b7b87852089d997c82d400771ba6db254c879a7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 12 Nov 2022 21:04:01 +0000 Subject: [PATCH 406/526] don't randomize keeper fault injection setting for inserts --- tests/clickhouse-test | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index c054e66898e..a07a39600d8 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,8 +456,8 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "insert_keeper_max_retries": lambda: random.randint(20, 20), - "insert_keeper_fault_injection_probability": lambda: random.uniform(0, 0.01), + "insert_keeper_max_retries": lambda: 20, + "insert_keeper_fault_injection_probability": lambda: 0.01, } @staticmethod From 661c5cd0ae2b1fa65dd24b90ffaea2b3b3693e51 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Sat, 12 Nov 2022 23:58:09 +0100 Subject: [PATCH 407/526] fixed type --- src/Functions/FunctionsConversion.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 57c191e7e6c..208da8a78fe 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2828,9 +2828,9 @@ private: }; } -#define GENERATE_INTERVAL_CASE(INTEVAL_KIND) \ - case IntervalKind::INTEVAL_KIND: \ - return createFunctionAdaptor(FunctionConvert::create(), from_type); +#define GENERATE_INTERVAL_CASE(INTERVAL_KIND) \ + case IntervalKind::INTERVAL_KIND: \ + return createFunctionAdaptor(FunctionConvert::create(), from_type); static WrapperType createIntervalWrapper(const DataTypePtr & from_type, IntervalKind kind) { From 5a0b7b294767afffcb65ca696d11ec35d173c9a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 02:59:01 +0100 Subject: [PATCH 408/526] Add documentation for AsynchronousMetrics --- src/Interpreters/AsynchronousMetrics.cpp | 532 +++++++++++++++-------- src/Interpreters/AsynchronousMetrics.h | 16 +- src/Server/ProtocolServerAdapter.h | 2 + 3 files changed, 362 insertions(+), 188 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 574d67b3ef4..f861fbb8426 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -33,6 +32,7 @@ # include #endif + namespace DB { @@ -386,14 +386,15 @@ uint64_t updateJemallocEpoch() } template -static Value saveJemallocMetricImpl(AsynchronousMetricValues & values, +static Value saveJemallocMetricImpl( + AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) { Value value{}; size_t size = sizeof(value); mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); - values[clickhouse_full_name] = value; + values[clickhouse_full_name] = { value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html" }; return value; } @@ -570,85 +571,93 @@ void AsynchronousMetrics::update(TimePoint update_time) previous_update_time = update_time; /// This is also a good indicator of system responsiveness. - new_values["Jitter"] = std::chrono::duration_cast(current_time - update_time).count() / 1e9; + new_values["Jitter"] = { std::chrono::duration_cast(current_time - update_time).count() / 1e9, + "The difference in time the thread for calculation of the asynchronous metrics was scheduled to wake up and the time it was in fact, woken up." + " A proxy-indicator of overall system latency and responsiveness." }; + if (auto mark_cache = getContext()->getMarkCache()) { - if (auto mark_cache = getContext()->getMarkCache()) - { - new_values["MarkCacheBytes"] = mark_cache->weight(); - new_values["MarkCacheFiles"] = mark_cache->count(); - } + new_values["MarkCacheBytes"] = { mark_cache->weight(), "Total size of mark cache in bytes" }; + new_values["MarkCacheFiles"] = { mark_cache->count(), "Total number of mark files cached in the mark cache" }; } + if (auto uncompressed_cache = getContext()->getUncompressedCache()) { - if (auto uncompressed_cache = getContext()->getUncompressedCache()) - { - new_values["UncompressedCacheBytes"] = uncompressed_cache->weight(); - new_values["UncompressedCacheCells"] = uncompressed_cache->count(); - } + new_values["UncompressedCacheBytes"] = { uncompressed_cache->weight(), + "Total size of uncompressed cache in bytes. Uncompressed cache does not usually improve the performance and should be mostly avoided." }; + new_values["UncompressedCacheCells"] = { uncompressed_cache->count(), + "Total number of entries in the uncompressed cache. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." }; } + if (auto index_mark_cache = getContext()->getIndexMarkCache()) { - if (auto index_mark_cache = getContext()->getIndexMarkCache()) - { - new_values["IndexMarkCacheBytes"] = index_mark_cache->weight(); - new_values["IndexMarkCacheFiles"] = index_mark_cache->count(); - } + new_values["IndexMarkCacheBytes"] = { index_mark_cache->weight(), "Total size of mark cache for secondary indices in bytes." }; + new_values["IndexMarkCacheFiles"] = { index_mark_cache->count(), "Total number of mark files cached in the mark cache for secondary indices." }; } + if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) { - if (auto index_uncompressed_cache = getContext()->getIndexUncompressedCache()) - { - new_values["IndexUncompressedCacheBytes"] = index_uncompressed_cache->weight(); - new_values["IndexUncompressedCacheCells"] = index_uncompressed_cache->count(); - } + new_values["IndexUncompressedCacheBytes"] = { index_uncompressed_cache->weight(), + "Total size of uncompressed cache in bytes for secondary indices. Uncompressed cache does not usually improve the performance and should be mostly avoided." }; + new_values["IndexUncompressedCacheCells"] = { index_uncompressed_cache->count(), + "Total number of entries in the uncompressed cache for secondary indices. Each entry represents a decompressed block of data. Uncompressed cache does not usually improve performance and should be mostly avoided." }; } + if (auto mmap_cache = getContext()->getMMappedFileCache()) { - if (auto mmap_cache = getContext()->getMMappedFileCache()) - { - new_values["MMapCacheCells"] = mmap_cache->count(); - } + new_values["MMapCacheCells"] = { mmap_cache->count(), + "The number of files opened with `mmap` (mapped in memory)." + " This is used for queries with the setting `local_filesystem_read_method` set to `mmap`." + " The files opened with `mmap` are kept in the cache to avoid costly TLB flushes."}; } { auto caches = FileCacheFactory::instance().getAll(); + size_t total_bytes = 0; + size_t total_files = 0; + for (const auto & [_, cache_data] : caches) { - new_values["FilesystemCacheBytes"] = cache_data->cache->getUsedCacheSize(); - new_values["FilesystemCacheFiles"] = cache_data->cache->getFileSegmentsNum(); + total_bytes += cache_data->cache->getUsedCacheSize(); + total_files += cache_data->cache->getFileSegmentsNum(); } + + new_values["FilesystemCacheBytes"] = { total_bytes, + "Total bytes in the `cache` virtual filesystem. This cache is hold on disk." }; + new_values["FilesystemCacheFiles"] = { total_files, + "Total number of cached file segments in the `cache` virtual filesystem. This cache is hold on disk." }; } #if USE_ROCKSDB + if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) { - if (auto metadata_cache = getContext()->tryGetMergeTreeMetadataCache()) - { - new_values["MergeTreeMetadataCacheSize"] = metadata_cache->getEstimateNumKeys(); - } + new_values["MergeTreeMetadataCacheSize"] = { metadata_cache->getEstimateNumKeys(), + "The size of the metadata cache for tables. This cache is experimental and not used in production." }; } #endif #if USE_EMBEDDED_COMPILER + if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) { - if (auto * compiled_expression_cache = CompiledExpressionCacheFactory::instance().tryGetCache()) - { - new_values["CompiledExpressionCacheBytes"] = compiled_expression_cache->weight(); - new_values["CompiledExpressionCacheCount"] = compiled_expression_cache->count(); - } + new_values["CompiledExpressionCacheBytes"] = { compiled_expression_cache->weight(), + "Total bytes used for the cache of JIT-compiled code." }; + new_values["CompiledExpressionCacheCount"] = { compiled_expression_cache->count(), + "Total entries in the cache of JIT-compiled code." }; } #endif + new_values["Uptime"] = { getContext()->getUptimeSeconds(), + "The server uptime in seconds. It includes the time spent for server initialization before accepting connections." }; - new_values["Uptime"] = getContext()->getUptimeSeconds(); - + if (const auto stats = getHashTablesCacheStatistics()) { - if (const auto stats = getHashTablesCacheStatistics()) - { - new_values["HashTableStatsCacheEntries"] = stats->entries; - new_values["HashTableStatsCacheHits"] = stats->hits; - new_values["HashTableStatsCacheMisses"] = stats->misses; - } + new_values["HashTableStatsCacheEntries"] = { stats->entries, + "The number of entries in the cache of hash table sizes." + " The cache for hash table sizes is used for predictive optimization of GROUP BY." }; + new_values["HashTableStatsCacheHits"] = { stats->hits, + "The number of times the prediction of a hash table size was correct." }; + new_values["HashTableStatsCacheMisses"] = { stats->misses, + "The number of times the prediction of a hash table size was incorrect." }; } #if defined(OS_LINUX) || defined(OS_FREEBSD) @@ -660,7 +669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. auto epoch = updateJemallocEpoch(); - new_values["jemalloc.epoch"] = epoch; + new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of JeMalloc (a low-level memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); @@ -685,13 +694,24 @@ void AsynchronousMetrics::update(TimePoint update_time) { MemoryStatisticsOS::Data & data = memory_statistics_data; - new_values["MemoryVirtual"] = data.virt; - new_values["MemoryResident"] = data.resident; + new_values["MemoryVirtual"] = { data.virt, + "The size of the virtual address space allocated by the server process, in bytes." + " The size of the virtual address space is usually much greater than the physical memory consumption, and should not be used as an estimate for the memory consumption." + " The large values of this metric are totally normal, and makes only technical sense."}; + new_values["MemoryResident"] = { data.resident, + "The amount of physical memory used by the server process, in bytes." }; #if !defined(OS_FREEBSD) - new_values["MemoryShared"] = data.shared; + new_values["MemoryShared"] = { data.shared, + "The amount of memory used by the server process, that is also shared by another processes, in bytes." + " ClickHouse does not use shared memory, and the only case for this metric to be higher than zero is the usage of the system's C library (`libc`)." + " This metric does not make a lot of sense to watch, and it exists only for completeness reasons."}; #endif - new_values["MemoryCode"] = data.code; - new_values["MemoryDataAndStack"] = data.data_and_stack; + new_values["MemoryCode"] = { data.code, + "The amount of virtual memory mapped for the pages of machine code of the server process, in bytes." }; + new_values["MemoryDataAndStack"] = { data.data_and_stack, + "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." + " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." + " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; /// We must update the value of total_memory_tracker periodically. /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. @@ -754,11 +774,22 @@ void AsynchronousMetrics::update(TimePoint update_time) assertChar('/', *loadavg); readText(threads_total, *loadavg); - new_values["LoadAverage1"] = loadavg1; - new_values["LoadAverage5"] = loadavg5; - new_values["LoadAverage15"] = loadavg15; - new_values["OSThreadsRunnable"] = threads_runnable; - new_values["OSThreadsTotal"] = threads_total; +#define LOAD_AVERAGE_DOCUMENTATION \ + " The load represents the number of threads across all the processes (the scheduling entities of the OS kernel)," \ + " that are currently running by CPU or waiting for IO, or ready to run but not being scheduled at this point of time." \ + " This number includes all the processes, not only clickhouse-server. The number can be greater than the number of CPU cores," \ + " if the system is overloaded, and many processes are ready to run but waiting for CPU or IO." + + new_values["LoadAverage1"] = { loadavg1, + "The whole system load, averaged with exponential smoothing over 1 minute." LOAD_AVERAGE_DOCUMENTATION }; + new_values["LoadAverage5"] = { loadavg5, + "The whole system load, averaged with exponential smoothing over 5 minutes." LOAD_AVERAGE_DOCUMENTATION }; + new_values["LoadAverage15"] = { loadavg15, + "The whole system load, averaged with exponential smoothing over 15 minutes." LOAD_AVERAGE_DOCUMENTATION }; + new_values["OSThreadsRunnable"] = { threads_runnable, + "The total number of 'runnable' threads, as the OS kernel scheduler seeing it." }; + new_values["OSThreadsTotal"] = { threads_total, + "The total number of threads, as the OS kernel scheduler seeing it." }; } catch (...) { @@ -775,7 +806,7 @@ void AsynchronousMetrics::update(TimePoint update_time) Float64 uptime_seconds = 0; readText(uptime_seconds, *uptime); - new_values["OSUptime"] = uptime_seconds; + new_values["OSUptime"] = { uptime_seconds, "The uptime of the host server (the machine where ClickHouse is running), in seconds." }; } catch (...) { @@ -838,16 +869,43 @@ void AsynchronousMetrics::update(TimePoint update_time) else delta_values_all_cpus = delta_values; - new_values["OSUserTime" + cpu_suffix] = delta_values.user * multiplier; - new_values["OSNiceTime" + cpu_suffix] = delta_values.nice * multiplier; - new_values["OSSystemTime" + cpu_suffix] = delta_values.system * multiplier; - new_values["OSIdleTime" + cpu_suffix] = delta_values.idle * multiplier; - new_values["OSIOWaitTime" + cpu_suffix] = delta_values.iowait * multiplier; - new_values["OSIrqTime" + cpu_suffix] = delta_values.irq * multiplier; - new_values["OSSoftIrqTime" + cpu_suffix] = delta_values.softirq * multiplier; - new_values["OSStealTime" + cpu_suffix] = delta_values.steal * multiplier; - new_values["OSGuestTime" + cpu_suffix] = delta_values.guest * multiplier; - new_values["OSGuestNiceTime" + cpu_suffix] = delta_values.guest_nice * multiplier; + new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, + "The ratio of time the CPU core was running userspace code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, + "The ratio of time the CPU core was running userspace code with higher priority. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, + "The ratio of time the CPU core was running OS kernel (system) code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, + "The ratio of time spent for running hardware interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate hardware misconfiguration or a very high network load." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, + "The ratio of time spent for running software interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " A high number of this metric may indicate inefficient software running on the system." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " Not every virtualized environments present this metric, and most of them don't." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; + new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + " This metric is irrelevant for ClickHouse, but still exists for completeness." + " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; } prev_values = current_values; @@ -872,14 +930,18 @@ void AsynchronousMetrics::update(TimePoint update_time) UInt64 processes_running = 0; readText(processes_running, *proc_stat); skipToNextLineOrEOF(*proc_stat); - new_values["OSProcessesRunning"] = processes_running; + new_values["OSProcessesRunning"] = { processes_running, + "The number of runnable (running or ready to run) threads by the operating system." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "procs_blocked") { UInt64 processes_blocked = 0; readText(processes_blocked, *proc_stat); skipToNextLineOrEOF(*proc_stat); - new_values["OSProcessesBlocked"] = processes_blocked; + new_values["OSProcessesBlocked"] = { processes_blocked, + "Number of threads blocked waiting for I/O to complete (`man procfs`)." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else skipToNextLineOrEOF(*proc_stat); @@ -889,25 +951,45 @@ void AsynchronousMetrics::update(TimePoint update_time) { ProcStatValuesOther delta_values = current_other_values - proc_stat_values_other; - new_values["OSInterrupts"] = delta_values.interrupts; - new_values["OSContextSwitches"] = delta_values.context_switches; - new_values["OSProcessesCreated"] = delta_values.processes_created; + new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; /// Also write values normalized to 0..1 by diving to the number of CPUs. /// These values are good to be averaged across the cluster of non-uniform servers. if (num_cpus) { - new_values["OSUserTimeNormalized"] = delta_values_all_cpus.user * multiplier / num_cpus; - new_values["OSNiceTimeNormalized"] = delta_values_all_cpus.nice * multiplier / num_cpus; - new_values["OSSystemTimeNormalized"] = delta_values_all_cpus.system * multiplier / num_cpus; - new_values["OSIdleTimeNormalized"] = delta_values_all_cpus.idle * multiplier / num_cpus; - new_values["OSIOWaitTimeNormalized"] = delta_values_all_cpus.iowait * multiplier / num_cpus; - new_values["OSIrqTimeNormalized"] = delta_values_all_cpus.irq * multiplier / num_cpus; - new_values["OSSoftIrqTimeNormalized"] = delta_values_all_cpus.softirq * multiplier / num_cpus; - new_values["OSStealTimeNormalized"] = delta_values_all_cpus.steal * multiplier / num_cpus; - new_values["OSGuestTimeNormalized"] = delta_values_all_cpus.guest * multiplier / num_cpus; - new_values["OSGuestNiceTimeNormalized"] = delta_values_all_cpus.guest_nice * multiplier / num_cpus; + new_values["OSUserTimeNormalized"] = { delta_values_all_cpus.user * multiplier / num_cpus, + "The value is similar to `OSUserTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSNiceTimeNormalized"] = { delta_values_all_cpus.nice * multiplier / num_cpus, + "The value is similar to `OSNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSSystemTimeNormalized"] = { delta_values_all_cpus.system * multiplier / num_cpus, + "The value is similar to `OSSystemTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIdleTimeNormalized"] = { delta_values_all_cpus.idle * multiplier / num_cpus, + "The value is similar to `OSIdleTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIOWaitTimeNormalized"] = { delta_values_all_cpus.iowait * multiplier / num_cpus, + "The value is similar to `OSIOWaitTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSIrqTimeNormalized"] = { delta_values_all_cpus.irq * multiplier / num_cpus, + "The value is similar to `OSIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSSoftIrqTimeNormalized"] = { delta_values_all_cpus.softirq * multiplier / num_cpus, + "The value is similar to `OSSoftIrqTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSStealTimeNormalized"] = { delta_values_all_cpus.steal * multiplier / num_cpus, + "The value is similar to `OSStealTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestTimeNormalized"] = { delta_values_all_cpus.guest * multiplier / num_cpus, + "The value is similar to `OSGuestTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; + new_values["OSGuestNiceTimeNormalized"] = { delta_values_all_cpus.guest_nice * multiplier / num_cpus, + "The value is similar to `OSGuestNiceTime` but divided to the number of CPU cores to be measured in the [0..1] interval regardless of the number of cores." + " This allows you to average the values of this metric across multiple servers in a cluster even if the number of cores is non-uniform, and still get the average resource utilization metric."}; } } @@ -962,39 +1044,47 @@ void AsynchronousMetrics::update(TimePoint update_time) if (name == "MemTotal:") { - new_values["OSMemoryTotal"] = bytes; + new_values["OSMemoryTotal"] = { bytes, "The total amount of memory on the host system, in bytes." }; } else if (name == "MemFree:") { - /// We cannot simply name this metric "Free", because it confuses users. - /// See https://www.linuxatemyram.com/ - /// For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable. - free_plus_cached_bytes += bytes; - new_values["OSMemoryFreeWithoutCached"] = bytes; + new_values["OSMemoryFreeWithoutCached"] = { bytes, + "The amount of free memory on the host system, in bytes." + " This does not include the memory used by the OS page cache memory, in bytes." + " The page cache memory is also available for usage by programs, so the value of this metric can be confusing." + " See the `OSMemoryAvailable` metric instead." + " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." + " See also https://www.linuxatemyram.com/." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "MemAvailable:") { - new_values["OSMemoryAvailable"] = bytes; + new_values["OSMemoryAvailable"] = { bytes, "The amount of memory available to be used by programs, in bytes. This is very similar to the `OSMemoryFreePlusCached` metric." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Buffers:") { - new_values["OSMemoryBuffers"] = bytes; + new_values["OSMemoryBuffers"] = { bytes, "The amount of memory used by OS kernel buffers, in bytes. This should be typically small, and large values may indicate a misconfiguration of the OS." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Cached:") { free_plus_cached_bytes += bytes; - new_values["OSMemoryCached"] = bytes; + new_values["OSMemoryCached"] = { bytes, "The amount of memory used by the OS page cache, in bytes. Typically, almost all available memory is used by the OS page cache - high values of this metric are normal and expected." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "SwapCached:") { - new_values["OSMemorySwapCached"] = bytes; + new_values["OSMemorySwapCached"] = { bytes, "The amount of memory in swap that was also loaded in RAM. Swap should be disabled on production systems. If the value of this metric is large, it indicates a misconfiguration." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } skipToNextLineOrEOF(*meminfo); } - new_values["OSMemoryFreePlusCached"] = free_plus_cached_bytes; + new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory or OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1043,7 +1133,7 @@ void AsynchronousMetrics::update(TimePoint update_time) if (auto colon = s.find_first_of(':')) { auto mhz = std::stod(s.substr(colon + 2)); - new_values[fmt::format("CPUFrequencyMHz_{}", core_id)] = mhz; + new_values[fmt::format("CPUFrequencyMHz_{}", core_id)] = { mhz, "The current frequency of the CPU, in MHz. Most of the modern CPUs adjust the frequency dynamically for power saving and Turbo Boosting." }; } } } @@ -1062,7 +1152,8 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t open_files = 0; readText(open_files, *file_nr); - new_values["OSOpenFiles"] = open_files; + new_values["OSOpenFiles"] = { open_files, "The total number of opened files on the host machine." + " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1107,33 +1198,80 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Always in milliseconds according to the docs. static constexpr double time_multiplier = 1e-6; - new_values["BlockReadOps_" + name] = delta_values.read_ios; - new_values["BlockWriteOps_" + name] = delta_values.write_ios; - new_values["BlockDiscardOps_" + name] = delta_values.discard_ops; +#define BLOCK_DEVICE_EXPLANATION \ + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." \ + " See https://www.kernel.org/doc/Documentation/block/stat.txt" - new_values["BlockReadMerges_" + name] = delta_values.read_merges; - new_values["BlockWriteMerges_" + name] = delta_values.write_merges; - new_values["BlockDiscardMerges_" + name] = delta_values.discard_merges; + new_values["BlockReadOps_" + name] = { delta_values.read_ios, + "Number of read operations requested from the block device." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteOps_" + name] = { delta_values.write_ios, + "Number of write operations requested from the block device." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardOps_" + name] = { delta_values.discard_ops, + "Number of discard operations requested from the block device. These operations are relevant for SSD." + " They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockReadBytes_" + name] = delta_values.read_sectors * sector_size; - new_values["BlockWriteBytes_" + name] = delta_values.write_sectors * sector_size; - new_values["BlockDiscardBytes_" + name] = delta_values.discard_sectors * sector_size; + new_values["BlockReadMerges_" + name] = { delta_values.read_merges, + "Number of read operations requested from the block device and merged together by the OS IO scheduler." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteMerges_" + name] = { delta_values.write_merges, + "Number of write operations requested from the block device and merged together by the OS IO scheduler." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardMerges_" + name] = { delta_values.discard_merges, + "Number of discard operations requested from the block device and merged together by the OS IO scheduler." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockReadTime_" + name] = delta_values.read_ticks * time_multiplier; - new_values["BlockWriteTime_" + name] = delta_values.write_ticks * time_multiplier; - new_values["BlockDiscardTime_" + name] = delta_values.discard_ticks * time_multiplier; + new_values["BlockReadBytes_" + name] = { delta_values.read_sectors * sector_size, + "Number of bytes read from the block device." + " It can be lower than the number of bytes read from the filesystem due to the usage of the OS page cache, that saves IO." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteBytes_" + name] = { delta_values.write_sectors * sector_size, + "Number of bytes written to the block device." + " It can be lower than the number of bytes written to the filesystem due to the usage of the OS page cache, that saves IO." + " A write to the block device may happen later than the corresponding write to the filesystem due to write-through caching." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardBytes_" + name] = { delta_values.discard_sectors * sector_size, + "Number of discarded bytes on the block device." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockInFlightOps_" + name] = delta_values.in_flight_ios; + new_values["BlockReadTime_" + name] = { delta_values.read_ticks * time_multiplier, + "Time in seconds spend in read operations requested from the block device, summed across all the operations." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockWriteTime_" + name] = { delta_values.write_ticks * time_multiplier, + "Time in seconds spend in write operations requested from the block device, summed across all the operations." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockDiscardTime_" + name] = { delta_values.discard_ticks * time_multiplier, + "Time in seconds spend in discard operations requested from the block device, summed across all the operations." + " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + BLOCK_DEVICE_EXPLANATION }; - new_values["BlockActiveTime_" + name] = delta_values.io_ticks * time_multiplier; - new_values["BlockQueueTime_" + name] = delta_values.time_in_queue * time_multiplier; + new_values["BlockInFlightOps_" + name] = { delta_values.in_flight_ios, + "This value counts the number of I/O requests that have been issued to" + " the device driver but have not yet completed. It does not include IO" + " requests that are in the queue but not yet issued to the device driver." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockActiveTime_" + name] = { delta_values.io_ticks * time_multiplier, + "Time in seconds the block device had the IO requests queued." + BLOCK_DEVICE_EXPLANATION }; + new_values["BlockQueueTime_" + name] = { delta_values.time_in_queue * time_multiplier, + "This value counts the number of milliseconds that IO requests have waited" + " on this block device. If there are multiple IO requests waiting, this" + " value will increase as the product of the number of milliseconds times the" + " number of requests waiting." + BLOCK_DEVICE_EXPLANATION }; if (delta_values.in_flight_ios) { /// TODO Check if these values are meaningful. - new_values["BlockActiveTimePerOp_" + name] = delta_values.io_ticks * time_multiplier / delta_values.in_flight_ios; - new_values["BlockQueueTimePerOp_" + name] = delta_values.time_in_queue * time_multiplier / delta_values.in_flight_ios; + new_values["BlockActiveTimePerOp_" + name] = { delta_values.io_ticks * time_multiplier / delta_values.in_flight_ios, + "Similar to the `BlockActiveTime` metrics, but the value is divided to the number of IO operations to count the per-operation time." }; + new_values["BlockQueueTimePerOp_" + name] = { delta_values.time_in_queue * time_multiplier / delta_values.in_flight_ios, + "Similar to the `BlockQueueTime` metrics, but the value is divided to the number of IO operations to count the per-operation time." }; } } } @@ -1221,15 +1359,31 @@ void AsynchronousMetrics::update(TimePoint update_time) if (!first_run) { - new_values["NetworkReceiveBytes_" + interface_name] = delta_values.recv_bytes; - new_values["NetworkReceivePackets_" + interface_name] = delta_values.recv_packets; - new_values["NetworkReceiveErrors_" + interface_name] = delta_values.recv_errors; - new_values["NetworkReceiveDrop_" + interface_name] = delta_values.recv_drop; + new_values["NetworkReceiveBytes_" + interface_name] = { delta_values.recv_bytes, + " Number of bytes received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceivePackets_" + interface_name] = { delta_values.recv_packets, + " Number of network packets received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceiveErrors_" + interface_name] = { delta_values.recv_errors, + " Number of times error happened receiving via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkReceiveDrop_" + interface_name] = { delta_values.recv_drop, + " Number of bytes a packet was dropped while received via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["NetworkSendBytes_" + interface_name] = delta_values.send_bytes; - new_values["NetworkSendPackets_" + interface_name] = delta_values.send_packets; - new_values["NetworkSendErrors_" + interface_name] = delta_values.send_errors; - new_values["NetworkSendDrop_" + interface_name] = delta_values.send_drop; + new_values["NetworkSendBytes_" + interface_name] = { delta_values.send_bytes, + " Number of bytes sent via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendPackets_" + interface_name] = { delta_values.send_packets, + " Number of network packets sent via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendErrors_" + interface_name] = { delta_values.send_errors, + " Number of times error (e.g. TCP retransmit) happened while sending via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["NetworkSendDrop_" + interface_name] = { delta_values.send_drop, + " Number of times a packed was dropped while sending via the network interface." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } } } @@ -1248,7 +1402,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); Int64 temperature = 0; readText(temperature, in); - new_values[fmt::format("Temperature{}", i)] = temperature * 0.001; + new_values[fmt::format("Temperature{}", i)] = { temperature * 0.001, + "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value." }; } } catch (...) @@ -1287,9 +1442,11 @@ void AsynchronousMetrics::update(TimePoint update_time) } if (sensor_name.empty()) - new_values[fmt::format("Temperature_{}", hwmon_name)] = temperature * 0.001; + new_values[fmt::format("Temperature_{}", hwmon_name)] = { temperature * 0.001, + "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value." }; else - new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = temperature * 0.001; + new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = { temperature * 0.001, + "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value." }; } } } @@ -1325,7 +1482,9 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Correctable", i)] = errors; + new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors", + " A high number of this value indicates bad RAM which has to be immediately replaced," + " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; } if (edac[i].second) @@ -1334,7 +1493,9 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Uncorrectable", i)] = errors; + new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors", + " A non-zero number of this value indicates bad RAM which has to be immediately replaced," + " because it indicates potential data corruption." }; } } } @@ -1475,44 +1636,46 @@ void AsynchronousMetrics::update(TimePoint update_time) } } - new_values["ReplicasMaxQueueSize"] = max_queue_size; - new_values["ReplicasMaxInsertsInQueue"] = max_inserts_in_queue; - new_values["ReplicasMaxMergesInQueue"] = max_merges_in_queue; + new_values["ReplicasMaxQueueSize"] = { max_queue_size, "Maximum queue size (in the number of operations like get, merge) across Replicated tables." }; + new_values["ReplicasMaxInsertsInQueue"] = { max_inserts_in_queue, "Maximum number of INSERT operations in the queue (still to be replicated) across Replicated tables." }; + new_values["ReplicasMaxMergesInQueue"] = { max_merges_in_queue, "Maximum number of merge operations in the queue (still to be applied) across Replicated tables." }; - new_values["ReplicasSumQueueSize"] = sum_queue_size; - new_values["ReplicasSumInsertsInQueue"] = sum_inserts_in_queue; - new_values["ReplicasSumMergesInQueue"] = sum_merges_in_queue; + new_values["ReplicasSumQueueSize"] = { sum_queue_size, "Sum queue size (in the number of operations like get, merge) across Replicated tables." }; + new_values["ReplicasSumInsertsInQueue"] = { sum_inserts_in_queue, "Sum of INSERT operations in the queue (still to be replicated) across Replicated tables." }; + new_values["ReplicasSumMergesInQueue"] = { sum_merges_in_queue, "Sum of merge operations in the queue (still to be applied) across Replicated tables." }; - new_values["ReplicasMaxAbsoluteDelay"] = max_absolute_delay; - new_values["ReplicasMaxRelativeDelay"] = max_relative_delay; + new_values["ReplicasMaxAbsoluteDelay"] = { max_absolute_delay, "Maximum difference in seconds between the most fresh replicated part and the most fresh data part still to be replicated, across Replicated tables. A very high value indicates a replica with no data." }; + new_values["ReplicasMaxRelativeDelay"] = { max_relative_delay, "Maximum difference between the replica delay and the delay of the most up-to-date replica of the same table, across Replicated tables." }; - new_values["MaxPartCountForPartition"] = max_part_count_for_partition; + new_values["MaxPartCountForPartition"] = { max_part_count_for_partition, "Maximum number of parts per partition across all partitions of all tables of MergeTree family. Values larger than 300 indicates misconfiguration, overload, or massive data loading." }; - new_values["NumberOfDatabases"] = number_of_databases; - new_values["NumberOfTables"] = total_number_of_tables; + new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." }; + new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables." + "The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; - new_values["TotalBytesOfMergeTreeTables"] = total_number_of_bytes; - new_values["TotalRowsOfMergeTreeTables"] = total_number_of_rows; - new_values["TotalPartsOfMergeTreeTables"] = total_number_of_parts; + new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." }; + new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." }; + new_values["TotalPartsOfMergeTreeTables"] = { total_number_of_parts, "Total amount of data parts in all tables of MergeTree family." + " Numbers larger than 10 000 will negatively affect the server startup time and it may indicate unreasonable choice of the partition key." }; - auto get_metric_name = [](const String & name) -> const char * + auto get_metric_name_doc = [](const String & name) -> std::pair { - static std::map metric_map = + static std::map> metric_map = { - {"tcp_port", "TCPThreads"}, - {"tcp_port_secure", "TCPSecureThreads"}, - {"http_port", "HTTPThreads"}, - {"https_port", "HTTPSecureThreads"}, - {"interserver_http_port", "InterserverThreads"}, - {"interserver_https_port", "InterserverSecureThreads"}, - {"mysql_port", "MySQLThreads"}, - {"postgresql_port", "PostgreSQLThreads"}, - {"grpc_port", "GRPCThreads"}, - {"prometheus.port", "PrometheusThreads"} + {"tcp_port", {"TCPThreads", "Number of threads in the server of the TCP protocol (without TLS)."}}, + {"tcp_port_secure", {"TCPSecureThreads", "Number of threads in the server of the TCP protocol (with TLS)."}}, + {"http_port", {"HTTPThreads", "Number of threads in the server of the HTTP interface (without TLS)."}}, + {"https_port", {"HTTPSecureThreads", "Number of threads in the server of the HTTPS interface."}}, + {"interserver_http_port", {"InterserverThreads", "Number of threads in the server of the replicas communication protocol (without TLS)."}}, + {"interserver_https_port", {"InterserverSecureThreads", "Number of threads in the server of the replicas communication protocol (with TLS)."}}, + {"mysql_port", {"MySQLThreads", "Number of threads in the server of the MySQL compatibility protocol."}}, + {"postgresql_port", {"PostgreSQLThreads", "Number of threads in the server of the PostgreSQL compatibility protocol."}}, + {"grpc_port", {"GRPCThreads", "Number of threads in the server of the GRPC protocol."}}, + {"prometheus.port", {"PrometheusThreads", "Number of threads in the server of the Prometheus endpoint. Note: prometheus endpoints can be also used via the usual HTTP/HTTPs ports."}} }; auto it = metric_map.find(name); if (it == metric_map.end()) - return nullptr; + return { nullptr, nullptr }; else return it->second; }; @@ -1520,8 +1683,8 @@ void AsynchronousMetrics::update(TimePoint update_time) const auto server_metrics = protocol_server_metrics_func(); for (const auto & server_metric : server_metrics) { - if (const auto * name = get_metric_name(server_metric.port_name)) - new_values[name] = server_metric.current_threads; + if (auto name_doc = get_metric_name_doc(server_metric.port_name); name_doc.first != nullptr) + new_values[name_doc.first] = { server_metric.current_threads, name_doc.second }; } } #if USE_NURAFT @@ -1534,14 +1697,14 @@ void AsynchronousMetrics::update(TimePoint update_time) size_t is_observer = 0; size_t is_standalone = 0; size_t znode_count = 0; - size_t watch_count =0; + size_t watch_count = 0; size_t ephemerals_count = 0; - size_t approximate_data_size =0; + size_t approximate_data_size = 0; size_t key_arena_size = 0; - size_t latest_snapshot_size =0; - size_t open_file_descriptor_count =0; - size_t max_file_descriptor_count =0; - size_t followers =0; + size_t latest_snapshot_size = 0; + size_t open_file_descriptor_count = 0; + size_t max_file_descriptor_count = 0; + size_t followers = 0; size_t synced_followers = 0; size_t zxid = 0; size_t session_with_watches = 0; @@ -1582,29 +1745,29 @@ void AsynchronousMetrics::update(TimePoint update_time) } } - new_values["KeeperIsLeader"] = is_leader; - new_values["KeeperIsFollower"] = is_follower; - new_values["KeeperIsObserver"] = is_observer; - new_values["KeeperIsStandalone"] = is_standalone; + new_values["KeeperIsLeader"] = { is_leader, "1 if ClickHouse Keeper is a leader, 0 otherwise." }; + new_values["KeeperIsFollower"] = { is_follower, "1 if ClickHouse Keeper is a follower, 0 otherwise." }; + new_values["KeeperIsObserver"] = { is_observer, "1 if ClickHouse Keeper is an observer, 0 otherwise." }; + new_values["KeeperIsStandalone"] = { is_standalone, "1 if ClickHouse Keeper is in a standalone mode, 0 otherwise." }; - new_values["KeeperZnodeCount"] = znode_count; - new_values["KeeperWatchCount"] = watch_count; - new_values["KeeperEphemeralsCount"] = ephemerals_count; + new_values["KeeperZnodeCount"] = { znode_count, "The number of nodes (data entries) in ClickHouse Keeper." }; + new_values["KeeperWatchCount"] = { watch_count, "The number of watches in ClickHouse Keeper." }; + new_values["KeeperEphemeralsCount"] = { ephemerals_count, "The number of ephemeral nodes in ClickHouse Keeper." }; - new_values["KeeperApproximateDataSize"] = approximate_data_size; - new_values["KeeperKeyArenaSize"] = key_arena_size; - new_values["KeeperLatestSnapshotSize"] = latest_snapshot_size; + new_values["KeeperApproximateDataSize"] = { approximate_data_size, "The approximate data size of ClickHouse Keeper, in bytes." }; + new_values["KeeperKeyArenaSize"] = { key_arena_size, "The size in bytes of the memory arena for keys in ClickHouse Keeper." }; + new_values["KeeperLatestSnapshotSize"] = { latest_snapshot_size, "The uncompressed size in bytes of the latest snapshot created by ClickHouse Keeper." }; - new_values["KeeperOpenFileDescriptorCount"] = open_file_descriptor_count; - new_values["KeeperMaxFileDescriptorCount"] = max_file_descriptor_count; + new_values["KeeperOpenFileDescriptorCount"] = { open_file_descriptor_count, "The number of open file descriptors in ClickHouse Keeper." }; + new_values["KeeperMaxFileDescriptorCount"] = { max_file_descriptor_count, "The maximum number of open file descriptors in ClickHouse Keeper." }; - new_values["KeeperFollowers"] = followers; - new_values["KeeperSyncedFollowers"] = synced_followers; - new_values["KeeperZxid"] = zxid; - new_values["KeeperSessionWithWatches"] = session_with_watches; - new_values["KeeperPathsWatched"] = paths_watched; - new_values["KeeperSnapshotDirSize"] = snapshot_dir_size; - new_values["KeeperLogDirSize"] = log_dir_size; + new_values["KeeperFollowers"] = { followers, "The number of followers of ClickHouse Keeper." }; + new_values["KeeperSyncedFollowers"] = { synced_followers, "The number of followers of ClickHouse Keeper who are also in-sync." }; + new_values["KeeperZxid"] = { zxid, "The current transaction id number (zxid) in ClickHouse Keeper." }; + new_values["KeeperSessionWithWatches"] = { session_with_watches, "The number of client sessions of ClickHouse Keeper having watches." }; + new_values["KeeperPathsWatched"] = { paths_watched, "The number of different paths watched by the clients of ClickHouse Keeper." }; + new_values["KeeperSnapshotDirSize"] = { snapshot_dir_size, "The size of the snapshots directory of ClickHouse Keeper, in bytes." }; + new_values["KeeperLogDirSize"] = { log_dir_size, "The size of the logs directory of ClickHouse Keeper, in bytes." }; } } #endif @@ -1613,7 +1776,7 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Add more metrics as you wish. - new_values["AsynchronousMetricsCalculationTimeSpent"] = watch.elapsedSeconds(); + new_values["AsynchronousMetricsCalculationTimeSpent"] = { watch.elapsedSeconds(), "Time in seconds spent for calculation of asynchronous metrics (this is the overhead of asynchronous metrics)." }; /// Log the new metrics. if (auto asynchronous_metric_log = getContext()->getAsynchronousMetricLog()) @@ -1693,11 +1856,10 @@ void AsynchronousMetrics::updateHeavyMetricsIfNeeded(TimePoint current_time, Tim update_period.count(), heavy_metric_update_period.count(), watch.elapsedSeconds()); - } - new_values["NumberOfDetachedParts"] = detached_parts_stats.count; - new_values["NumberOfDetachedByUserParts"] = detached_parts_stats.detached_by_user; + new_values["NumberOfDetachedParts"] = { detached_parts_stats.count, "The total number of parts detached from MergeTree tables. A part can be detached by a user with the `ALTER TABLE DETACH` query or by the server itself it the part is broken, unexpected or unneeded. The server does not care about detached parts and they can be removed." }; + new_values["NumberOfDetachedByUserParts"] = { detached_parts_stats.detached_by_user, "The total number of parts detached from MergeTree tables by users with the `ALTER TABLE DETACH` query (as opposed to unexpected, broken or ignored parts). The server does not care about detached parts and they can be removed." }; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 6e32bdb43b8..32a82bc106e 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -18,16 +18,23 @@ namespace Poco { -class Logger; + class Logger; } namespace DB { -class ProtocolServerAdapter; class ReadBuffer; -using AsynchronousMetricValue = double; +struct AsynchronousMetricValue +{ + double value; + const char * documentation; + + AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} + AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} +}; + using AsynchronousMetricValues = std::unordered_map; struct ProtocolServerMetrics @@ -42,6 +49,9 @@ struct ProtocolServerMetrics * * This includes both ClickHouse-related metrics (like memory usage of ClickHouse process) * and common OS-related metrics (like total memory usage on the server). + * + * All the values are either gauge type (like the total number of tables, the current memory usage). + * Or delta-counters representing some accumulation during the interval of time. */ class AsynchronousMetrics : WithContext { diff --git a/src/Server/ProtocolServerAdapter.h b/src/Server/ProtocolServerAdapter.h index 850640ab70a..514354f9723 100644 --- a/src/Server/ProtocolServerAdapter.h +++ b/src/Server/ProtocolServerAdapter.h @@ -6,8 +6,10 @@ #include #include + namespace DB { + class GRPCServer; class TCPServer; From ba05b7dd2c2218b1e63d701c5b762e2ace26f1b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 03:36:20 +0100 Subject: [PATCH 409/526] Add documentation for AsynchronousMetrics --- programs/server/MetricsTransmitter.cpp | 2 +- src/Interpreters/AsynchronousMetricLog.cpp | 2 +- src/Interpreters/AsynchronousMetricLog.h | 5 +- src/Interpreters/AsynchronousMetrics.cpp | 56 ++++++++++++------- src/Interpreters/AsynchronousMetrics.h | 1 + src/Server/PrometheusMetricsWriter.cpp | 7 ++- src/Server/PrometheusMetricsWriter.h | 2 +- .../StorageSystemAsynchronousMetrics.cpp | 4 +- 8 files changed, 51 insertions(+), 28 deletions(-) diff --git a/programs/server/MetricsTransmitter.cpp b/programs/server/MetricsTransmitter.cpp index 8ad519ba5aa..f7829a49a39 100644 --- a/programs/server/MetricsTransmitter.cpp +++ b/programs/server/MetricsTransmitter.cpp @@ -123,7 +123,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count { for (const auto & name_value : async_metrics_values) { - key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second); + key_vals.emplace_back(asynchronous_metrics_path_prefix + name_value.first, name_value.second.value); } } diff --git a/src/Interpreters/AsynchronousMetricLog.cpp b/src/Interpreters/AsynchronousMetricLog.cpp index 228934d5f4d..6176bb781ab 100644 --- a/src/Interpreters/AsynchronousMetricLog.cpp +++ b/src/Interpreters/AsynchronousMetricLog.cpp @@ -47,7 +47,7 @@ void AsynchronousMetricLog::addValues(const AsynchronousMetricValues & values) for (const auto & [key, value] : values) { element.metric_name = key; - element.value = round(value * precision) / precision; + element.value = round(value.value * precision) / precision; add(element); } diff --git a/src/Interpreters/AsynchronousMetricLog.h b/src/Interpreters/AsynchronousMetricLog.h index 900d84868bd..8a19fae29e9 100644 --- a/src/Interpreters/AsynchronousMetricLog.h +++ b/src/Interpreters/AsynchronousMetricLog.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -14,12 +15,8 @@ namespace DB { -using AsynchronousMetricValue = double; -using AsynchronousMetricValues = std::unordered_map; - /** AsynchronousMetricLog is a log of metric values measured at regular time interval. */ - struct AsynchronousMetricLogElement { UInt16 event_date; diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index f861fbb8426..6e8d745ef0e 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -394,7 +394,7 @@ static Value saveJemallocMetricImpl( Value value{}; size_t size = sizeof(value); mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); - values[clickhouse_full_name] = { value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html" }; + values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html"); return value; } @@ -1482,7 +1482,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors", + new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, + "The number of correctable ECC memory errors." " A high number of this value indicates bad RAM which has to be immediately replaced," " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; } @@ -1493,7 +1494,8 @@ void AsynchronousMetrics::update(TimePoint update_time) in.rewind(); uint64_t errors = 0; readText(errors, in); - new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors", + new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, + "The number of uncorrectable ECC memory errors." " A non-zero number of this value indicates bad RAM which has to be immediately replaced," " because it indicates potential data corruption." }; } @@ -1519,24 +1521,36 @@ void AsynchronousMetrics::update(TimePoint update_time) { auto stat = getStatVFS(getContext()->getPath()); - new_values["FilesystemMainPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemMainPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemMainPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemMainPathTotalINodes"] = stat.f_files; - new_values["FilesystemMainPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemMainPathUsedINodes"] = stat.f_files - stat.f_favail; + new_values["FilesystemMainPathTotalBytes"] = { stat.f_blocks * stat.f_frsize, + "The size of the volume where the main ClickHouse path is mounted, in bytes." }; + new_values["FilesystemMainPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize, + "Available bytes on the volume where the main ClickHouse path is mounted." }; + new_values["FilesystemMainPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize, + "Used bytes on the volume where the main ClickHouse path is mounted." }; + new_values["FilesystemMainPathTotalINodes"] = { stat.f_files, + "The total number of inodes on the volume where the main ClickHouse path is mounted. If it is less than 25 million, it indicates a misconfiguration." }; + new_values["FilesystemMainPathAvailableINodes"] = { stat.f_favail, + "The number of available inodes on the volume where the main ClickHouse path is mounted. If it is close to zero, it indicates a misconfiguration, and you will get 'no space left on device' even when the disk is not full." }; + new_values["FilesystemMainPathUsedINodes"] = { stat.f_files - stat.f_favail, + "The number of used inodes on the volume where the main ClickHouse path is mounted. This value mostly corresponds to the number of files." }; } { /// Current working directory of the server is the directory with logs. auto stat = getStatVFS("."); - new_values["FilesystemLogsPathTotalBytes"] = stat.f_blocks * stat.f_frsize; - new_values["FilesystemLogsPathAvailableBytes"] = stat.f_bavail * stat.f_frsize; - new_values["FilesystemLogsPathUsedBytes"] = (stat.f_blocks - stat.f_bavail) * stat.f_frsize; - new_values["FilesystemLogsPathTotalINodes"] = stat.f_files; - new_values["FilesystemLogsPathAvailableINodes"] = stat.f_favail; - new_values["FilesystemLogsPathUsedINodes"] = stat.f_files - stat.f_favail; + new_values["FilesystemLogsPathTotalBytes"] = { stat.f_blocks * stat.f_frsize, + "The size of the volume where ClickHouse logs path is mounted, in bytes. It's recommended to have at least 10 GB for logs." }; + new_values["FilesystemLogsPathAvailableBytes"] = { stat.f_bavail * stat.f_frsize, + "Available bytes on the volume where ClickHouse logs path is mounted. If this value approaches zero, you should tune the log rotation in the configuration file." }; + new_values["FilesystemLogsPathUsedBytes"] = { (stat.f_blocks - stat.f_bavail) * stat.f_frsize, + "Used bytes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathTotalINodes"] = { stat.f_files, + "The total number of inodes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathAvailableINodes"] = { stat.f_favail, + "The number of available inodes on the volume where ClickHouse logs path is mounted." }; + new_values["FilesystemLogsPathUsedINodes"] = { stat.f_files - stat.f_favail, + "The number of used inodes on the volume where ClickHouse logs path is mounted." }; } /// Free and total space on every configured disk. @@ -1553,10 +1567,14 @@ void AsynchronousMetrics::update(TimePoint update_time) auto available = disk->getAvailableSpace(); auto unreserved = disk->getUnreservedSpace(); - new_values[fmt::format("DiskTotal_{}", name)] = total; - new_values[fmt::format("DiskUsed_{}", name)] = total - available; - new_values[fmt::format("DiskAvailable_{}", name)] = available; - new_values[fmt::format("DiskUnreserved_{}", name)] = unreserved; + new_values[fmt::format("DiskTotal_{}", name)] = { total, + "The total size in bytes of the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." }; + new_values[fmt::format("DiskUsed_{}", name)] = { total - available, + "Used bytes on the disk (virtual filesystem). Remote filesystems not always provide this information." }; + new_values[fmt::format("DiskAvailable_{}", name)] = { available, + "Available bytes on the disk (virtual filesystem). Remote filesystems can show a large value like 16 EiB." }; + new_values[fmt::format("DiskUnreserved_{}", name)] = { unreserved, + "Available bytes on the disk (virtual filesystem) without the reservations for merges, fetches, and moves. Remote filesystems can show a large value like 16 EiB." }; } } diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 32a82bc106e..3529c162944 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -33,6 +33,7 @@ struct AsynchronousMetricValue AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} + AsynchronousMetricValue() = default; /// For std::unordered_map::operator[]. }; using AsynchronousMetricValues = std::unordered_map; diff --git a/src/Server/PrometheusMetricsWriter.cpp b/src/Server/PrometheusMetricsWriter.cpp index 9168eb5f24d..843d1e64463 100644 --- a/src/Server/PrometheusMetricsWriter.cpp +++ b/src/Server/PrometheusMetricsWriter.cpp @@ -108,11 +108,16 @@ void PrometheusMetricsWriter::write(WriteBuffer & wb) const if (!replaceInvalidChars(key)) continue; + auto value = name_value.second; + std::string metric_doc{value.documentation}; + convertHelpToSingleLine(metric_doc); + // TODO: add HELP section? asynchronous_metrics contains only key and value + writeOutLine(wb, "# HELP", key, metric_doc); writeOutLine(wb, "# TYPE", key, "gauge"); - writeOutLine(wb, key, value); + writeOutLine(wb, key, value.value); } } diff --git a/src/Server/PrometheusMetricsWriter.h b/src/Server/PrometheusMetricsWriter.h index 4422ced625e..0c2dde1f66f 100644 --- a/src/Server/PrometheusMetricsWriter.h +++ b/src/Server/PrometheusMetricsWriter.h @@ -3,11 +3,11 @@ #include #include - #include #include + namespace DB { diff --git a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp index 70e12440678..e2f62b902b7 100644 --- a/src/Storages/System/StorageSystemAsynchronousMetrics.cpp +++ b/src/Storages/System/StorageSystemAsynchronousMetrics.cpp @@ -12,6 +12,7 @@ NamesAndTypesList StorageSystemAsynchronousMetrics::getNamesAndTypes() return { {"metric", std::make_shared()}, {"value", std::make_shared()}, + {"description", std::make_shared()}, }; } @@ -27,7 +28,8 @@ void StorageSystemAsynchronousMetrics::fillData(MutableColumns & res_columns, Co for (const auto & name_value : async_metrics_values) { res_columns[0]->insert(name_value.first); - res_columns[1]->insert(name_value.second); + res_columns[1]->insert(name_value.second.value); + res_columns[2]->insert(name_value.second.documentation); } } From 9e8fca3b4227beb1681e5cca20acddee47f07c31 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 03:59:10 +0100 Subject: [PATCH 410/526] Update test --- .../0_stateless/02117_show_create_table_system.reference | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 4ffa31a5618..f2726f17e88 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -23,7 +23,8 @@ COMMENT 'SYSTEM TABLE is built on the fly.' CREATE TABLE system.asynchronous_metrics ( `metric` String, - `value` Float64 + `value` Float64, + `description` String ) ENGINE = SystemAsynchronousMetrics COMMENT 'SYSTEM TABLE is built on the fly.' From 3096534d6a845cb06b930973768a20cffa7ef9a0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:07:29 +0100 Subject: [PATCH 411/526] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 48 ++++++++++++------------ 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 6e8d745ef0e..9f657220999 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -703,7 +703,7 @@ void AsynchronousMetrics::update(TimePoint update_time) #if !defined(OS_FREEBSD) new_values["MemoryShared"] = { data.shared, "The amount of memory used by the server process, that is also shared by another processes, in bytes." - " ClickHouse does not use shared memory, and the only case for this metric to be higher than zero is the usage of the system's C library (`libc`)." + " ClickHouse does not use shared memory, but some memory can be labeled by OS as shared for its own reasons." " This metric does not make a lot of sense to watch, and it exists only for completeness reasons."}; #endif new_values["MemoryCode"] = { data.code, @@ -870,40 +870,40 @@ void AsynchronousMetrics::update(TimePoint update_time) delta_values_all_cpus = delta_values; new_values["OSUserTime" + cpu_suffix] = { delta_values.user * multiplier, - "The ratio of time the CPU core was running userspace code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running userspace code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This includes also the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSNiceTime" + cpu_suffix] = { delta_values.nice * multiplier, - "The ratio of time the CPU core was running userspace code with higher priority. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running userspace code with higher priority. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSSystemTime" + cpu_suffix] = { delta_values.system * multiplier, - "The ratio of time the CPU core was running OS kernel (system) code. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was running OS kernel (system) code. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIdleTime" + cpu_suffix] = { delta_values.idle * multiplier, - "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was idle (not even ready to run a process waiting for IO) from the OS kernel standpoint. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This does not include the time when the CPU was under-utilized due to the reasons internal to the CPU (memory loads, pipeline stalls, branch mispredictions, running another SMT core)." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIOWaitTime" + cpu_suffix] = { delta_values.iowait * multiplier, - "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time the CPU core was not running the code but when the OS kernel did not run any other process on this CPU as the processes were waiting for IO. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSIrqTime" + cpu_suffix] = { delta_values.irq * multiplier, - "The ratio of time spent for running hardware interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent for running hardware interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " A high number of this metric may indicate hardware misconfiguration or a very high network load." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSSoftIrqTime" + cpu_suffix] = { delta_values.softirq * multiplier, - "The ratio of time spent for running software interrupt requests on the CPU. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent for running software interrupt requests on the CPU. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " A high number of this metric may indicate inefficient software running on the system." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSStealTime" + cpu_suffix] = { delta_values.steal * multiplier, - "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent in other operating systems by the CPU when running in a virtualized environment. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " Not every virtualized environments present this metric, and most of them don't." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSGuestTime" + cpu_suffix] = { delta_values.guest * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This metric is irrelevant for ClickHouse, but still exists for completeness." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; new_values["OSGuestNiceTime" + cpu_suffix] = { delta_values.guest_nice * multiplier, - "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." + "The ratio of time spent running a virtual CPU for guest operating systems under the control of the Linux kernel, when a guest was set to a higher priority (See `man procfs`). This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." " This metric is irrelevant for ClickHouse, but still exists for completeness." " The value for a single CPU core will be in the interval [0..1]. The value for all CPU cores is calculated as a sum across them [0..num cores]."}; } @@ -932,7 +932,7 @@ void AsynchronousMetrics::update(TimePoint update_time) skipToNextLineOrEOF(*proc_stat); new_values["OSProcessesRunning"] = { processes_running, "The number of runnable (running or ready to run) threads by the operating system." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "procs_blocked") { @@ -941,7 +941,7 @@ void AsynchronousMetrics::update(TimePoint update_time) skipToNextLineOrEOF(*proc_stat); new_values["OSProcessesBlocked"] = { processes_blocked, "Number of threads blocked waiting for I/O to complete (`man procfs`)." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else skipToNextLineOrEOF(*proc_stat); @@ -951,9 +951,9 @@ void AsynchronousMetrics::update(TimePoint update_time) { ProcStatValuesOther delta_values = current_other_values - proc_stat_values_other; - new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; - new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSInterrupts"] = { delta_values.interrupts, "The number of interrupts on the host machine. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSContextSwitches"] = { delta_values.context_switches, "The number of context switches that the system underwent on the host machine. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSProcessesCreated"] = { delta_values.processes_created, "The number of processes created. This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; /// Also write values normalized to 0..1 by diving to the number of CPUs. /// These values are good to be averaged across the cluster of non-uniform servers. @@ -1056,35 +1056,35 @@ void AsynchronousMetrics::update(TimePoint update_time) " See the `OSMemoryAvailable` metric instead." " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." " See also https://www.linuxatemyram.com/." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "MemAvailable:") { new_values["OSMemoryAvailable"] = { bytes, "The amount of memory available to be used by programs, in bytes. This is very similar to the `OSMemoryFreePlusCached` metric." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Buffers:") { new_values["OSMemoryBuffers"] = { bytes, "The amount of memory used by OS kernel buffers, in bytes. This should be typically small, and large values may indicate a misconfiguration of the OS." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "Cached:") { free_plus_cached_bytes += bytes; new_values["OSMemoryCached"] = { bytes, "The amount of memory used by the OS page cache, in bytes. Typically, almost all available memory is used by the OS page cache - high values of this metric are normal and expected." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } else if (name == "SwapCached:") { new_values["OSMemorySwapCached"] = { bytes, "The amount of memory in swap that was also loaded in RAM. Swap should be disabled on production systems. If the value of this metric is large, it indicates a misconfiguration." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } skipToNextLineOrEOF(*meminfo); } - new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory or OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + new_values["OSMemoryFreePlusCached"] = { free_plus_cached_bytes, "The amount of free memory plus OS page cache memory on the host system, in bytes. This memory is available to be used by programs. The value should be very similar to `OSMemoryAvailable`." + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { @@ -1153,7 +1153,7 @@ void AsynchronousMetrics::update(TimePoint update_time) uint64_t open_files = 0; readText(open_files, *file_nr); new_values["OSOpenFiles"] = { open_files, "The total number of opened files on the host machine." - " This is system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; + " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } catch (...) { From 884603cc5b7ce6bf1bcefea80fc0784a1f83ebbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:08:17 +0100 Subject: [PATCH 412/526] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 9f657220999..8e84dec0dd9 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1054,7 +1054,7 @@ void AsynchronousMetrics::update(TimePoint update_time) " This does not include the memory used by the OS page cache memory, in bytes." " The page cache memory is also available for usage by programs, so the value of this metric can be confusing." " See the `OSMemoryAvailable` metric instead." - " For convenience we also provide OSMemoryFreePlusCached, that should be somewhat similar to OSMemoryAvailable." + " For convenience we also provide the `OSMemoryFreePlusCached` metric, that should be somewhat similar to OSMemoryAvailable." " See also https://www.linuxatemyram.com/." " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." }; } From 90643aeba1b8d7b9918ec501b31de028778c6157 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:10:20 +0100 Subject: [PATCH 413/526] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 8e84dec0dd9..983eb36d042 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1669,7 +1669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values["NumberOfDatabases"] = { number_of_databases, "Total number of databases on the server." }; new_values["NumberOfTables"] = { total_number_of_tables, "Total number of tables summed across the databases on the server, excluding the databases that cannot contain MergeTree tables." - "The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; + " The excluded database engines are those who generate the set of tables on the fly, like `Lazy`, `MySQL`, `PostgreSQL`, `SQlite`."}; new_values["TotalBytesOfMergeTreeTables"] = { total_number_of_bytes, "Total amount of bytes (compressed, including data and indices) stored in all tables of MergeTree family." }; new_values["TotalRowsOfMergeTreeTables"] = { total_number_of_rows, "Total amount of rows (records) stored in all tables of MergeTree family." }; From c06bff661514e24cab73c13e96aae471f01f6f40 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:12:58 +0100 Subject: [PATCH 414/526] More praise for heroes --- src/Interpreters/AsynchronousMetrics.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 983eb36d042..6fe4a1bc92e 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -669,7 +669,7 @@ void AsynchronousMetrics::update(TimePoint update_time) // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. auto epoch = updateJemallocEpoch(); - new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of JeMalloc (a low-level memory allocator), used in all other `jemalloc` metrics." }; + new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); From 7aec54b95a6d69984e2420f7d997f5dca4b5bbd9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:19:32 +0100 Subject: [PATCH 415/526] Clarifiactions --- src/Interpreters/AsynchronousMetrics.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 6fe4a1bc92e..d4018001a49 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1210,7 +1210,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardOps_" + name] = { delta_values.discard_ops, "Number of discard operations requested from the block device. These operations are relevant for SSD." - " They are not used by ClickHouse, but can be used by other processes on the system." + " Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadMerges_" + name] = { delta_values.read_merges, @@ -1221,7 +1221,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardMerges_" + name] = { delta_values.discard_merges, "Number of discard operations requested from the block device and merged together by the OS IO scheduler." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadBytes_" + name] = { delta_values.read_sectors * sector_size, @@ -1235,7 +1235,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardBytes_" + name] = { delta_values.discard_sectors * sector_size, "Number of discarded bytes on the block device." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockReadTime_" + name] = { delta_values.read_ticks * time_multiplier, @@ -1246,7 +1246,7 @@ void AsynchronousMetrics::update(TimePoint update_time) BLOCK_DEVICE_EXPLANATION }; new_values["BlockDiscardTime_" + name] = { delta_values.discard_ticks * time_multiplier, "Time in seconds spend in discard operations requested from the block device, summed across all the operations." - " These operations are relevant for SSD. They are not used by ClickHouse, but can be used by other processes on the system." + " These operations are relevant for SSD. Discard operations are not used by ClickHouse, but can be used by other processes on the system." BLOCK_DEVICE_EXPLANATION }; new_values["BlockInFlightOps_" + name] = { delta_values.in_flight_ios, From ddc01fd05ee29a8568952f0b7b00d316730c01d0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:24:51 +0100 Subject: [PATCH 416/526] Add a test --- ...0_every_asynchronous_metric_must_have_documentation.reference | 0 .../02480_every_asynchronous_metric_must_have_documentation.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference create mode 100644 tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql diff --git a/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql new file mode 100644 index 00000000000..3f0ab58cc43 --- /dev/null +++ b/tests/queries/0_stateless/02480_every_asynchronous_metric_must_have_documentation.sql @@ -0,0 +1 @@ +SELECT metric FROM system.asynchronous_metrics WHERE length(description) < 10; From 045fd0651032cb8c7ec7bc7a99bd46f340d0f711 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:30:43 +0100 Subject: [PATCH 417/526] Clarifications --- src/Interpreters/AsynchronousMetrics.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d4018001a49..291bca4277d 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -1200,7 +1200,7 @@ void AsynchronousMetrics::update(TimePoint update_time) #define BLOCK_DEVICE_EXPLANATION \ " This is a system-wide metric, it includes all the processes on the host machine, not just clickhouse-server." \ - " See https://www.kernel.org/doc/Documentation/block/stat.txt" + " Source: `/sys/block`. See https://www.kernel.org/doc/Documentation/block/stat.txt" new_values["BlockReadOps_" + name] = { delta_values.read_ios, "Number of read operations requested from the block device." @@ -1403,7 +1403,7 @@ void AsynchronousMetrics::update(TimePoint update_time) Int64 temperature = 0; readText(temperature, in); new_values[fmt::format("Temperature{}", i)] = { temperature * 0.001, - "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value." }; + "The temperature of the corresponding device in ℃. A sensor can return an unrealistic value. Source: `/sys/class/thermal`" }; } } catch (...) @@ -1443,10 +1443,10 @@ void AsynchronousMetrics::update(TimePoint update_time) if (sensor_name.empty()) new_values[fmt::format("Temperature_{}", hwmon_name)] = { temperature * 0.001, - "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value." }; + "The temperature reported by the corresponding hardware monitor in ℃. A sensor can return an unrealistic value. Source: `/sys/class/hwmon`" }; else new_values[fmt::format("Temperature_{}_{}", hwmon_name, sensor_name)] = { temperature * 0.001, - "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value." }; + "The temperature reported by the corresponding hardware monitor and the corresponding sensor in ℃. A sensor can return an unrealistic value. Source: `/sys/class/hwmon`" }; } } } @@ -1485,7 +1485,8 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values[fmt::format("EDAC{}_Correctable", i)] = { errors, "The number of correctable ECC memory errors." " A high number of this value indicates bad RAM which has to be immediately replaced," - " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." }; + " because in presence of a high number of corrected errors, a number of silent errors may happen as well, leading to data corruption." + " Source: `/sys/devices/system/edac/mc/`" }; } if (edac[i].second) @@ -1497,7 +1498,8 @@ void AsynchronousMetrics::update(TimePoint update_time) new_values[fmt::format("EDAC{}_Uncorrectable", i)] = { errors, "The number of uncorrectable ECC memory errors." " A non-zero number of this value indicates bad RAM which has to be immediately replaced," - " because it indicates potential data corruption." }; + " because it indicates potential data corruption." + " Source: `/sys/devices/system/edac/mc/`" }; } } } From f5270a6ebeeac6221079524b4a602908e26d6f0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 04:51:46 +0100 Subject: [PATCH 418/526] A workaround for LLVM bug, https://github.com/llvm/llvm-project/issues/58633 --- cmake/sanitize.cmake | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index f0cef54b0b8..3e3bb7ec2b2 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -16,7 +16,9 @@ endmacro() if (SANITIZE) if (SANITIZE STREQUAL "address") - set (ASAN_FLAGS "-fsanitize=address -fsanitize-address-use-after-scope") + # LLVM-15 has a bug in Address Sanitizer, preventing the usage of 'sanitize-address-use-after-scope', + # see https://github.com/llvm/llvm-project/issues/58633 + set (ASAN_FLAGS "-fsanitize=address -fno-sanitize-address-use-after-scope") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} ${ASAN_FLAGS}") From dee11f11a1949217d29c3bc91a03fae8822e5151 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 05:07:05 +0100 Subject: [PATCH 419/526] Add a test --- tests/queries/0_stateless/01606_git_import.reference | 2 +- tests/queries/0_stateless/01606_git_import.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01606_git_import.reference b/tests/queries/0_stateless/01606_git_import.reference index 6b599307ba1..44ae4a3e039 100644 --- a/tests/queries/0_stateless/01606_git_import.reference +++ b/tests/queries/0_stateless/01606_git_import.reference @@ -1,4 +1,4 @@ 913 888 2931 -160553 +160553 4.6 diff --git a/tests/queries/0_stateless/01606_git_import.sh b/tests/queries/0_stateless/01606_git_import.sh index 6d425c9bceb..8a2091a99a8 100755 --- a/tests/queries/0_stateless/01606_git_import.sh +++ b/tests/queries/0_stateless/01606_git_import.sh @@ -118,7 +118,7 @@ ${CLICKHOUSE_CLIENT} --query "INSERT INTO line_changes FORMAT TSV" < line_change ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM commits" ${CLICKHOUSE_CLIENT} --query "SELECT count() FROM file_changes" -${CLICKHOUSE_CLIENT} --query "SELECT count() FROM line_changes" +${CLICKHOUSE_CLIENT} --query "SELECT count(), round(avg(indent), 1) FROM line_changes" ${CLICKHOUSE_CLIENT} --multiline --multiquery --query " DROP TABLE commits; From c2ddef897f59fe7f60bf062eebfb9e510e0553d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 07:12:25 +0300 Subject: [PATCH 420/526] Update Dockerfile.ubuntu --- docker/server/Dockerfile.ubuntu | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index c552c26f731..f50160321e1 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -88,7 +88,7 @@ RUN arch=${TARGETARCH:-amd64} \ # Some vulnerability scanners are arguing about Ubuntu, which is not relevant to ClickHouse at all. # ClickHouse does not care when you report false vulnerabilities by running some Docker scanners. -RUN apt-get remove --purge -y libksba8 && apt-get autoremove +RUN apt-get remove --purge -y libksba8 && apt-get autoremove -y # we need to allow "others" access to clickhouse folder, because docker container # can be started with arbitrary uid (openshift usecase) From f6b334fb3af9ed5a6b62f6749889a7d9e2a35e26 Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Sun, 13 Nov 2022 12:03:46 +0000 Subject: [PATCH 421/526] add columns to information scheme --- .../System/InformationSchema/columns.sql | 12 +++++----- .../01161_information_schema.reference | 22 +++++++++---------- ...information_schema_show_database.reference | 2 +- 3 files changed, 19 insertions(+), 17 deletions(-) diff --git a/src/Storages/System/InformationSchema/columns.sql b/src/Storages/System/InformationSchema/columns.sql index 80cf2f911be..cf609bab0f7 100644 --- a/src/Storages/System/InformationSchema/columns.sql +++ b/src/Storages/System/InformationSchema/columns.sql @@ -3,10 +3,12 @@ ATTACH VIEW columns `table_catalog` String, `table_schema` String, `table_name` String, + `TABLE_SCHEMA` String, + `TABLE_NAME` String, `column_name` String, `ordinal_position` UInt64, `column_default` String, - `is_nullable` UInt8, + `is_nullable` String, `data_type` String, `character_maximum_length` Nullable(UInt64), `character_octet_length` Nullable(UInt64), @@ -26,12 +28,10 @@ ATTACH VIEW columns `column_comment` String, `column_type` String, `TABLE_CATALOG` String ALIAS table_catalog, - `TABLE_SCHEMA` String ALIAS table_schema, - `TABLE_NAME` String ALIAS table_name, `COLUMN_NAME` String ALIAS column_name, `ORDINAL_POSITION` UInt64 ALIAS ordinal_position, `COLUMN_DEFAULT` String ALIAS column_default, - `IS_NULLABLE` UInt8 ALIAS is_nullable, + `IS_NULLABLE` String ALIAS is_nullable, `DATA_TYPE` String ALIAS data_type, `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length, `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length, @@ -54,7 +54,9 @@ ATTACH VIEW columns SELECT database AS table_catalog, database AS table_schema, + database AS TABLE_SCHEMA, table AS table_name, + table AS TABLE_NAME, name AS column_name, position AS ordinal_position, default_expression AS column_default, @@ -77,4 +79,4 @@ SELECT NULL AS domain_name, comment AS column_comment, type AS column_type -FROM system.columns +FROM system.columns \ No newline at end of file diff --git a/tests/queries/0_stateless/01161_information_schema.reference b/tests/queries/0_stateless/01161_information_schema.reference index 3be800888c7..5331e30b899 100644 --- a/tests/queries/0_stateless/01161_information_schema.reference +++ b/tests/queries/0_stateless/01161_information_schema.reference @@ -14,14 +14,14 @@ default default v VIEW tmp LOCAL TEMPORARY default default mv SELECT * FROM system.one NONE NO YES NO NO NO default default v SELECT n, f FROM default.t NONE NO NO NO NO NO -default default mv dummy 1 0 UInt8 \N \N 8 2 0 \N \N \N \N \N \N \N \N \N \N UInt8 -default default t n 1 0 UInt64 \N \N 64 2 0 \N \N \N \N \N \N \N \N \N \N UInt64 -default default t f 2 0 Float32 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N Float32 -default default t s 3 0 String \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N String -default default t fs 4 0 FixedString(42) 42 42 \N \N \N \N \N \N \N \N \N \N \N \N \N FixedString(42) -default default t d 5 0 Decimal(9, 6) \N \N 9 10 6 \N \N \N \N \N \N \N \N \N \N Decimal(9, 6) -default default v n 1 1 Nullable(Int32) \N \N 32 2 0 \N \N \N \N \N \N \N \N \N \N Nullable(Int32) -default default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N Float64 - tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N Date - tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N DateTime - tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N DateTime64(3) +default default mv default mv dummy 1 0 UInt8 \N \N 8 2 0 \N \N \N \N \N \N \N \N \N \N UInt8 +default default t default t n 1 0 UInt64 \N \N 64 2 0 \N \N \N \N \N \N \N \N \N \N UInt64 +default default t default t f 2 0 Float32 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N Float32 +default default t default t s 3 0 String \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N String +default default t default t fs 4 0 FixedString(42) 42 42 \N \N \N \N \N \N \N \N \N \N \N \N \N FixedString(42) +default default t default t d 5 0 Decimal(9, 6) \N \N 9 10 6 \N \N \N \N \N \N \N \N \N \N Decimal(9, 6) +default default v default v n 1 1 Nullable(Int32) \N \N 32 2 0 \N \N \N \N \N \N \N \N \N \N Nullable(Int32) +default default v default v f 2 0 Float64 \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N Float64 + tmp tmp d 1 0 Date \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N Date + tmp tmp dt 2 0 DateTime \N \N \N \N \N 0 \N \N \N \N \N \N \N \N \N DateTime + tmp tmp dtms 3 0 DateTime64(3) \N \N \N \N \N 3 \N \N \N \N \N \N \N \N \N DateTime64(3) diff --git a/tests/queries/0_stateless/02206_information_schema_show_database.reference b/tests/queries/0_stateless/02206_information_schema_show_database.reference index af437aca989..821fddbb933 100644 --- a/tests/queries/0_stateless/02206_information_schema_show_database.reference +++ b/tests/queries/0_stateless/02206_information_schema_show_database.reference @@ -1,4 +1,4 @@ CREATE DATABASE INFORMATION_SCHEMA\nENGINE = Memory -CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` UInt8,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `TABLE_SCHEMA` String ALIAS table_schema,\n `TABLE_NAME` String ALIAS table_name,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` UInt8 ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n table AS table_name,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns +CREATE VIEW INFORMATION_SCHEMA.COLUMNS\n(\n `table_catalog` String,\n `table_schema` String,\n `table_name` String,\n `TABLE_SCHEMA` String,\n `TABLE_NAME` String,\n `column_name` String,\n `ordinal_position` UInt64,\n `column_default` String,\n `is_nullable` String,\n `data_type` String,\n `character_maximum_length` Nullable(UInt64),\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64),\n `character_set_catalog` Nullable(String),\n `character_set_schema` Nullable(String),\n `character_set_name` Nullable(String),\n `collation_catalog` Nullable(String),\n `collation_schema` Nullable(String),\n `collation_name` Nullable(String),\n `domain_catalog` Nullable(String),\n `domain_schema` Nullable(String),\n `domain_name` Nullable(String),\n `column_comment` String,\n `column_type` String,\n `TABLE_CATALOG` String ALIAS table_catalog,\n `COLUMN_NAME` String ALIAS column_name,\n `ORDINAL_POSITION` UInt64 ALIAS ordinal_position,\n `COLUMN_DEFAULT` String ALIAS column_default,\n `IS_NULLABLE` String ALIAS is_nullable,\n `DATA_TYPE` String ALIAS data_type,\n `CHARACTER_MAXIMUM_LENGTH` Nullable(UInt64) ALIAS character_maximum_length,\n `CHARACTER_OCTET_LENGTH` Nullable(UInt64) ALIAS character_octet_length,\n `NUMERIC_PRECISION` Nullable(UInt64) ALIAS numeric_precision,\n `NUMERIC_PRECISION_RADIX` Nullable(UInt64) ALIAS numeric_precision_radix,\n `NUMERIC_SCALE` Nullable(UInt64) ALIAS numeric_scale,\n `DATETIME_PRECISION` Nullable(UInt64) ALIAS datetime_precision,\n `CHARACTER_SET_CATALOG` Nullable(String) ALIAS character_set_catalog,\n `CHARACTER_SET_SCHEMA` Nullable(String) ALIAS character_set_schema,\n `CHARACTER_SET_NAME` Nullable(String) ALIAS character_set_name,\n `COLLATION_CATALOG` Nullable(String) ALIAS collation_catalog,\n `COLLATION_SCHEMA` Nullable(String) ALIAS collation_schema,\n `COLLATION_NAME` Nullable(String) ALIAS collation_name,\n `DOMAIN_CATALOG` Nullable(String) ALIAS domain_catalog,\n `DOMAIN_SCHEMA` Nullable(String) ALIAS domain_schema,\n `DOMAIN_NAME` Nullable(String) ALIAS domain_name,\n `COLUMN_COMMENT` String ALIAS column_comment,\n `COLUMN_TYPE` String ALIAS column_type\n) AS\nSELECT\n database AS table_catalog,\n database AS table_schema,\n database AS TABLE_SCHEMA,\n table AS table_name,\n table AS TABLE_NAME,\n name AS column_name,\n position AS ordinal_position,\n default_expression AS column_default,\n type LIKE \'Nullable(%)\' AS is_nullable,\n type AS data_type,\n character_octet_length AS character_maximum_length,\n character_octet_length,\n numeric_precision,\n numeric_precision_radix,\n numeric_scale,\n datetime_precision,\n NULL AS character_set_catalog,\n NULL AS character_set_schema,\n NULL AS character_set_name,\n NULL AS collation_catalog,\n NULL AS collation_schema,\n NULL AS collation_name,\n NULL AS domain_catalog,\n NULL AS domain_schema,\n NULL AS domain_name,\n comment AS column_comment,\n type AS column_type\nFROM system.columns CREATE VIEW INFORMATION_SCHEMA.TABLES (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables CREATE VIEW information_schema.tables (`table_catalog` String, `table_schema` String, `table_name` String, `table_type` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5), `TABLE_CATALOG` String ALIAS table_catalog, `TABLE_SCHEMA` String ALIAS table_schema, `TABLE_NAME` String ALIAS table_name, `TABLE_TYPE` Enum8(\'BASE TABLE\' = 1, \'VIEW\' = 2, \'FOREIGN TABLE\' = 3, \'LOCAL TEMPORARY\' = 4, \'SYSTEM VIEW\' = 5) ALIAS table_type) AS SELECT database AS table_catalog, database AS table_schema, name AS table_name, multiIf(is_temporary, 4, engine LIKE \'%View\', 2, engine LIKE \'System%\', 5, has_own_data = 0, 3, 1) AS table_type FROM system.tables From 38ee1a81a3efdf939df6e2891428d517fda128af Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 13 Nov 2022 19:17:41 +0000 Subject: [PATCH 422/526] Stop config reloader before anything else --- programs/server/Server.cpp | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..573c3fbe1ba 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1486,11 +1486,6 @@ int Server::main(const std::vector & /*args*/) #endif SCOPE_EXIT({ - /// Stop reloading of the main config. This must be done before `global_context->shutdown()` because - /// otherwise the reloading may pass a changed config to some destroyed parts of ContextSharedPart. - main_config_reloader.reset(); - access_control.stopPeriodicReloading(); - async_metrics.stop(); /** Ask to cancel background jobs all table engines, @@ -1789,10 +1784,17 @@ int Server::main(const std::vector & /*args*/) SCOPE_EXIT_SAFE({ LOG_DEBUG(log, "Received termination signal."); - LOG_DEBUG(log, "Waiting for current connections to close."); + + /// Stop reloading of the main config. This must be done before everything else because it + /// can try to access/modify already deleted objects. + /// E.g. it can recreate new servers or it may pass a changed config to some destroyed parts of ContextSharedPart. + main_config_reloader.reset(); + access_control.stopPeriodicReloading(); is_cancelled = true; + LOG_DEBUG(log, "Waiting for current connections to close."); + size_t current_connections = 0; { std::lock_guard lock(servers_lock); From 22095b3bab2d5b8a06662441fe6ff8af0e91a11c Mon Sep 17 00:00:00 2001 From: FArthur-cmd <613623@mail.ru> Date: Sun, 13 Nov 2022 19:56:40 +0000 Subject: [PATCH 423/526] add test --- tests/queries/0_stateless/02479_mysql_connect_to_self.reference | 1 + tests/queries/0_stateless/02479_mysql_connect_to_self.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02479_mysql_connect_to_self.reference create mode 100644 tests/queries/0_stateless/02479_mysql_connect_to_self.sql diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.reference b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference new file mode 100644 index 00000000000..573541ac970 --- /dev/null +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.reference @@ -0,0 +1 @@ +0 diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql new file mode 100644 index 00000000000..3f8c693dd63 --- /dev/null +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -0,0 +1,2 @@ +SELECT * +FROM mysql('127.0.0.1:9004', system, one, 'default', '') From b9471e492a859bca637900107ee7034736c09059 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 22:52:21 +0100 Subject: [PATCH 424/526] Fix typo --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 7e2d5e1727b..05aa8f469b0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -307,8 +307,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) else if (!args.attach && !args.getLocalContext()->getSettingsRef().allow_deprecated_syntax_for_merge_tree) { throw Exception(ErrorCodes::BAD_ARGUMENTS, "This syntax for *MergeTree engine is deprecated. " - "Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause." - "See also allow_deprecated_syntax_for_merge_tree setting."); + "Use extended storage definition syntax with ORDER BY/PRIMARY KEY clause. " + "See also `allow_deprecated_syntax_for_merge_tree` setting."); } /// For Replicated. From 4012b46a87daa5ea375055674920205cc52bd5b4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 13 Nov 2022 23:11:00 +0100 Subject: [PATCH 425/526] Adjustments for Mac OS --- src/Interpreters/AsynchronousMetrics.h | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/AsynchronousMetrics.h b/src/Interpreters/AsynchronousMetrics.h index 3529c162944..22ed2e862ea 100644 --- a/src/Interpreters/AsynchronousMetrics.h +++ b/src/Interpreters/AsynchronousMetrics.h @@ -31,8 +31,9 @@ struct AsynchronousMetricValue double value; const char * documentation; - AsynchronousMetricValue(double value_, const char * documentation_) : value(value_), documentation(documentation_) {} - AsynchronousMetricValue(size_t value_, const char * documentation_) : value(value_), documentation(documentation_) {} + template + AsynchronousMetricValue(T value_, const char * documentation_) + : value(static_cast(value_)), documentation(documentation_) {} AsynchronousMetricValue() = default; /// For std::unordered_map::operator[]. }; From 44ed807eaa6f909921269e6c37cf2538f6dc94bb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 02:42:02 +0300 Subject: [PATCH 426/526] Update 02479_mysql_connect_to_self.sql --- tests/queries/0_stateless/02479_mysql_connect_to_self.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql index 3f8c693dd63..a7aa6a96c1d 100644 --- a/tests/queries/0_stateless/02479_mysql_connect_to_self.sql +++ b/tests/queries/0_stateless/02479_mysql_connect_to_self.sql @@ -1,2 +1,3 @@ +-- Tags: no-fasttest SELECT * FROM mysql('127.0.0.1:9004', system, one, 'default', '') From 3c603c3d8c41c7533badc150e448401b257c1a36 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 02:42:30 +0300 Subject: [PATCH 427/526] Update columns.sql --- src/Storages/System/InformationSchema/columns.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/System/InformationSchema/columns.sql b/src/Storages/System/InformationSchema/columns.sql index cf609bab0f7..b01352145ff 100644 --- a/src/Storages/System/InformationSchema/columns.sql +++ b/src/Storages/System/InformationSchema/columns.sql @@ -79,4 +79,4 @@ SELECT NULL AS domain_name, comment AS column_comment, type AS column_type -FROM system.columns \ No newline at end of file +FROM system.columns From bfea3deed98a85f2d94f82fbe2fa1f5e2f11bd58 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:09:03 +0100 Subject: [PATCH 428/526] Miscellaneous changes --- src/QueryPipeline/RemoteQueryExecutor.h | 2 +- src/Server/TCPHandler.h | 2 -- .../ParallelReplicasReadingCoordinator.h | 1 + src/Storages/MergeTree/RequestResponse.cpp | 19 ++++++++++++++----- 4 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index 78bc9f611ab..e2ebdd3cbec 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -45,7 +45,7 @@ public: /// decide whether to deny or to accept that request. struct Extension { - std::shared_ptr task_iterator{nullptr}; + std::shared_ptr task_iterator; std::shared_ptr parallel_reading_coordinator; std::optional replica_info; }; diff --git a/src/Server/TCPHandler.h b/src/Server/TCPHandler.h index 9c8d3ca60f3..0b296aaef4e 100644 --- a/src/Server/TCPHandler.h +++ b/src/Server/TCPHandler.h @@ -19,8 +19,6 @@ #include #include -#include - #include "IServer.h" #include "Server/TCPProtocolStackData.h" #include "base/types.h" diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index bd2082be6c2..4800533e919 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -3,6 +3,7 @@ #include #include + namespace DB { diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index a266540b99a..f4e09190596 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -7,21 +7,23 @@ #include + namespace DB { namespace ErrorCodes { extern const int UNKNOWN_PROTOCOL; + extern const int BAD_ARGUMENTS; } -static void readMarkRangesBinary(MarkRanges & ranges, ReadBuffer & buf, size_t MAX_RANGES_SIZE = DEFAULT_MAX_STRING_SIZE) +static void readMarkRangesBinary(MarkRanges & ranges, ReadBuffer & buf) { size_t size = 0; readVarUInt(size, buf); - if (size > MAX_RANGES_SIZE) - throw Poco::Exception("Too large ranges size."); + if (size > DEFAULT_MAX_STRING_SIZE) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Too large ranges size: {}.", size); ranges.resize(size); for (size_t i = 0; i < size; ++i) @@ -95,14 +97,21 @@ void PartitionReadRequest::deserialize(ReadBuffer & in) UInt64 PartitionReadRequest::getConsistentHash(size_t buckets) const { - auto hash = SipHash(); + SipHash hash; + + hash.update(partition_id.size()); hash.update(partition_id); + + hash.update(part_name.size()); hash.update(part_name); + + hash.update(projection_name.size()); hash.update(projection_name); hash.update(block_range.begin); hash.update(block_range.end); + hash.update(mark_ranges.size()); for (const auto & range : mark_ranges) { hash.update(range.begin); @@ -118,7 +127,7 @@ void PartitionReadResponse::serialize(WriteBuffer & out) const /// Must be the first writeVarUInt(DBMS_PARALLEL_REPLICAS_PROTOCOL_VERSION, out); - writeVarUInt(static_cast(denied), out); + writeBinary(denied, out); writeMarkRangesBinary(mark_ranges, out); } From 9a0a21db94de9626b6c4d04ebf0b3e8303c1bb2e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:22:05 +0100 Subject: [PATCH 429/526] Miscellaneous changes --- src/QueryPipeline/RemoteQueryExecutor.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/QueryPipeline/RemoteQueryExecutor.h b/src/QueryPipeline/RemoteQueryExecutor.h index e2ebdd3cbec..8b8f21a3ae4 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.h +++ b/src/QueryPipeline/RemoteQueryExecutor.h @@ -45,9 +45,9 @@ public: /// decide whether to deny or to accept that request. struct Extension { - std::shared_ptr task_iterator; - std::shared_ptr parallel_reading_coordinator; - std::optional replica_info; + std::shared_ptr task_iterator; + std::shared_ptr parallel_reading_coordinator; + std::optional replica_info; }; /// Takes already set connection. From e080161ba5dc560ab2a8b2ea52982c50192e80c2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:32:18 +0100 Subject: [PATCH 430/526] Miscellaneous changes --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 051854d8bc1..d85fb4d26a6 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -45,7 +45,7 @@ public: const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, const Names & virt_column_names_ = {}, - std::optional extension = {}); + std::optional extension_ = {}); ~MergeTreeBaseSelectProcessor() override; From 60c96072e1d07edb7aac36800a556988ca7a9fb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:37:19 +0100 Subject: [PATCH 431/526] Miscellaneous changes --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index d85fb4d26a6..99a5a2beddb 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -189,7 +189,7 @@ private: /// It won't work with reading in order or reading in reverse order, because we can possibly seek back. bool getDelayedTasks(); - /// It will form a request a request to coordinator and + /// It will form a request to coordinator and /// then reinitialize the mark ranges of this->task object Status performRequestToCoordinator(MarkRanges requested_ranges, bool delayed); From 85dc692f7ca4e9bcf33f910e762e6870aa14fff5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:37:55 +0100 Subject: [PATCH 432/526] Miscellaneous changes --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 99a5a2beddb..e385f5f4d25 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -194,7 +194,6 @@ private: Status performRequestToCoordinator(MarkRanges requested_ranges, bool delayed); void splitCurrentTaskRangesAndFillBuffer(); - }; } From 4966556f4732d37a996d8586a15778a61e92450a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:41:51 +0100 Subject: [PATCH 433/526] Miscellaneous changes --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a450a9ef3a9..22e416384aa 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -227,7 +227,7 @@ void StorageMergeTree::read( bool enable_parallel_reading = local_context->getClientInfo().collaborate_with_initiator; if (enable_parallel_reading) - LOG_TRACE(log, "Parallel reading from replicas enabled {}", enable_parallel_reading); + LOG_TRACE(log, "Parallel reading from replicas enabled: {}", enable_parallel_reading); if (auto plan = reader.read( column_names, storage_snapshot, query_info, local_context, max_block_size, num_streams, processed_stage, nullptr, enable_parallel_reading)) From cc1855ea33657ffa109056a297b698a4102b4bb8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:45:52 +0100 Subject: [PATCH 434/526] Miscellaneous changes --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index b63e08b733d..3612b7d47a1 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -692,7 +692,7 @@ MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performReques .mark_ranges = std::move(requested_ranges) }; - /// Constistent hashing won't work with reading in order, because at the end of the execution + /// Consistent hashing won't work with reading in order, because at the end of the execution /// we could possibly seek back if (!delayed && canUseConsistentHashingForParallelReading()) { From 53ee7446dc88c02835b87565d78b43062cf4c242 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:46:37 +0100 Subject: [PATCH 435/526] Miscellaneous changes --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 3612b7d47a1..38ca103f4ce 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -669,12 +669,11 @@ MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performReques if (task->data_part->isProjectionPart()) { part_name = task->data_part->getParentPart()->name; - projection_name = task->data_part->name; + projection_name = task->data_part->name; } else { part_name = task->data_part->name; - projection_name = ""; } PartBlockRange block_range From 15202833b1868fcd96e553d8ee98603cc87af3fb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:48:49 +0100 Subject: [PATCH 436/526] Miscellaneous changes --- src/Storages/MergeTree/RequestResponse.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 85c8f7181af..fba0071727d 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -14,7 +14,7 @@ namespace DB { -/// Represents a segment [left; right] +/// Represents a segment [left; right] of part's block numbers. struct PartBlockRange { Int64 begin; From a77032fafa1c87ce5fe70e3556142f928b8bb84d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:54:58 +0100 Subject: [PATCH 437/526] Miscellaneous changes --- src/Common/Stopwatch.h | 1 + .../ParallelReplicasReadingCoordinator.cpp | 24 ++++++------------- 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index cabc6d8ba1e..e17f3b18b5c 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -63,6 +63,7 @@ private: using StopwatchUniquePtr = std::unique_ptr; +/// Allows to atomically compare the elapsed time with a threshold and restart the watch if the elapsed time is not less. class AtomicStopwatch { public: diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 730f9a05814..7802b650a30 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -1,24 +1,14 @@ #include -#include -#include -#include -#include -#include #include -#include -#include -#include - #include -#include #include #include -#include "IO/WriteBufferFromString.h" -#include +#include #include + namespace DB { @@ -45,15 +35,15 @@ public: PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(PartitionReadRequest request) { - AtomicStopwatch watch; + Stopwatch watch; + SCOPE_EXIT({ + LOG_TRACE(&Poco::Logger::get("ParallelReplicasReadingCoordinator"), "Time for handling request: {} ns", watch.elapsed()); + }); + std::lock_guard lock(mutex); auto partition_it = partitions.find(request.partition_id); - SCOPE_EXIT({ - LOG_TRACE(&Poco::Logger::get("ParallelReplicasReadingCoordinator"), "Time for handling request: {}ns", watch.elapsed()); - }); - PartToRead::PartAndProjectionNames part_and_projection { .part = request.part_name, From 8e87e141a4158e401c232a127292ea4ce12f88d5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 06:57:53 +0100 Subject: [PATCH 438/526] Miscellaneous changes --- src/Common/Stopwatch.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Common/Stopwatch.h b/src/Common/Stopwatch.h index e17f3b18b5c..32d1fca337d 100644 --- a/src/Common/Stopwatch.h +++ b/src/Common/Stopwatch.h @@ -63,6 +63,7 @@ private: using StopwatchUniquePtr = std::unique_ptr; +/// Allows to obtain the elapsed time concurrently with restarting the stopwatch. /// Allows to atomically compare the elapsed time with a threshold and restart the watch if the elapsed time is not less. class AtomicStopwatch { From df01689ccc2770fd34c6c76240605ca92b254888 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 07:13:42 +0100 Subject: [PATCH 439/526] Better logs --- .../ParallelReplicasReadingCoordinator.cpp | 13 ++++++++--- src/Storages/MergeTree/RequestResponse.cpp | 23 +++++++++++++++++++ src/Storages/MergeTree/RequestResponse.h | 3 +++ 3 files changed, 36 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 7802b650a30..0f772a0fb94 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -35,10 +35,8 @@ public: PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(PartitionReadRequest request) { + auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator"); Stopwatch watch; - SCOPE_EXIT({ - LOG_TRACE(&Poco::Logger::get("ParallelReplicasReadingCoordinator"), "Time for handling request: {} ns", watch.elapsed()); - }); std::lock_guard lock(mutex); @@ -70,6 +68,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); partitions.insert({request.partition_id, std::move(partition_reading)}); + LOG_TRACE(log, "Request is first in partition, accepted in {} ns: {}", watch.elapsed(), request.toString()); return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; } @@ -85,6 +84,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa { case PartSegments::IntersectionResult::REJECT: { + LOG_TRACE(log, "Request rejected in {} ns: {}", watch.elapsed(), request.toString()); return {.denied = true, .mark_ranges = {}}; } case PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION: @@ -100,6 +100,12 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa auto result_ranges = result.convertToMarkRangesFinal(); const bool denied = result_ranges.empty(); + + if (denied) + LOG_TRACE(log, "Request rejected due to intersection in {} ns: {}", watch.elapsed(), request.toString()); + else + LOG_TRACE(log, "Request accepted partially in {} ns: {}", watch.elapsed(), request.toString()); + return {.denied = denied, .mark_ranges = std::move(result_ranges)}; } case PartSegments::IntersectionResult::NO_INTERSECTION: @@ -111,6 +117,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa ); partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); + LOG_TRACE(log, "Request accepted in {} ns: {}", watch.elapsed(), request.toString()); return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; } } diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index f4e09190596..c7fbaf1e3c2 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -62,6 +63,28 @@ void PartitionReadRequest::serialize(WriteBuffer & out) const } +String PartitionReadRequest::toString() const +{ + WriteBufferFromOwnString out; + out << "partition: " << partition_id << ", part: " << part_name; + if (!projection_name.empty()) + out << ", projection: " << projection_name; + out << ", block range: [" << block_range.begin << ", " << block_range.end << "]"; + out << ", mark ranges: "; + + bool is_first = true; + for (const auto & [begin, end] : mark_ranges) + { + if (!is_first) + out << ", "; + out << "[" << begin << ", " << end << ")"; + is_first = false; + } + + return out.str(); +} + + void PartitionReadRequest::describe(WriteBuffer & out) const { String result; diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index fba0071727d..c8fe81cd2cd 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -39,6 +39,9 @@ struct PartitionReadRequest void deserialize(ReadBuffer & in); UInt64 getConsistentHash(size_t buckets) const; + + /// Describe it for debugging purposes. + String toString() const; }; struct PartitionReadResponse From 99f84d8095e226ac7571333dcc64523085321c5d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 07:22:02 +0100 Subject: [PATCH 440/526] Better logs --- .../ParallelReplicasReadingCoordinator.cpp | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 0f772a0fb94..009127046a2 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -37,7 +37,8 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa { auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator"); Stopwatch watch; - + + String request_description = request_description; std::lock_guard lock(mutex); auto partition_it = partitions.find(request.partition_id); @@ -68,7 +69,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); partitions.insert({request.partition_id, std::move(partition_reading)}); - LOG_TRACE(log, "Request is first in partition, accepted in {} ns: {}", watch.elapsed(), request.toString()); + LOG_TRACE(log, "Request is first in partition, accepted in {} ns: {}", watch.elapsed(), request_description); return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; } @@ -84,7 +85,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa { case PartSegments::IntersectionResult::REJECT: { - LOG_TRACE(log, "Request rejected in {} ns: {}", watch.elapsed(), request.toString()); + LOG_TRACE(log, "Request rejected in {} ns: {}", watch.elapsed(), request_description); return {.denied = true, .mark_ranges = {}}; } case PartSegments::IntersectionResult::EXACTLY_ONE_INTERSECTION: @@ -102,9 +103,9 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa const bool denied = result_ranges.empty(); if (denied) - LOG_TRACE(log, "Request rejected due to intersection in {} ns: {}", watch.elapsed(), request.toString()); + LOG_TRACE(log, "Request rejected due to intersection in {} ns: {}", watch.elapsed(), request_description); else - LOG_TRACE(log, "Request accepted partially in {} ns: {}", watch.elapsed(), request.toString()); + LOG_TRACE(log, "Request accepted partially in {} ns: {}", watch.elapsed(), request_description); return {.denied = denied, .mark_ranges = std::move(result_ranges)}; } @@ -117,7 +118,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa ); partition_reading.mark_ranges_in_part.insert({part_and_projection, std::move(mark_ranges_index)}); - LOG_TRACE(log, "Request accepted in {} ns: {}", watch.elapsed(), request.toString()); + LOG_TRACE(log, "Request accepted in {} ns: {}", watch.elapsed(), request_description); return {.denied = false, .mark_ranges = std::move(request.mark_ranges)}; } } From 582db233c46357a0c172a741936edd53d7e4d659 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 07:22:17 +0100 Subject: [PATCH 441/526] Better logs --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 009127046a2..54b96ee2015 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -38,7 +38,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator"); Stopwatch watch; - String request_description = request_description; + String request_description = request.toString(); std::lock_guard lock(mutex); auto partition_it = partitions.find(request.partition_id); From 9c48664f11fb69023df3b8429bb10575111819c4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 07:48:50 +0100 Subject: [PATCH 442/526] Better logs --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 38ca103f4ce..227a5c2a0ca 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -690,6 +690,7 @@ MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performReques .block_range = std::move(block_range), .mark_ranges = std::move(requested_ranges) }; + String request_description = request.toString(); /// Consistent hashing won't work with reading in order, because at the end of the execution /// we could possibly seek back @@ -701,6 +702,7 @@ MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performReques auto delayed_task = std::make_unique(*task); // Create a copy delayed_task->mark_ranges = std::move(request.mark_ranges); delayed_tasks.emplace_back(std::move(delayed_task)); + LOG_TRACE(log, "Request delayed by hash: {}", request_description); return Status::Denied; } } @@ -708,17 +710,24 @@ MergeTreeBaseSelectProcessor::Status MergeTreeBaseSelectProcessor::performReques auto optional_response = extension.value().callback(std::move(request)); if (!optional_response.has_value()) + { + LOG_TRACE(log, "Request cancelled: {}", request_description); return Status::Cancelled; + } auto response = optional_response.value(); task->mark_ranges = std::move(response.mark_ranges); if (response.denied || task->mark_ranges.empty()) + { + LOG_TRACE(log, "Request rejected: {}", request_description); return Status::Denied; + } finalizeNewTask(); + LOG_TRACE(log, "Request accepted: {}", request_description); return Status::Accepted; } From 17693de05391ea86f57f3d9514d77cf9b974cca0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 08:01:48 +0100 Subject: [PATCH 443/526] Better logs --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 54b96ee2015..e07f19fb64c 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -37,7 +37,7 @@ PartitionReadResponse ParallelReplicasReadingCoordinator::Impl::handleRequest(Pa { auto * log = &Poco::Logger::get("ParallelReplicasReadingCoordinator"); Stopwatch watch; - + String request_description = request.toString(); std::lock_guard lock(mutex); From 4edd8b013593ddfa559608a40533eab839f4a235 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 14 Nov 2022 08:07:27 +0100 Subject: [PATCH 444/526] Remove unused method --- src/Storages/MergeTree/RequestResponse.cpp | 14 -------------- src/Storages/MergeTree/RequestResponse.h | 1 - 2 files changed, 15 deletions(-) diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index c7fbaf1e3c2..2ea6b0c9f9f 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -85,20 +85,6 @@ String PartitionReadRequest::toString() const } -void PartitionReadRequest::describe(WriteBuffer & out) const -{ - String result; - result += fmt::format("partition_id: {} \n", partition_id); - result += fmt::format("part_name: {} \n", part_name); - result += fmt::format("projection_name: {} \n", projection_name); - result += fmt::format("block_range: ({}, {}) \n", block_range.begin, block_range.end); - result += "mark_ranges: "; - for (const auto & range : mark_ranges) - result += fmt::format("({}, {}) ", range.begin, range.end); - result += '\n'; - out.write(result.c_str(), result.size()); -} - void PartitionReadRequest::deserialize(ReadBuffer & in) { UInt64 version; diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index c8fe81cd2cd..ce9dc55f479 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -35,7 +35,6 @@ struct PartitionReadRequest MarkRanges mark_ranges; void serialize(WriteBuffer & out) const; - void describe(WriteBuffer & out) const; void deserialize(ReadBuffer & in); UInt64 getConsistentHash(size_t buckets) const; From a0fadf8ba7400dd4cb5d9022811f390755691d0f Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 14 Nov 2022 09:59:47 +0000 Subject: [PATCH 445/526] more improvements + flipped sankey --- .../example-datasets/github.md | 195 +++++++++++++----- .../images/superset-authors-matrix.png | Bin 323772 -> 283789 bytes .../images/superset-authors-matrix_v2.png | Bin 0 -> 322584 bytes .../images/superset-authors-matrix_v3.png | Bin 324610 -> 0 bytes 4 files changed, 149 insertions(+), 46 deletions(-) create mode 100644 docs/en/getting-started/example-datasets/images/superset-authors-matrix_v2.png delete mode 100644 docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 1faa63884e4..a571ad82458 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -745,7 +745,7 @@ Our core data structure, the Merge Tree, is obviously under constant evolution w ## Distribution of contributors with respect to docs and code over the month -**During data capture the changes on the `docs/` folder have been filtered out so this is an estimate only** +**During data capture the changes on the `docs/` folder have been filtered out due to a very commit dirty history. The results of this query are therefore not accurate.** Do we write more docs at certain times of the month e.g., around release dates? We can use the `countIf` function to compute a simple ratio, visualizing the result using the `bar` function. @@ -804,7 +804,7 @@ FROM 31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.71 million rows/s., 950.40 MB/s.) ``` -Maybe a little more near the end of the month, but overall we keep a good even distribution. +Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unrealiable due to the filtering of the docs filter during data insertion. ## Authors with the most diverse impact @@ -1318,7 +1318,7 @@ A Sankey chart (SuperSet) allows this to be visualized nicely. Note we increase Alexey clearly likes removing other peoples code. Lets exclude him for a more balanced view of code removal. -![](./images/superset-authors-matrix_v3.png) +![](./images/superset-authors-matrix_v2.png) ## Who is the highest percentage contributor per day of week? @@ -1570,7 +1570,59 @@ LIMIT 10 ## List files that were rewritten most number of times? -We consider a rewrite to be when over 50% of the file are deleted, and 50% added. Adjust the query to your own interpretation of what constitutes this. + +The simplest approach to this question might be to simply count the most number of line modifications per path (restricted to current files) e.g.: + +```sql +WITH current_files AS + ( + SELECT path + FROM + ( + SELECT + old_path AS path, + max(time) AS last_time, + 2 AS change_type + FROM git.file_changes + GROUP BY old_path + UNION ALL + SELECT + path, + max(time) AS last_time, + argMax(change_type, time) AS change_type + FROM git.file_changes + GROUP BY path + ) + GROUP BY path + HAVING (argMax(change_type, last_time) != 2) AND (NOT match(path, '(^dbms/)|(^libs/)|(^tests/testflows/)|(^programs/server/store/)')) + ORDER BY path ASC + ) +SELECT + path, + count() AS c +FROM git.line_changes +WHERE (file_extension IN ('h', 'cpp', 'sql')) AND (path IN (current_files)) +GROUP BY path +ORDER BY c DESC +LIMIT 10 + +┌─path───────────────────────────────────────────────────┬─────c─┐ +│ src/Storages/StorageReplicatedMergeTree.cpp │ 21871 │ +│ src/Storages/MergeTree/MergeTreeData.cpp │ 17709 │ +│ programs/client/Client.cpp │ 15882 │ +│ src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp │ 14249 │ +│ src/Interpreters/InterpreterSelectQuery.cpp │ 12636 │ +│ src/Parsers/ExpressionListParsers.cpp │ 11794 │ +│ src/Analyzer/QueryAnalysisPass.cpp │ 11760 │ +│ src/Coordination/KeeperStorage.cpp │ 10225 │ +│ src/Functions/FunctionsConversion.h │ 9247 │ +│ src/Parsers/ExpressionElementParsers.cpp │ 8197 │ +└────────────────────────────────────────────────────────┴───────┘ + +10 rows in set. Elapsed: 0.160 sec. Processed 8.07 million rows, 98.99 MB (50.49 million rows/s., 619.49 MB/s.) +``` + +This doesn't capture the notion of a "re-write" however, where a large portion of the file changes in any commit. This requires a more complex query. If we consider a rewrite to be when over 50% of the file are deleted, and 50% added. You can adjust the query to your own interpretation of what constitutes this. The query is limited to the current files only. We list all file changes by grouping by `path` and `commit_hash`, returning the number of lines added and removed. Using a window function, we estimate the file's total size at any moment in time by performing a cumulative sum and estimating the impact of any change on file size as `lines added - lines removed`. Using this statistic, we can calculate the percentage of the file that has been added or removed for each change. Finally, we count the number of file changes that constitute a rewrite per file i.e. `(percent_add >= 0.5) AND (percent_delete >= 0.5) AND current_size > 50`. Note we require files to be more than 50 lines to avoid early contributions to a file being counted as a rewrite. This also avoids a bias to very small files, which may be more likely to be rewritten. @@ -1858,7 +1910,7 @@ LIMIT 20 We can plot this distribution as a histogram. -[play](https://play.clickhouse.com/play?user=play#V0lUSCAoCiAgICAgICAgU0VMRUNUIGhpc3RvZ3JhbSgxMCkocmF0aW9fY29kZSkgQVMgaGlzdAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gTk9UIElOICgnaCcsICdjcHAnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgICAgICAgICAgICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICAgICAgICAgICAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIGF1dGhvcgogICAgICAgICAgICBIQVZJTkcgY29kZSA+IDIwCiAgICAgICAgKQogICAgKSBBUyBoaXN0ClNFTEVDVAogICAgYXJyYXlKb2luKGhpc3QpLjEgQVMgbG93ZXIsCiAgICBhcnJheUpvaW4oaGlzdCkuMiBBUyB1cHBlciwKICAgIGJhcihhcnJheUpvaW4oaGlzdCkuMywgMCwgMTAwLCAyMCkgQVMgYmFy) +[play](https://play.clickhouse.com/play?user=play#V0lUSCAoCiAgICAgICAgU0VMRUNUIGhpc3RvZ3JhbSgxMCkocmF0aW9fY29kZSkgQVMgaGlzdAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzaCcsICdweScsICdleHBlY3QnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgICAgICAgICAgICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICAgICAgICAgICAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIGF1dGhvcgogICAgICAgICAgICBIQVZJTkcgY29kZSA+IDIwCiAgICAgICAgICAgIE9SREVSIEJZIGNvZGUgREVTQwogICAgICAgICAgICBMSU1JVCAyMAogICAgICAgICkKICAgICkgQVMgaGlzdApTRUxFQ1QKICAgIGFycmF5Sm9pbihoaXN0KS4xIEFTIGxvd2VyLAogICAgYXJyYXlKb2luKGhpc3QpLjIgQVMgdXBwZXIsCiAgICBiYXIoYXJyYXlKb2luKGhpc3QpLjMsIDAsIDEwMCwgNTAwKSBBUyBiYXI=) ```sql WITH ( @@ -1867,70 +1919,79 @@ WITH ( ( SELECT author, - countIf((file_extension NOT IN ('h', 'cpp')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, code / (code + test) AS ratio_code FROM git.file_changes GROUP BY author HAVING code > 20 + ORDER BY code DESC + LIMIT 20 ) ) AS hist SELECT arrayJoin(hist).1 AS lower, arrayJoin(hist).2 AS upper, - bar(arrayJoin(hist).3, 0, 100, 20) AS bar + bar(arrayJoin(hist).3, 0, 100, 500) AS bar -┌────────────────lower─┬───────────────upper─┬─bar───────────┐ -│ 0.033562166285278416 │ 0.08337307389808846 │ ▏ │ -│ 0.08337307389808846 │ 0.17470067710547066 │ ▍ │ -│ 0.17470067710547066 │ 0.25909878535992237 │ ▍ │ -│ 0.25909878535992237 │ 0.3775444108257119 │ ▋ │ -│ 0.3775444108257119 │ 0.5108436376911997 │ ███▏ │ -│ 0.5108436376911997 │ 0.627700343453621 │ █████▋ │ -│ 0.627700343453621 │ 0.7417374581723406 │ ███████████▊ │ -│ 0.7417374581723406 │ 0.8467725898688147 │ ████████████▏ │ -│ 0.8467725898688147 │ 0.9427852671078976 │ ██████████▌ │ -│ 0.9427852671078976 │ 1 │ █████████▊ │ -└──────────────────────┴─────────────────────┴───────────────┘ - -10 rows in set. Elapsed: 0.053 sec. Processed 266.05 thousand rows, 4.65 MB (5.01 million rows/s., 87.61 MB/s.) +┌──────────────lower─┬──────────────upper─┬─bar─────────────────────┐ +│ 0.7943704340352385 │ 0.8033711363614333 │ █████ │ +│ 0.8033711363614333 │ 0.8199118213401927 │ █████▋ │ +│ 0.8199118213401927 │ 0.8378309872629326 │ ████████▋ │ +│ 0.8378309872629326 │ 0.8530295909585852 │ █████▋ │ +│ 0.8530295909585852 │ 0.8828273906355679 │ ██████▎ │ +│ 0.8828273906355679 │ 0.9237540060993992 │ ███████████████ │ +│ 0.9237540060993992 │ 0.9477682629346298 │ ██████████████████████▌ │ +│ 0.9477682629346298 │ 0.9627948304418104 │ ███████████████▋ │ +│ 0.9627948304418104 │ 0.981300247585602 │ ██████████ │ +│ 0.981300247585602 │ 0.9928451178451179 │ █████▋ │ +└────────────────────┴────────────────────┴─────────────────────────┘ +10 rows in set. Elapsed: 0.051 sec. Processed 266.05 thousand rows, 4.65 MB (5.24 million rows/s., 91.64 MB/s.) ``` Most contributors write more code than tests, as you'd expect. -What about who adds the most comments when contributing code? +What about who adds the most comments when contributing code? -[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb21tZW50JykgQVMgY29tbWVudHMsCiAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZSwKICAgIGNvbW1lbnRzIC8gKGNvbW1lbnRzICsgY29kZSkgQVMgcmF0aW9fY29tbWVudHMKRlJPTSBnaXRfY2xpY2tob3VzZS5saW5lX2NoYW5nZXMKV0hFUkUgKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoc2lnbiA9IDEpCkdST1VQIEJZIGF1dGhvcgpIQVZJTkcgY29kZSA+IDIwCk9SREVSIEJZIGNvZGUgREVTQwpMSU1JVCAxMA==) +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBhdmcocmF0aW9fY29tbWVudHMpIEFTIGF2Z19yYXRpb19jb21tZW50cywKICAgIHN1bShjb2RlKSBBUyBjb2RlCkZST00KKAogICAgU0VMRUNUCiAgICAgICAgYXV0aG9yLAogICAgICAgIGNvbW1pdF9oYXNoLAogICAgICAgIGNvdW50SWYobGluZV90eXBlID0gJ0NvbW1lbnQnKSBBUyBjb21tZW50cywKICAgICAgICBjb3VudElmKGxpbmVfdHlwZSA9ICdDb2RlJykgQVMgY29kZSwKICAgICAgICBpZihjb21tZW50cyA+IDAsIGNvbW1lbnRzIC8gKGNvbW1lbnRzICsgY29kZSksIDApIEFTIHJhdGlvX2NvbW1lbnRzCiAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwogICAgR1JPVVAgQlkKICAgICAgICBhdXRob3IsCiAgICAgICAgY29tbWl0X2hhc2gKKQpHUk9VUCBCWSBhdXRob3IKT1JERVIgQlkgY29kZSBERVNDCkxJTUlUIDEwCg==) ```sql SELECT author, - countIf(line_type = 'Comment') AS comments, - countIf(line_type = 'Code') AS code, - comments / (comments + code) AS ratio_comments -FROM git.line_changes -WHERE (file_extension IN ('h', 'cpp', 'sql')) AND (sign = 1) + avg(ratio_comments) AS avg_ratio_comments, + sum(code) AS code +FROM +( + SELECT + author, + commit_hash, + countIf(line_type = 'Comment') AS comments, + countIf(line_type = 'Code') AS code, + if(comments > 0, comments / (comments + code), 0) AS ratio_comments + FROM git.line_changes + GROUP BY + author, + commit_hash +) GROUP BY author -HAVING code > 20 ORDER BY code DESC LIMIT 10 - -┌─author─────────────┬─comments─┬───code─┬───────ratio_comments─┐ -│ Alexey Milovidov │ 30867 │ 356978 │ 0.07958591705449342 │ -│ Nikolai Kochetov │ 11128 │ 113261 │ 0.08946128676973045 │ -│ Vitaly Baranov │ 5120 │ 84504 │ 0.05712755511916451 │ -│ Maksim Kita │ 6184 │ 78778 │ 0.07278548056778325 │ -│ alesapin │ 7456 │ 72279 │ 0.09350975105035429 │ -│ kssenii │ 5804 │ 61852 │ 0.08578692207638643 │ -│ Alexey Arno │ 4430 │ 61674 │ 0.0670156117632821 │ -│ Alexander Tokmakov │ 4022 │ 41964 │ 0.08746140129604663 │ -│ Anton Popov │ 2067 │ 38448 │ 0.051018141429100335 │ -│ Ivan │ 947 │ 33711 │ 0.02732413872698944 │ -└────────────────────┴──────────┴────────┴──────────────────────┘ - -10 rows in set. Elapsed: 0.047 sec. Processed 7.54 million rows, 31.57 MB (161.75 million rows/s., 677.77 MB/s.) +┌─author─────────────┬──avg_ratio_comments─┬────code─┐ +│ Alexey Milovidov │ 0.1034915408309902 │ 1147196 │ +│ s-kat │ 0.1361718900215362 │ 614224 │ +│ Nikolai Kochetov │ 0.08722993407690126 │ 218328 │ +│ alesapin │ 0.1040477684726504 │ 198082 │ +│ Vitaly Baranov │ 0.06446875712939285 │ 161801 │ +│ Maksim Kita │ 0.06863376297549255 │ 156381 │ +│ Alexey Arno │ 0.11252677608033655 │ 146642 │ +│ Vitaliy Zakaznikov │ 0.06199215397180561 │ 138530 │ +│ kssenii │ 0.07455322590796751 │ 131143 │ +│ Artur │ 0.12383737231074826 │ 121484 │ +└────────────────────┴─────────────────────┴─────────┘ +10 rows in set. Elapsed: 0.290 sec. Processed 7.54 million rows, 394.57 MB (26.00 million rows/s., 1.36 GB/s.) ``` -Surprisingly high % for all our contributors and part of what makes our code so readable. + +Note we sort by code contributions. Surprisingly high % for all our largest contributors and part of what makes our code so readable. ## How does an authors commits change over time with respect to code/comments percentage? @@ -2393,4 +2454,46 @@ FORMAT PrettyCompactMonoBlock This is particularly difficult to get an exact result due to the inability to currently keep state in array functions. This will be possible with an `arrayFold` or `arrayReduce`, which allows state to be held on each iteration. -We welcome solutions here. +An approximate solution, sufficient for a high-level analysis, may look something like this: + +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBsaW5lX251bWJlcl9uZXcsCiAgICBhcmdNYXgoYXV0aG9yLCB0aW1lKSwKICAgIGFyZ01heChsaW5lLCB0aW1lKQpGUk9NIGdpdF9jbGlja2hvdXNlLmxpbmVfY2hhbmdlcwpXSEVSRSBwYXRoIElOIGZpbGVfcGF0aF9oaXN0b3J5KCdzcmMvU3RvcmFnZXMvU3RvcmFnZVJlcGxpY2F0ZWRNZXJnZVRyZWUuY3BwJykKR1JPVVAgQlkgbGluZV9udW1iZXJfbmV3Ck9SREVSIEJZIGxpbmVfbnVtYmVyX25ldyBBU0MKTElNSVQgMjA=) + +```sql +SELECT + line_number_new, + argMax(author, time), + argMax(line, time) +FROM git.line_changes +WHERE path IN file_path_history('src/Storages/StorageReplicatedMergeTree.cpp') +GROUP BY line_number_new +ORDER BY line_number_new ASC +LIMIT 20 + +┌─line_number_new─┬─argMax(author, time)─┬─argMax(line, time)────────────────────────────────────────────┐ +│ 1 │ Alexey Milovidov │ #include │ +│ 2 │ s-kat │ #include │ +│ 3 │ Anton Popov │ #include │ +│ 4 │ Alexander Burmak │ #include │ +│ 5 │ avogar │ #include │ +│ 6 │ Alexander Burmak │ #include │ +│ 7 │ Alexander Burmak │ #include │ +│ 8 │ Alexander Burmak │ #include │ +│ 9 │ Alexander Burmak │ #include │ +│ 10 │ Alexander Burmak │ #include │ +│ 11 │ Alexander Burmak │ #include │ +│ 12 │ Nikolai Kochetov │ #include │ +│ 13 │ alesapin │ #include │ +│ 14 │ alesapin │ │ +│ 15 │ Alexey Milovidov │ #include │ +│ 16 │ Alexey Zatelepin │ #include │ +│ 17 │ CurtizJ │ #include │ +│ 18 │ Kirill Shvakov │ #include │ +│ 19 │ s-kat │ #include │ +│ 20 │ Nikita Mikhaylov │ #include │ +└─────────────────┴──────────────────────┴───────────────────────────────────────────────────────────────┘ +20 rows in set. Elapsed: 0.547 sec. Processed 7.88 million rows, 679.20 MB (14.42 million rows/s., 1.24 GB/s.) +``` + +We welcome exact and improved solutions here. + + diff --git a/docs/en/getting-started/example-datasets/images/superset-authors-matrix.png b/docs/en/getting-started/example-datasets/images/superset-authors-matrix.png index 8195749099bfbfe82fb40539c1f23912010c96fb..bdfc6b6f304345189cdc35ba0d9367fe4aa4fe0d 100644 GIT binary patch literal 283789 zcmbSzc_7sJ|35YzC~cubsI{dq$yqrD?Z!%u+zdu(C=6QnFwROBDk0}6xsBV5oRu== z8uu8(NOI1|F=jC1_`Rmx_V|3S-QOR0n|Z%y-mlm5emtJf_gzzC{ha~_1$cOPb{hQh zvl$N$_i#(=_O0NL)z4oR!5>~9GyQWs#T|#H!9PDEYz$nCjCf?h@pc~G=<7UNxTk=h zgW!jU=etZ^p6|e4UhZp|eBa&`;K}^%+wqoG?uizvM;v*0ba)JY*0l)ar4R2|b9Elx zTxe}0s*#)9Z1z6hoo$LUC3}{Oyv#oLr5bYd$4!A-V!@TR=k6T0EgZ$U#(CWOQgrls z^PN*dO&*&;@Gry?V%p1&)r`62;06v2r8yB$?2}Hb>F~>#3|S`HyJj3vc4<4Pc;)BA zN5Xmdwjch-zjT86GlC|PPob8koNk0HQ-i&psv+%u-OcFG@UjV>ts29T9D|LJ+cyj^ zT)Ok`J~@?tF>+Hys1XUiV289zM;duzv`k~mZXF)pfBx&5o)c?0tZDms-pBv$LY>Tj zEqbws40xR8(oX8#`&YOADwVIorCmUJDf;HWx{imh{^%=6crYzQ(e9rwk-Hyn+LD!< zwjcVpOT=%PNOqOv;|Zoc`M2MA?l8~8!ha(P9i9kDk*A8zdixdK`->zGssE(+uP#4) zGx*EP+>^JH;Gt(V7vD?!RZTayNb5b=F0Hcj@Q#0>65vK0_nOX+KsofivEY-;ezKk) zp$5-M?1HX*G-FmI6xXlfyy#)7YE^Z%L{}MnOwk!%8iQ3d4ls_|;_2}({gsZNP+PbETo~)aH z^Vzhx`qRDY^#;xtLr0& ziT@K1f!y0k4-?x`J&U8_Y4kcwdt)K4Nw%=4yK|)^qX5h_`nhjxVV^VjqtLO6x=R#*YJiB~)l!=;q;UOxIT_gjr(8u)EFH1~o zT3|4;t}2$`<+W5c%K2qnp_X1=+GhphuTNZ`kJi3u`Ap~^sBJ{-_Nl}MK}e-(;CO=d zo2)o!%AqLS_2yT4QJcNfVyPjgLYRV^5=kG@E6Yx8zuEN6a{;1`6hFZJBetNsu#Ls%2{s{3BVbOqnI4%vF>`20uoTrfOv z;oZJ57Fa8Um6a)5WjS2;q9#s@HE>9|gpZ z{l$8I*a7xOL#(??^Yxyb)niSoF3rwp_mMHi#4TLK2D@3&gMu z<)Ki{tS%{mujKQ`cuAQ!?Tr=t@q&0fglSU=bx#(H(dYE+#m4r?NMIzbFQA58@rCxJ z>S8~Q5m|--g=w=qIpS5qw0WX6mQndHx=yDraE|)yqhHtaWUy=$3k}*xS5p zdZkqA!crZhLr^Bo`&=Hgy60?{Pr{&BcCWJ?bM~xJQ1Hd$7^4$e&ly-&5a}s1-Ov1` zR9vhaC$JQkPq?pbVr4R+zVAZ&gVS?1_nKj}g)7b$BQ>F_c3a9 zg=P>LLRs%#i`FxQfp2^W?jw1 zkO}3{%&1-WbiWG*-o@MW_4`NCP1X&I38hWDG~V868I5&uJ^AwU<9Y2(hW4n8_Nu|x z)(Xn27Wl?)229@)VPK)eaY6Kmo|k2fG>Rb z)21{~-uvo8S#OVuJdNKFRbsA_6 zbN2|jhpR_6Soq72&Wqjk6F&1Xe16&_OuggBsVHebA?=md!k-i&RlR1HH&!R5cFp%J z#iv{`Gtm73!x@^qvOB52N}GS92v0@nDl53r-HXcfOif3DN80zLdMws^Wi)!*aAJSA zzBaon)0r_HJi?(K!n@0BWGHu>k(MlBR&`&Hd&kHeqk1*27oCY64`}2J&(|n%-AL2E zhWT5g;T7-p!8JoRIU~YUV`YQs5IIQW+M;d+ZThL`miPeb#>gfoZ1lFc=N-~}?x&Yd zW(42n%r$Ci+RQX#U*pPo zq0lz*+Q_RjFR9k{H{Oljm{K=aDeovenr5pArXaesT9o6iMO!MtFxP0x}@kCdt`zU&X;&9Mp z`s6_(knx9?tmRSRurtt5=0NX)t@k6o0}1|v*KC!+2&MrqXx`w6dM~~Phc_yTuhElU` zfAW$&Q@G6`NnnNog#O8=%OPM`^t*$8=`#z@!ml2X2O75!WXzj^GG2Qt1O1-JIo+@j z^Lgx~s=t^y0}W!V@kt7dz%Cy1xOZc5+6_7N zgZ(HEk+X&M*>NGCr4`@X6U**Ha5h4NB=;1@DCWVONp~``ZlQUUNrJ~E_9q_wCqXsp zu=i{`{uW1aZ6p)M=0WH+h{II*~5!fX&`Q*pPtleiq@dDMCYCheQ`?ScW zmBjJtSxE}IZmxF4g^EONifnnNqm>N;iF(J)Y@TC=+-$`8<9)20lfi3mfzSnm2txt+ zF32uFjkb6b=-I9QCk9TX_D&jAfJ=F@GiCS_(n8tOik*78I|V8)bwfq=PsF=CG!uu2 z`d)qQ;V;zSskXTg7pD2eaGU}Z{*bPQ`em-Po46TQRCsfol+*^B)YyDdr1beKnV4C3 zILA(?)YK^z^T^6wlG zz8XIMS+UlXb>k0`JmSd8OLg@X%|-5d(7`yqSLGFBeLb#2AL!aa&R$C>>&6-}ku@3j zhLqKvlQDey2R;g9TREG{-|x3 zvVIXQd&(3q)5nP5t1GW~JfS7lqRH}r+p!7H_~w_^a;_3ZYcfRBfZuLy{s`sYI59KG zhhWw{U!XYmVH70h1muc#ONk2zRX$m+t|*&p+}yaaIT$CbSIIgbcfBwx+r`x)cowz_ z{NyoD-1>_QjM*fG-B!DCZ>R{HSl(NPfWvB-3!exu*20yzo%K)UoG$6cq2y;)t@7VD z^gCx2Rj{(0ERXWzY?^p5?Ldht{qpTLHBIYLH4L__s?#15Wl!kX0P zf|TNbUIp$1`9Xl#)vKba(xjNv)cvkm-AD~pPWofd*ViJfQy_*x?HeuzyizF$W96NQ z_;&AH*s`10@%FVziXEE&sO+6zURwz8RlGWuAh-Pkt0Y{m{o-W&;B~!XZg99fRDEN? zrfS&b7{jk3NP_5+iWIxAn)R43K>mb|L50tW%%(_)$knXFJNaX$BzpwD#AkUL7U>)j zbTgMq=`&Q0xb9MkuN#My!E9xZ8|wa&GHswdw%YqXP^ETIYk6p|Yhr2uH7I&H8+PMl z&D8VDJw~TvY9FSbPhg{GG6pXQIQA%xsA%jqSFT!p5sn4UNwou&`>WuTth$$gnYmeJ zT4cfJET0k`+4EH`W`sh;LDeJT!x`okM0jUbp;v2P7?GCSnK(oc}B1z?%qu8{B1}4_)9U+=e!EYEA zG<1d&7bZXW)Gaxu*%?37?|kF!o$s6Cz86&M-tfg<>sRJ~Kd74HQ;@v;M5Q&w1=Z`} zonlFR%*Dvv^3kt5t4E!k12KB6VwnHa=A}WH)^oaMdV3Tqy<_Qb&-Rrxn1Rz z9+0iX^|(m%ghN=>P*HuPn>L3`Y@XDU>nVpxDTLPf7rSdY(iL+*+PB-~k>potY5g@& zcP+IR^;QdxRrxDz!Hr&;);}u(&!w%J>`LJ+?JJfD}yQnu}VU(SVWeElr) z#4gcJv}@*G-6eo3 z^rMPw$RD% zIL=cVsVpL!XPDpbeAG&IrQ+=#Lm{s|`8z3>%t&ckyFTZ?AXG*=R5?t(dE7YBl41vI z45611KDqfHd%w>y@N#uZqa?ENU?D$s(D~fXrKg>KnJ;LDybS`G-5zn^77H?)>1{X4 zA~@!>vdMy7u1F8rQ~LZ2TOEvLZ`pLv!eXmJ9DWBW5E0lz54RbpP!K!gF&;P0d2q^{DKq=vQWUaP_ z#?&PP-Qf#CB|BL|Y0Cq7erPsHU>2?%=hm<107IQL4af7R5+dfe{gd+Q=0IRa~WbIpQN|~Vne%q44YkL8` z9uMyt(dUr@zQIHfvwGBH7nYf6RAi{gAzy0HWP6`IuZqCiP9aTuvoci;1bvDfFqw8;oZ@05<0k!31s!!?!$5pV$u*$aq{RQ{Fg z{RaF+19S_urc9*u$H$)1Sf@g%wU65tP^PCoNo}qWUfy;>x-rjM|76`sXP$IUH;=Kb z@V_pmhViYJ%WD;aTzgNt)IdLrBfM@B);?6*+0fU>*C`_O^aAs#U9foEtO;+EfLnK? z;mMaT=22Bp`IQfR8y22%!sDtZwQh@q2ayp;?`T%|^nM72@i?aN5g}vFocF_pn%Po` zB$r*5AM+y%sh}Vw%+s%_u67EVLVn5>rc3TlQ%=dgRy!JN7dbZ+v=daoS-au#9jU z&z7My4?Y07c-Hci_eC^Xd=T}L{o^F@LZjU=+ zPiOG146!poQ_rHCDbf^3uDP|!U)p7dMgb9QLcg6mHqcXa zC^y`>-QTsMJ0I(Ly@)o;ts%!C;YhJ}U0M3XK0j7j=X+;jKmm`omZM1ZDnQ6o(g0R? z8Po(jHRkGkgiiLT@vo8@sWUWG&r6H)kLDw;j!;f@PgPN-Tl;gFeeEj8&9w;ES!x(J ztsL~QN5g|ltPoeu`(o0st^LGepj+}&M!Jb9M=U1atYI-I0;Ppui1UiEt)R{9)Q52F z;$l$gOg(*r*LP4*^H;L}RJqn}9g{b2=~$X1sTvU92r3NCkJ<+_kfQ@q>EBn?`2fsf!jeAS!`z297nWIE1;9y(a!IhpUyOdVNsDLrgnn5Cq3 zX&Tkf%=3A1)!B>(W6y;}3AS=r2dWu9y-yx4u-H|VOP&&$4fz{GNh8TACEM88ijniE7Lqi5{j(phn%uviu4A2ex%Ij_9ehwe!o@XN`RH}sz!M0F`K zt2F=+=N$gbUL>zD+#RxhKZ#w$e2Oa@+&~Uz^tc^B5b?}-e9^cIhv*Jp3hWxwA*B@0 z0-Qt|8g}B9-G|s*>H=L7{kUBx!m5<3_~Cx=$`GHw-9P?EI`XmZHRjoe8gt+;+v zMm)UBxu$==+kxy30OY3LkVDl?srQX3&Xv8mp{hhhQF=j*?~#paxTqD{I}tBZV7YY@i@wZmj1`oA9u-}_xWPB@)##89eptM?mL&Z ze|Jwp<*)91FTI`T&}YGLXZ?S}22C094zC5g1*})F&(ec~{~*u%^OOi)vHglfO@`3? zsL#~DyWypb@GG0!dFb_w<~!g2D;{#Y^r?S!J|L;6d!DMgTkTi$nP{UPRLwg7Z}vyW z=@tJcNVxt3r#-KD8+V-v`6syWH})_>a_=49@HU`E3sB5DY=e`W3)N7xu493h-;aLP5m$ca;bJm$b zt(@{PxAXO~?2AEFSr?sU{s|(%U6fGHt0LQqHxD;n@L8&#SMrx|hY~$t_$SkgS-tX=7 z37T20OKq(Q^7!?na)Xke8CrGl^7*?aAwSgK^#h7<*jk6TgM4^$sKsMSoR@!(+TM!;zraSCCvVM=7>fwn zDhok*f+bxWnZf)M{#xhs_O`UM%0hlkIvzp-A;AC*cjjpoRH)mnB8_pGNvv|z;=>%)X9Rb6B);3okoqq#TN>N8Nvv2-ml<@7HY6FKo!vLDnRC&GU zO^aN~p$;I-`D9|qv8|g%z^e^`r1Zv`6uUF}(~AGl4MOR3PU;?>{o^Z9T<>T4qs~Xy zN5(YViSW*C-zR=>S3%5-_g%-;&*44b4UW>YXmN~n*Z=I7zw*=2yO z#6{fTGK~WOOK+l;G)i*Ww*5~|yR<#^Zc`-Ogn8AgCer5p1mP0zb+<`~Or&}>l(s3g z-gJ)BQnU9bBNW3;lm!abxvX0;z`4EG76`;_)}VU{Kfj>c#OrrrFnItyi;q6Pe^MaH z5p{h4tg{JZiThlC453h{VjQ#ZXM9`ZpzNvn`|!0Nwys~tHE{Xt$27lw+sm){_dURGTJC=4a_q<7 zZBr*;$L>Je;nn?$SKfNNk`L=PI$Anv6c&_NEt{+KUsh52zufYvQa*Njc_ANs+!*bx@&GAF*X6xa11WV`H#g(?#D#H>8K>=c{V~nGQ{<2M zjs79?M7}|c;1v2-(TF<@QmwE?T=6BlWFpKKi_k(^B>kV3`i}1&TD)6GjpgicgW2zi z1}f3S#W1~M3bob&7j_Rn()S>=6@s^~ODJg(_05fN_t!i+j9Lbp=4yjTuzbU99%vQO zJbk_U1=wlWHk$vNnm#Yt=@n7$T=I8Z;wg~h#LNoUT|d13>ApfUPh&17R9!8MTO@zS zQn{U-XeZk-A)`HO&bicWqjId>X@yIqB%auB%DLpgzm~sls8BEB56ZSPQ6Gb;jEvIx zH-ZG7bFW>h9*d%krHLEw+l~eQ>|HO!@Cg+7SzQmEm&ISk`9bO+iH~Ci2)R>1i=Q(y2{*=? zZviYya}8|HPIVUz9H3o)kOy>N{t@%rf99F5bha-Ahi7TuWnseTjPOKgNe#K&%8U4= zCoW0zf87P0fbG8TAmPj=DrlhC?<+Xsib1Qj0E%_jLjOxpIJ*}2eyN?4><4(YH$b%C zC9`l~7tXk6B!XUn8>Zoo$uD%y@o(6gWoCGuFFLR$B2@kPvEgmOlhud&yBIg_`DrJf z8jrJ|UBuHIoFe$Ii1UA(4&(~3wDX+cJEd`?6zJl)vpyij%%yCnKyG2}c|-1>thqK6 z*Zdr6;RR~&`^w+BVYvMBSkHbBd4bQV>V9r%qb6v;Y+t$I0d(xKr*H@eD7Ux_+XD#= zwf?>a6}{mao7WFPy0Hr&UAI&(-~ChC+@sF$aEwD2l#`Q~uQc0{#0k-&6nS0>F3dXm zzX1+g4Ry*8_x4qcsD1%W3FpF@{V)YjYoTGy3fGZTOPkua&F}oz?C;ub7B9X%K-dgE z`l!(L7xgI@*Vf~Mm^rv^q{eKed7WyRg2|zCmml#7s5)o5Y>1!z^{VELjs0Bud;Xa) z`r4WO<)sym#qYri$}-E!W#7A!3%yh1sm~DM~gQJLGv%j;JL?2A54^OrO8<0FWM%n zGwBr>HDt6aDRsIq59y{|VDhKmn3Ck>hvY@a|9euMsl5fYs_KX_qGLMrWWX8Agud68{&4UA8Sg63=>%Yo@|22} zFMU;lPOAuuQ`ZZMuu)fB{RCHZbTQWYp;`)XT!z<3J96y|8mc-$_y5Bdbb@&66a5*q z{Klh|^+bsHi{rN}9OPPAldTn$miuOV|L~-LmyS-R1RtBY`V%{n($I+{7?Zbs+nR@a z=I8W7pq5GHgM83CuP6JuOHRj^lq+b*S}|djagG^P7aYIlIT0ITQ#qvkUTg-F+P>LC zo~I?^e}xqcE#j6CP|TpDIOBf;<@Y=BqT|%B!gF?&ei!{mw!D-c9u_yrKnG+(G!ILi zy0%I;t0g8PNcU$%kQJf#W>jh*eR%c{zxe=D!_3sk?~z3amtQO#>SfB8&xcMk@=;Z| z+`+4@mBxQ+!rx1=0t%Yrnk=tLr*q_F%Q?g?> z&w#t@gN47m-+6>r%Cz&@k;)J}S?Bz0DOa2G8$vWN_?m{cNpp&2zo71^gECxG-><@} z6-{9T=Ee?tj?kloW~;QkE?QYrAYRBd6iy!2crBRPaN@r&s3QjX*wnJeJg=HI-7^M* zeN-E6uxXyyvLf`20UKJgT~#nJU0w<;BBOF zIikUy!=weRK1bo&@$GWbHkXvRI*^Ix8(ynRt&gdw-{qd1lw@CvubT5HllgBgyhYGr z4^LpPu6xqR7(FFEBe=MjZK204e&^w@k|bVMe2qg=Ez&J)9cvx62oJ>n8@tF<1twAT6Z>*g!&SF-&nZ#fEnAHa zbE2j*CA10m!?)0@`V!B}B4$>g_t~{JM(x*P>!4(9`_Em-; zk}ODgj3%bqnIudPSU6TgCkJbQgfsz|yoxSGzXrq}#jAlu3sBf%aK zW9tjhPiC!|zW}|}c@(atKrYYoJnbnbelxEO=CHYua~xjTzC6jm2`4=I>bUOlqfX@2 zK5{xAjAXXf)xD^Mc-1PH^g$9`NkspGaityjQ)Z&vRY*9(l-?U3^Z#@aK^3s4e<~91K$*>~+w6foi z>g9HO@eb8CxqkkOldZJU(LF4-y?aorEBL};PFBIp)v)7636>7I*Q6{x%%y-HKQ%wh zJd;tO7f`ZFiql(EuhAPMyIbL~Y+AzMLoo?lfmOmXDHv(mPSd55EU1DtmZII5(E>|` zn4_cm8y(U4@>!|t-|WH3m{;nR(q!U>JpcV7 z#S4oHtzGyAMmEJW3+m4m1scS}X?75u#(#p~d=9iIuO*B8}q ztdic?A>805=n;55((=Dl@;Z$>gvin3p&F-R|L&2ejz&1HT*fD4u(HZnquRN4)I_F7 z2|lLJ1Y(W|l^@Ko)(86IdtZm8j-0FxG0kMT^WM%6DJUYd@}cNtW{J|*99##@SI=_n zrv{7FN4I*mHr(#7b$HIK%%2l7{SPy9GUZ>gjpuo#tHn8V`fpm*c^-U5){T1K4Skm? zqxJAK*%HeM{Ny+i9z;P06?k4sMz)^pli3kbq*G!qBes1i30gANjnYCic;c{FO84?y zYk5Y|S430gWT)Yr)-3u(tQyD8jW$cQa%lucdqeIo`N3BUx^~eiugs1&a=!W{ zX&W*KtJjP*A4gQ5uh2zv97Reix#xq8|>0TO> z!@gUfznG4>_0=$uhO>l4yIm~|Bh70Y-0TA_+bKsJ-%uqxrZN)MEpd=kuRfH22<{8v z8G7<4oVbXy5N~x&3R=JPM@Z&gh867$W*p%>uBobh9`Il8OD9vCA2m*JEAblHMi#~o zx&*;;T2h8xW~-KhD49(i zTk0@K|GQp)mG0D@h)@B&MeS3RxMDkY-2ifUk_IWnhx)_&sS0REra`-)Z$-ZF!Lkst z&N;Zrg?{QHt|&L7&l2`~VQINzsVBHy_9A_~fQgv1tjtJK7)&HJCnX`_I7d0oNsX<2 z(tpOxHpnT^;Bo2+x=AfS^};uN^>0yT$~Yo)ErJ~SBW`YXO-Z4oA|7uv9<+I{qhOB6 zHc1d=zQ r)##^4obKq_yizMo!~z8AJ69Yvy_nRg&R}}*Yw}=$jYC>No6w$6?6G? zt%BJPC6nWpBrikkX;{AphrVLh1ur1#EmN*(#6*0<3StBw96DRB3^>Mpn{wB?N9|RoO;?*RKyi z&DIf|3w>p*&}3{95J-hzNN{WJ%vDagI<6A;dw~8+(v?}ySI?F%Yj7_O+fy+oY1V3y zT$TST>=Jxdt8lpSv~9BW={Tzq2lBFYXmS#%>?)-fq+A?F^pHQ}rHjXJ(*C?-g~X~; zf3jSxL*;b$2KULf7SIjZF)Y%xS{K%INn-EDx<5t%hNsY0oqvb}`myvpoGQL^+Jr1K zRIsbN!KrMcoMzRrhj)veOlJWhJHK#!;2u6i1RVI8Q**d>_o(RX?^K*Amm+Kys zHEvmidJRlM!YbRJxPZd{_}1=5Xrm zM$KQ{ zE2*!+DtY7!1Lp-}BNIt}uB-HhkeAHMhK=o$$ly!x7`k$8T<;j-VauRjQU3ah&qc|< zuD)RJ)a0wY^4BpeG7+qD&G17g=TKZO_~P5R_y z<$wW&X=@)F*ZtG!PC4>!W>g33e|XQ0_-FZ-&Z4e${LufK1p)Gh`VhZdhX`x?Wj)+k zx`mQxUF;u32?;95K;jWNWB+FOeN72wh((=X~6D z>@GK%`2FVDMw#%DNjs;kjwJNS)|0g1!~zV2!8sm}XXV!n_TT+X=)Sv7sx{#p%FQMg zVyu!9$nZvJ^Nf~B?|)?3Q)%H)+FqW4vtg_UUtN+8QD8-Z9LzdgihLf{m1L>Z+`5Pyfj2BE%hqB|2CSDn+kfl9>dcxZhR~@kFO_ivxM`RUOfsMLu5p!~iE>OZ7;%;MgHR@?iO* z_l9McmmIru@*u}!EGZ}`=GjGY@xnR6pjvr5o{6@jvakLp*dWfwreEZ7@=BA_`}P+4 zWT%3*Q`IMxX(M-*z7&kKA@2oD7nntXalu6<)~HLV3Y&*zoiBhmY3m@8+>#-!#rSgL zEHuYob0Ygs`EG?d?WXR{f$p0wqPQN%5alXzetQ&XRYc6Ui70e-&B8FHFy1)K3{XrpV)`{Or$Po5I$u5$;;Ut$y_dEjfcsx z=PWRxIN_Mr)cD-Q{PH@rn11H$lVw`E(#&neO4`7>s%H?)O(+qMC@8uHBeU~S*yCf- zXY>CU8I0K2*BNZF39Z@}cV>=pX@+qD*VgYh1TN4*OiWNF5)$&qmXYf#N+Z$Z>)n%R zjS*`ZhJX7it>n0uGjrC$Y7UjGMn;osyxbAE&F4%HiNe;uh!2z$`Z|qgcagVMy*}#^ zEAkh?b_gzA;d#Ojxj=;KKXFj&)vRh zMW@CU`e*ee)*m`+Ph4fFBSTZ+%c~yg1-R8vGO4MmwZLG3AJ=A@YV7uNQl+dxJ<8 zHViV{@fq1Fr_hsgQG(WZ(E*+d&oJ@5_$TKR!2+0lUYNq*#`jA^$ywzhiAU%!-KgXc zoTHnT>*E~#wY=)oBq6g0En%&k*+e{&jFMgEv=>e;TdKji zG-s)^8T^7BF|q#Tt;XL%p%CuX=MV8sd6%-@beg(P=B0uu%g%-;6PU1#YLjuxjY;jz zvqfdi2kqL86zZ4EYyh7Drn}%_T0;PA=Qb^;XKnQ6n@3TJT41p9a@$kI%b*Q1**Q4Z zYN?@68qdS>TA_brPl3FvV_UJ? z9V~*CJmuoHOw0&s>9Ie{~53qTI>kKrmt>wF}G>l>+k6k`k1ICrAd% zdo-@IE`tFm0q$_7p?ulTXJ|zgk&&!e^_e_xFbn5u*Y>jJ!?Dzx+H0f2W%Ga;76t=( zooS*jCF$HwGjNJ9AbC8`01DyFU73tC*6GF~Q}yTFo8)|A3o`cgqh==Pp0S@~Ikd?J zdd%dZK+)p&9jwc(9?P~PO3Xo9{jv5hCNwqAMo21NoeT}pV$37RbN!ky_c-5TOZF<3wGk%%P*FuK@;qVWpD7;Tsf4On=&2O72vDtu(Dg9@_m-zggSlzDweA6|9CLD(XBs zA5JT2LFG5lZSJvL{XpZ-dGso(g;W6ZAF(9KDXVED+7Xr38%7JIvCu?~hoZvr}1?pi+j zn*VD{YX?>Cbmisl7Y0qv6*qjZoCSJuP68!i3q{jJ5Pd#uU0^nqPZ zA2xn_7J#dp$Lv9UH|>MM-_V7Y9lp+)47qUievLlQhVc1&1V{XIpCE`7OJ>?0_5w$+>44acL|-nrZn z(4efzp2`{(gs+?%a4)HU1NKU^5i{gb(f-Eh^sopEv>JkjXUc8NI^A^?j3Dmh&d8hr z;(HSZrs4+tX;}O&?t9Kon8-yI7$X|J{K1F>#(^~ViX6|QYk>c_0*_ zk=2fXH&b0%V;3WqN5aD7NBl?*O`v6B59gAULgU)6Hi4tJoOeG9g%5x>gQn^mV{gFB zPE#^??m;XV1U*xj=_0g7gKr*8v~$ZJRrHn33Ma&Z2AKTsT!4!;#fE2S($HO4|1Imo zsqI$I`qS_!#$rJ#9#T@5XVj%eD`_+H3(>+XbKWPqi|5-MX6~0ax5|lSrC}#YXef;2 z#aJz89=zUjuc2bB6Z*SxsmOK4=-xd~xH>n!7OW~~UW-~JLuZg)%&H%c9Oex(V#eR0 zMh^yG0-2EnQ2wF@xY$cuEsC`p*`0X(JfT(XkM!?b^bItB)!=izIm=3_-P6m(V|d7F zgs}7z3V5d`K+a0^+xxWtHuc!HoUMR9v9z1zF;E~sjY2gMGt z2`!)S!Fi3C7qofFjMRlwwQXhNLE_A8>Aj9CaT&~QPLds4HEj8cN{4M(vY*4u)sDYk z3;sEHGYcUdJtBe8~Oy+xP!3*Ip$93o>hg(%w1rzv^xH4e+3|~6&t6X-*2vK5H>f(@Xe{XV?wZ&T(gC&rCdZegj&2CJNuq z`Ez?FIfs;8E)GSztW~Ihz8A0MLciex?o@cJA-1ISjX=|gHjQ&V=B4=KX2o7e)b!Tf z&0lem&UC)@gBP!A+Zhf!##OYU@YyrDEgsr2p60={CIv`IDCg|z*wSh!biyUJeY4sv z2vz_GiQnv?Ri%{nF$`YKD#!0PP%oIjW?btzx+`mW3(I6rq6uW3rNt>#M%13eta9A0h z3*F~BAO8wuCi{FTj%csXjjit|RB0ms$nT#A6a6>A1U#KIe4-hk11LU#EtLmU*GJ>HNgnN$1k{JtE@Va5dol;nd`@eNWj&MUwQRhz{ucSUev2@NlD3Vl>>oUOUQ>cfeW*FZIP~d_@9y~*GeJ$%da|$>W)gN#@T+6ZRE+RM znYeDg{*fcu>UbZkS>@=GdNV|;6&BWbq5eF~hJe3js z){aucOPYm>f0QEMY+WbQ=Hmw$%!3N{iSQ1-MSsv1l?j^;1rx=Xtg?orOWgsf()zIv zbc0J%2j5xZA6ZbESI`_t0+s_=N2c%H?P%vC3b$leY+D*+AC!7i;eOv)>V)` z6S=sghVS1COi4$e<&U=%rk5lEjzo2^8P5m`-IA!p-9rGB(I4D7m`$Z!-Rv`W%qgA$ z1-#R#Ep+s=*#1SpdRbXC_N!5fvx$@jw5=>L*w)}T4&z&D#Jh!Us}m7i(?0k1?%rr& zAMQ+pu5U>_%MP>_)YM>0W}u)0p4(TqfLWC7!A77R+QH~S9q6Z%mIqV8+=nQ{;6-r~ zdF>$5AXT^Sk-cMrX(Gf*h!dVucDOgr{Df!sLBjjuSlEwEJN6!X%L3yR2>B7vI`rwx z1bEnnO|ja+9&XP~2l(avW`~T>$jQJ!LWQ!_+I+{TLfp{I{R`uw19{G+n^S1uGR1J9 z(Tn-XU*bF7@LLuhKYxQO1{EF(Y+uq&asN4p7UL;5=*rHS_fmHNA2V(Iy3~AOyK# zR;wVvm4sEGWB>`QhMKKfB+toI)q5nfN73aX|7o{-I+xL8;@$jmmr9WpS3Ey1)VCHs zEEWi~vFiERMDUb_z%*4;rdj@8-tZy5Gk(X8|G}`TA9vD@9Vn0b>2B>wc9+K7w#Da* z&yj=AW{ER?>=klYfZzbLdpN5|OI?dPt2hUseaCWU!H_2!h9L|0RmDXf1jUbzE5G`B zouwDg^Gdp3TVhP2pH)V^bhu6y-$vQUHK}!K2Q_st*iyo)wsj4PrZzu(x4?bIDV}pSh0WqF_)DnozyLBs+1guL~2afuF^V4-`#N} z`IqFXvTSyFU7}@JW?cGLN<7advxjKOvip>z{&SVx2hd3wHi zu3cZTEdUv&iVuAM9rr_jw~UXQTb9NmHhG_=pURhdU$trRGC>rnFz7P(0hwR(f-sgt#%E!?N$e8kia=08eVHRa$*!Z5ZH#sq0? zi!<&*6}3rmK^jB;H6OdAU1jdK$482d?_cSkc7#%2Xq{Uxi{w|_|jupV*ZK5ieP?VfUkLVc*n}*sIK~#=gwFrRNJeT-R8*v%?kyS_|uPX<%i_Pg z_uuw1!gKE(JkJ-LuHs**nyt1v@KBDzD+=P7@HVxmxJ2^jFnK(xp~wSDQJ`&KN)6W% zt68sJ^NG=#6oFJQ!Tb@l_T+uL*>#pD-Z=!^Z93oZqUM#k_!n18cG$U)%n^v07lW(i z)zV)Xob5|7Jj8`}x!us+j_|M4@P#o z%CXF4o1fTl<$S4?gS5w1f9@{$zWaP^`_9fkyU-|+m<<+;KiU5U#$kDQC>UN&*WLrxQ)6{wZ|CHshb%X%XBz>~T0uF{Bi)#p6yOmWf|#kivWA_iXmQRjS6~#^ z5X;wtXsE$~YOvutb33Ni_wa_-2T=Db{*~Mm**;|$V-;3*S=*E%JcRRX)oT^pb>9eV z;}NjQ0oRtBh_0-<3Cvi_gsOj`JAhen;|Ua;O@ubr?M3M1ezzNWjkSi&evE|yhSn1w z>M_kJQ-TeuLH>f9jc-|Ulub1QNHU;7bvRoJRx z@9cM1fv){Qqk-?h@0g0J->_W&4!>^W(Td{Raeu}?RccI^Ct{J0B^?cE#d{bT1#d76 z0oTL{q2lyf`fhW#?ulsWl4#`LO*TTJ2O8oys#kjsF(>6l3A-CXp zoJ(93YkTE|k01aJ78J*?&uBb6vhI|vAoZWoCSCMepKxEkR6>RJIaoTXRYR_s`n^u( zFdxgl>&v=Z80*k=?mmI+pwP1~f_lRnxXy?Ok*S=?y}^ySdq*)q6%tla`AiF2oDt5L z{)D@i--yHCjyfyDp+3UjuaPRHCI0@N1LR{AqTo;-)PfzxaxPB*6#~g0&u%%bwn~Hx+ zP{6!tr|VT09y+Vy?aH?ozk2D2lcJ?gNwU$o!UfR7r)s|dCJC2;wLYuH28#SzwA1I_ zV~EC)%fy#s3_tEJL#IB;=@7(A}S*_kdJ#qFm`>)mk)zT%^wgdla#aApH^ot z7c%)#dSFoj{{>ic@{sdOmFOmkRaE4l6ct6C!h%_*(=HYa#Z4@!@=6)ml%LzXH@HvB zJT97F_$cL|-@}@QFL&8iD$#m!>dvF_@@pcl)~9gKoL{zQIvIBopRL;cgxVNA3?OOp z|7grVO^u_&8f{th@mhL8G$@9mcShB%*cKX6;zk zp~R{Cl!tErGzdS<jG0d58z? zXoGU?eMdG$cm4cd>4vTq#I^;JUg9@{ z42)wp#82Z7kd;b0ahj6S-MB?Htsx4cmipA8dt%W6$l&8ljQPaN-774{kMedehbE>IaC+@z<`yrpTDyBqc}H-iXypYKltfw^s}djL+Hg3Q>Vm|E~ht;hlpp_H$BFJX|&*< z_kDRJ%sr9M_!F;}|IFfgv}2y5^6fgDhWQy4e5o~_Shk@5a^bRmw0>vI!95d*x7Q~+ zY-LHW95X!D1*7Q+#^5JRguF8?%-@3Uu|yXWc;8+Q*$km3!tM^uMyFd57s)SmD&l z$!XJ7L?*Dl3?MDwZhEWsD{wfKv2qI#WME0JCT?_h2KS*5)vuKTBF8pK`h zZnb?rCEF7HUDM}sgM%tsaoEZZwoz|*`e3PpZ#f7k<lNoGy&E|Ts(JK6s7a{yB ze^rQx1tR=I;0jr5+R5CiDcZi=z&C56U|B#@de=tmFC5O%0Sd*dDMWE&sY&K`QlOfR z5}&!L6ZrW-DAB>WrN)&KGW1y*im@{7q*R_$2oEkdKIsDG?J>7PIWOB~t5a^$$i;#1 zTOi=8NTDM82nxaCGqi;m=XeSoU8q$-y8P#uDhX+DBk!uQ|_AHr)hC*j`^A5XE*iZ~qef~u_9@Ixf zzKx-f*RAkhOCryuR)aDgu1HXsuHBKR^X+2wW&5xDIxcV<8-cl_d78(Wf{~iXK-`0b ze)YB1R@0GY+^a;sT(QsbBGog$!U^@X=9tva1}QHtr8{5reXGD5MTvE6cv>KJt&VbeKjOBmBs26WW{@SBPboXVvz*-FxgU1h&9G3Kw?=9{xe|@#QK}+{x~f(p-g&gf26IAWk|-%or-7S6w^&n`R1FBI#ZV? zE_`+@q2IWxMu4R=5Pf%O=@4$nZKRmOopr57hb25OyXB4X%LPa7q0 zDP(gLgl?~#i#dEv_mV*3J4%h7oN~NN9GD9>4DiT^DNc$ixCOPuK*)6vjmwD*}4~Fu?n46A4~%}f)}EoIPRwMr?Qg$rC_nJ;e%5_YyLj?m#I~tT5uny z*|oRRpc+Xy@O}WL5)7#Rt^<-@LAK5sz=90QMLA^5Q7A#hKwd{0p{a2yK%~kO`^kG;SJ5+z2n%<1UvD(Mn{8&=&bnK?G zTa#WK&ufYtDk;sF>`CALLC(D!WQh1}>GV25CZu!>3tT#VA_SJt7h*%CYds>n+LTgUe!zuFj5( zEYxM@gAD{7ShrCxD0IY@-B{JjUX(dhunMpTJlnV_CKfA0yGcNlKuBx$n?N zatpYBqo(#)4enff>yHz(?Cw`lP5k(QQb4s{Dp$s>t-kchN7@uHQLsS^soPeciFfi| zWm?oN#smQ(gIbXiePoIgB?rSV0Zz7Q5P(*Vc07%t*y-V=^&;J$FL?eCeOfkrsWiW) z9kk#0u{dlKOeUQJ*(c^~>^$|MJC0V-;U<9ibmrRNNwD2R;iwn$h~+=;IqCBDO*SSj zhJUpv3A?yA))C%vdv``Clu`})hGrD|&lF+KTs!aGkwu*|$Nca1`4L+U*MS<8gy=kA zo#wWSxl8)IdZYqqQeAQtD1HkC1-UPg6%?A)x;Xp&${$jEuzS6kOKPV;GMwM)t=#h@ z8~zsO+aOc-QHi~u$2Oh~RHSAuCdut?NPE2ckdoGL169=Y=PxS)Ru1gzak*_z|Hj+R zSGoN0c^|$`u^AGGJ=)9GOU(AJeUL?x5q;;;DBm8D%m@+t^KMJao#l5i=-pG&Sz`y~ z_x|%tj)N_Ij>1?Gsaf@m+_9ko@w-KBPJ&pYuQnY0oD09POcv)o8QU&2^uz9k2=xac zh)b0ZguQFjrKBVgiF-PSyLS^WdX+zeaX5<}jnw1sM5UkjVod}NCAUX_>T#(zWeZ@N z*NFhI#);-EPmcJQKkt$*nSG!&sPukf?+SG8Zr@+_#h{EC+h3yB?ezG@{7Q}6T~Qpt zm4aXa8RW>q=~k!EJ=milJzOMt&u~>Z<(&7uRx<~X1YQ8NbzC%!oM~Rf^f4|meEcli`$TwtT$vpFwl;^ z)31}LmynKA$!Thl&UfT{xoW4np7e|euDv>DfWGc;TI+bEL)-eYHeiIl{=!GH*6wBE zm2$b)aaYRZn3B4LUC#EoA8B)Z8p`=+9jr6fsRHm*-JQrZ#pwkx)`RWXT?MMc6;1mF zn&nQfYFX6AkImOmJfsgtqR+9vA5*A|mdkr;BiaLU9aW&Hd{4Js+~Gvs`^he$0h7fe zVeHyUmL1hy4#jII=+2m4|EH27kJ`EAauOm<#z4uvn3Jt{b8BoEqa(`wxi(6NQ`*70 z#Nt}8g4#cix=XbGzOnZ3+6jY781m|AalY05mr!MT7 z|CsEwePGTmKNe4W!ltm$-z@4*_I4^U&uMZ^OFDw#{3%zF`irj%TVJ>3N9v}Wd)d09 z`v=N#gJU?lhZ^=YHHciA?MI&~X)bPe0cwO#0;g*$E_=2bT0Za-TD8PPfT%2~-}XVP za+KBXT=L!EYO$t-japjY6fus2CLcjYhdv2h?QQS35r;wh;-WxMDTFKp0B zxcSa;KOI`CHcw>1GqLUD46O$xKP|%gKUeoBOA5uU-grk(f7u)wd0Cx9&aRtO``6$G zr}pbzxtA+#e@DSxbw6mXf20L^X*lDm)Dw&Bzgk{R1o3u86}(+=J8`${({Z_7(ObO< z#G%e6KNdL7L?f0bmagjXobH4slQ-e{D<7ArTWpQCp5TGuJ zMDm*{AKq$g^9-XdyeF3bJ_|khb3XKkSn%x43mKe$7(sE0j)YQUQVioRP=VDMaDeoD z$5z7&+Rp4AcbBa4ly9s52GgC3)Rd*Z*%^!}qXKmlUYX`o!qS(v8s5WT^oSsp)S(kR zTfB#I!ZZ&-6lVJYt<0W%FK=uU*KCGtgWXSlWV|=S6&L1x?!;CP{gWn4WGqzs$NYZ! zc_*>>@!Dud?{fn-wI-tz(`?L^>*X7&q^4!7 zZ}{sC3O{#4lbPx&9mZTqKZ0%3v9MuF0iJ|T6CP5axLxy&ZO&%6XJyX1N54m`e!RVT zC-o|}UJxB&M=wv>@cSP(PRgfITI$j)cUdc9Ww3hE4w04;R4Q3MFR1I64TP~^0Cgpx z#I#bWmG6fe|I@TJOKH*>_viN?V`9w@J_bNeQ0UNRyS)BVSGaMWO{n1N$M9gX`dhxe zztNIlP1o&u?k0myn#oxsv^$n3(AuuAf9+{PcNWA^S=yF{?S5VCuTrEtiLu!q4`Q0S zTH_(qag7;U@XDsMHQTN|btrkpwT1S5Ta8gI>JMkqIt7k5IZPi%_qQlnBC?sJZ+SVRorH&)=f_scgx?fUqD2jL|a7bMr!Ol@H-Wv z2&q{flEomBAChNL*^=8yb%J2G16&4)fzqPo?(y zH_3nn$(rgP+~BF1I<3rHrw8|UbHd8sXnp%DO2YvQm zfBNGaWH#g|hw8Z|X5PBMw!HCoZ3>4YT^(IfLxE}?lHXtJC@Ez`61@ih{>7FbjFK_h z^sKX&3Q;IL)s=BuDrsD9R;FS|m^IWm>6$5@K)meSYGxaTw;SR= zKW=>2t7gB9kCMHcRFvq5Fd?sjYqsOt4>KeLsEY7O$;5x1r>4bg+UTK|w-4#cnsURc zA)%dy^EzD$ghi#^(_cj`N_qN97E9A@y zF`QpRyXxWYTqoWnm>JI9vqn|n%d%OhLqb;gX97|bMSru9Ki<%M3nLQ7i6ca&?yiki z4!t;(hR*h6=8f@WAwaq@972f|G zvZjR^jlt^6h}q@c`?-z6w%uwST&Z3TD~M9D(Ss^OJx#~4ZG3r*&UqlZRmdiQEP>3d zJw0}Cd{(+N!~ckZ_BVaqEZj}{g=B`@_zO3EqyZJie8NWA31v*O>NOs*JBsd97z^w! zIh^o(YNlE}#K|>LvRmj#V@^ez+&&$Rr4D@ORJq;So#8UYSevJw<`4c;OkZ|bhajB6 zy)Jrar+Bsfc2@2Kteq9&)EMbMOa4pB-!~{`7o(H-$0uTUzJK$dAGifsb;f$S@@$)K z3zmG-?M-6XQbKTRcFIx}3<(C~<|~a@{(HT?3Y`1U1$rtqYPa2gsc&swHmc9p(IW#O z-QC@-2LDhw|M$qgjRb>d81E`LOXcRJE+Kko%Sn9u(~UyvZtGa>G#5Gf-u>0h)k+efGwOCa2R-!MHYl z^RJo@=h)rR+nbG=pZrjI<80y8|9_2&Atoj!tpfYDDG}x)G*&xK>aM(DLMXmn^l(dp zpYnW^`vL#{iI;LVLfZWhtntz8)z0&;h5ygTH$W|#u}8?GYqPs6)%hB8)FICBN*fbT zjcjyhTMrZ^J+xf!=a~TP{gjnyH(#S?4fp?HL4v8^UhLc_FQi1JQf2c-5Co;dHM^%? zqENb95j$?&Gy``hP?p&UIV$k|cu4eC1ujUtkd* zFg+A($W>oD7x@RAnMOiadE)ZtClCMoNpOAEv~9r|$P11Kfd<2ecE1v^j>Z+&yJS33 zP~yfmdN^ffPb{KPs2re9W7pxQ*|yj|n|0su5Ym5P(gA5;`02{}e#bn7@LXx|sJFkq zg1lH+c@dT~EwSosRadZs&@{hSE05~G772L+QhI%27nNycQ{&(3whD6r&Mal4e69a) zuan$T+u+v9nfq`_?b6PkBP!7mjUNADXDmW!2eRNHi}3mMP#T3rS&6yd9RIt)WYD2*M51fJ>-MfXshQg-w*k%B>->axSf>8vb<k%>JEEhG*%GtYxMcsPY(ImG0%RysY)#`8#!?B@ilt?V zTOCquL|c+(5NG~N8IZD&nOVXy{#0`7t+o1-?@fJ&9O`^>bOb4UFSQ8`}x7OJ6lEBBw8Zp0ex4;^fIOtuE@%{HB z0m%+2r=jw`^6||Xi%wdZPsnjX&p$-@YMI9VbH&iM>a?DzOH+Og+V?~emMwegh^H#< zfvGO_Jrwi4^v}wgYg_7ezG!S@xO-2T)l$+}da(LG()<2`!_m&NW$eKR1Hq{3HW0!)%N#k&DGd&y=3ffdme zw%uBdy((fB+_KpB47Z3e6VeI`aaf;cmZSe_+B3s2ZEXhfxB1E91GgS!9J})9i}ICe zu2jgP#IXE1`wo0=Vj^71#-nWRFki>h-zGptRlaP&6o#w)(m1c8Ntttxf)4M|kA^i`VTtmjYRM5hf(bpe9cd0( zT2*Pv!0+14r)XokrrPf6@^AYxeBF$fL45uicysE@@`*jtqzjF9xVV-VYa@Ux<|i~t zwH<^#h!AfAYXU#T@M(dl+VrSm3g_`oh%*|CY8@#K^!zvH_7C_n2d4Nr#WJnW)o3T2 z0r-zY+6V_DH>oy#I@IY_5?{B1VSXpWH8$Z2ozjRTn{y_~ZqX=a9_>6%?7y5!nF!4l zK6_S;+jkEpQ*`r-;56ej>Dcm|c=N=ZXGXeYklX~3ZP3Q$A~Ii+wX%A5y$tDjzH>H4 zl^pm;5R)5!NG}%aGMW0R{`b2D82#J)MqNJ)Qf7$e%_q^8q9@?iYHbFSiiGFOTEtpa z6p{1Q)FU)PW?Lu^L%dCeuT!6++`lZbW~t^1^BF4Shtog2Z}>}EnUnHF8d<(rons9c z;I!gabN8jzTRm&=^V~ipZC$SXhU~EYlPyn@Kh#8)RmmYp4@y?Q`|jIKVGGOFEFop* z+V6BA*!&$;cdu4_B@yzo5%k z`mGLWYQYzIUI_#;yjx(gQ%UTT+VGccngV3)oL5uX~Cn4DZ)4MSGo z@BN_Z85K~$H4~3~L9vP>}7(rBK%n)*B)_ftQu}|x^RD(>}lVg=)&(#RV zk&fN@O&H>?p9lR#QgD5?+lu?PQ)SnuS|@x_{x7>A_LxAG`E19NO3ZNJO@>m`P={}( z=lkJ?>ETRx)grpH+;!n0Wv60hB6iLSvFTAjC?Yr(P(-t(2{vUD4!~Rb~3|Z991AVu0*PrR>DP0i)me z&Y5V<9`Ah&YguIaF~*Im1YDJ8H2Y7swJV4Ta4(2_&ft6CY~9(`*S;L>n<#ZVl}&s0 zWEC@u&ErapO)xS`qiX>78lx8W3~po~Y->n@3FrKeTI9)>KJCSV=@R z7AwKl*l>32fTmFxbvNW@BYBuxu{Q4@s)k1VU~o2It5!eSws zJ+4H?e(S}9nea_ph=J9}AFh7<<{dpc8W8uW)9EyvwDo_OQAA|hA3>0)4!$T^A)^}W zJQUcnwGr#Ehkq#bW}=#$yVbw))99(-XK437q^uR`Hg3wz5+61R6K_+%Jz$8_;TBtvJ*bp9 zE<0V!9No5gkFJZ|f`fKo-%~>>#DI`(6-jPU$;NT*>myBUU$ z4tx1Z-29Q7_9|irV<#-l!K)ojLW%>!{){Kc|9Qv)v|HLxORi z?TzSv`EShW?I$tfDC4S|gU4_1aEf8A-6ToZ8P(dPCqyG;89KzO^E)|7od#iy-uRM) znD3(f{s9_q>2c1}$;1fh zBucOjmLfDRjc?(>8s%&D6!O;W$SAV3*?4+?d%gw0BQosL-ox$0Tk(qY(A`rGWp57U z!`@?JKoPO!9n7q!?YH=yR0hW4FU3|Hv&=0IEhB9>Lw@6DmBF%fhGF?6B_%O2rF6g) zq-MK=wT`Gj@tKgFF@lXUMjf}jOGJM?_Y8CdCRu!Gr|~csgL9jY42oT|aHf?RRUpuw zrb03^yt$m@rRI!{adaX~y)&SreJ>Gm)7l8hXV_*H@vA;t$b$ppB_mb#u&r(DuWCrc z5Wy;Z!sS4i7rs8(JciX$z1*#(LtAJ@ zpC1DcWvW??Kh~4>cxbq}%5dW5&wsG4K_Iz6_i5lE>}Idb!Pp@r(!wODlM_qKvA!)e z+*RP%ep1)QPHIxt7WU5L8T+1JrGQMrBqQQZk{VU2-+)FYy8JNEN^CrC2yodTD7a-9 zX*L^h&~vmty|wT14!TuUftk|2RTm;aD^`>MJZ*Q$9N<$ys4RVYkg<(L6x zH6a{ewJh$i*}-n(hG3?(QNwXRAlewupU>d+JZb0luHV8JeP&yTGMagB^z6%DG*ID` zz+|JC8P)U=9p67;d84(Gf$poyl-NKk02UcgaGU0@Mm#18+6}zR31%6T{_}`^3q@F> z#~aX{pil*n*oVFG0+s&yYjZkldTM~KhhOFNuX~W9(0{@is;s%Qosv=#KVoEdE7_ZX zTNsY@qa1cv+L~MHd~O4J&LP@5=+~nHuD@$^gw0#c%$??=IzUdgD`ugW8eI*v_W532 z-HrYXjRMfTQ2l^Ye@q=#EIvbgsRK(_0Zw8%XwQLK1|3<1%A@Pe~vCJvj)5fuuK&I<;Htr56(zl*C5YC8P3pDKG0Tq8F<$sUn_P7 zkbNKqhA(D{;@x!#fGo$GLq{c`qk5l##@-85Qu$lPZp1DpBl=Bv1jGnDhn=FU>(2SD z@~DeV9nw+JQdUQnY%imt3cIQ}6#H+Xi&LNzg^Qgk|EI&Y1 z0Hq)6or(0;qjbn#IJyd&Zs=+pnF1}UpsV?1l!kF$uO>%=$Rim5iabHoCG8`oX|+<9 zDa{{a@w-4QZh~d|&Bpbaf;O~G8E=#ZBUA#s0fg7k7Vrb1gQ32EQ`&vE$^wKq9x%sb zs?R=dYN4)V;G!&!%g&>eBXGcgI3rjB{;LmOG&bkcc=#5C(h@qxcPuaD|$hn zg@P`f@^@-4?Ik)4Qkqz|fAY@69_GcwR>bABJx)= zLa|91!mNr3wy*+sbMKzQeH`2o(-u4ed<99K*TrXh%&>~sM{rDX{J2TYo+vVDpj3m z2_bQ;*S;P+ zbCXSMrQQ&oJ7s@^#QEa*uhttxcX#Z*ZDATfET~oMwy}`vJ{?M*hwvRw^BR(VUU#l5 z<|BrgCYscU%E!$!^UQO+iN7{c_p@o4WxMUqdUlou<8!G`OJn)1$Z@}%Q)!a`s+c}t z4?Z?am=E$+mKNEw31a+v2KRXdPYJU4{z1GIw0O%aSSm4->sX0l*#HM9(qaJa6^rSDt0{n9n|G*MkD+Y}J1Ae~dNHK{3Kx=wuAhrmsl!jc09B-%YfN6PduLR%|Y2b zyXXr7+uUIi&U#8x)j`GzSQ`>DxA={cI@;z#9W5=cxj$`LRn@j!0lQgSKyKl7(7~eJ z)R;xwx=pH3U-`8`fYL6f?jTyHknbdo2cG^E~sEc96sU*P3eWQ_#9p_YloOC>!F)u)h> z=2^VfWo?Ry5T6=0t(K1!3{8Ml&Dx7ws_GgP^dQSP5T%+MDDT!X7`qHv8|fMdJBCIt zWahm|PZC&^)tHqfWeE;Pwy)IrdW0Ji>r~LdO0*17%y$Waa)fdajU4&dNA9pfBcrrL z!|WrYgm_LbPLBjmj->PUPKUK}7@A&$YMFYM9agG}%9*EE?Wgv2IJA|A0=z3D2e_pU zA0OwQ_F~uzF^*BJsN0*UT#ISE#)khdOA^fu>w~c z6WDcBb6oD4c3)DZmLl&81jT|=g>NXw7pY6;As165!si-=G{!8d31cyS^_3)^4b5~v z_GqEcA=mLF&=^X;0qUac-N|{ZyeAuU7s{OE+7*#X8ZwA%LJ1Grhm9l!1#(nk7*=Xa z22S86I17tTd4XPKhV7=F=M|3p>ISdh;W3w(PD;6$e=K6)LebK7ch%jyXGIrsY?Wmp zmAa82hLbVno*10(t~>Wjm>GTvthkCWufdCb$4VqjOf8CQ2?5W=6Zwi>iJKGrK?5yY zs9jERHaICxExP{2f7^5Gm;z-47TJU*$l>Dq?v|CGUxqnbRy*M|Eys0A3qm1-BlK?* zp^d93t9q1u`@rTazR`L3qW}1xbX2KBk*QM?HJm}mnuIt}2^!`sPFq|ozFZuKMS;(W z4haPpK_2b~>Bdz%Aio1f+(Y>$bo6;nOsv}7iW5)xHiv{}**l7;`d5S-e|43=UPqmt zY0}{bx=4&%HWqGw%Y@u)vn&%N|P!=)^pV{Q8Ps702M+L|p96>x=-MGJ9 zJ;%#1&u+Lkx>`z|3t3n?{Ruu-TM3A6)KS3}OM?Lxf~$llto zpo{@|1j1Gl*B%YWSW;JxH|08;qO2cVm1Zl4Je-v%&BksLa>EG=si-rJXExi)b>;bL zMv=u3o{~7jS4d{sXb@wtQOiN|{R^4i49QMv7|DG7om-R`f*qE!`c;wB^A%ySiefHts>u!>mC< zGmQNlq-L=#Zo%J7z6BaGyOTRQgP()7A=omaX*tcK!_+sxw-@Edvp>SmsS)K7)J9g- zg6*%^5U-NT(cDWFzbH(<;k)kx{{Epk9J9YVET`XHuu{f?dl0uOK}1nI6kA#Hph)V)S(z~pEhjEb3(~4bVEYTy6cOdZOG4hA+~qhb&fGl8;@pM$}aO#_D{1UGshiWz$iJ9z5F(CNOXiMI zllUW6!_y@axXeL40X%!01|!k&jvYmIP3TO;1*^$%oL}@%hn-1ilTh2Yz=VRX0NF{d zqv5RgmKXXB`p|qTnHlHBQ}N!agpS426aeQ!pVzzHCg(bpwxP(X64=6fuhC*w=K^XC zqOH|PU0wKXt6!BMvpkrpzbmLl;{L{k*kmr&MH~mN%SFsJSzXna!=i z^PyGi`f_8$K)fk$RQbO#Jh5j4syv=UuFIZRALv)L&5JnE)phz@fop*B;gMG;)HD=T z+mSi4nCWJxrtE<&5y8b=;Yan2f&mX1ljdL$hn(BN+&Y>E7GYbR8eS<8Qw z+OOZh7*5(8D9lwqW8!OSJf))Ld(JotY}edG&%#GTMY-w@Xb;mcn(ko<(m-qvzd@5H z>YihqMz=@6WXx=g^F>MQu$Negc$>7ty|)?;I4^9h$Hb7dvP~P=a^%$#mRM+JTdeH` zi4&IMp6B&smo}CCMJ0b-0gMOZL%|#)%VLd8b%!2ghS*5Gw6k+#vDQ83qk(MB#VOg= z-B0SRY$u|MZ5~_w9seFV`-!iS+{53Nw6l77c9C1rr6GT{QT|h4Uc*w{#1OeiI&U6Z zT-SIpTMMQFpO~1a7Y$C$^W6P^j0KGDtca9EK-)NfVP9Fp@S;t{>Wrn%={%+y(cnUx zq&ntbW%|2sF#Bl-UOzCQz(7AKCu|Xf{8bcRz8O7cfg2R|zaaNh4&&RwxMAykw?E06 zSJ!B$xELn&em9(7-j*y0&q5!T`Tvo30!VH+$POB;PpP3ZesfyxyM5Z~!|18xK-%fH z=v!3t&+Krj{tea(EDV+u*|F2huj@Jh?gLzQb=?N93D6E35qX!SGdI0xwvA->=e8H7#{0 zYbcd+LJrWB0G}{|;CN2VyRG;MR&n80!gJ;9;=b#qp65I(CSp-239LN!A?S3ayjFy^ z3F7+?Jk+x(dQ40X;@; zrgxI1yjCRR&N>e3plE03PQ`S-R>g_{io>Jg;-$c6A2l#nTZl}++rPtSW7kLL_ z-I0@tD5*UNuRc>3c2*gkX$$Q%XuCxmi5&%K^Z$65Jfb6+6Ud&Zn$>;h2w7#E?*=G-^)3Ah-%SM!w+{yVsKo?w8J-` zPorkaQ{B^x%)R4lR*>R&3N5e5*4}^L(9**xBNh^7MvBAMJp0@shC}5hVo%5Y{_6E* zM(GZYjv0GuSafYypUzIHe@Z?4xoqy;S)o0y(|(Mf7R;gtvvYApry}nVQ3;5%CNa{^ zv);V96LVtTPJ-HHoRI3OBhG902zgoY8JYABKr7L@gZ0FS z%fSCyUl)su0Ogo1#^u#J?EB;?#zUhHL{yh4?nlQj(fm#3i*hEl;zWZlf^@ef%PjVQH7<*J#G)JhH0(>bh%NwJ|)!-a4!rI=N85uR=#Ft zm&e5xcSe((VP@T@8{fW=#72l(se!Z1hU<2p;}XevzB47xGCJ9Ew!=fiKo`VJ#|-fQ zin)#xWYsX?J!$uxS-`38Md_bk)O{8bw=2G(|58}Y@WpxQ5v1~U^3hsG2C-Y3EBjve z358$2$V10x7`2uX$@{*oBF)i#-@h&2EGxIz@-3g)A_mW|*a4jAeR$lE*~i64k1A7F z@^p=(iLuUqub!zJ?s>3Jmdtoh02vRNDNpR1nl-W{FW8$8rO=d*%yh;+`2C)>0C-KY z^HgqS57@pO1eh?Ut4el{cs}20;z`+~ABkrQ$*ss#B!x|2rH9;|iArs!EqWnC3j+WI zxweBxKhpn(rAbBbM^z2^e?dVSoUlOd!R3Du6j(60F^$0y;TBE?v5?Dy`1gE$L$Ssy zdrLdPgPrayMUDANNouKL%@sj;l$cPG zWuVs8`=U6d_eFklEWC=&&^LxU<1zO*g)7#UZ@yx+h;F>_@gI`1PLiV>*KZF{i}wJ% zdurRqit}p6(W@iIO1z0xyH=kt8&tQLC0%YyW+^Ni~v=hwdl2CviQh z2ztiHB_M{4%)pxd-h;v%W~^ztrb_L`uHoDMJq!h+B{{S28VFD{5FCZ%$1%n^|1^SZyOLwMJHGnpf5G-f=*P zZB+JL?1*>>1uWN&=5D|4lz8H|p)RqcU?D1rKHDp&zOOL&&>?jHV-{=&Q`*%cDFD>= zeZ*80e^2RDpt^g`Owk%Zi0zIgiczu$sL{ON9%-wLHLDB9Ds-Q0J~|A*ke^mFXB9kp zZXTnWiEx4vn=Bp!L>N2MEJ-BfhH|6L zcCiN*E3x?=G|v^=t~(&}igPG!+9HxyF=f1OjdA=Q`yYec?IRgcaqubtHM3V6&i#!@ z;Jyqs8czH*_eP5>H$<_%*z!82Pr5@_uJ-9f;51E4jOl2*mvJadD=ajDO^`JjM$0%$ zdLgb9>+Ecu*C_NmWa1N0QV@Ksr{x!9-%-i?mN~xJSB;dUCRrlzxZsaz(zB)u>2J(R z8a;M2YWB;K^5Xk)HtN6+PX$=mkRj7({hy~b%X+)NqYP!EBsFj4h>Ceylv()5x2LSF z^v~IpAu*l95yAF!^s-JW4i)2~1Yf6zzp(=y#?+u>&^XBUr(@Rf0IOh*uaomfj2!^; zn5?Ct`I4TN_-6X@j6@I;4{3Ikr7%Mjs^Qi1dF5G7Z=5$N!jGBM0Wc^OLA7t{D$P%} z4jBUBk&DCbl40hy=GhaUpNMU;x`$J-?IN=M6Vap;0$|?VCDAwsYqK9u`{AFm#@^EG zNd+K0-&aq5K5UG(!(~j1%F93Co*ZCR6w?G5Nam)3M?*QCeYmP}-&{_@bR21=%^Tn-wS_;aKo8En!v`fcNAY6*wRRD0M9gPPp zI4wcq`ucXRNZ!1=N#{&;p~tEgBTDG#JWY*Hf7ojCWBdS);?=R0D_aPxIaQP9&GOEK zf>6f6-nRT>)#?cEM9*6JVXrqHiOg&YU_JdiVc(`D{dKa74wYfsGM{$m^pKR9n z$1jFhiXM(W5z3Q^b2BT}INQx9Q66(-@X5fPoA1>hNreRGJ#^39Z05eY-((sZDpH;t z`%kI|oHfSnC%MJ9*@NoE@~kp(RU`Em$gJMBXn0L8r^4V#9)Wo4CvuWWasymgLNSMfqSZG1Qzd1lErS;>5&o-5IAjAk+V_T^8q8 zFJ-|`G;SL(+QJ_@dITv)jd9E_PU$N5_x9?ryfrc64rsgD5r>p1`FkB0Ubl4y%U=Nv znluG1aP$NAf(&q{oI8)>ZNuc6Vvo|#^`Zpj*;Zi+phupYIsL>=<%74z4b1^>Whws{n&Cpl+G*v+e;=ur zSaN{b2lN=zr=V55=H6-Rg%M0z%O|}M zc$M~IZMCML)%GBL3h!h(1+hTBTj!J85BN4O@OwA;_vY<9R_8mA(R1ZZEQ!+oig;5< z^{LaeueqGZ6pihDLCWo_Q=?CRg`H%%tgy6KtZA)<(}A>TLSw+|Le zGy}ao@5&%bJ_BREfH_fn@AdH2G%XXS8T<$68WD=1P|vkatfQxrR$eD%k{6@aigs4i z#`?%WZxYqlmYNJH#EQgrAn8_(m1+$=gioz#`Yq6YuT^sGv!o%wgaw&@7~m{S+qd*e z%YenNIV7E7q#U3@vJ=-gQK@Aa+|~HF#|+5fQYCr(h9@&9~I^cam$_S)dCg z?X4fH7^c+5&gi7K^IF#Wyz8eR#_%QW&c|WB2_0A?5uHq_I&ce*tXd>=gWF9>p z$VRoY5VW?iy#042YDk+VFzk(Mq zm$uLC6>bQRJ1p06{oK;Y>d~Z;=*l`JE%Nh0`=)m0T?*T7yzMJ+?vhuE{I(X~O?h$< zya-yLM~wm4!oXH2c?fC%C4Y_~2#wUfp9-8>2t+L74J$5dEJ>~r$DymO&{iE>xPU3m z5`{!oAM_47;{^s;xD!9oTV7f+cHZ@JxEeqfBKED0Y8g^+Pk4X=drx&hzz1*(=D0*D zAK>QFnd3#R;t^yAF8~VBD=ZrlCrm>l!rd%Yb0pi`VoG~n>^xL=*)!aCwywuQDvfZF zq6a>(d+xdu?os}x9E3YUtaSyaVo21pqrj&}BsbqrJh zC*qn(9(ER`jgAs#b(dMWJ)y*ear__~|T>_+>% z>jf*au&iI=9!c4*L|tmD?2dsm!Nv&QJrw|8)|3ViKm`4Y#qbKF@^Nd)j4P2II~0pw zpT`GV4kBTtLyi{)#zOiQ>lE>5696mnS_QnWcB|K_j`n=q*}B8UEkvbx5omJxP7(53 z{?&_W3UBy7Vi+XCE|!s%m|ehN+CSF%h*&0$6b6wBGuBp`)-osk+Dj)*PUwBM84SF4 zeS_Qg%>!eNH4<>iwm~`%gqZBS?eFcoCIRs`!#BF~%!PZHn=~Y>O&23P|FL?4nuu{; z%CAD@BIF<6`dKTHB9IW@UgjrsA5(*tQ!`3GOt^SaO;b!n1_cg9I?rB-_DOIl`#irm z7=?h6c%PPZ3>?1T#B(#;F+BBCEy2&#yR=)Mav~Fg?(qtiyw}kv_ZvG4y-dGm4z%0w z7Jr}*%&A*)=4QpBA0P}h(^#5n51eSn%b$Ej=@pp)$Op?xDpVDY!*l9Xm@Agd*65pNYvde(@LxeyC`fFkCgP|8=-V4;j!w6t2FeEFs` zq2(8c(MSPagvh-{jqy1@d1Bw^`zjrrmKDRCI(`1H&zfJVKm$Wo?!p137)X5ki~_cf z{2+1eA3W&%myxuan~P89L%1Pr1;K18U0dnn&E+$JGk5!oI^~_hQ@2q(8DfS%1$c%B zy&gTw!+ciVGNlt$xdp2M^QlMQZVHKahL#%sqCO4vV0`gjz}?tNIrW#1+=%jRuv)js ztta@Gn?-I!QAjs4ipc8_v4Q?6^5j$-b(NA<8CtQ*pUov;v6|m}bu5W&zbeuntLUO# zTe9EHr~XK)UKDKHDxwtYg2b(vq_Co57 zKLGi+wDV9`x!u0sM?s7i4n3LW%V~>MtP9mamnh+-r$CcNGlXEG3^%s!0o|bA+Movc z`CXyCltUPS_#b1}9Z&W5{}B>G$;xb!5oKj7l_W07wO97uba8Q$G>kGX5pE&MUf14b zuW;>|z4uQ=`l>bSJcBQ)@*?>}$Wa{H7In!mU# zhV*=*(a}`N1UU6L@O;IJ%*-9p8d>}~L|;a!%L+B6vPMsy&Z5=m8qkESNPF@Wf7!Y( z2AdvRu_YoLJZx#wwn>!JhL#%tn-sr~SopD3h)I3?Pc1*yH%z_P1Oq$VL=e zcdwU!5jXE;ZOBlY&tTTCO}%Jdzj#Stj(j~mzP)3P%o>E-=P%SP=!FhHPUa0@Y?m;1B_yce z#O8`_3?!^&MLTNw{{b~%vtzFKSaS4kC_2h@lyY;-)>}mfzS?YhhQ;JBcTMb`aL2z) z(a~fvoB2_0C*@7q#o^jZv&K8_=7gmBZqXXye#9hWVVpgcMiIJt(b4{Sza1gi;x1-k zvM=0Af8DFtztzpdKaJl3P_dqrBvO2%PM%8so;pe0_6S4Rou0h6Tmi2DSxyQ;$KabG ztI-qoTEElLHww}W9*m^htFu#eT?v>nUw*3p z(O0&g`_mGJp5F%dGvdO2fD;+yrp>d`8hP3% z8ZuK0Nr&fdh;9ZfM}ouQaKIV^;(zyb_#&4rz(RWEp2dJTqdPOLaxkA z3%A5=@|A;nSECncFgNsiw*2yM1}6!y@^$n-A;^QjHmw_vE==R2axkKlh^f-uqG2uN z&BrlYxTcf~-=T+HQKPwgXj^zv6T`u+-+2$QQScUL@SXcs%D;-Cb=o3e5{9XCL^CJP zTKJPBSeQrHt5=f}+@#aU8;BFHxg;cT`?rK;`&CIC8%FUP9p4bagTL~}H?*gD<~{Xb z-K0ch9Qre9#TM57rX=)8{kmo6_Yiu%*8;wshH{k$CqY;JEA~y{q4%_e?LVc>?r^*~ zm1i0dJ(f#)$e%-h4t=R%t+T0F4p<7R3%2hzCY#< zv3D znT)+sNVYXg>m6D(<|n2w=z%44a^Y;0Fqr*0F~6@%m>i0kL^3yjG&Zn z=IeO5GfBe?%cLBYu(~FPw!A{Z{8^Yz+Y-f4?Z5Lh$7A48s2)R z*2;;fxT2#$bh)Q{OFG-znoiy~-O}P$K5V?l)SB!%<0BDew4tl$^x>@~QGlmv0)e67r5#u!+Ez%#x534~X*e^rR|V_) zvDxH(W>Tp0=Kr8#2e#FDALfW?INb)vbc1bPwEk(ymfpAAJ@%TurX;Ib7?~VH@YM95&|e-;mK|9gTxnrKG^oENN<^`ZG`P zmpQH^zC(HQxr{WU;7lw~H-UFuwVEzj>;W~G_Vgp>433dn64Q$IzSGb5I&Z2uPtIR9>J_RL%J=nKeBODV3(xcU z3n#LF-^~ZCikJ9$r}3+DX_s=;HJ7O`q9p)O|#P`pC`?qgOUw>CvjN;AsAC&{a?~ok8TjlWFj%suJx99$O zD+Tbh+c*B(ERbj{HohcsV7mUFhcp+Iw&NaVXx@%J|J(8Y-%l801(JqWnQ==c_Vxz^ zKC?$J`;N=;6*DjdaLt(Wc8mo65QtO$Di9}>7k1{Q#YFuNEr5fL?LSt4hZ%8|xwUwr zylMTCpAb<}vScXEP=?=Aa{bQm^QPAH1$QhfQZnmK)?o#AjKRTYubIi!{ofpnANBxh z+Q$sijGLkW5yrgj3LKxPo~?OQCPJ&SK9|%d^@Sww3Dr6B#H!Jw)Aa4)Q(=nm*7}@k z{aX7^j{hk-_5JzN=M_ruR5z^q#GxHhK>v#fw6tj0bBwlj{&92|XH_IIKU zJMvCf{|^)~0SJl=#|3Y248D*O1`HsAU4igI-Ig%SmzWI8fPk$)E*tx6#Wz)C{}|%G z%`yUZV zw7}x7=EIy4kgouiRlQEvK_5+oTy_c>dmJ=W&Wvjf6orZ0(@4)*@%p00BmQg zMFDkh&rqQ~#-gWo5A9;F4Uw`?`X_jEz=5J95F4;q99@qnBj8_4{fP}|^`&XjPq8*h zLXfsyKw87RCtf^p*7UU}r~Zp~sH*ZU zP?oUhsbMBc;YgXr`f|i7I8mK}^6HlMcRPOHpI${EVA}YT(LWNn2lfe3M&= z``b*A_e$+4X6Q(etsz!ZITIT$+aT$e4s@jaOOdoLGPS~6Z z+Vn2P2P~@@d>=VJK92zDS3{*4miM-Ei=R^8=lSV){`{aT0|-z8+PHITy_hnLm z~=Sq|m+e3g+9r$N|o{>nP~ z7Uj<;99a2p;}RyC=`Yg2N%IOi^=qx#wRO9+3&Mv|^-h(+fOPSEw+E#~HNlv%Xh2;< z+F+WU<`bR(bAJu=K=g7NFYBK_{>L30kitZ-mr>%dI@=$lKiE@Ov$Va2@P1y|4E|`e z!E?w^tuj{48X2@EmnG@ttjQuV592!Pp%ZM#%c;7Sjgj_??#OTWLyYRzaZoSKGPh0s zt262<92io~rv|zov+YVnYW(YPf4%w_|SD%sIWxKJLNa*sRPVF-*&x zN;+X)H$P<5MGXofwYIm^W4@mz|ClKti}$ez4g^_0K^ntR&|Q2Pr+U7@sEd?XU|YYWE)15`_4?r8 ze>q4{)NBh;$Fa#2w^#XnjgBHnZTQ2-^9!=Gv$+juS!-D|LNlf}>@?dfgMgagr7p$3 zo8#+1#9P;<#ijA{Z2iBe#FpkC0qVj%tWz$(79H5J?j*tKk?e_NAiA2%Ri`T3E*%o& ztFf}0+_Kq{o62uY#8az>TdoKcE4S1`|7HF`^`h;(2Zf;LaAoGt>*ja!Rn^wBV$ z&vW9Wv8``nqCYn750^I~#WJTug0ptY^v}x-5cQ#^Bjc#>ta#`%8iw9V7;Ybxs-2$O zcB%9xsv_Y|Gr<50tbv2gRLI0n1wQm88M=d2528)f7v*qVG73;ZQOb`ik{J{S(a&j@<%=~2yMyXL|_ zY=rKlqp=A+EJb7LQbYEq^hoo&CJ}w#iQ`V!Dqq2C6dJX`H}C1CqXoo*!4J^t9=u-F_eedsAr9NXSx`k zG1cJ_a6DEZJ*imKJS%SRToB?S`E|3PAF0(? z{i`|8Lu58!A9)RYSpzK~^IXBI!3X>OMdl_zgH%E1Y7 zxXu^#0Up|Xq$Ye715XLTk@lq>@|unE*n^dn_gm)ljU5`#dN=?wM{{cf9v;rEa)O&I zA_RSYOkys@PeI;)zyS!gaPO+BD=KiTMuO(ab4Ba@%Z0wL>?~E1ScNL(6$0Sg>8TogX}Q+m?;(^UdVvj`mmoZ1|$ z@JGMkW{R%#xykBw4eO#&72Gy6Phy<5+DF#Ko9}ewnmLvTe>2M;HugeBQn30Yqb`zK zGhzFxjHS%d*;iXg{+fZefs8R%$h}C-EjenXXMX(o@3t+8qQZ9wNQUZ!!|0?s)u4-L6DTGoo9Rf^iHDdcrQeJn=-yHhH{~y<<2BmgdWuO&ej3sS z*)t2dnoMC*Y^puhlcbheI0sl~hXu{r*qC(__Lh`utnjyqt^cx#!SLf)bjwWrY%1pt zdm|pf4E2~pD%|}RvY`=U#>u;xTV3gjQPc{wb;(wFtICo1f1PU_gwgTlqRf^goClz1 zWseHf=7bp4jsIKg_Z zOG(|i&vBRCb7>No=o-hCskx5jzPTXIXv7XPkFMuH?SrkANvk1m8Ykdi3PeU;NEKSE zPjauxEh~5qnk6QWsSslOQ15>5P*+n}3(nda01d)N>k6;_a&rd^!~u@y)bxJh)s(x^ z4)G!UMAIOv&Kwh+>2%}e7k;PF;z&kD%DzKcrKLO&4%XF|7yoU$s^Oo=X@3);nJl0G zir=W7Hh$KIsy?32qLW9^pqj$D`Rd_w`SDyXppPzfYG(5_l1<%QRYi`y$Om7ZS^=-; z-13<^kN8FFM=*Akl4nt9JbBsfeCO@ioORrXQ#u#mpd+XaXiy%oE~C%3!98bOINDRg z8a;#VGNSi5bZ-->VKoVQ;!+9%Kj*x+Ili6jX?cf$2HW@D{UY(Rw%Z8!_S&l8eHArB zs?@cG3i5&qa*5vj7gB$n6iL*1ynLOGYMd+{mga{Gfd&{fhOg_Ph$+*Jvm%7G6*y(0 zt9+5CcghL>JV3tAtR#$RB3iwKXWdAlZ~wN-$Dx?SaEy#Zmn{>#?}U;~6go58(#myp zY7{KwuwgCex~erGebriB3)1uDV$n*A)!Iy#d!0^Kvr}pJeHDS&nAl`zszFCU-wBT;e@0u6Petw6$X-ZFe(NqG(1 z>mg?g(_R|*3bJ$~@mRRnW~}a2KfN9X^Ba6hg=P(~^3)lNyt!mDNuB>OZ!FDzlUkMPBF--JOQFgqc=g&XYUK<;6eIAfF zMK9LB7RppID)YFTMun#gEk#+?UQlRi0sRk?ChQ3YC(ntE-UiXrEAqWofqZJ+ar(Vo z0oc+=lJR%h-;urQc<*9X~5F=6YA7=(7I6M4`6C zB>CUGN&ZR1T?<@I^0vZ#y^~Xn&b_u)ADfC$le?J{X!D*t1!Oa0#+6sP&sfXF^YBu@t*3za7`zJ7Fo9MRS;DYA>OtJi`Ds z{RUq9w^c*+5lt}Nk}eR^dQ{TQp34}N%n&x>TDYVaNOt*O-Y1FL!FL!&)4lJ*J4%1d zKvgacML=&fO8ubIcesl7Z!eKtm(4fCVew)5>oj8boTM{rv#+ESNjHf-0Ta6n^g6|T zu_{IJm5I+WOq5UInGf-Y1xQZ%j>}cr!4NV-rwcakT9y=URq}D&N;?Tlh;VjW`6JTAE@!p0 z3tkSD~mclZyI44&!T{uU>rR53t$o} ze|N36GF=hI8vr>a$%FT&f%iSc(ZF$jUQvJc)E^NCc5;4B?Eb0ddR=s^t`rHO#r3^t zHGy}*f~5++%Ttvuev1r=2mbQw$7T%CM1DAzXF<23q4=NwcFl{N&TO{71-zU5k8Sl1>QTvq`rn9SSR1r z==a%B-LOu$YOQCZr`lK$u6M48Fwalzj~DbIOO2!9W5W5fGq0}v#cB_ZB;lnbuh-Ea zO;RtsR=guz6pFja4C88+C(l@{hnyTJ_bkul40^+vq@sb0%o1-Oqk8a%l{OO)f&Joz zQ+m;c;T7J0zpT$!G7*m=@rY>Bi7=D0C&CU!Dx(TaNDTBkvDeXMB)^q*pF%2;M>_8B ztIOi&nQ~Ob;n;l|UDIdP|IeIZx7&wOsWZ{63OuAIlkTC74cN7b!T>Gv$peweWn&Godh+38u{y z-ujTPbtQA89r=;ZfH4%qom%hEYDX`^MZO(k>Ud}K#;;RcD+CAEQ){*QGWY-RHkKes zhHB*dUP3<^jx)8hm+`sB%2*{@A5vi?#I^lWeDseuhm8}xJ@kDVBr}gp?4(Mu-f$}K zusBRhe$a6o9PuxAg8lZ{sbwr9j9_2$yjr`(*Z4C{uSFtae%Fj=b$!FB#qdKoU+ zc^3D2;sf0$Xnsyow~wUsgNzn&tKkd(|MFn{wu2#_78mN$wCZGDkyt^F<`xt!8tiSD z9U;%KKs)_D#Q=PPbG0UUhHAX{z10@f-!Yax=Umt22$`*rZj1UWf*N{ENI40)^FrUl z=Vt$}XA5x`-iE(O)jj_G=z!h=_FV;`9X+zq?BH}|v268MqmwGZ{zl5dT$%c?)sGW@ z?92lzd*CBWglClW(w)l@FSFSbnjaoJC(Du8@bFaTN;%7?v?Oso;+}H5gdk{!-agkr_d$3=*;@)@>%){|L& ze-ajG^~%6>xqetbP+n{;W9yw?ocKS_ou7{X$ND+Y;JH{;W@df~K;s==eWn!7iL7$$ zdViCeS!vZWi!&;5vvtYQk7|?=zqKn;8yZoW9N)KP1TT(LdavakJISZ-o3w~Ay3kNn zIQuR%q(kIp+@6|#EAkGiLi|xp!>zAQ62(Y#79Pr4)2(-Lc0~Wd{rjU=I^6ER!UT0M z)n-_3IFS#S_3vZYZ%>iv$!we>8`{rD&7ui@yrGD{WuV5b*BeVoXP zv%-!<%dEC}NTxW1EA$XBO&KHx8LN|(6ty@?@^AOGAB*zCSttd<&f05|O`alq1N++^bI)~l?yI?#lmGM_ z3hH>&3nnmA#^k{;0=)LXjuz>;1f3#yvWNHNXVRa}@yE~kb6&t7pfo6&GpETO?&p6> zHwyeY9#oJg0grlrSMyB*62khR@z0;zq>4`0Zb{Q0|6evcXom|;l{HctG8vmA<8E<5gYU&e6xz#Lo-CYmcqmtZ_U{V*6y}6 zk>J5pbaVP_R}+V9gjD}ta*OOzK}c~m^S7Y|M|NU-BV06VSY-L{xYF0AOr1lEl zYWrbCr!xxbKWi|_Kb48l!mg_t{z_3SP|1i}*m>V&NOGNYIb*Sm`R9uN{*oVd2fm$v zL4Eqmb@9`_ekbXAJdMSBZG|aS3QuR}3ynzPqP2r+Dw;C}v-UnVx-IHlof*llY|Hec zYXHIp7tk5k(xuCHy4}qyG~K;H+$XEiDb2wf_#c+@-Cw@AfqKE^nMqjPBcNYNx;_~n z?XvobS|ce<`IdnSNB9cpMB7SL6HKjXb2U0q+`OGL-nK3{>cN*I>v7!h8ZJOeU(QOIn^3IF7gLpZ6Thp$tW7(Bkpp32;bTODZ4~d$2e3X97 zeI&f}A66x{ zH4)?fsh~lir(O*7U$FW_rQ(1^d9a9VPEauulPRWS61`26qQu%J*u~# zRioi+uUcz8K<%w_9~NK7$|~o+SBh^GM6&Ge&U>tLESh4({Pvfl_j?cH_%jH#B4f_f1eDDH*Zb|QpQ&KzZ-b(N8*m4&xJfy#3gDL=> zA^HIJhM8Og=;64w%j7vh-BO(WmR#=Qjkggzde^=XBO)0u{8-y&Ln4O`NxH}YHBY_Q z(}QC#M?cGC#JSrJB)LE}Omqy7Hkopy){oQnC zfhV@VO^L&1LR=eh#rExZd%6MR)D%vWDi)EzSRz$`9oAxst>TKj!h06h2^^xG0@uP z!qogJb}V{VVm5SxMr)*9&6rWpq?u*C+dTCcPMAxu=?`Ct;-E;AM5XYA+K{O$s$YOK z>oP@lLiypb^ix%i^K8}>CM<{r-i}he%W^%=exrBMcG<9~;#ZB$ME}pJVn-3iYwSPW z>_zI@85#qTQr=sMm)OK91~nfT(}+Ia#w!(AG&gl>eaMG7{~S`aybp|MuAGE>C2`Dj zv>2N;m_vDy*{RDw68HN+LPy+0UVzDCXYrl~L}J%r!0F3VR$+qu3#QpuyNIgp;-Hus zwe4!oS$R`IZCc|Y&S!4=E&z3>+@Z%&TD`0_5?u-`F*_a4Jf@aGIahlRHrudM`%tjg z#*1FeC2Rf0-aNX;s@2fAT7umnJrV7*|puk zPfvflLH#h;Umm^X2;2GO3Z~+?ME+F6_9V4Ey*GFxCs6TJxXi9ImrAB0A1))+%`TK! zra(89aC(&>VxuVqbHXJj2SK`SSleUQk(rMzfCeNJn0dm+cGr5W=jBCa`Eu$m7UY5q z2_|(Rln=Q|!)kdf>8yn7%EXWgn;SrR?Kk5O^AJynfMjmbQG%74MrsXdQANOq5lkr5L-mS zW+b}F4)Ij$D=586Upjf&6XF|#UhdPqSPc5`wjW5s9d4Wj#r|`^M?0x|TbB{gIG87A zBQ-zcryhP7ybG&0+!+PXT*RbNy8p# z*tDIvt^&uBH^O^SwT?5>*MkdeU%(Dz z-5&RV=0W#XO?z`~<;y&1bllR(je-^Q*{T)F9>TX)7MC$^GB&Yi>>OZ!IDkxmzs|8m zd)dN2QYIj7rIs|xE6mAQGMffo-{6gw1d(-LxMxaEt2SOxyU~)mLq5OF`M=`gUxPH% z^W(HdRn&7Hl||~SGySgfV$9=41+sq365_3urHDiD0;jIs^4*$#?$UmYb=F_(X6NNX8wr zNOB9yJi;OT`7+C?*dL`n3UL9{5xwy=aA3ZszjXgmgec6;SOL|b2h4nhgQ>gLYls${ zy67B3%SghXF~AR(DyfWrLLtrorW{EEg{J8-*d&~~qb;YwpKq#vLR74t0DmUzcB~fu zGkMq{g1FGN7Wk7~u0lb@tftOCFv1@X{CZuSBAVE<>%qr?3TE4xLBe?S#H+q=1xOKC z8oowmxt;ml;K6yDPmN20X0J=)yv5Zxs>7Lhuc9pp~PGf#92r<*0 zR=-hvx|4>#L14e{{Vse5iEJrBHAkh~sH}?1b#1LhABvZ95*{T_$+Xuc_3w?@DmodY zvT5Aty3i7T+Cn*8<*uu?TDS*nTTqtUlwtF?Y?7o3E6RSzD4N^{@#u^Bs!OfYKujQ) zc}`&>^MK1(2kFhV&F9PNUL?uPxQ|yEU|ZJ?ASAziG(3%R&1Q5lkG~gNa--Xf_3;Kk z9#246Cf~4SY#y@c36TjL_~h{{?Lg?jf+xJr-ntn&$7oPAVh=6ob4`fVpUsfodMz3A zYQo;=Dq=0~`Lv9%%;o8-^nYKB<~w-#0kzbn4QCrw0oojT4#nMe6t4 zvIexoZ2QsH>KeFGc6QVvecWZJYcV%}+3xIh)T?n`>IJCW9?)+{zmId@BsH=VM2EH) zTp|p%3soginCI1oLI0B82Eh=^c}X1IJCB6F`6?7Ko{Nde1+~i&h#I71xa~47q$-uutqDbYjUhQ$Rvxr zk#C?zyUA{${&K0R^uzCCVq?bL=E^C2#iXwLx-u(cROYgX3ST5o>dgU%5a$pB&hSgj zaOg~&y(mW&3>x83;nYYKVLL+m_X8Uxa*PEhO@z^M><9WRDSM>T%R=ZFQe)~KC1Hn@ zjx6x~b{z4eUsreFi9naYHgb7EJyz!1&=79;wXwPlilAk*W;A-G9*LW+peH@QGX05kaM!v6(%aI9Zj>()v?Sj>n z4lnz2uAco<8I?Ic+w4K#7bE#B#h$o8ExCxqtvY@ybqyw1Z;9eETT4nq;;jf%^bD;a z`*`7bKheKBx@K+sm8N2Ed>r)s?)*)vcezYxMQmQOnUN?~?8ysJdBc|>pU4;AV?vGA zp9h?C!~!E)A)fFOb!{}PyW+uSNRDQ#RUlW*gD`I5Z-6D+L&=S8=%XT0fu*E73HFCX zB+|fThDk>~S>N6)i`nTleTcnP%fy){@V}re`69S|9h1#+w0)XqzGgZkT`AKdR%gXX z3R97i-BPD&hVKP)W>F&fwN)V;;~0|d@!24iRA^5I1nXp2@V1zqmdMuLJTEG!pmqWI zX(@J1-B5IayN4Ce13O>A(UQC#B1?qy&%$@-Is!u@^Bdm3bnni)9XY$;P8S5Yc?#cP}oy4zV>O*Fq*UhOEZ z0Zf6^9+&CladWBfXPSusuB&IujBUa8gGH%~1vmC&P9sxG*humOv%jghdQ^^_3z$X} zKmN-4$-M@2&6A;@FL>LkJjUE-q8koDC_l{yZY-XU9 zL)oZUp!Zsa6N<0%h`beMTmf~3BE7@_+Zoi zLSwYfTa)nxUSz@v?`NzE7jJeHIM08)gk@z|6|F*Zy3G>v_I9^fr&u0+-xVlx$}}Ho zEm!2H#*3P^>a1^WBYg~mKEVRz1M9c4GvxGv!6No4(n@6O_FG$e1L4j+728G~Tn|ur znMQnHZ2_oZ8|@RSmcAKF85mel4ybpplX6W@3Mp*h-fWJM4k*_;(!|IfVt!PTv)7dL zF9&2Z7m-7-9#UO-{uxJ#4JE6*12$T51-Gu<+nxYm`an(?9Drry&|fxqQUc>OtUa*~ zuEM#9Lx=Mr;+r2q_}0&rR*HvZ_sqM!*ues29XcL;mT6^#@@3vYio?J43Pg1AQG~GL zhaQ6>#Q)!Dvm&B>FHM>!Q)vm09EF&{1gzP+D2HS zUxi{>cXL2EDl*990L+i5#Shn!yFoN zZu%YDZ;Zz*1Y&a6ZYnHwumC0v>DIF3>A-^g2xL z1a?jemSETigqui8-lFgbR8;i{1nov{IaeGN%=G}B;M~2hCSUGM)setnvkCyD(#8t| zVsCvQzMdr$-_VVxI+A^35zL#`S3oTqsDdS5`;>cN&z@t|C7-@MXM~ya?l?MBp+fiJ zUQAgrWNAti^l zBIR6;xbrV9tP9VD8YDjPG9Nyn$5C^$#X8qtIj~S&jUxcoqWSWlx!8P4+&-lVBIYmI zW=q}B_3ASL4!lP7Hh@l>NM61TE_1tPu4Ao$r#NqwKY5-RC!w?mz}5xZa|KPwIsGp3 z980E5UW7-N1TL2B*p)m)J=}!?@>hw=AeM63@4jvM73{jl22WnfX}c?%hb8A7UNPk_ z1G0COsdyVe0cKNDBAGT}@FVtK77&jkX-S?`di5LD{yn_Sr$jiprx|yM4S8X5lk_jB z-HkfS+xND+UK^>?FVxSXa0aoEsT}k4L%uRG(~ESz$h2gnors5yY4JB(P4M&FO=)W{ zDApDp@NF8O$?353)5RKUkk#D+nlZP`MY|6ju|_+?nw90cQW*Wn+pB5hCChKG{)z*C zteBg;WXDdNDy{;}3;>hXX?E2S+!su5ND>^uQpNX-uI}&c!X*clRkTKpnxbY4EP$ix z_n*o%YFHWt9P#cPQ^PX;cGU{&b&jhgd5OEGJ%Fyto;wfPfG)W(GBOsYvSazRB7kfQ zXF;tPD9P-Z@n8FGHdvaO@bhb>uDROKi{rN+ z!<~n83&u#7(~>UT>8Q2UepK9bCnHYaZpT>)=6R#0mOZ0kO?Ng=7Gyjo$l-{j$=YWF zQS7ru&oEfS3B}(iF zmQmG<>zR*0l}el&?Phw=mZ0c9Dp7-l(*dzrJ7=+yUc_=BG1=3xEJ+zRTYV6~0%L^a zOf&D~BF3=%UT4r#hDbx(tKXM%nioG`qbI+O(Ps)x@C_L#habijeZk; zbY#NB!`xQ;Lbc?(VR2|ck2;-Z;2~W0uSVW{8qaV8p2l+hRN8=6-E(A(l|?Jb$ra>U zfrZ`ZZ9Os0jus<$@Q+~b$G(t6i4jexr%72m4Rznn>A>R3fS)P|$ddAcSzA$ocuT=q zmj|p0X2)CrB~xWbNp}G2RXq@iLKBa@&`a>KSH^A>DCnWW!G|QhfL%VWvPCcMmP>y8 z9?zQ(S5pk6F{I2FU^ICXEj{gtdrD$h}RfqF5pzpMbXxp!`RzIZ!DMUi3b01E=5rH#BsT^#nM7`^9c5Oivr-qc06;W(ZtDtr4MyJVD6Ys}GHu_Uc)6zg zBS((-nq)B!^7%{I_oIA|e4VjbDHLZ3ot`dg!n>qHZPd(Rc=-y3--nEt_jaDcq13`=e*YfuX%&f(79y?lvWJ)J~?(~$IO%0xREctX%?^P z5lHeq$L1{}SEyBWaZA`y(6zb*m{!pW@w~{Ilg%kEuyT7dFX>EifD~<7%8>h zjw$5yl|+kifA)x^@LuN+3r0=nXEd)*sg2&u@!9KCt^LhzOr*b^GQu_4L0jsqK(7$e z)*a#wa2%R5x-^ZjcOfb{$}Ztj?VX@TBZE&wCLd-1bNE1)d9v@pOs;l7a#FF(@U0&g z_wTm|FdAR-%;Ccg&>YvhdVRIwuR2PmYw}$dd2qQ*5L%H$tMQ{^l!vC--|Ndr< zm{CAz@@ZYcYvqgdTqLvcA)^p=jqD{COH8-Kiqh|QtORe*Mh*>_)%)qDg*c%{js@Zk zXA*CARKE9C6Is9H-w@^nM{{I zTusjy4Kbju4vP)#cs!|Qg?11W_~rLtH$Xo=RkNazZ8d5|B#C~&Zz~004s2=&b^doTEumDmjQCpxnk?R zt=L$94uq=CEs394XEjAY`geU`e|NyBdmNO)go%LS!(JeVC>&6DcNFB>UvsM}!cz*D zZa0~M%(4@<)(+cisRug}VwL0W?9M{Jq8iRxrr%*ccM zLsrT~EpqHSt+v{!$DC%)WnyPJq?@Bl?}quYv)@i9-Xp#s%Y}{0-3<%Mybf=_p&~LV z^Q1g~$0k60D`C1LHCpE>#4$N32UMR4TV4Xk?0P=<^X|}Wr5PYdJ!oM3#+%yi$9+?$ z!MZm1f;)c+xNSPgTO$alZ2&>RAA3i1W9Db0T>$-s9gAF6?LZ;jYmw1lB{v)0nO1Dc z7s$u+@4JB6eGMJb00B$UEQk;5SLX)?4j;W(yzxc*-69sZFAXU1vQzGIp%S6qeC3eQzvDrYHNpGI1}IX(VGVe>Nk@*&+-O^)CJ@g zF{vd=#54@n{v_tt%a*$Db4BN5ySA0vt3}u%?+DwvK8P~0lyP~($>{?>+p)-_rV5R? z%as9IGLk3(vW>;wt0I?eN;cE8HQ0M?o3vz^SvQl)RVRQP%#oJ!CN_z!zs7d;?!_MS z*y`GXLu6MWKu`vCw048yQC?OVAI#l5Ut~ou87n&jZI-r0}o&(>^ zikMgwzz@Cd+q2_d$FNCf4`w+;0Jg5L?{XdIFjCva)EO+`rjQ4~ZLFG6SLc8mtr_T{~2aWP7`%DFf6mAL=+Z`IR&vbemWCS?-01MoHhEnCPVAJN9RS6Uq^K_ z5uWgPvs~Tf^F4Tw@v^nRXefGl*lY^3c7dKI!klBL#rTu<$pI`~s111*5*Rlb-b;U= zq>}HcXQY>Ef2^LY*d4cvX5;O4!Vsah#81#piR7eV!{XQwj9@Ysvgb-*{5kX1TtGvW zf+fYHo5$0vJGmih@Z||_;VCJ}IJOro&02fax3TLyN<1p5GhBfAcg`P(5Y2IH|o@^;{LQRL{%ZK-yM2Bi+DVfqS$NHf?z z!lrk3uYkbI0b2t}eybN$vb`RA6S9{jJg~byFh71FIs~c%cZ`-z@vJ;{=H^B^pR`{A zprqttFZ`+6Y##30Jp=gwn5wm=+ri%5ZN>LC^d$u0@n#q)S@moMcZ-ag^XJXfQU++y zI97D-ga?X7s-;S;o~cH~Lqw)K<>>18X#mKtCah*@1QCV>PR5UByljWcq!jmh9w+#| zEm7OY*y@Bl`%Kzl@kI`NNfzTVdSa=Z=%BV#PU-%!4t5WRe%j&@Y1JINaLrj$OH;e) zVE~r?jQ8_}5DRXJ5iurPE3J*6X17i`u*p%^4d+{Yyo?#P|8iFCihG>h-Tkv_7OXPy zOllSqiB_b1QRjlxv|wy?i5gi^J3J9@Bmz0SGZeUkJyL31omo&GL}L3p%aPFRf`xLD zJ}k_!2M(-!tW<7fYY70(iZnGgr}m!UjkL`JdEIbVKo8it?fJpnump?DVcQiHr&^d9 zUbW^Ea!D; zNRU;teQ{Pa@Y)D9Y8`rR&WL)u?2vP+SBqc#*rEy1l63Apj%2jOe2HmR<4^noF_3h3 z5`mm8E@kVh<1Rz8)a5R!jb=gG(1PrcA>MJu(Sv-M6ZkW3nI{?tuZNz}?O?^o-`XuV zZ;stF>RGuTSvWftYEZG&(ap3Sq?~H#*2T*lwB<9)&N`{OFFM=u{+t-7KD6qw%vnwB zd|v{^*;x$mm?0Ayp|F;B$M(jULqv*{uW=NB1{L>sL#RBC*7(lA=`fmNL(v6uH=lC5Pl9Xt+@{ zfTs#-O?z}KHxqGg5*#*rJ_L&R%bA<7AgsHg#O7rXyUcq*$!0K#q2nkF8RopaI-=Q4 zfLZ{_p=zJZI(In7kKXhKU=sXLaZ9*HX)*yh+so8oxd zv`~#h!^@B@6YO7b+)ODg0-Y&D z#3|$x`*uNZgoU16Y~B3fL+myXVNlBFBLXe}k?1WglQ-V4(PfzSVs(VP1|bKJ!f@PT z^OOtmVC$48$;DjV_^3&ryh>mYeZ5$^LP>2jl&kFuako<17j^UG(JaQa)z1c4T*j(+ z(@5~ag6@1Y9v(&O?A8N>SF+g6v9?%9edkvKzARTbmA{tXYS$kFn3l3`P_oS;kn# zG9>%H3}zUtlCX{ zC@!-^sp%`*+Q0R{ZebW0B;@7x0y92!h+7;e?{lbdj-_eI6RulkEkn^sZ$1TMok`>f zKIQiu_JLd~!IGn!jR6acIrDGV2yl1H{EZ`o_Hnv!NL_fv9eW*{B<=<131cg*bQkCqWssKqJqOJG^ zCZgxGtKs?hZQ58u3jKO1A)kJvBft6z^kI(Na_P+()(HbKEUD>-eOv@a1Tz zT`8L(oUjN0y4wLOTGSgfOw(b3PVUs^8^=w#=w`%?RsOu~H(ry@oFkj3obyqF)$G=~xt-0;b!JlQ7|@kH@=^z2l*V4U1Ze$GKY-_jK=k;wh<&D^hH zOk?{Ni(6VgpRbh`Y=Q>%CB&;TF^HU{gV<^K@r`Y!LQwuy?u{^q4@uJlIiC}!`_6&S zUqZy&ceEEA;Y@QxEfzZgqEh^rGo_7X3^#%O^`^ zZ_F&J`fZ`{jv()IImQL*s}QpEE&%W&-+4IQnNP`p=(!hAy21701XobLV|yKO{gogG%I?!@Qn5X!EaZ2LkEX#Q z3dkmv03T18E5p%XTqoJr5!`j)pS2`|6y|hzdrEuo)vaoWJ3HU3AAH6YneA{6nYU7k zmR|H;4>tdbVx1(k%S|7SwTOe}@TofVD}ML9knYC`EuZ&Krxg=KQ(}aeEzBfl4RbhG zMg1Cn00}Hgsnb`s2)oV_Dbb6i#p54lcuSkCyzpkBU!_p)7Q-rF1t@*PGxXwlFtFpOVjt2KD2(RAFFat)boH38q+$imx!Inr{I zY!#>^{83e4Xh5*>x-p}H6l1J|z`o7V_k-m=Qg(`0dQ8A_Ain@c6hs~$$YTO?8tE@Q z7f~H=Vzaf{onJyDz6hq6148G{vA0vecAlZtXPpD>H=&|$eTX?=JimPOI3HXUBFLJ$ zMB)mnsJXp&A06m}$Zk9;FjXSl;9Z!ZLwN?*t#w5?K=t%zvh@Llc|UKW{g=4fj>}Ll zvR%rg^bz9>#oZ$98)j2DgrhG(6*9t; zI`?Q^DlLYSS0*V|Zbj-y*%%C=Q!tPybUfla;YQwqer7Szt}Dp>Gs~#H1?s~L`Qx=q z)28tc7LE^j#m9(-#MCaK)J<}Xa%37Du)hA6#DDgDP^wZFrVm+|ys6x!g#KdIUSaRi zn~v7`=o+S*OIHmRbl@R_8QFs~fYIoT1+c5M5kO#vH#f%Qu+T0Qhqd`3*#{~?u-zvf z1mcgNtPkQ{mOdPWZ!YtBP)6dg;1+-A_gn4vcf)~k1~s)Z*`3(n z2TXfBAWk7^w?WgL6u*@o|333@i8`U;kEc|9ly13$R>70KxC9I&-FFQr_r&^Yokq~F z0{vzBh!9G0u&jzKzDIh(w0~Nzu0IH6G3`USeb;oBx-?((>4UjIR zNDV)~=&L>el{~;`h+l3M)F>Usu5F4AD1uI6OYEz&+#0<{?Rp1Z@+e{09p1j{_Z$wB zedYw*a9qSmz9r0ypb3dmlbzUk#A3YLJc4X^EMUgY)CFYSWT{XtQVg6U;NSUaSW(8cxc+~xZc4_Nic@MCEI61~_cI7X_ zstyAtF1;?zQSt0Q;CpzQ(@67&s{W*C$V>nVfKcn&v7rUAlLQ`f;FKpZ} zv*k7-;=m6q04ibuOnl>%I(VFrxsB=$PmH`YUr0*>U9EWEOW--X-s4>s7!u^(Iwqk_)65&!+&l8`&EVfIwp5 z-Sd4I%}aR>V%CYqNP16TQbDPfK~_OS)9sdS@CHSgRJsq$MYQNr>6sJ^K`)lqr7K6_ z!x|jWco(ye9{O{7h@k7!WnqZmwB6eTu#s+3x93QVuJ2&Fj#e}?1kd;Yf&k6-q5e|F z&%#hX)r-M(z@Aw*LEP5oIg9MaixD$d43ZEsu^-5S$(E=yVLZq7($8Fz4=`L=)#k;` zU@vW$z;t@*_AXs0ddK$908U<)SAC_rcpg*tIju%oO}}2v8#QiV+N2=GyWb^-J$X7w zdAP$s2UogDD2>k-ulrq(48(PZT2bb~!s^pC1&W}cVtoKIeFOz*eWvvYb{_`6&;8~; z)kMI>WquLWvTszAJ26%p!W8<|0;rlo>A$l?{(#u#d|ak82;up^o(z|HECqKw1<#=4i428tGqa}8EPdvg-^w3|W|(R>Q^ zT%cM35>0H}y%jG+$(iv$SrnlsweP0-#;Sd1xNqxH%FgHvnnnsSp0{Pavfr{Jsnl-{ zEcBVy-KC|T?t3RS?daLN?exb`BC8PjUiZu;cyv?21z**x@Uy=)SYh%#TwgON#t5Ua zN%ty;a(n%r#nbwC3?oh6Sd1#&akxN{e<;thr`zh&;`QemNR|HeXQsu6Kmo7g(`V_{ zWTBLU^$VaXf7hIyJs0FoGe(b{udyA|wJD!?JcEj^l@+VHpC&EH=Jj#L_qJF`W9PuQ zbVIN^P4I8lf056Cyi(YcA0eirfS}lrrI4*zt^;0U638l`Wr-=pkf>7ybN(B{T^K1;-u4FXH*d`<9QW7q3>B&CFMSVjmf2u=u;12&2Nc!y4U^1yrhBi|{ z5XQXo{`Z=SvQL!}D>k>x7)3*(n$Mw8ZP(c;F@I(Oz`c}X9t7)IsQKginiaDj^#vf6 zoZ~yUTu=7l0s_RrG>ywOdnSZ+BOCdAH`9r3i3*)9&B9Z^--EMULGwG*C=~*oJjT0w zZpF@E&VH(O&ci|y9Q(mMeo(YA;=Qo5V zu(^Nli>(EB_%PwSHA0FwckNG5l-i9DkE^T@yC=7}5=ypb()aQWkI{)cAjK?js-W|_ zlwWwUC;lgT0oWba53A?tpF`7iD&x>~gjm!ZZmSBuT4+`aDWUp+t=!isgBjJ{op$?; z7*RGc+c||B2c&YBVLS2y#TloQAJps}_zWd+Z@`4ooeMQ1Dzki*VV}YZ#Bb9PsEzdyyLso$^j0(1_ zxGKMOYG@rKiSzgy)(=47ZDB>PeMy*Ux!Sf@kS@;&Fv67raEXk#TPI`mWk^AmQilF> zlh{=?TE?JrFf{{g$?E0+HImMme^t4yf6`<8OLr>?Ii=QJL3pKFbwztEN=rKHdlSYD z;K#FWl?j77x-Z)7=E^G9=%PTLp@edZE-!m#(<7^hhu?Azp!~M7228ol97>=;eS{*3)(bih9esQ0wV?fZPpB6;=|;$u+|%|m5ho#tKm{cXb`u4`F= zGK{U~Zbz@i#zi8}S6WMT!6mB7beEiKZ%aHQ`8AalxiRvYNnp4&3#=Lc_y} z%-iu_$_8Rm=*eFr(0eZLG3^|V0`gG%EKtdE~hi!8*Dyr=5{^-#y zOhs>L>+c8v17rZz&-c<#y@)p@-n-lo>pD{MTH5X1eZv!>ahI(Jh_CgOM@l|RHWgH# zH3SAdD<7^IvdPhw@!vbF@YQ;cUl7r6!ye3m9me$r#lg;AF61Tz`y?w^`q^+&U1SH= zFH_pL!2y79krhXdF+Wud`(B*UF)SWx6}{%kVOF{L@wSp*p1oJ3AVNqP7kPW;W--F0 z@h|Pj_^OdfyYIIgK;?-XO;N;q`q>DvpLN>!O@`}pMNih@P6awL8GxkNQlHU7TU!{e zw;~oM8>8+Hu#N6|MuWS3nvQAHbELcbbmDy`aGG1IM!&C<&-P+sC-1#b`Ml`IEng0jSz&Rb_Auvtu5M&WeGlnqlud$tdfDg_D^f zUB5A?Wv|IIE7$4*vW%)V57Ov8?NXS^FP(!m(+*?J_g$>V%MJ9q>!ronoea;m*fzb% zTDpVRlo0yhJsFkGHD%FLg6>w;V{U3^o-&g_2o;z_hP^Ft{n5Ke$H@!O(@9O=&#llK zUA>JUjwmbk=Ly!R9ul+vtVoPhL9U6rj;rHtrrFo8fQkzyg13GA6Ak}X{{+fQ9eP&a z)Z5^(q1$xPL zI)LSEe-WwXbg|~=sSA|v*%M78BJ3-Uk3{4vOdFg&@#eUG9($LCcx;YC*A6jtn|qU%?=D6;;lHxp;yM5jh>s++9T>Q zt`O-EnL<6?5Ov>Og$*4qctR=~5V>!PEb}0ezM7vcsY(-E)|kdlM9s9T6(6GB z6+<+XQ0*%6+`}|;D)+F9!Ec(!o7_oxFRUzbOXmo?zHED}syrOdMB{tZPp6`r!T8U! zc%i~Qk0jfXcr2>C3)y*akBCuCamC&#a&%}ZE4A$`R^Pl-Q$BF*2lhQqWFV6Ju66BP zX+Kmv^(b9h+0rk>zNwgy4?or1FiHrK==O4Z6>LFC_ddJ72~`^G7inKMMS4E^i|NZR zPn~?Q)$f!JU0yJ^-w(2@yz<`1%nY^8%Kbt}!8S;dLbqNt2FVw{U1^xqy$>m~znMqI|R`gJnysh3{oOU+7+VjbTLHI&kO zzU+AVKeQv7l#Az+$_`nC^*F#?D7Sx!q}!bzg!^0w0zv&Co!+s-$A7tH0AH11_!8Vm z?&fzf_0bdH0tQb6^qC312K%IR^l5z*wJ59*NBj{|FGa{pF*+0e%sUE~dV0mrNx#Ef zzgtd_Z$fM``-!Zm?Q??}qF2&lp`cHZV2%tpm&t08tZkO=7NOr$rF;KeS4hE(6l${V(2;pInQH#jfpCeXbE@3vyFbO_kTOA<}&8X>1dhfZ@OmeZ*MH z+Hdy(d>*K2 z1*|Ui@2!eniXkL;z1$2lbek7G51Vr)zpl@#GZDpjjYcZ*&An12HcDBFdbj_>fdfC| zuQ{^(0BP=mb12uTCZ&ci(jYCWR?y{owH4mzb>V#nUa-f&Nh|s zX|Yu`_BM==`C!_1{YVK*2O>v#3O!Sj&PEj|XU_Hio#0BB?LkIQa*}KzYWT&$$ zeJDP}4^HDY;99c2G2VcH!*@xlDRaVdA{%+5(i?V$?D!;))jX3U)WENyx1fFO(LXOn zHaxH@Dyk}Os;tj*Kc*8k)i3SE$>Ei(TX4L;JCnLdXZRg62YTYtra7CV+1bZ>DY~A* zM;f2s6u3Y1N{5Li9#84cH5$`hkkAp_-&DzqDk#ho%kOqF)8K%$;H>yRKI_pxW&x4LW zHHl%FcG!P+48FrX3=TBcmj6Ps&iC@>9?1oRD6fAKrAoa12Exs zfxLVTK_4hh5FhZY-4+`F7mY?m3RqV9APeq~0fh9aKH#Q2n_oq>g_z_PD>1VGfMfRn z{2h!*rMFiy-Q06S35EGguBpe1wvFv0?@)jT@K3Z1<7p-M+qtP8 zfyaGtYkO|tBy)=ofsEup)k`|bG*uC8Rg}kI#TJW&_7|F9pTOGh>jSdIHtxtIEwxCS z=lLtg#2kL3`HoLAlv4Y+x3l1!O_h{_@wWs<4jjAio;==GJ7xC-Oi^}wCwd#+4PxvA z$mbM5H>5p)xq^2&RsXL+KDh6fx(Kd0vKYzNrDXyD+^z=a)x7}yy?>uT-XKlSJWSLG z=TEE1pXNVzc7ZYm_kiGK2+HvRWABZ|#{i6}N|g1(_yEi8e#q&=jdeadW+$tsCzvy= zabGpIZ`jSqGIfPeXmMhbJDsDgmiGmR(dp)7XQlIM8dw+Xi@sp^^lRA|UKF@%QIo$0vmYhuAD@gZX zy1Qt0G||!hhxNCuhe7h(>ri25Q`)3%KQuBa1GHu=z}35UPe6F|EPxYN+!E`*i(1Nh zEEBy9fqW>5k51iNj@lN{2O-W}XRY4_VGg#6)fgy00NAWsLB(RoTmU3=fpF;EN0R~vYU%vPYrQ`zZD`4%|6^O`~c$9G0)=NKR{L^ z_!BH2-vWbjs{D1vy$%2yb^w3 zu42VXjgEkxP^0*?l(PBhi6_N{NgZ;^v7JiJe==b9@1VgR2XI*=mCZJHKOzEaWILv? z$-}*);+fCkM}5E52Vi{>iR|jIAYtyg`6^4WDbuPv>`3QQ?9!z6#p~m5$eQ^}H`!Yn z-8mD|tzeVp7ER42pcuoKNar9q^-ZwHAAL{2P%f`xxLLwjNKcB@w>*X`P7!`vwVWYB z!94s@$~q4*J+?i>4Pa{T91UXTg5@^S@OPeWgN!aiX}}j+MZEv(PCCc8L%YPim&Xfx z;!t1U=>0k``XSHPH0LvDAr00LSL ziA5@bM~D9tS(wcRkR~3cC`vk|7_HZ*|)|y zGVkMxt-Yv%t~uZ`?W3m}?%x_P7*a?%KZCuv!6}sZX`|sHm+N=1D=_fTo)*X3*G<@i z<|Q?c)GYOtI&)U8B}tUVWZ_(+^WfoL8O6&><2@4AZ7Y|mB70i+yNE}KpNn6nvo&@6 z!qZK-l6uLIPT;NTJAXjcp(d5cVw4rL8zoc_sY72{jHXMG z_9`aL>I(dkn+JO|lddW&ploWm#^0^B!-van`DL6ypa$ZQ?UQ$4*wNEeYLH65&hjGL zNSO+V3k4)G&t$jG38Q1`E^QyK`o0^KTKE;b+s<~Phwn1JxbHed&XTEaNNlny|NtaF=tCZ+p6 zf3T1dtS1)=2?l%fmKnifCtt>g{nbS&>M3=xK;1ghsUU+gG(>tAAL1=xYs@taPyJshuJd@0V^eN%HgB1W<)wrVzp+`y@hP{;YPP# z{q@e~j6Ce{28cEoP>IN%rvu*WwuWZNZV!O0erDSk+x==ib5MctPur#;Ku#m=rne>I zAWE*W(+5`B7jxi5F9&%}n)B^yMB1aHxxf+xe*trtreGqFZse~6TH)NIJ*b#D;FmI3 zwyRSIw~wg;P!MUIRTw%wP!9AZI#x|I+f11uL%;w^d|TSMTY56}TqW>cS4A-Y`xt|+ zIgGb8nNx29)0p?kn#_o|87tnlz)X=dT#3Y#d}9i29?^yrBB75?!Q#| z;E$N7N*@#$rwg=+hZ29}S| zbg*23N#ebukT00zt|9$L@E>3DBCy$C>OLd5 zCUb7vI?}su2ssA#bU(_Q_c;@7!(;<%ZZB~n2Jy-MB8zoprXP}UUVY^>Wf|`(qh2aF zHyy$$j@E5LrQ7StHQ`4OhsR~~-T(}m_uV!we~j1+cW>~cvM9E+P-|vd#vhP_C3eE! zY?iBaiy+k|Q6iq*`2O+VW$BZ4AEeH}ULs0ClqiRQ!9=qom$K+2CTtK!uzD#!sw*yp z1k}iY_AOupyzC16NL6Jtcl3?CU`(R=;AcY!v#QT!5`b3zcmnJ zbRi58_zu+e?YqzA3L0L`p_5}Aweh9ogJHw_EO%fr1=l%|1SGg{_}7F-Ao8TcUWjHv ze5V1f35ez*OQ(V03O{zSE&}qGGGzNt?K@r2J_f;T4O{n%#6`=Gk#)9pZ$bJBy2;?` zh<9KO_A=04NgxErunKU<5Px6?P)qnv-FAog3-5v~-H)B~(;{0%Fto;&p0E(g)?vG@-7hKHMs;F=@Z)l_o<5>MbxD zfAd;x-xvJRp#!`Pg8GB0lW9mBe5MyG#bGm;mKV_1^0oHqfu&vsfmiA~0v1pHaS*Jc zfU{q+e!+aYRQ9(@_Ir;Y`_|V8o9LRwt``iAg6#sSry># zyO&%U@CO$d4LEqs{IO5SmeH{1D?HEZ|F?&CK4`tNFOX<~=Ba4Y5_o#?bDj^|kU{Wg zf?81E%q#EJ+2qL;P#`7{ukU0qP)NXV0|= z;5X6@3#xkdE*09ZoG~9!{F$46z(5|r;)N-rm;S932i@qtp!^%$74iBu{)BOzVnr!X zSozWq9dKu_Y%9Lr+a-Nm=K9N(&{7K*wPKt4xZjc6v%7<4Ilo4;(j6OcEOt(sI@`(L zgl}<;tjj?owc(eS@QXSWjrVWelkLjw4Zx%{9`(da4=oGk23_s>vi-CDs1t-SU$+DysI!@v6|EJaY-Xj(;J?|A`>IE_6C6_bAisiQ%HV0 ze`&fOMC5-8cv-$yj+O|a@$(h4pw|n%129~DNoAN~Lv9gI|EW9eeW8WDf3PcerSX;+ z`Rhr1jv_|h_oBuFXhcD(f5B0I0eG$>j&V667?WKxwtUd*?iAq zW|d1P+3RV;67fr|pGFRBgtxW5jDM-GudT1W z<(uMi>cW@M84o7E;@pRyE{ie?h}i1taCSbSJsv6d`mit;$a49hJlzh4e}1{Ke>@;5 zvHgjMuE|<@mhw6JJ=}qG-+%c_FQ~&VU90D=6`{YahS^onF~<_-bqmp$JJ~y#)o|I2 zeT2e&bKi-d|A%&@RrG-;N8j86FEXHVAk=h;AQPwGFGdStURN1GG~Tgm5Zb@4!_HA| z(Q<7pfO#wNV*_F=>21<0UcV7^LzqBRKP z?lIMUO1Tn7LOIniYl2TVn$VclSks+L%`7s*{slkY(in0FWu=h*>m^@Tk?#WUo0P2! zR~KaxWid&%z!xXEo88th8Blb~agKR1Qa)`XD=g_9OdbF;+hOvg1equ3( zm-YnKNX^d;opN(fl78Aa6`gSp>3HYc-IP!LySl<_$w(lA&cwZ>(k?8Zi&07BAeU+M z(;XIWAp!^y#1sm4S!wZKSF+~@w=c)>Q9g}&+3vG9E$(RPYNgyn3p4ko={`BYxQ|>34HU?R;Putv&n!7t8jgvHcS}gUJ2ycHm z2;c(c!viZbi!Nm~pZ?Izz8Gse6I*sk+tMvI_5%w+-!Q{$;7e_>Rti?;X2{O(r%|3R zXn}2yDf9C!Tm6&t`}cMG9FG+~dOkq8|84GCt~YyS*X4Cbnd$mQ`#&qT6BG?V%xSZ`f(dFFbpZua5Y&Zsp?V(e_IR#$qQGQnb%okSo#5V_IUWN7=?^ zJWdH0xD42FRiXfYm1lHXFfJ#oNrhT&^ApZW^(O?~4^jc3~?-&WnHPO^O-u25ed28q@#x$!(dXM>*jP5E|PLr-+CkGx`^X! zK~4u>)QmUS$i?{J0xu!Q90P-acH!uZ z(Ys3mH+$C3iZXqF8GZyA=w4k{ke_REx7;}hp}H%5E^1|c&Q0Z@AL^+D4yYbVoL*nm zPXj@T->TZyvKro;%eD22126@}k156j*r3I4+MLS;Y~!=kxG}s4QTRYt;WZ0zgvvy5 z#{qhpAmFmkX1@tPp3p;o_%-yS$Fl=vzZNrNBqbFyMa}3HxuV%(LbI~-vn%pc^o=zF z6h|!lVh9IvaaY1L(VfMZ*A&*6II^}*SE(GiA!z$)RiEDfI6$Fx<89*O0{9(@R{ZU} zzV<@FPh*{iMz5ZFOvdi zWraKG2$;1V$4%MUL`7f>9cf`cA#HWnV3)OeTYyK;ET0_Z?=9~sL(*h3X|-yVq7p_d z%|^k!Cv&zLvFvPs(dt6V5~JAc+;%IG1uW6^O=jtNd2x?fBixZNC(r?|FX~n;#>|4l zd6=E9au?}mz9q8kkqZo87ea)ZODNrKXS82>o}tKTj1)=G?RQ3E9TH!!iLD7+^=WDbIB%T#gj?wj!$SmNBeF}TzsM}n2MU3UNqmes`3(gqced3;3#FvSWO zs|3i+K;;a~(FG`c3%CBR0t;FfJ&5bTPJ6b=wZ00{i28ly;aZ?sFC}qa+~~q>5=byd zu$5^uKWM1nk=S5QnuN6p(K-veG4c5!#yoBIL^gJj1MIBD5A(YK7v0OS8yoY%a?@iu zMUKG9g`{Ia{L2%}aHvuzAlkbBo3u`sHbpjysNLbzx~`}4Fr$t%?8(X5hZyV-xT~y= z83$f&yN~lKz!D&L7aGP@{K~(=V$kf~?kTfdSaAi*$JH0<(Z?FwyE7chiCr+fj&GuQ zS&wc%86|(W>VAU2`SAq1w>W-c!>erj*uh<(Fl6Yxlg5F+SklbhFtG3kwK zJ~y2xCHq@G-M-;L4ToCSrIO2crgD}b>es7AMGKrDQ8YklXZmiLEdGgengwnINdu`_ zQafx9Cou`hN3;<7sjh60ps6JeI^C-|v-F;Td3#w#N0)uN^w1ob{*gUsQjjJVD4xc} znG6D|+DKK6=jgmdC~{33wPV$1S{*bduADPkisb6zcmcl)4iq{z)>o?w_PWDW*@ zBWYw`^q~*ZYeGu4EN7&4@E7vh00c76;I=mX52p|0wHcHSX)L9PJAygf^?FEKZ)0xm zX&n}|mT7D)&-p(m%$u#S`E=~c0FMlJt6b3Q)mo6)qIlg@h|yu%9j2g`GqrGgH0h{5 zTw2-rV8BUMy6f$b1}M8Y(H!>&HaW_)%nfB#U8e3hsWc{(7D=$7EOw^f|Fv+2avHC) z{CYzsX7TC!EIFFZY7JXeD~Ur*?lFXQlRl_IMJJI zV5h4%*eL*QMx?0tOqyDFGt7HbEpcpl-h+*oIGIx0<1mhXQ-)PW5$8LTY0J+{fnaDL zRSyiuXdH}VU^5xS6!xe)X7$#ia?05McK=EB2ueRF?`U;$uxX!n`%MPLrEGT&oy$u4 z!WEV$b<6?iU-6n<$04hl36~4s%YfgCndjKRN)p)6m@OSXd?LtfU-$Y_Y@N$FeqEy| zOg`%GvBU60B@M6mN2sEJ{emdjn5Pzv2o)nsCpT(|Y|Um~p8ZDQ=eh$;;LE;I*q9U6 zbHL2#1bdfL_-{XjyVWlUj2K+4b`D}mG>0wEfh}sm)*^d9n&j`373K#<&xMfjK|k`E+tz1cJ#7X+N@fA`U87bePvx!M;Cp>&MN2LaPwm)V^0UKjG;P3 z0p|Mfro6BZ@5yEXX3v{N1T_a5hy961`DQhTGa75W&T?^z;Rz`Jx7DlDc@L23~@s#b`0{g7CMi-7Gkuq%iB#Dzeq zx+(03sfCLT+PEZhjv%T$h&Fs4pU&5>`EPIstY>0pkQIxKn@$z(yXCcWXaKCDjmp6W z5@meQBV6@lACmD*{m`8w>FQ=$m}Oq~1am?GsU;%t7l(M;&1bE8W=bG-)6jMS-=`)RVK zbjnGrtIx|o+J?OAeRuXWsgT#r*x_)2jPx9}B6L)@OxnnWpOlV!EjEeE6Bd=V0I>%Y z!*B3dIOO}K=63=ESM6?oKPTulZ)GQanEYr&XJq~E*v`nbHZW5qbgA;J# zGBd7-eT{j&_}0(Yq$A&~3~E8r<2|xT|_ygjV<0Xv-^}(+9WS zSZEd)=DWM~E=yH~y6_t;agXMnca3f~dqLJ-nvWV`|Bgrf?5+O&RUq5hUY5guj?_JY zeS}`GH^Qwbzq8&6Gts;sA_z1J8!enQ#v?%S(}O<7+|75Hn_D*n7`+!JGLXE4 zkrm?i->ra?+1Gw+T>pKOzD3KRPDaRnQ~blOyiVhZgF6`NkV#co& z4#RnugxQs7|K0Zf^X;l1FfQx!MbL51wgk5QcCBC!M)83h3pYH+iz|2K>9rGg`sB>% z_L@*x?ey1+F($+n1SwZSo3LtUQn}n)mxQS6I{G`X^fOD!$1!Z|s?NV{4KNW^Q%T4< zt_-Y+7EK^RbdfSJV8lSRSaY&e7Ze&}Xwo3qL5c-eP0beRLlnR+TBZPls@KJ^_G-ayAal7oq>t?m z^53TgJ35_F6^Ax($*qhbsF*9*amDpYkhiQ@8ZOmm@$>eafAU`;=kU3C!0f_UxmS?i zZV)!z<_u(vyw(Q+0tHwCTxIDL`Xm5sETqI%9L z@g5;CecPtCz&F0~)jnXWM=Z`b24JNI19$exZ4S#x!rFU*Ub7$aYXpf|h$QB)l%wX? z=r^vwl~6#MKOJ5Mt@~)*76_6Fx@E5?I*2DA;|Y)fvVA}+n>i);Mzj+6gSg18_q6nZ zZOV~$MctQCo5PE*moN*HRtI5Jnd|+VQ*{xcnr(9jM*e-Rw?pI!KJ?n;s4wp&ck`*E zoC=%B#wm&Z4=t7i5D42|GPM}R3K||=Bq}>O#xU3a-*xvv`N27QqCzK#Bf4^xMaa)- zQcE9#d6!aFE9xt^XmhI`TOcROjxtG(bFcQa$d>5OLsq|7u+^#`GnGGugZc~)U@Ifp zk5n^-Xk5VB>_Zwe&%lS4WxH1Z(rm3k4=80c2_9 zI(&=5m4)4J0=a<2Tf{m+2nzf&IDnIL^JR;T0_z6?F~cNCmR?q~PvC8{IH`{$@gPb? z%n9;Q_5t!#cHtmtdLF)m0SEttuVT2VQ!0f*I5f40&h2BR%Jx*+sB?F)(`N7c{c^^d2o^ye>iQ;UT-E6G5*JJW?Eqw7JtYB4%O7pWyLJ zuM~|$<>lM^+t&YHtuCLM@6c2$KUXx{S16hNpM$8bNzVoOU=1@)C-VlY;pNE`&*kPqF1JVnQZ`&id75F6g0%;^I6T`JR zA3Qr{x`mk&-^grs#~5x)gtr?m$8kLGd%z0p3Z9sKjNwUxnRIYamtTDW$VjDso)tKv z{u@BChhNI z5OK_gCP@N)>b6EwoTqrlUmFuIxfFxeR3g@VS-RdQZ3nmOx40{IF)fuNtlO{qiQ9;0 zi>ZvUC5py7_9=O)2Bcl5`RdXzk|KvK{dr%)-Fxzy%T=zLQsmrD{Sz9;Gv(VzQ?C~t z@*nDlVddL=z9^0?TL1SWAWSxnzCQXBO$ciHv(>IYC%Y%a|D!#Wsgzp zNUY(;^~8rE#fP|;U;Ww9hlRZa>XSiQs>Kz zeKA}z&Uz8YJI#Y}>-}@o1A&maPoE2X-AL~1EmP_p(kfVyS6K^aA9Lb#pwA6~K&U8{ zP?qmih^OY%J8=tgcL5!bg?5<67*^e(L6;$FdqW z5}scKH@p-2<&RyWWtqd5U|k(ohe3Xl-(m>a>=opnBi$Z97IZJDc=c04hp5e)9##~> z(4NYz6iO03lXU#5O*-XwU#{&3!Ao|A1I`wGgC;Glz>!KdBGcs1I1ffJQxe zc5OI<8R5J6H($zp&S=Ik=uK84s5G!NZhCV`ZVMlm;|Me8;Q1RdnyIv9>jy1I!`(`g zY~5AswKPhpT7Gr6e@2~s2QXRkS_R)~-Pv#W1L+duvH$5{j=q-P2)gYn`#$IVC8!dp ze!=)HnX72i2Y$j%5C9y5NKb2@5 zKBS^!|5@86ilR+crrqDmQpqXAAEz3H5iUDtsM%H#+So+pN3d%JJUX6bkF;>JSkjb& z4rVGWNY#O^W7th@;cvCI`gPmkUWn4RY;6h~#3wIJ)3=H(^kkN^eCFC(Q^Nf{4`_fc z#5|7<4JDN_=3y4MvZFnD|g+`1(8w&dYY(BL$zXc%Xc&086deq(pXn%=MTd2v` zsmq| z`79!4C+Xf7k&K5T@A^oW9j+1`AzGJ)du{hcs~jn=Z6yX~i;KCNcVPA>GHgv*HML3C zxrUPNP;w=*2L$iTXzLJ^G^7OPP=23J6{`Igy#{?WF%8Z>Aygpf<~zq58; zyf7pu2tjpH(}iFb0z6SFeej6`U%8m`q;?8}IZl;$|FL=20#C*3EsuY;P5;+dQRfai zT;8)3mGdrlJNofzbN9uuj3#>PamB`xHCZ&ZI9IgEWrluQAEF7pCmQcvf)`VhBp%yM zT5WQrq>OKoC97?>Fdj(ZeCdsn{qLukytfA!UHuECC)GUYJJRaEPPS^r0iX*4yaRSu zJz+EHmUGZ8d)aC344HduWt;XVs}<$yauKDon*D%(+;Uk`r!j4s$#+s0Mz$tF?vBl` z&ezsY2?sPH9B+|7n)JU^r`i0`Yz{_kPC&c;%CSJ}{7vor3St2WVW6etw931HyfOt1 zbbRwVvm|nwA(Q#J4G>MIC}A(WlM3tb$qa6*IAHrpfYxNHF4+w!WsR)!%nHbsUH&ma z{&#KpBP#y#7eS1S;RCQ+s}1`djS64C{u(YdJMZ(pwCi>~7$uny?|~v=flQoRVODc| zye`*E&?AO42JPwlR@16_2tQ|r(N%!#P)B77gu$fk5nQxC#sxN2Zo|taI?z(Ii~viC zj141~=n!+>cLfv26HD2(*lE{%I1tL~Ur48I|Bs&N*Ux(W%H5;vbXnaST}aa~XjU~L z;3hpL(c(lsKK2Glsz+0gS=n_yg=+b~4CqzFSjB93VxaOQvta|oaNgld`Yox%OU4nS zY+SIFjH(HdBl#W!v8IN!?&*n^(+L?xroIDtx&|kIv@id2-sIyLm$iCwgwZ0TST^CHfYcPgk3PL(qL)jdDKI9IYdrk^!d+TW&11ndWHmh zX~*_V^CnBl?O3um2rbUtZc1N%U4z|U1@zGl>ac5AhixA3@F4AzTF&3b zQDSKx_>~nO#a~J{AgTm}i!D%n$&8d9e(}p3{ntkUo>q_vBXMA`-#wr*KVoJ3=h}du z%ieBElqf6RE*ibILs#$OR0f}9v%8S!*w>=l$vEw;g^`RR6&2F15# zh1a6TAI{$bDGp5Dz#PsED6Uz2S_L5_K{p;QMB}V)>0I6_#s*%$qNWjkfg-X@chNr!% zab30nphgrthy$q=3vA`E5Y3+KMr0`i10QH0aX+YAl9A{Nq5kE7c~GEQLb+$Uis>qk zQkBffYJLV4^y^^L=?rAVBrZz+RCQ}g*9{Dq9fqD(Qh1}BPSpb6v|$_=K^M$E zKCJoxkStw_Q?CBUTl3@N{`(Sw9QKbNs#cq$O~IB-4R3991<-U?LAm6M!9tS%O0b33 zmAGdOjH=Br8_?1jqI`B(D72Vxd#DP^2npkY0|C(PlXXxr48b#yWGJ*l4nVWe*ye%` zt=m{Q6~DKdD@6YFgTBDjr3N}cWoNZZq0WYD@0ZNiylmS1229ULhz=L#zA zwI#?C35KJs$cixcFQ}o>DdNDC7BYBjbT$0%Nz4DlxNq_0&6z=QI$IuNws2@_M%vLj z`ucziKi;wo5|Ob@oyR~^XJ~rg2RVqaxBvsDw~)t|ze^%#4i@ZUSi??F2zzo-TZ4VE z5Wv9Ux7_B>Gh}D7bR3wYJYuFw4chRKps5hF28Ih!w7HJk>aDlkdC$`qYVe?aiulWr ziU13!>|ncv@z@+w_GQ?>I_MiDEA?B6xkeG(o8wAyll8^Ux7 z9NJ1d3%p`iha0XYG6NIJ`>R3BzSWRLAy}ue;$jdS4wXO?FB^zQ--4UCj+L~nh0k8d z#xmssYBz~g5AlvS7jR%7`ofqmSoie*|CAjS+A)JnY!|K;J^=&Ol&N* zz)gsd@$KBA2Y9h4{-q0Q9$)e1Pu=3Gp}D)i`5@HjmoBJA)XM;&+eY$gUKFG=s3$?b zBVY`*%D%WcV!HXEWdiuH!tuuY-S}!fH8Ws(7DZPY>AsZ|rtkdNN& zSD6RHTsMr=AaISE3GT?Of%0`fXuV}Se?toXmCXL*w|^a{_rBm@B}O;t;6}W=S^}qi za!l1UVc&8c&krCPW!10Q;(HIPNe|7=z_vGFMi6 z2%06$@n<(XLZN2v)`+8LExYd~J8A9ddf%ArnCwz~%Uh^{uLQb4ZIoOUSo?()0lk_3 z&J<3CJQa7Sb%HG6HtMikU|j&Tak=yX+%|3m=p?-YbgRI+j!@Oa-8e{ASkK;I6w!8Tz7_e9?;T`prW(Pt`B6#si}}dwFhAn$?ElBucYrmSHQ@@PsEF7>x@$q{ zhy#BtBJ7>pm`G`?== zf)0c�MP!CC23B0%#sc!+uEP*nKeuHawM65w7QgS)VqQAk0lCLpsTAQ&z6ROA&fv zwu0jMddR6+52)W81KVJ-`^dL7>sD$UJ9{~$VM-62_*u~5iWp{1ahDe(s%#fyvyd|u zv(Bxqu^mi@3i{+xbNMF}P$y_X@oUYylW8@-A2v|Q#$zvggh#SrcYrHe+5i01SBK3SuYKD=w#ZD zn4Eo9n)ZhI>PDtxMUw<`@9b(ya|i@uK{D>aoczCH7&y##6yo?D_nnTJ)hqgN8MP|X zG4-;ze=_-&V|K^T*KU9ts~V9FkWJslzK)(0Xqt)a9P|F+y+4N*A0QRgm14Wu+V&bbzb51_k zM=8rAubcg&2q1qS_XFzIy4#aO(C}`A*w&4n z=RAJYxH!H>BUH=zWCfIzyroc3>Ac~9O63PpTigqV@F;a26PJ_I5~5;GtwsAp*BVV> zMdR`lXmytd#`^4fj_Dn*r3U+7%UMJB&njg!^GKR;-5K>I$Hop&WE$*9DgqcYA$VLz zjY?9(5ewyLBjS5~CcHkw7N=TM4~;r}>HQ!`gIcEg-ZSH-Y{dfAjoG^%NOoZY;ME<) z>=0dg5YVG1nSpGYL?E)`!OiBIjKxe9 z%CAQ)fz?PDcJA54XLvt^V$DwBe{s?l8IDUrYxZq4f`)J4t%_~~snDl&np$j?*c=3@ zXf61Uds3e#hYIb{CSD~7Huw3ClGox^ zWjr3ayn3kRC2}dp&5qa7_&~SPTUdZ06hx{^!XrBB%RqP{@eGOGgHH!K6ne=s5c5Y` zNy=_?X*tspR&^;cl}|9X@aPW4&Y)uhbSc-EQhQ$WU1t$ld8**i;e_YlR3Er~R`Y%- zxTW3WD~wdM$adBgDPwrkouL8W$h-AmWDS~h)FQF%<{>B!~9xyFWtLvsD4RJT={wJm*(G8%)1tFaF`4$XKm{RGViEJ_3`t zki5WDI(*YyVyCx)DvRoNvtrOrgz;QHk#9>Wd1cSu{dy91yhpt@t7CAD(7S+I`lW%0 zyuj#)!OHqeR)ey(J@IIk@1R3a5ws?MT)96kR!t<4N__T8=KxJ$_y4%YR3{wzV)X}| zPqB^0V|O0A?w6;62YQO%1)!v7U6vO2B7mV}gN~X^OxnS>*&}|w)+p@XA6XrDz!rAo z%e#le@cM!-b?r@m5-Dm!ufVLQP=8=EU_i^Rr{l}YKd^s)C%8D#M$m7(@PGDe#O38C zFF_ruKX1j@9fvK8M&zIAm~hga;b<_AuxpJ$PyBCC39k#eD7n*;I~Zy8Jzh;lOU_RG zv%3|vS4^31@Rc9~WCn{Pzhp{DNc=1}fB*Y?x4_Hl@Z?mLD0#8A#Hk05!9)7zOU_>Y z_F>^%|KA_x_2J}uXPO&qcg*1p;~N)k8}cjh_1a}`LbiPGFEkD;qzx$0Wi<;3a|W0) z*E{gZ+a1AaP-2`}TWnsgigkNv}~U z*l*phwJRw+cW}AH_)GfhhacZ8boSoP8Cl&VW$SOAZNADZA;_(xpooel0y9ZBta~bM zvIfKTJkXb$X`HyHMoT~bZv5R7q}sUMI)j={*9Jxk%u75HvWZeu>uK(~j_QI~y)6f; zgQnM#wD^TqIk4RRJf-`dGkgKxmx7Hh&a6A?GCbmRzCmL3SUd^07G*+{0 z$l#Z=)>l&3Thf&_*jN*hEq=y0$B=$GI7(j?DHIsMQh1mE*Xi=%3oGgz9nDW4t}lem zmxQjo3Jbu>RGBlh3W_V#EeT(q+2eU4E_y-JBIKsth$C{uq>rbc`Pbs@yx0Wod*@bO zwmTJpw+4mMU(cZHE=J0>YvH!T4OUGH`93H$zh4Mj&ihDFlXFnMhFh-@MC8*ZqEA`#;bnr_Mt<6#)1B&Jnw|u4jrEPs z71wvXdj4i!#KmA~EsH*n#m=AgP!JJU4onAhCt=hc!u=L7Bo$56j2{U%3s;)h*`vuQ z&tV0Btg_{Ld4NEJof7(n$ql~vVRiGZ+k|6T%AMSzHKn{+5ns!#Dy3af>OyKU*9t89 zVX@XRHiSaC=1=t(Zhyav-_bO9$goeDw&whO)rz_a$SGAt7c2Uk*}>t&(6Qcb4ZM_8 zON6f9xutTu24%*dQTv-Y#$P<{-fQYCQ*f+FyWc@G6g=yz+R^LC&ncwzj z2UX!pC70^c@v%;NP3KR}y*_mbjic_iMp5}@?bg9IQTd0FcP_E$Np&_`1a#LcX*)W_ z2G6PM8&=6JU|=IIDnELFz=Kz_QY`gb|4kc@q2)ka{z|jGS}#0e6{_`#;v0(Z$3cOL zpijt0eEh@|kg_D9Fm9++vE+Mz-^x0bb25!#jP)N^OG&uFWn`7<|V(|JT_ z2DCI8gN1{d(z$+Ngvo|gtE`Sb$_;BV-{Z7)&sv-x`SQ&N(dL;pnxCA9H0-jsd|PVW z3Gd#`STf4gx^U#FAVz#Tgm)tYxTOxfKU>LW4HlcdNt-_GY1%PDhA2QlDkDSdo*)31 zYnr1dU7kU^<6`nif#*0Ebry9xr@TqHa+oilV;YArO+(75Q$=|;qu2eZ2%Of&zvY3! z?7Y^)qJsHS`xMxP33tAm$Zk}atdzv>uN%CUks02boiWZ8ur42(+-TeXl-4DI-iJ`bET00ESRs5+tK6X75@D5|y^BfQa-_bq_HpOo*;LQ<7c>h(a{XxfEZqlY8T)mLH ztB>frfZO2~&va&8#aBW63DV5YJ$dQI55Vtka_J zu@l$vy1AhPdJm^5`C5woKeb|o&&f^I*{>$4N>Cbd;(o~rq#c65w(6niU|GD}I0 zVKlIwb?H#mt{$`aN}+m%>dc&){43UJGz8TLIK$z_(R|J%vzUO_%Q|d-d+h|6IE-gz zKn)>28?EAuG65wC=jC>BD+V;`R%&_b$1Qp2lH4e9J4D}neQ7UM|^tuAY z?tC5#Q!}A$*o4XE(@mevS37)1NrTP1MG z12SM|J!F?xhx?zHDGrkY2s1jwU0hrI)i^?Ys>Q{O^eai}nawam{SxDedQB{;EXfT& zhw8#8H2K&V>hTG?Ty275o9Hlww^5&!b8Ix7m?y>xo*)2|a`am0&jvOi;UZWW_N`~g zZts}nDYh4je@L+1c?7RV6l4|l^f`DGOJCnCqZ@dKE1BWs?$QG^p541ycAV#MXAaFe zr)tRfnEgG3>ev%vP#1d%n))m0ABPGk{kEJ=pL{RN6@OJ1e* zpCLzc8SrbbIK(K0Hr5-Bx=FNjnPy3SawCLUFauwU(gwbkW<6Itq#s*6=%%QXiY_(r zErQw52U;L7xw)l6;{K|;-h9WgR^O+Rm0MoWMA9DUcl*??rVl4xm@6cir=j&GLPUH9 zi-qkz%h$1N{Uz=O6P<2zLbi&ldnM3;XfS2 zPPT{vKX`^AybfpM-$k2f=%y&EGn^1Wd}*GRdz0D3qvv{1aizJ{2=-Ew4^l(3*D=_w zx1Ll~@)UQ&xxn~B>D0?TcVe*QnQ6)PpBTlr?!V2(zoOB%F+$uso`Y>1IB0tRcP~B( z#JZQLBvz?R!`T%=F>`s^h8v{eL^h(0@U=rnl2MmD4KNSMeN(j7Rl9 zgIaXf;Jsu`JoTORui~Lez?WY2rddy{23@(@vu#8_-CB**8=AxIDw>+Q?Bc2qkQ!0G z(xIVs;a!+4MUl}efhZ51Fk$t`P+Z{$kYVQ7jD=Jo+kd$OdkK@f1PFE!CaQ$@pY;1B zE&?SgPrboDAEz+$(@ol1DrdtutfhK0u_hxL08B4y2RZ_%kiLGXJmYnZY~522q~zC& zWaqmt%hr|?L)m4zRO>V=`fVX2-8`AydTu&O)-0l@__oOn1Z=KY!GZx;;V3Eu(_GB7 zEO-+$UNG`;p#-CoYSd)Zh#z%V&ar$IeB{@3;;X->-g+S@EJ*HMV*^`?8k$+lt`SO$ z+a@;P=sVbGb~($VRAf`rKWJdwl}v1~VAHkfyVXOlhC#jvfTmaXBV*J<3v_vsi(B7Z zgVFleSf8=`%OIa(RL(Z4LYJT!#hxcDLwU~Fn4NlYgfWB#cVT7j4x=e97wlZgj{Ogl zuA~;(ZpBLPB|}4>uI~kaKP6XPKZ#d1-G?CX-xhX^^|<@z)0u1tF~|F|{@e`-6bjg%`t)ArruFuNMyKYRD} zj-Y9jcWwxDhGed_*!JASFYi*2C>!0Gv>C*)v?@kybcA7Z)x*KgXjIsEVTaB)Oy~PE zhXhfYl~+{Jw-GkZ@|K#iin>Db5$d8Sngqx=D*%`_OnJK$Rzz6jrCzj&j`1;zC@O?p zg^HU-lBkq5`3M%po27?f4(oHa((Lsq2xziz!3te{^kb>MEm~Wkr`}_QDecuH84=wM zD)N~_PpVLcl|92{y*(+dn3fGYW1ln0*eqCsRbEyh?!h)!!HtH@2nmlH~g^2zHqdrO2Gaw32s)F)Ubp2udMZkv676-iG)&Rc&tupY;zBsyA$qh#s zQ4q#SaMU}w(abnJxBtS=D6{*ZZPs3D%w!wryyfW?(J7YSu9aSuM|yqc=o8q{sS2TI zh^lAOT>L&rrx~rbRb^EP+XJSQ$Emi1*VmO4%g=L((k$9oq7Wv?O?dys3zzRL30~XK zrkf}zFP76_nNZHbq?-#32I=p^i2V{dCSbyRMxm@sCP?$All5EB>Wk~o zF0rzA4~qCDVcmK6qiT|KJvn5LvM2m^03qLPoat;fO|ii3rUo<3n7Sm8UYN|AAQ0=m zW3`pLKN~PJiZ!_hqt8whh>OrHBAW2^4aS=5yNhm3wn<%BRU4)N&ZhK-er>QXaMj2< z_Q3ywNf<0qEN%V4uup0ZSMR^aA<(5%LrP(rHxWXQHoR=QO8V96o)E)=IfFj$uIuc(pA4;qoqb$e|9_rsai2Abo?&J;3V0bxsHTU19&vT&j@)8hC%urM z1`c0>k+Trn;;w~0q<|?XI$bl z><9zx*&d-eZ{`2{A%elBKRJ22c~n|kl<>p$qYm{_26V$jc^WwiQ!=fbvycL2EJdG6 z0w2v{7E4WRJiehPdHcx=`Q+2GS6FO2qG31qV0!YFLv(d^th`r%MCwz0dO^`53(T9` zH_r9*4tm`Zs2ShuS9i0pU#|g@24A5%<=2Uw%BO#YC-SRGL7QTrOmhX+1ip*~aMF^b z{$%&z_>juJsbbi(a}Zk13ZRkfeUiko)sAVJ8cDY%>@tT5sk_@ zy6hZUrn8KG{b{1&O2~RXVHPzVA%|H;rA46S`z=|1_=f?*aA!9x_UW36rs`%~ze;nw zd<9TAp;ISwjd^|jqeX>{W)=~#em{Pwc&j^ydvB8c<0pIr=a=X8vZ1{T_mqJfaBy&>1*5w_e$A0aHC9_*kaJy?HmuG$2Be6f{EdKpe41mZ z$KSTQ03bG|6J~IKTm0wG1@S>v65agu(C2ICzw=ObHp$>2xyxH41|YB^mzmPi=o)a< z0(&Qh1X4h+PsgteFw$DNr=fcAsAafufH!O2ze-m>mPh$BU|QbaZ-9zhdivAlNJe_X z5)xsFm}b_d+WDFri3=m;H4cr0lvHc_F%cY~1j#0N!XH|co|k{vT^LYV{CMcEE&AM@ zM$Gq*{;_!Ad|sq@@G;`;!RnW*(tOh2-!#rk5^wXSTdvrQgDuO!5G13*CUWca_sbro zzj2#ZI%_JP01l=kErsRdXiKAimcDBNOq{fO#UUItRtvKc7cUZTYE(5P{$DGqUgl(* z3e(CikHt3Pa!b#h_#TnY02igO$;9$k)D)i}^ zUSlIE$33(^yiPqPuS8zy`;Fndum0}As#lMEajzVp@#^eV9qV-+R<%(>)5lw}bgM^r zBNOTm8T&05vfSn~bT*GJMLyZo6ra8z;9W;8+@+J$Pq#!sZ^~v^6$)>R6^tIfW1vD| zyS1MsIigd`wrPpKIY6#1pmC03@rN;aUy7rBvut!71ehpOMO_WOv!)2lYO%kZUHgAF zH+yXe2$nIEn>onOpmU<*n>uhV^aD2YawCF}jDz|DZJJo=u`*9ok z+HQj}cDTj#pnFYDty|U%A^j4g;73QBWAKkxb;hntP@3>QvpL9jCkAc9!LZL`W1C6U zRMzO%gDHQcmkK5$Al(FxBuE66`7qb~)afU<&YkK zROeH8)bfcUnHZWkvKBcj)z@PKqjRGp-c*jxpB6MXsb~?jVnG)fzQQYS?qtb_mbl|Q zjJ5mr^{%T-lI=YarQ8PA?J9{{QmV2#`pI(gmR3g@CN9=}$Gv~wN5G-ujB8dPwHum6 zWU*60Iz*O&vnC^2wdLJRu|8|r@^c7H-M)1_!0xj~sM>(S;ed*cY9xN<(7I(BXIC;& z?mJMinF1t;9|P2We)OT!M|x2jVeD;CoHWg;sQ;4Rqp@r(+x~ zJRB=z9)1{hDhcC!H*jy%&`nJ@iy_f3pQu#0?<^xA2&!e_nDUgC?gzxMdgK$FVI^m! znNDvqBaSaD67V-SpR>41y{Hv5oOPthcfmJc09UUy z08*mvphl-{*}VZ@_jm*lR2Vx^t-C7CO9^>JuULv2LA~^N_i$pFN5%!_3!nq?1r(yw z%*qCxHFeAx|6A*3Nneto)fa{~TY6JkT+UgTo+&ROXC$zk!0du;uDoyH>?+aHQY#FK zwhuTNrV<+W7@MRWBs4YAZ1x;Df6glgJR7v>n%Nfx9W-pBZ2EdZ%}!HM0@Di_yRdQh z^MHoHR@bStbhtSqNGN`HK2ZEl}rGwvaxF%>QTW(%W}qDv14W`060%>X;kgkB*Mgrx4Uk!}0 zz9Ne|oyYXp4L*`H?p*;>aECramEY2y7y22)z^_6M_nOk)M)h^VhUrXA!o1;xFiki1 z40dHPsY=fZo{an+H>>qGLa@&9?ZIb1+#0kIR)=K!{IGgSi-7S^qP7U8lclJMzWvPr z_Wo~MnJ28P4@W*fQIf55;VI5gJem6pnmw;_R5;XbNs6`LLrbGCtgefz=VKa43i`U5esPf1)H$PN!B9&j zGh$E}HsH)&xwpZvkKCAzSHOY8=;q$%{blcr0Vg1{gWOKAQ?*X5%ZGv3Xr$V}Crqvi zq()xx)-NC$$24Fg(ocSic5z%}6>2frn3K{T53u{g;+V)PDBm&d z8^}qZ{hpQla1uLzWzy3V9@3#QS{|wy)qhc4&2;nWOgg7!I4~hf@~d8t@?!9{i$eOU zS*wF|Ebo6+GKhQaSpid9JXW~<=IkKW2*HQClTswqQ%a6Ue z^7=;Ngpw(9#qvvfiF~}N`*69lRqCwK`i=9X-)i&URtc)QGtEqI-cGa;S54~DEJ+0Q z$k;~h$V6CX%Z=j?+2aHWwLHXAAXMZE*M)XzGdb2Q%I?8sPCk=dC2>Y8;fdYZK%(Mm zmW4ISPuHh`4>5W=4aoiXbIUa6g!8$?6!kY)$TtI(sCz`!v4jbE%}Pmz%B7ccLwCOH z)*cRCME<8I;I$+h@Hb^~zXBhV0e7FAMZ>J{`37}KmeVRoGvaMN$hY+BeJjH5Z13iY zHj6VN)5O2=qUlW^(t@IkLAw!&FAu+_Yu(@5NXvA?rn@q=>P=;53#m=icEQ93(Cd-7 zLPf^yJpUb$K`&9V%`%Zh>t(4wvd$^~x6QDUt+UF^W%apQ8q(v&ghUkTFx%O~3pOQS z-RMFwNw(XnllpyRvMUR zTb7ik0^gAWVnNV1z=A`GEG<&t4_AwIaNSi4-_o_Cw>+ix(cFsR`>z9jF*NC-09NrW1Qt!a5cvXG1u+*Ye z6a*9iRXqvaGZ>^Tn7|IkJfyKokZ&@jzyZMPl4J5k)1^t={Z zNX2Mr8_ry~22igAH4Vc9E73SnHqOht@S(l__!OssW8z)pUE1(I%14G#!L-&Lm6ung zYTKnWSO?vD9P~I1(HB0x>)c%(It#M0mhL}lEgopw%SYB7%?%pl^P9L&AV|?$8xh6_ zGn22k4!Yytoh5#_T;jg*ppETYEC*{6OISKs4ULqX0aKQt0n=1CvCa)s@x^m1Qwkpa zF<@jW#4Vrg%5q06yFqe4j#G)@r5@WW9_Xy*!)(!?cgXAzNLpPFVGH2mPAH&cMYIA3 zlj;f2H?ccgNH&igar$58FX=%s{aB)6-iRe>d_o|8u=h=66KS&5cj^oPjP&}H-@vH7 zI7Qp=)yqoIWBKC|$Zy6Ckk1n{A7~@s3=JwVKAWm_-bBM;(%sOfh}5}J!yc|E)IwPO z8gGk{FDRsd=H}UDbVn?Ku8QE{K&EK+l?=Cy3}?<0*S;pb3FBj8OZSRM9gbhM z!64wx@FUQC`xP)rq}{AG26mt&lD}bU))hiWTgN+31$yC)!7R@OlOTz$V(WMn#}g_Z z!lRY;;cx9)ZZyDwq~7qKj(=^`vTS{Z0k+&qv$cE23}D>5FQU1<1=NQO?=9*90ON+s z8CRhBL$Z_D_b+%W-F`1USYMwH;_rc$ATll)Nds2Ukj)|+@9$C5H#{TlPrvyolSF};DWx?;mJ#im_uIWswky=ugTJw+z=t8_KM z?OT2vNmkx1@=~+G5bm=xqizv0TU#qtbI^FqB0%IQ1CZt(Bw*ym4s#|u$G9zgm|f~n zaD=8@27Q_kgnx8}RO^=L25EpGfEA#B*tvY1)gE3ryW%Za{&sUd$ip6*I*jz_FJ1wH z0Pmp*m^*WX9Y_6o(wd(Fi7(ed@%8Dk7NN1D`WX!by+>~kSOKa%z&97gh=4i8(EN{? zsigiF$u}TvEPDXrb#@=2(tiXbZO}Vh`=lMR<4g%Ib{1f9A3b->k}+?v5|HH#56 zTNhBrgG6S90JYz}JCQ^J8d`JmA~V;ER!9BHAf9}d6lx=%>?=^~w+9Typb@M?kVXV-p9}SG$qEo-9TZKrbJb@B%?BgjBhZ{)Hl2UUkKaoNy$+qCu2uCHI>Ht2aBmfD~qUarQ~s6F7Ggm;ZTrfWTU zC`+WVr3s%(Ix+xy!VLPjy%%VEImx*dg+8j6h6?fwUmU=wQb!=Qwps>Y^qvkyqRny) zL-`mv#{-=YioruC&jJyFdDLcpq`V14#>7S!K|^CJ^CjV)9TURvz%qz1tG`WB_YkZA zlO{+FIQ%hZVLc!YMyezV4bgzPzgyfcSW83{M7}P+TYwc27Y#R$GIIqY_fY!^$RG&-gg)Rj zxotqyhB=_AN98`wp+Cr-(3T0(~d=LIBReyPU+t0GK~8 zb)sB2qzY&&4MH$wzy%lAPm=qmk#GQQ6&@j+-)BusW;qOW#K2UnZ6Twfr(Z9ytUa@S z)9eWgJhBvF2)9y6y)`4?J(20w8*a2jHwKYeh91>YZ-tFQK>5vnFu3E>l=D=}6Lo!L zfd@tI+>$l6h6Y_@`>Pfc5JQ=+6>t6^ZV##oxv3--RPME|zHlOZUd_rZRnG(%9B&`_ z2<7em*Aj*>g?&>!+d32hCCAkl;o#wF8(5_FVh^%iE&GKr;}q zy4OU$Zh}MVSh|kY#*)-l0<7I?t;t14o}Ah+b}CT37c|wCbQ9Ly%89_EXiNYP_ z>MhEx)hIFX9{i}q>YE5XV=3j_Ax)ef&%@3^jk>4vVWu51roYN6ENozF1qD-7SLoQCV{dH!}cNlp-&1@iu(Q0#UN;`2l#2_ z9jmy~X^aFDm;T<4FdQIJ9$vo=u2-Hf+UO$x9pZHk@E6m(s-Om6msF8Ec8JHx}49)1Q|4g=Ol zR3xg~99mbBp&ZQ-2!~!+OCS{xpR4XRp z8c62i{ZCSCuG7NV-GEFk(gP5`XIqjRC5ECCDW;E|#Sd3~b|OWU zUjb=NXrp#nZhm?T>0^N64rq=SL+-+oI2%4ph3R;p1lJVYNhq|c^>{Q=uYf8{Ywl9| zm5h^jH)qsQDqwo$MQ>_nRS9bR-dh2Hi%HEDCa!;ZD%c$xwH$7?joNHAhlD50SeBpk zLUXR=Lhl^QZih9So#-!r&;ZRU&+BJR=7FE=te0f!_cZl^t@VNR$2aIPrpC+`76E1@ z=OaOJi+B}QQu}K@OU5DHd-}qb;`FtFyw)av*CHlGr}a^rO2F_&7}JMR(=zdMM^l;u z`lC|>1~$&b3rdTFQD7~&`*7)>PfFr2pv9G5&Z`CRRuX_C@%UX$%X4ouQc*{wun^L& z=yy>PcVhDzFu92^b&ICt_FR`Vj{f{)mz_EvK%KW)oGmL1$|5#3CZ6t21}#3>uq6XE zR4MFk=owwA1S8d?TNvcBi~Z|ZyC2_rsr}y;=%qnxl;V?1X$TPqoG7;+2qGA2l^V%yE+nd2WB6yc3&kcJ2vUqkGakNFMgHzk;#gn7BTYT z3Hn1&-$nSs@yZcK`NrXgv+9~w+w>BGg=RV(4iz8abjIEiGCzVZNJMIAss*e-KXy*Q z>IK1=6RPAuyk!9=ZT;J*jZ&`aX(jRzfoHH8A6%HVQY^jSHR!MWQmxXsXK;mKA3V=H zkxHI0cL`p%QPF_cK^fO0pW*Im~gmisEGKfa_JcCZ(BLzRO#byfynAelMW$rh)u zp!KCQZVcL9UrXy@eNRmqX+r_z?I>!ge!1uA(_f2;AdTVT1&`AaAB^ELu>R6%)jM*~ z<$|(L#cMNX89)5GuGJXF-eR7C(t(;D*03U#%K%lB%qi$ilYVKpL(rze>i#aBl(T~F zmqoP%ImGiA^ho?YCg-R|8x~%txiOiRP5!T)qD$}`-j8Z$pelT6{^BwJS@O$q*qcmY zTF&9^U*!y&n_6niL(e46qgrm;=xPbt-Tbl7bd{dEY#h{8*Ymy(F2F<5Xkb|BZ*WGq zSM-Y1^#`BUFPDtNTTxqg)T=XJ)?6PBq3u31J_gg_2$-Ft;QzfxZZJrZw3Gxvyh5mY z89c@nD|*5zp2sX6bXJv}<4@SB=tD+k;C8^iIr)OD-9q!&!H`%vx@bKh~fz_K)?{UGEi8#FwXVTArFH38SyT?tk#Rn#DeH;sJ;_ z4}=7ff?BL?f@H42x%&Gjn^&R*iGD`Dja@w7Yd)+mIF9QrUT;o|3FPH!@g7+%9Lrerpt>j~jh$Bf$8|l57)Y7G|(^6m-dTlp?0Qh-T zcR^1zJSqOmTb;7220g}Mu)c)kLYp$H%rMR0%@kBbf`R_LHPD|A9L}E}hXib0%;X2i z4o^~M*0%HNtAS}4Ff&tlXm_)Ps5o!fpG7q5=L!nmwuxVVE}??4V{UNJe$lt{vG#JT z;`G*@C$?PBg*tvoFQtM!DjRO#5#9}3XWx*c?6517J zL%_!#?2;bqEu8oh0y$R~LsJn7x^owEOH&Kgure&S;a!5?ce;}{yGPLkHP>m8S}E^L zmj=pywa42E65z$V*Al~z=>M!f-T|8MdAVed@;$sy-Edv^jPvvBVJLaLiXZKlGL$cj z%nmO*-A}1mkq~cC$9O_APJ*1oVCXZ>r3&ipav-xLiVWAwC$i_+Hb4FzSxearvXTex z&Il+Ds;{Uu`ybc17iVr4n`yyS4^464=@q%Odu#u#es)yNz&yK_kDh#t%Yemg$u-m^ zz9b;)tDI%?joCabfv#@LBAgM!mSnPEfDHl=v9;Rh{$w35=AL z%wj_HTD%vUk%|eJqMTqLzi>Iu!g4)#bQjC4$7>yJ{<{Dp#TP3>Qs?dry(SL|Bi+l!*@mvW!qtJ+U064^>+$K9%0_K;aB zvz8hv@9%=2s$c1^46m#y1imjiys^f1PX|x;WOk2wn7LIL&);cAw)iu(Ue!WWPv%Z^ zHWLJ%6y)@@jak<9{~oAz%my)d~01%=F5c z8-ag%EVLc`bKtL(#nro^S1%7*wpt%7P8xeMbN%dh z8v@m@b@FB5zc7Jc%demr)H{<|o>?D~&GytIs36Z&I4FeX8Cw725vlQ`d%CBaR9$^w zX;{#4AES`S{=ZorFUZO3X!DgUrqXAniaSOR78lm1;geJK7Q0?DgflR>-&M~y)HKHL zKe*mrg6#7-VQ6jxqfF~?b_>Z!PT^J^mWiAQF=uHlK;xDTEsCAn+?fGi?XDGTKFPhs zuX0Zw$`eNrE@SR|7?u@0QApS}rz7+FbhgdnmMb;GL|Wr%1`iDN&-eSW$uv8_TqBaMAX1WNjoh$;(+C;C>TbSX@v4J$-26w zRt2dMvw#r}4H|eY;6rH9L~kNLwX$v~IXh*#X@qqM@_je(>VzYc{n9>}=+sKvu}%fW z^sLXI{<3rtoRDb&RRDN)MQW;bcl`D6-%ft#BaGSNHhD8F_YH!P2<9WeM;7h2BLP8X zy0%lVzMWEF*)V{VL-S}fKUE&-PcpPP#G)FVm-L7~4t_eFy=f|(?zu&6F+p9|Rj-!|4$KoXj}wd2ry_LCk`Xy(*&Gto6T77&NkNL~{>IFLX#q>8 zSoV(T+KAM}T=CNLZ#lO-mRk-?gr%jgFJ7{4y&=Z+wEc<2-&D5GCCQDBCV1YvU_egp z`+D%*v*RcJAU}BKi_w7-HxHbQzjFzUe*OENSKJ>bwN4)mS2+FU4MkhOE!TLvoxFl{ zWmeUV4F$Agi~zsIgVtf!2g@Fp7R4<(+j%k zSesM4WdHLkV!xMS54+|bD-DqQX;QhZSB_kZ$6{tMw7cS;>5l6z zl>hzu5!cAXEacXoiBl`Le=1TX#pfipT9>wm5OSZd@#*y^O<(K~j%vTb{K?_2IPxA- z$pUggAojJ==G`ViB1gf6iUi@P`VJo{EuYn;jK?l8rpb7weS&gBGm+qZtN%LuNb$3b z)2)9P$x9BgbN3}Mw6_#C@_M)P9Ia85kHFZmx{X)A&N^mgA#2q`AzMu$%c|h%998Ys z(#zA`(5xeD=Cy>}Iuy=(y9k^pjQt`LDAa_6<%jKN^63X7+J%TPFfP zV};Irve(7;gEJL22XYgEp&Tfo8#{UhCz&JLF+d+QAOLL<;`kLrZb!3fh?%Hd2UaCK z$>bD~K$lFo4*emFJRPYiV(vGURNVcczW7{;t3Wr=>v{sp78&Om$R&o{`6DE~<$;`3U?*H7zWMn$k|@`#PqdHmOr z+Qi%DW!tMk+bOdSMP&u<&k3L#kPcdm{dk}fjJt}+GsXhFU=^s%q3|KE272zKKP3aL zkfe@r*1O_%25yG^$y;&*_{&eXhCwwppxG|w!DCH4T|`o{y__}U3x4n&us>Oa`#!}@ z_(cCjODxI9)0rnP+}*KTP)W?_>c6k!H~}^jS6z*SmhIQCUU9D_O(rql*2#KV9KLa? z(;ZtLEmq36e+BpCbuPwI8F^z}DF)A3&=6=!ee9df96$V3{t=H}wDCmYsKoZ7guaO2 z#t6%n;m}h^2Bs@$vo}Z;dhOz6Zm3g|`l$bc4iLrcdU-)A6?)e`hYvuRswu5OY}TgF zEd=OVt*1e=qFjKGX6M7;rqyQbMl*h#$HN&hTY{U&wHQj%Bc|dH@HpLsZ3CUowBh+c zM-|7CCf!sK@Jmew4&b_E4FV}h!qm273E`RV2QT})y^7`KI~H)e2a~kffY1&6M~L!O zSg?Vt!VC#}otYqKpiLnpWgi>z{v<>F$r&TjW|UxmG5V8r3V6kXwuBti?>AFoJ)~vSARj;z=nQM_Mkb(%l4NSgr`5l+(F> zPNcHMv8Qa9_{DvDplWkKxaFU*PeJbX#E@jsMEN{})npG+vhUUq5M4{N0$ZkT?b3lt z?J1uS=wh0EM=OS1t_H zZBijEq=Jt6tc}Jj6&%{b9RVb})uA>_DkQ^0T&|`R&mL`hwhbOo8#E?|ozLjYzB_k9 z>NdU@2xaT@RLPw z->u((eDMM=2EDJIY%bXypKQ+NR|VcI3mW`^{0sO5L3NMK9uM73z|(z1JS8!Is^Bq$ zMAmwT#AZhqc5?`;T185$(h>s=%&eI4E9+p$Q#lwRL(0NiZ%tUILdslS{IT=?q%2K8GL6bXc76?_V206XEfX^b0 zL`Vs_EIs5xFGk~3Orve?AkwslQmqKOb1v!`J!qXRpbxx^2ZNb2n5fMb>I9%)%#vgZ4-mzWE=jM>7=xdodWykfqAUbbids$ZRO z5#J8`jCmo>drSXKuv|+emo=wr!KaI64R3dxx4{jHrIUXWqOTa2HwTLTD+u?VAr}>F z5)&b`XGGCp^JfZ$9SuDmVN(O@N6&e^f*YY*h#1pnAhG*b+_1ISCCls`7@$= zxe}8%X;Q0}ec}gQvFpQlpyric)x828CL-<99}1O8EB!r{g>lN5E+9sh> zkc67aX6Q}E#Yfeyo$<`79tAJ5wjs^H^q8BE+SY-vmr{}38V3-4v*KwsWo&hu;7@}z z!x5Vs_vauz`QX5_UnIVqr@^?-^tfk?`gyMuH;AQVlEAE|dqY4;JZnsd^a-ZVR$j+t zr5Hp*Sq-$t^Bh>y19s1mtbskb&R^pc>PpTa3yP+^G*;{L%0G+hyewgCoN*0dSLrg4q_flW&==9MDAAOZ0E1x{C5w)9$RrgZlyFYtHaFlhd8<=jG z%_8vs8c%q=at?iEt zq3rGv)#Yk>`)d+p{&qv4O*=xeaNykr%}>RVyN+c8Lz%+|++v=g=Qly~ea7YOGk!7& zN550UL0gkSYE@f=D!mcOXC2HEG3z)Wkk3~2@Ll-6wGwQD_;L=L$4Oe8q`lc^oTCTM z(LLEgt&R5O>K|C(6sx)kv8>X$*zax*1pX1 z*Y|*?=?vXl4OMSZa{bd^a4!^q_|z$1hV5$(jM9gAT}d5iJ&*ahu%hzu4}9|r2ic@| z4b}^F=$+5Chhw!7Nluudh;iq;Z$)_;%2e3rl3s=eC0a)cWoi^E=xhe)AANWJp8P+r zY)?)r7r~)e28FX@6vc(OLRfOyL+pK@;oWnwC$E7?d*YtDJV2y7fh2E5RW>g=2LV4N2e?j<9(Xi4v?8#0=|l!K>t~RK1YkX4m@gN+liugKV4EAfzX_4Y9( zNA$kKkS;u-QV9zNPbbYvs)6S{e(sy zjIk^#2j3WA5m*v(b!g+AVyR=-WBRM;>fbb?#VC*Q{5YYeB#eFh_RW3d5 zU?TImae#6A^rOoZcip6)bTCfQR!rVztNOy`n$YhjJnEJhRJs&g)ks<1FXX=AlyNy= zKlAxzIsIe)YeFlEfi9}v|A(zF4~Kev|7V(#Q&}s@Ue;vII$2MPWsIe%I2hEBWF3s` zOPxX(vW*x^*^&$jF}CU0vM=gYr!IJjwIY&Wz{JA`>s6gDb=D zed{Dfa{cvpmZ--nM}zI|fb9SqbFZ2F?7Jos!9I;fDq9dmYT3>;sD~Brt{)3L7AGf4 zbBspTswr9n*J?x|`@8tTwn(CvMDLtX!K z=KcUrC$_9&Dqud*b)A?R9IR+9vxsXF!MU0IfftMEY6|F95?DPR`zf;mqNe&Cuh~~Y z#-Hucw@YRVjme(>FzNVb{_)3ZXI_2Vp33Sg=;z^r*OJ8L#X!|7lRE6}c9~lIX{m08 zE#W(g?jU};T1Tn!!P(G$1NnjFEEHdu&sG&uC8$Yt;~~pb0o=};qrXb9T@Cm(`5=&g zm9EJNlbIpESM2jYe);={k^u19f@#-%!_B9OV$-xC%@?b@3Y)1KEYMHtX=}L^tGu-V zPQT(M^5V=(XHzHIVu$1DiVQ45Ks+m4%}bX0#6A0Lrdc&0v;Be%Bvk*wW|vK$+1qc-Sx@@}uL{9c*IsL44h?Ty~{BMg8+N7`D#$+vb%@zh4uLRn&7EjnPp$w5mh{a4BRt% z@7I?NBz(dp5-bn=ya#LlK~Tc{I8vq>^fZOT?Gm_OkPo(*xaOKHzR-#>xfP36XMI!u z;*y;#rc)fmP8jyomOE;-flPQP>>rLg+~{S25;zdlcd1PtRIbo<*E;Hd6VbJ^S&oMZ z?66Pk;d4?Hd0SAgLTmszWr=UFb|uiMlKP9G05n)$08}$7I3p1n@|h*Xqs%PnGr&`PhyKnp+xfISg4a6oJKg~oCpHYZ{<$W` z9Hfs%%xgJ3j()naMX(v=*_dp>?pWv;oWrw>Um970Kc>gxxT$-qe??yzBuPT2y|%MjSbn;6z;1D&pe{Z(tegI{j%Rq)3)kG;dU0}(AR^XvmP zp`(SlHHkKNOn+bI`r~%@RAli21FXPXIm8vfvv3EVrllVE+erJ1e^!bfD@)L^Qk%mc z=hh;wYmBHgos#y0ZNdE8T%R0pWgN#}9^&=G9^EHCL6HP0Oi)MEOytG$*ihWYwbosq zS8xdSo%Qd6&wl*|@CS56iJE^4szzw&djR6#4c7@|;-OIb>t0*FNQk&vAuF(6oV`?L z+}Ed24*`D7B|}Gpy~Pbdy`R`fiQ*mU|0)-0j0rX4a*Wb-5KZD~NovlHnFv_whnb?rlh9?Re+lR#NQ~Vlu@xs7qY` zqdJ}7%1vR9*jF5szH);BPS?C^EI~99cFdx}OwV5ZPGB*6v;v7c($i;Q(z;xU<$q)JQpv+Gv_X4ZRi*RjbmuF~ZE3bKgbhi>dn zoab-X`CBzkscv(Sor;AjZi1-o^7VM?Px}YHkp}#C;c`95;}i_amj14D_#Tnx5A2$0 zB$XCd)jWqJ(XB$x=U3Xh@mFnUcP*n1e||Gh3_CE~Hl)qJ>3KBznjHU|g8`Dy(x$GO z)iNb$4LlWRzsS%0G(X+PQo!-N=gG#d|H<|2-W{IPI@7i$cos(Adr{QkX0~>B^sqzx_Eg>QBgM5{FZi!bhn!4=TZfCC4YSdJgZUM}8;ydg8rF#Yr=3HhPMi&AyRs)vN8 zO4*-4n*+%WzqMhbecqIg35HeLRtz3_r(q5X@LH_=$GwyYf-H%KKIJbSMTl-ZfzEon z`l;{WFp#plG1t&^=HkA7PIO;BWDNx$?JJPUMb=FL&@=BtChD0kZu1+_VrP`x@1_P9eo{L;#V^UC8>AUWa0u_HWs2VVb@_nAn z2W#pWJ0c4 z_5E39;zgcmzQ*qPIHVF~;BG@!K3W-xh^AZUua5N5vT06lWBl$xXJoK?p=cc0lZ9Py zOjQSSknF#GPZ*`n%H?Qf-cxGGr#j|w3B5Mg8w4wu@O10%W=(LrsE20LGe9W7E)`$` z=RaUIA-}?04DkLIya4UizuDV)Yhj79!P*$0G44ETZ`=~}yzT5>{EW)KvjBd2`*tTl z{&|J#a%fKM`N;MP*zAfYq#Qfzw0lVl)v9EyUi9>O9*oD;A>T8;52}(RGVu@Kwg;|k zUYG}|fBV}P>~$vV%2o4B=tA?G=W>PQq-4NtxaKh2hHSY3sY+2DVNlgC4oI`8URtg* zA4uTNM>6qPR!D)<-9PZ=yDOy6Qv1$j*GPK#&GA=U3ZR~C!KG&%j^O72=h^PIaY$&{~$3$Fe86dE52#2}NB4#8?7cIToAK`qwXMlNMMidLkmcK4X9CU!VoC%$rDACTx>UKt zF6D=D^x$Uxh+1+I@kkg^_)o69Se?KW1-H(v5l?o(lEgaLt7a9xnJZRJSkW1~`;)}5 zpoZIDJr5hDA854#Y-S$YgBt4_Ay1c&JfPD|EQ^~l*6SjPU8izv_@NuE(4D!TAocS} zI<2P-=u)kfWJ3ycW*P;17`k+7nkX@xXS*(xoTuQqxnJ!9j=GA8jId2 z;CwdJSlNy(14%ygsn>LZ=%S>c zgRQ26m%|IZ%4ebfUdv&y#ifO1y{Mu%>dXIrFGWWr?2CM1iN4(1FFh~kMdDllzRjT= zdVDQ}bgSdY@LE*ZxxTdlBD#(Vxp2>XH_pEPN(=GJ08y+cwq>#q$=|9)j=C=@l6S)V zRBqFF-@jtB(*k=7m=x0u^A*e8+cMdiH@zdl7DDv80+YaK^g5THztMM{BZ z`_tR?BuSwjh=<@d>7_H;{)b-CYx)ZHA16xJ&papi{kIXdN1-90FULi1{ufda%MQ=o z8YB281wF-h%~{=`SD+`=a;Q)S2zXy^MgZZ5Fz=hfri^_uSzn_zfi63J$P0p zRQc5sU)-|R85%N>jp8Zv{YcN62;&QH5F$k{N1?MNHus8D|6}%4L9_8 zT;RWL4Dm8&1T46|R#vtDDKC&W0QnQue5h-mW3EU5Qgu&Aq#Vq^twpSI*ZUXf4Wkc^ zXaPt>ZIg^ zSSFb^Ri!utNg?tBl`H#20o8eesKdW`OjaTjJ=;6m<&`uHo{Z)dG0*EtnH0ho&tFwV zBVemH4NN?~EXDt)Ip^b<=w=t0WIYG{O@EjV@e&{<{!$SB9JwZ`<(+sz7~{X~ndPlf z)|i~7sGh!JxN}v&=ji!dWjtcc{1QK_x*)4(yuR2LP(`1Kxp-#q&UY{l3|hA29MpOO zy?U`p44tkEEZLg%Zjpf4vNsaeVrlG&r=aV<%e+}6tGv(DK+dooL!xx{zi<>T=trvz zB*=D-b{rmi^51$>h~lgz;uQxb#Ura_{(BAbVM&MQp35eEPC)<3K*^G~8!nmHoNJb2 zY156XvHqYbIrCoXxfFZQm?tJ|nfh?OTxYl61}{VgQjVP`@{N9`wSCLs4Ucgup4YY# zqfi6L?>T*mA1JlqX=?Qj&2vUYo9o?O5QV$PUll^Xl0F4s7rS3HJ!8I$ZB#VQYPH1e zOyfE!&eu{8F(3V{k^TaoV5XuzqdbosK7~XL{%{!#%VrdM-!|a9c)7=f$Bx?%7quUv zU6mk?+|(we-RwF*#(4>EE07ys7o`bKy=dWls$=EJmiUl2eg)hL`CB_dgc#hbwKOWV zp>8!cQ6M|pQd8&pJuaO!*73@kZPJ?0Q}lA2+Dp@sVkYB8EH=+!`XS=9e}kls1RdH- z9ui?Q+BrK#XwW>`X5KIU?x*sl8MHw3Jnvw+B8|{i_HM!YCs_kTn4`?(?SS*NpF#^p z0fFc69P=08+Gd|NkM$;^%27MA6ED7UAH=i)?mJlETU}QfJblmm~sM%Wcu(e@VhQ@!G*P&mu zr}blLv+}Z=y$nO;6B$a@F>5X-o!b`k>Q|O;%d5mEz_y^X)1?f?%zw|}FA||w{`z6u z7%R?aRSHrzWea1N_|{?;+TSZFL|@a+BZvxK-&4ePX52}XY)&Sl$nH?q-c$f*C$^f` zi+TC=U3peGO6cUU)8_DOxoW`f{ng{>ycpcr5P%$kl@k?N8&`3@=8!ISbt`X1NuU#K zao=fiKmt?r-RKi%58p+IKEsDSrrmccI-Ys9u@e8#jE7S7wR6$$-Oskf4%99@naO0+ z9K0q8?Doe)zN~`@3uhC641?HTp+5X2#lfQ6G`RfYrt1ve|-EMuUWp%=O%@(uiXWus8<=be?@0j>YO-7L5KH$U$I_Bw{j_{nSeM7TB)=F@;Z z)V%aAqw`T`TF;5p9T&!z*_@6rGm@6wxiNBoI2hb$y^~w!9XKzZuT$$AHUBa@8i+I> zg<@PWbYCq=0f#i#4y)hzVjIADI>LDE%H|}OMAUZx!Xie;`hb++;6)h91!-ecOwM*l zBx`yB7RNN+^MC6%s}jW8zCG3CRUu%s>`x}%4rFzf2uhP%E^Q(rQt(>NqFn#Z|!}mgxD%P%Dl9TpS%qKyQt?-kfOwB>{zKx~2lT9{bJ=DIw!ZCln z=@G2Hj@(l*_2em3uD5ZX!;4O?Km3fuqMvTrhWk%Z@%%ifD7iDaShoSBq&CQT5GK4tSL)INI#MUp_mh?|C(cPrVDusyGP_xI)5tkz zDTcAU*Vs_9R=lNz!3%TiHwNJ!MND)C^KaxVg{{a~IR7GMhGrh_7h;+>ZzKbse}0gh zlez7TBBTsiN>5UK!2nMeW41kOAH^K5vwbrC0CJC?QqyQT6BYj1-M^Jv;9z#bgSOYs z6P+mS=mE#lNB1+cusB@ENRRUTx@VaQU$woo*5|Vk4h0=ak!Y^_fVQLn45;lCnZ8zq3sdnPe@{|>MuZ-7}unY zV{F<9H{oCHB~U-mWs*=<;s1O)4k_H_-zYY?4Eo(T{9b0jAslF3>{5_|=~lHc z#X)G^Az{jpDhg5g3$o!pu_xCmm;u2Www&11ltYw1H`hqYMHiDWS_Pj9T-u?oX?^sC zYC6EC9AXRejp-}bSL}qRst@evb3TCn3ppXg3^JzTT?q)}yEfk{@%e3TiaMghJTR#O z@pIm>Z*lSrLj$@r_tbteri_7-Nm04veV1?r5VSctpKz7IM86Fyb0UXSKiuR zSC&};1Uwdpm^;nl30HFlk90alL(0p~PAFmy#sYakRyq@1swhn~d`^gtZaRFXq9AMQ`^ z`YlI$d*05VUYO6A-v_D2o+me`&%JgH7QYK2D<}r`lOzi$CktfiB$F{8&t#vtsoQ|4RsXZ=Fw1Z5w}^IKN1#yDSe=? zRIL`myuoe-oW)1+%MNGO=X`&^2jA`Gl;as}w_6|u4wO1ylXHT$6!Cl-d57quBBlNq zl}XpZk)~?$yyWxgl#XieV8Kkl>waauyO_J4Lguiv@-|oyXcmjJN%I_hfy4YCs}5L^ z_Oo^@#F<5S?1%q+DzM%|oW$X9nP3G|+tr3srd?skTBWH+{p*9aw(c}}Irf>Ol93HB z8vGB>IV=$z%s;80TzkNk|7`3CY*-(I=J#25HQ17GSEp3OJmuNwIO2~g=K0!}8z0ER z4b(-|LV&D}Ghc1?;8TK5xyWOxS!|K2MtWL?*5Lt|UCAK~~3uazEKl2{-$V-=I z2TY*;qvf6vmh66!J2VsnO&>JWd)9tIlX09e@;B5}y>@bJLA4 z6-&-7wVY**;MvJF`A{1=(Edj&4wk9nGknsggV`nM0gn5{G=HqqRNs2RXaz|uF2mW$ zl4_qgRI-ot6LE(d(a_T-#b~F~0pXn-J?rB4HSdwO2xZT9DK~$t=*Nc@@Mm|cMt7&$ zS>@t_C4ez>oe?+!lXITXfXnvGf*VgQ{-COgQyZ}o|6iu+Pp{iJ&<3}zmIjjh3)omI z?87U$4{KbBJl%S3*K>9FLBn(GY?u6dPmAqF&npS8DvP&oaocHo8I`QF z`in$Qt6}D?0|zKICKJha(vNeIPcLNggR-NS`?g(FqnTRNcZ?MQ#N4HgkNgI@Pcgs4 zIydqc~DY=JS?D z5&Gtf2F4xqvjur`Xv!^VEk~MM=dtmwRN{-)8WEEhb zmTdz!v07>Zm49G3-|pHz;U#^KL8uu2o9+I~FQ?uX7S|CVpK4ZVJde#0=o{a>8*yM;-96vdcu(y;W$D zj4>hwZMa*dd38EN?qoMy)(9?j^kM{;VO1QHmPBhIRm71gv`BNgR(d-xcBtD|;9gV< zJ*C(Ls==ip{);Qx|CTstjO7!G)Nv`sHJmYgW*`@9W2reKWl|suT;5FEXJI;Mt-0*8 ztK36&M)x_NG|pJx%j*Cgr_O&hZNj>Cd!V?>XIH)3&luf&%JKR;!a&h?#wft`#nVWc zd*18)`C8>4fm`yv(iqImN7+KQN47c~wcwib1}3jEHp=n`c7`v?Ecm2UMNTSWd9FyFI)0=XyK{QxIexxg3|h49XXsAIOp-a-_FDje z@cvP+)Y$t8n1iB^7(@!Lri?uQM4;mfKLW~T&Ui!gDYfn8CZaRGDv!V*Rx<8*EfQer z)>!QrLCL!|{ESeCjf-U|D_!GLIxZBiozoG&!L)h6C9ukhIXn@_8v@TuoUoC%0p?+> z=WSHBWU+f5o<2g!5iqiYgQs&$M>1@p^W{2_5D44t?%5tAorP4H4eEzh_>L?jqfu=u z2(+ue>7HurnmCa!Fv_sF#EQc?UZT#oy!B^0RDj!4s>iv(-nG5T7itBs1VyI-j`Ami z{Lg2iKS0cMP-F`|x(r;y>xy11A0)H;xdHUb8w}Di{pl*ac5bWco4FKH0Dw=~_z#m1k`HxVvY^|KORYemy4uG-AZf+8?O`^-~jjtrX zdyNs&w!zMbTnS`7HHsSh$@|*vf~S?ze?2;u~ zCh4Fp|4H63LKS;YEfe}pH&8P_Wg4%gog=2AJivsoytk@eU4hnvJy4qEW0&e#nB(=O z$p0fI9Z`-?4M>fVwa8dAmPPbHL#>llM7MR?Lf$n!__yYM6Pnja#zp0Y-}%>{cNh%Z zi(o~IMUJ{Y#&D|V{$Cn|Yyd|6#4BKc*b41LvG0{DRlGkaj)e@4*a_Rul>A4)$gnXQ zcjlMIRzivPx2ukSu?J2bK=8yq|tz``8?R=dM4sA1vW=T#p{62ye+2T7T3a=zg^bK}1E zZ#nF}B5{?NEfLqsXjYf_P=0yC(P{q4vA(Uws1S7!hY2Znz3V*&rMOgSAW@)^`}9fF za9pWkeo3*%=YB0H+A!fe2w%w6W6kY=b4*<;te~$@xO&hpVYHDL@BedF6PY$2xG1HEql{bPP<|%PS5aVv}j7aY!X5RvDH85KuRix7%uefxkEIs z3bnjMCZR}sg6s$KR!UOjHZ>U>XbivlHGq6tTxhG<$h_CXzH7S9{_QjSz#|?Ti+X$F zmfR*z)NHcW+07s1-(3X{KFs!IO42ewfb%TZk&+kdEqYg@P2ZljcQ@u zhF4Ic+j!8uLLp<1^XRTPB&P0gYjDD<@U{60m2^jd)e!zUBl zgtP(`jS)jyjNYG+?M=bWNhlpeU^lzV=_W+pZQ&yV7>w z_IFN+*O!vX1smBx(fM0|nFO+E17Yg%N({@<4$iT+C4cjNc+8CfcA%3HisQM~a;R_Q z3)l0hq1B^0GcT0~o)*hY(3YHLzMiEL^UBET)p{S!s)Q|a_xMFlTVE$g2`*Ml1Z-x- zFbx*0LGtMqHF*Z%zp8E+nyY6w^U%9kZZmDWNoaaeW;D7#det2u$3W=Fupr2kx$K#S4AVV04ItJmIzY7q5~( z0UCQbcgHZkPAa3V&d+tjX&jmFG`gSgldtF)`;`Xco?L4o-mtA)iF=%R><-W*M5oPG z4o29risIFkHd;{J7H62dq`Ho%D>wSW1y$4x8--hx2OcKEg{km<_{Uq?%*(evIv**q z@8WWw8o7qB((Jkrh7ga-kcNbdqBEY}Opljy@WSLZ+eoC2(gWT7<` zn+viptL7Ntm*V}mqmNi+l+XD=6BV>hRM#Qj)-sH4$1|pGfeoc)_W z?%%plt4ZgOK;1aGgIq5(*=b*@Zq=4J7q<9)PlHB9i0xbl5u|Gddnu5Cq{4SH?neV? z42li#@M4*?C)$0sfj*_zDQRLjK;@qHKYX2VRrBna^K>voEm$nB@*%6yGo}_40p*qh z=`+)wO|fuC)pwCk-rjNg<_VOljX6Re@k>{ye0av%s&(s|q#%Q60EE7CYUI))@3iR6 zM_&A^vYYC)!SmbiaVgEczwm`zuuOE%qWY&Jb4gQ9!3H*EX4Z=2!JJFlN^@Ia#h84j zkvK#uHZR-NTw}ISa33-THbxR9dJVU#G1+)^DqB5oY_F-Xqt>fHx&VMA0_=U36WxeD z%dw868PZQz%iS|qBu|(R4r;ye{@1CJbwks{fM6e>J06BC{hKY3UGik_yyy8*y5|U$-Wt4<$y!+29M@ zO=eLW|AMiRanUC@xY)y#S zS~|p>p1A;YvE(fHJnV9dQS2&yqGy2X{E2&$tRJc@?~G7+@V!4+I%e$hl*R>dHu2#6UQAn$tMj*UTF9r zDiB+=mLYUO$g6s2@|xHAw7$BSVVl7(=|8i%N{)hFi|jk`)0}c?K>nwGa3u?sL_E;N zuoWM#v)dj_ypWZ%ay}m#iqvI)4J3N~;+<(rb3Vn6o*L&)sqdKWTr)mbRpu>*mK}fW z^}NxHgRP@wL`ckYej!h*t}-@t17WPaj;ac2IP8IM@iSX>NK_C4qK6;;dq|v_{0fyV zryxLq`2pm(*_E8a3-8v#(H5;&Hw_m?Tq=|SB^G^6KF;I)VA_S1KxqQ| zR`NYOSu!^(7>k^{m~5dEv)^3(XMA;wc`dBSt^GXGvd-%nByxfu758p;%()*VmA!TVL z6YEUUDJ4nZrSV}2m@V(x;6bS5O=a!Ueua!YukAM@uFCg8srlDVQRP;mu-M=Zvw(Hx(r zJk)LoCQEztooiGhq;vsvlL}9m->ZCR^U6@0*1)^mR<9r11#(7i0CHLcI)c|teM8TM z>ja6F^gHE7KBDE=eDnTXkT~_wr3otL;t+7JlbixX>XV^2Ppe&>v1=$jO;LzRZ(s%8 zCM$2!Y*eV{Qy}ck0UxVSd7tKer@>MzTB1IN-F`J*quQfOa;p*ehfh=u5Eoh*(DuI% zIhsBsoJPKg9(kQGqEhs3#HEbfHf$91b?HZ@K=fxVQF(J<(*IPo54XiLB3@f*0LCvI zX##83nwdg}{~UUPn8`=CuVwbDJ^vuZ(ON-+#I4MmuuyuQSInfmRF2+O6gws4M!!{Q zZ)MC&eB)Lg3;L3cdO-f{WI zgjMS>wP@_CU)tP}F#W#QF*jMqz6eSt(UJAtRNQECIFxs z$F#`~saKyE1yNsB;NWp9H`5bFw&mYxSP-8S$;@S#Vtlvz1i8}jEDO>b+#7EepCo)L zsRpabq3dWC9rHfeIK6+#@T($O%GsiH-6r^O;$TiD`=gBvS{3s(KIcqpjq(_v2id)| z>c{@Xd)T*6WsCIu|Mil9 zLfn-IT@A;fOYC8s^;{8fJ8ZT=z!B8}gW7u!>E-+;{VXGQqS#wcnB=-tkNDI9L5?@; zXJSN=^j#Y}LA!)iPXDH)a5tWZOJS2z658Ik&!0kjJm6ccY5SmHdX%)C^Uhi@%vV+l z7&p+)SM{S5ND|LN+noC&KNpe_R&_g{ei{Cg6d!As`G zVO8PNiLLXLkx<{$cg@+t%DQ<9e4YN@uPk-27a%RUPt5Z0Co8eH`bGMkANTY!gV(0@ zU)Y?_bv+&a$h-H@ZVwYMq(m^!Z3HTCFgI%Z`L0LR^5MBJ)D-uNEK}+VHgtUJKW5)2 zfr+kAG;rJ`m+YSFKP;0Z>3p)t>bME%wR!3az##nB@s+?p*3~B}W!GWwn-1OxxaE4D z;w-z<1sS0oj?rr&sVlwh;^e9>$nEi=@>@A{2Xbco9?H<6x2gM2qwmaHBM z{Re9XJdQoHepzl-J7tE+&F``>PydHO6qYVQRF>D0j0IHuE)RuM5+m6Eo@aNB`!CPR zNS~moSFw;-Ac5k^6;MAg#lc&DeJJ0J0Vyo7F_W8fy&lc}K1;}hb-{-=SS>eK<5k_% zgZeo$?{Q?~_nMa56y#~Qae?8UAc$bK{RE}FhNa?PLi`Lz&uOE0WkahP=@4~$g5F=2 zmz?LhhV4n?@iN~tR+TF{gE}K;ZO%Ane;T~ zie~Fy({EPGj~Wez_lVqIKkU z#C3t{<}S+qbPI71!F)iO_2N{)b(o4YseuMWZ@*+Mo^ykM|); z9}lwkUb+xw15Q!>%1x||8y@;c{T_4o+z#|TwkTFrLaq4n`8+n0mbx%%bsx{L3Puk@ zXDxFCQ=!;AsB1}Dz@&7L%Wow;JfIog$~HBrb`y@boRt-m5fKcy_$$-ugqE9z z&%VP6+v1!YNyoP)UQKSMs;;2?J0QP1cL0sttylevFE$VJ;DK%syC87oNsbrMs2QMM zC@oY#9FyQ{@>+eqB8hD`p*LJjYkP^C9lu@-VYFZw+2O&y!Hz7%swK_M(SJ711zLVA z-g;WBH6}%+HtPnYEw4sg(}9sI`Xq$T%4qsJ8D`6s4+BL#y-wVDsz%c6p{Fpmu*O^3 zqNc{{*1#C}_4S8XL(fBvr=FHElHsqcCCMZR`zcCW`+VZN`Eub!g>NNSTIR+wde-%T z2&Jh?Rw`z^XG?K#i64Etci{f=VCk9GJyZSyZ}XsO=S=-K?qwl;zKqEEyXOs);+}-G zt|!6%o z_})Y=r{o9PpEkGj${k58T1nfs+fSSrIlFK29TwsMFX3N|V&N$B_)7&l8+^=(sZQtU z7M9dY>&W%qbwB}dzWO;VYN_mfO4|T~GCQWqI4U$4@}=kbC5elVxhIw>EzXWwqC4*T zSvdZqH!}Qe*uwNj7?Hz}EHl~KU&Q*1UrE_C6MM6TqOp3Wf#)t~Yj!dDyT5A}Bd*)UMuo{5 z!%y8?fWs$B%S&oMdG9~?9xI`Y?5c3gJc~w|oLbFPI zWll)>=&ezazxPiMJCLS%trBJ(@#Zf6FJG0}+t$)XWEI}Ifg}#kjp^pG``@-P*Jy#S ziwyJoV0E@h@0X8+eI5IBu7$-RR^Z}IcaA%l8v&k2FEeVrJ(;0oHGNmhmg$ujuy15* z5Xkd5|H=*B*@ih${nQxeFI>M{7dDpOi#}c5cd5CEVzm=iHU+6O_?Ny;e?|$98ZBhOwOZJ_XydJ2-!3Eikw3Z&4I`F5~ z^4&su7ylNayA$laP;Q_peDAGOyR+N4x_F5eXxng-F(O3u0qJot4J!wZpRE+&lGGD$ zneRT-E)ur|AZ9*~Ldiz}VTE|--hb4~-_oj;h_p2S_J_25ahon)>f?>>p{?UxGlNNwzmz4B)El*W{EmzHFJ@Qx;{a?7ki5JQsADztTX@@6w9l$$XVH-y zq)g9=iNM(L7rb&$x;m8n!wgDcVV?CuID2Y}+ltga)|X4e z2QE|tu4I&E8bTJEPYXdhA-oZvT6sbs&u1h4Ot>t8I_;3WTM~G_k#NMD z>E~v6HZPShgov=j$z2zD>ZPy4c2jeY!EN)74;gORTsOe;(6qGG-teS|J>m`tTxr|_ z8(ew`*O+xBWV*TggT1Z2gKn5&++SNA2PP@BTJyJ?Ryau)IQuJ^tVPsCnVAcrwMPCZ ziAr4EH=MO#B_S`w0ylQiKj2->$a?O5{4>iVN=^IeL5d1}%OwE$D+H1K={=CtOjyLg+M{=~ta zlvE!HnS;W$Z)bX;l1RRuDt+i-?-{rW7G@r7AVB*B$q`^0)Ez*}K4y~eXf>!QV7~Ca zDaL0fM_IQbY5q;GMRvNkS29Xl`lC3~#;Qh!-DcW3%Bg9W;~*ECc!>)L#Wv|MeNLq7 zQzPXP4iIb9%*}ad=9}9+Vg+y8uWq?k1a0530uD_uIOUQ@4-!s{G_l>denS8?Bo=mU)gI_H=*9HTRUNsudGLweGtAeaQe;fG4csapctt9DI#h@Z z*Qc}2l*if~EdAjfk5V$%uZ9lxc~FgzK& z?r$a@Yb|}lS7H)y2rEI#wP#$_t{xs~z>xw@^`45AH}xf3&TNC4p0qKn&Ag~B;j$3D zz3B!Sp!?ZXf!E%?4`&y`vqVxN`HejePdD}V5NrP8IX85;L%WPe0Q83!kdNSD{Omz0 z2^YeoIPJP9%)X=}LGw?Cd=7R!ddR{~w(7dM*s7}o7^elpk@oV^;{p`wHu*&A?3p!~ z7u-C(i$SakLvu%Xnd6F&W9)#A&y8ZdmCXdid(a zk?EQkKI(lVdQDgBU7mL3m?nSjp&KCFeQcqoYZRhlVS9MG-H2^fG^_aD zynk3U3NHkJF8~dPc%609f+w90$Ra~j%iE0~;?~92RUPjK--TzAl#q6#YrKy!L5LY% zZNBq{4`)?6{ZfyVyXWZbr-nM*VQSc;k*WjEL8q7zEZNikS#*bf={ovghvJ(G#jstl zQoTs|e4RYjtvx|g=I*h4w6#$_!Iw?%;+g_%45)Kl7S6a(r3U$3R*u$6{Tdep$0@q{ zwY5B!$s<*EnaqpbzyQn_K;Kg!%w**N)NTHezbC%!PJFttwhl~mG6;a-Bu+fjk381Y zTxR8nC|!#8g;c~V=Bo6{KKyu&d+0Z2Ogl^Ib+)B()BX$WAX?>_O z29KxAROU;a+UR|Q2X`G&VDGWU-D2Hya`(qB2h8L{=8>8O#;!83dmV*iqR#kU{xn0N zE$^l6Y4g|?VCkW&s=;AT{LVtaT!T1u`>~qqQQCY)fQ`L86*mXW2SMSDfw6dAm1~Sz ztTEVP=dz<(AB>>!$#wo2Jp1iK-uI8Iv=q-Qc}!{X7`q_C)Dwb4TE|Wa986kz z)xsdv0tl{pN3%L!IeyCQD&Ks*`U}5}Cv>aMq5=pa;hU_t$&ELh+`ux*X!lbw&s37Vwkqo`6N|$?-X?E(LG=n!AVh;IU(MC;A z6=2p>^M2~WhktOLIX$~#`>CZ@E5LO_^NOgw=)-eo3T0c?;wsx16V0|CAqr%Uu(EL9 zJyWtoIV=0J{7bT^98dTBbrX`Bk6FEpcxnh)23u!B;TNQwzCRshJkKjsh@+W}CX>!) z?P_~>-Ki69^ut!EbiH%Rm`!<^DR^r0131|rbd{||L*<}g+p5=l&!hYhAyc96VNs4# zZ01pnHeYnt^Jk&6J(9|{SJP+zm``=~#sDNjCi>I4*Ui>1WCpNm_h2E7boz7LL8v9i zebJg*?c%jV2|jA5n1^ul9-NHhdo^3A1m6mS8`Uh9wFHBd9yqi=b73 z`a8$7FW0HO>96)%&%jf4j(rHRMn2|o4QMq*quflfNj3omJV7j zy!N_q4PLKyXi8X3NnX#CWfVP#FxL)j+G1w8TxS%4oQ&_f)rmmWJt24-lZKvp{&uQu zWX8zO5UjTO2hWEvBj*_RM>#NhjI-GO1aKfop^LTyER3L^{&mm1XsV* zQNq=hswTgtJYS887qX5L0zhv~voH~r?$)11yG@X>_4|-VQ4(x%MWwUDRLU4gi@YFn zI<@qww?bJix(k`@d!CO1+nOV-l)`4-7H8mr<^B^_h62_nxe7x6n<3QleaJ*ct za8FCj)))mw8Pb%wyCl>=Zlfadc#@~rRh-prjbwUz!-lxLDhSu`+F<>K&?$<7yBpFg zB#l+RyVWdiiy&3<4c+~M@;kxTh2y@OZm5?b^+sevE#5?5{peLhbXqB^5kPQ<7ClQ$ z?wIc_Rev3;L;A|UuFz#YV+Vqs$*Ph7h6~QV#?{3{;#Ub|sHYpsy(qA+&AG3ySX9I)HZh0Ysiy^hGxiRE+3%wP)-48Fkx|_EvG@f}n))bN<`e zJtrh=RTS_m@OYXdJo9E}{znDcU7Bq{T_HPi{cT}7J}~`kF2yomAZVQOAdlNGJ*LWS zzjl)6NYQwF!9}#`)xVqE3=SF~jC6|kg;wC=ZrPm3H%9$d3 zlzkf3QU+n}tnKfgmDI1psN)YTe~H(kl>lkl~a@ssH+oTN_42dQMpi}oA@VsL9$D$ zum61GswR%MK9`XIsqpeU$KL5X1|+iH9q}gC-xh}LV*$J}@Apt-M%JqC^7S^hS`Y-| z$Y@tqneT%ny1j{ApXYBg@b2@{V)$&M1frls`F`E&GN|K7S}I?e-JCS#ulhn4{?Yy^ z%k#Z*O=8)y8&+HKy>2HE6Ds{TvC+Fxu9A2pjNRU#h3`revJPW5Qy|;!#Mu*ky4sxI z3zZ$*7c_e?Db}HRDW84;;7~Opee`QlZZImmGq<4M-1~7ERZ(32AV|&UGL=p6c>B{& zjIn~nd_mg<^wZSe!nP|hraYMNz!+2G2+unaR6!Kr^?~qVDrzrJjrxr2vID*70HOml zDDI+0x&i)%Kz{tP@6~)IXGnP3;fO(4>PE_jL87%SdOWGWoKUMmHBJvDo1sG9H8}vK zD4E<4f41yZi)*XteP*%_3xX}4tak-Tzn^lv!pJV`ha^*qSda z)-jiY;zKu{DD$~e;hIWEf1PzHMFA7&SUqaErxf@}PtKCEC~0q=w2SA%Z4%!kMEfQK!m1Vxq?3RrR|f@vc(HO*JDO z=WH%6h?_I!<0ftR^U<7@kx1#WflnJWC*TsR>|cJ#He4mE=8VEE3!ML-0wmLNHPfa$;ie_59igVm0mAMcq(d+BZGRj!y6^`;_(iOQxeuk9mc z9S_K1BwQi0-qBZrI-u&S>=-ujxU*Tv;|8V#sgL&Inl0IFP2SOP8=vy!n2^Mq85p|c z4Lyyn8^-JV;0rOHbwS3@LvXb0a1PIq63iXyomSSm?v$`-^1R4eC=YEPGs<@oqj7YJ zZ!&Ro>I@u3yNOd&z(ckx>f1Nu&wdVd(C>+faKk-)lF)Jztzfb&vPkGqVm+FEz%#pd z3)kk`C2j22R2qv)>Mp;P&iXoysYS@vNctG6%ZO^6t9<_)!t&jsPi!bAAJ;KIReE9- z+KQ?X(hd%sc#LAH{}fD3`>p=H^s?!dxY2;0JLh;`@Ata7Ox30fb32|~NWVbf>b5T5hac@V91 zmeh-hl5K`g;T(6-8%ukvGA}&qq^m> z?)vlNv(r;&La9}9iHIj4)LbG|@+Iv`-4*ts=WoX(R2Ng4^=aSeDGYl_qsyP>>a=C0 zdyTtF#MusJEJiiFj$c0!MSUfAtTWn1(|%5(dZJlp13&NPHR5#?qVxcU4BC@$4z#gSBi590sCd- zh-VDp%iy4rL6I(W&T&xMX~F}ctK|(BF9|enx8nk|Mb8AX;0an}bPAvwNR-Ugj=00J zu-dpH1*Ue?g9Hc_n+&hpC4+`dQ>s7K=g z3dJLKJ_EmywHqz2x6Y#Zpd8td7mCO+b#S{0wbe zrz{HSRW2$>en?g9y(r7@7JMcKIHOC$4AWFFNBVZJQdN9C9T&TStX~rf*2$~Cl|U^b zy96ohWQz1P1=8KC)2wg~$`|M5DuN8AfsxHh{tsMT$roMZAq}s&9xqfTO3fu&>@u1I zz;g2w0BPFg2rODPjZP6tc2Aeu-{gVdiXC|yOswgQ%A?axlpHFlxi0AYjshnw_ILg_;-?bbE3Tgm|ys4OAHqVzf` zO`fN-&3Q+kj`&bmF_;A?IU9LfYziCeu@*`g>JZDu-x4G(y%d$X>F$>;u3(w$0EWVK z5lZrsA$cha-<}n_Cv&T{i!meEf!E63p}Hfu0sdc?{OU!QsT%Zopj0P!#hR%#E*27> z$~4qyI2XpkJ(oskm)^-Np$z0TOk5glr|b+&^?mqdmo-1?(n8Bwqj&> zxJY(f>tEf-g6n`RW?7gwL~)K4%MtoacIC(BKgm|l-z6JDbBM_O)FwsBH9i^Yfq&4f zR{JpqJfJVKubf`ecgs+(2nN+Pu9P=2)h&FXc8>@XJ?ozzP=P0XIdPRCU!YsO%>=lI z?y^rtO+<~BTj~I0Ql%;^(8ID89xH-8$<}21DO?(3f$aTaKUA)rcfcIrzfqfx*%V!l zy|^dXT%TFPo4#qWeJUK6U=72{L;};n^TH#p7;5SC*^*o^QXTs$V^s+o%+%8W>J$eBC%C5mN?I?Zhe-on z6W{|#S;UcFopnGE3Cd*g*?s}shZnD*K^}M(lf7SY#brqftHf5> zuo?sX-8+CN#rR|h&J1oZ2hh2EwNKpI@^saU4BQTgGMr%LrnyY}YJT&SSHj;E)hbdB0fNo_O7u+H7QPZFLCJ)WW5`0qzE2Hl z(nzw{tY`AQbb3|xyT6{IR`2E=sbOi=dn7b4G;L9KinVW4PJcH;e(L@zl{mkGFmpnN z*XH^ZX56IMcU)m??|J|gaM1$DH=Mue0}D9my4P{WB*ujB?P!nww+fIye12m^ieU=? z`Z=mC8lG*JmeMRfGP`mTnJIAt3n^;J*2Ii-3pXP-J+$zM)nTNxO(okblaB}|CC#_dW=yQ#9`3Ok)=vK4x@MV5;Z zx=JJkE(R5REZ-Ze8{kxE@dHS@S&pQg&oNV->Yvo|2M=Z3V)=uh|0P>$B?oXlV_1Jh z4B(%GCT9CkH_@H7tSX%@yP$#an^vv#@iy;23gilFR*_mI>I z0axwS(-t*aR3NXC5hA!q5H=H^oqhK5mlvKxy;j{wzt~YyE#Sl*;8J4Qi*{xo_XZF< zQIV4_9DtH+`Q)v>Z3v(ZLtG4vaz(Q0AEb6I9k1{$%3B`%5n_$k38;wNpwldNG^a9l zMicl_f7@}2G`&-ifx|tu8mU@|nbia4a05jsIowp>Lr5QV z`y#6j!fx4>D28+AY@uz%-{R8!tSQd2FuR1K+~p#f>(l#btX8+@kcZLEk+ezoSrFd1 z0~xvbHp{ob8k(ai&`N{oYKLc%!>iT*M;|P`_=A4S`Mj_%x34?-Tv}7w^JE5Np#?hJ z?Mn;hQVX34=ER6v6xvHO^#Wm<`qUK`C87-mM08SaG()~6L_x;}kER}Qx9+}*aZ?#n zQQx60jT%(Z%yE2$s$$C=S@6e?|kAn0Z#1=mH zMMnD!|IVUVU?6`Tt=qd8Yq~i~p$Ef^Mze47KcIm4}g=Ovv%5U+ed<_m7D`kKY zyC&rB423o)ayWiYGP_7Kv;A-tKGTHuBRX(gHjhTA3w*$&@B9L&-f``H z#VGy`vRVejk$+sKI zxJ_{Ohd2K0jjvlj8Sqd9Hy?g%-b)pA48-$z(1wH9x=$xmf8zw8M{&6}RS}A{+J9>; z!WC=nFCL$=hk5hO7YUzXv9RY)?cZ8tL@vEF#ftNfi3GQQtRBgRu+m>NZ9(C4G^MgR z6GrI+#~as@g(QbCUJhuyY#z40B)@U}tsIBwvRF)Rl8|ey{%c6%B1|}U^uSJbMPuxT zMS8%#Sxh*|P%>Y{YWFmwGlHbGSk^m}m}&bjVq;sYw)lMwmGR&;HOHlY)p9*^2xG{k09JeYkotcS-XqR;TEV7h$oCi+GkQN4 zrO@W-p1LGISi;M;YhcmgRem;-A-&DppF@%x$~RYMuat>X8qa`m-%0NrOh3-egl0jX zB(G8){9ZndAU)sC^(p#0Y7dNyTc{)+XyW@NbCfv6nuI3fxL0)@!It6RyiBp%5o?{I zCCdhCNQROKX_37g-eGnU=2>G`u26;#xxcFkAX;bNeTG54?)ToCRp=-D-vEE&1qub~ z)v=VrACI>8UR(^LohKCef8H+)W@R>0XEfu|Dl3T}P%vSa8l8k8yxBAq2)Fw z=#zu@h*MlS7L7{<3rMLJ>~+{kX=b|j4*G?L?`F;FIeAj=+OL!SN=Ck_@L16f8c_zP zgV(xJ0X;Cv(RU**jqf^+^gIxMnOz#R4=noJuV-U%lBBoz(7^m?kR1=jCB+n>M))g+jZSYa(xeIMtsH-r zZ3{-`ES5J(P8jX`${Wo>GOJ&eullCG;qFY*U4HiV%-8AwbY(;krxp!4t6Yx74&5O= zCWU*ATj>=`>)X`p)-A~bB*$jz&Wrb&m=l}EUZovPfVx{o78mfTBJmO1Z^jG-Z8UbC zzcY2tSBy7c%LZijuvbM(a_;it(tnqqt(R_girSSGz>$8Oy6b$9jqlXb{s)(QpNEV5 z$9L|!ks0iqllAfi3wI+UR`dNgHMJ#E+W5*kZMlfB;4tk#iEAlbUYft8P#!8n&-w-k zreU9rQa(I3gC%2MHM!@euv$%dzgGm1kGSrCH4{Z8OF+5fpZ4FWb9>AWd%g$| zgeO$#18vj2Un>kAdMM3mFY0bw?}7cYBxzpZbXx_~CEhNoyvnC)894Iw`%|*mQg*M` zZmi(0WAf?AW5L!vEhvvhp(P(-!HYRj`69BY$UH*8TC9^P$em!rvh#w^GY3CZ6O(SW zgxO_e2ax~rw34*}7)ruR#JSK@bG2Xb31ZueV^w~u#9X6dLgZcHigX0}kq*|<%?-Jq zw{MfM*XZ;*KegKbs;-SJjosNxwBeQV;t|2tn*CckvMBI%g_J*t6@b?-^hzUCIl+#d zF`Sa!2n`{m40FTm$d%aM`LrL>+eRL3;!DpgA?j<1b$M3@v9}2wP_aNq<}KZG&b|!Q z(EmlDL>USv{=Mk0uWpU=XQx<7J}KTyW!b6~TdBSM-s;si8iyL1bpb!43O)a1euC6| z4)My^bJFS#KvD(Q=^`WqAE@`yLt{J=B zC5A%8aDhRdYsM!YO6rI7GN4;3oDS3rKDghngPR4oOt@lzvKkn#sLiv_eiZ|QgWWR@ zn60^)-JuELhD*j&?S|gwef`Ytg>Qr<-Fq@?g`IC?LVqPk6g_D5wO{AvCz_l3yt?#V zu$Rhl+t@P(vfVHCO~NS_dgxfX>E@t`e!5{CBFkx15U%IONHQcDkZRPC1C!BRD}KYL zT|F1IYxPCeimUr@9EV@iW^>m6s#*^#1ENa9D#=m?>SOtE?@OHeN1w5P*+cuM@|~AS zgY@j9^RojG1&30Dv_Jf$ry9*53UiH4&%cor_?pG|JLr3KXZ3df<)>po4;5&KBlOiL z3`#dp-pgjIOUnvWkIJ2A3Ka|W_PNdnd>``F;qVqufCm^ z!F7{69)QbJDgWacg{uVE$07~#2W8b=Q?FTc0iw3+Sll^)>ZXDf+*2_8ma*D6Y!R(p z`^7#?5j^s*nek`a+Kh_Eq0?7RN@}Kk=YN;b57zUJr9Oj#%Z4DyF?wp_FFYtAQ>YW9 zS8PE!>U1F2TYWio5K^y^qA66c`&@jZE)`Inh(I-Q;*O?p^+bNOT__y^hCLZ7nx zIj$W5Omn>LPPE;w6311KzG2N-JwE3-0#9^c0p0kU{Z=#1rQ_4xGX4ffEo(K}AI3Ah z0)aH67GIVr40ZZ?EH0Hl{(afbK)xp-Eqe#x9;4xz?&We(z@w1CeAH!)=c$$7j}%l~ zu_vkGwLPHoLcxhomecU*_KQpr#JMkX=nudh0rf$Kosk5KKyYw^5M=V1?2yOM5J5Z~ z)}e~FGD<6$yDteXLHN9h$+$Q?JpW{LSkeIjV6x65Kkvv=6MZnMedOt+IzZZom?Vl> z4MWP6jy_h@Kb)4Mn}l%`o~RNYmM73lU-&0M6m z*f*uYsG6A*eoU2bmLn=EAEW%@O4ihPHk{i$dY)dtZPA2tE*(zII#H)TTgy7hKN0da zP)GCdoNm{Lk_7YAu!29Fr$yBi*TW>2-F`5<7)x$0rF{ce&SmG*_RvU}^N1Q4H@K11 z;_6w>3pMin!>|DODY^kN4J2*iskpr+6tFNWCJ#YE30ZQhg=OD$VLLHH3Y*@2P;c!} zvWb&N(%I?e8FF6O2@-Y_W7-#SNmQs8n1nX08|ZfS!?O?d0G}$9fz~Y~+kfBf1A$U> zL5F?yS7?GA&P&}H!}Y}-EvbP_|G3qV9j7v)PXOPF{90hvt$^&MlQ(Db;t$AUO{sG6 z$2r~y6Xs5L5T5Q?Ns1#{n9{87K`N!eE(fN*BjU?#ph4W-3s>%V)i%%6(lL~v**R=1AS(s z%KNrtUFV)O^wS?yNplZeJE;4^eEY#qnY>vM|Fl+H#O|1@3q}bM{vz@CPKn9e-`>A% z{Y=5?r8j}yOrcZp3Q>2`nLZ8jn z1h&urba#k6T)YA}v)FrVx3L%{Sa_Nc0vY@=8>8w0qmUGK-J-gsuVLS%8kV&^YBo*wa$M*L-#UC$OXOEpOL>WjQz7h2HKo@80~C#Z*P84{3k7QS-npIM~7} z^_P7qLo46%Wb5nShQG>N-doT8{3fDU@>NTVy7DT;u*~A43=!GIX(abbr!+D91=B4n z^Un>bxY+rHq$+{KCwt7p22_n%m#n1vFECJVR87HzufaGYsRFIV83MmpWz+y3%%<|k zS9txR{9!413Q2-YPpGd1D$-V~Cdebl+Oq}Mxu~mihtjEryg^mIjBL4ilR`cE*TFv< zLdSip)MXuld2sAl%3%r*B-{SQMx6oM`RS|-Rc}3q?oIQ=ioRJv8mm`$S4iIRtv=JK zr~8#(=7`FJ43LIz5f&WiTMR>J?Ynt<3DWd>aQp*j(9e2j>vZc?zoe?(?gUzX%C5)F zt9|mR61lpHKGQzU$-n&Qg8X!Bk z$ia=`x8sWaa)j|)Eey`PyUnU&hn+LPsYUG5eQ>D$A@k6(GefAZ}B{}(2N60O0xjd~|K_g=8@sAl)sZeBkEbW>nH4(`>dlgHEWn2Wa zO~n#g8-;%@3sGE}V`AxqL)M1_^&IT6*EM`C)GRa!an-j!u2p2nciqG_?+Q@tFmeZ@ zSQHqaz$hPOc+v0E1rDaf0{a@;EY9en9fy{XT5;txy+Fi^Vti>b@N-L9@~7!*@N2GT z&Gl&?>z!le7Pg*j!@>xem&6nn%yKwL)Ay-3pnFEMDN2>T>6PMK)F^eNIZ9JuIVvOp zONDOA72QRb5%z?05%35>cwDcoOc4oAf;icJB*Gn3fWl0t3<+{Khc2x2gp$X5iH?Rt2Jk%uEs;eyT8xg{`*=};@-r75#|}nqGxizr9fBF-sSp&~ zdSD|S0c!Lfh)ExGaY3l~WUG^7`509gS!bT)p+bDrq$|4&u+oaXwSAizs$7T=U*LCoz1&(RVe$&vw9QcTbY3X6JeGltVq2tO5 zc!imQar(vkr>8;6q4ZKOx996cy|Q{)&f83iwbdk)_gJjIgm1fg>br1x#zheySy%B+ z`z>Rf7rxm4Gd^8=1wV3SSi3T)auhapG5DoXI?TKk;YM==(KRv>*E9 zMCR&%dSUDw@|Za}m|{l#&pmSIomP0~eJ=A_AE{g9>bu}c`@)0AsG2IJh2;2hDDAz? zE{|V?FLM_5)CadKNd5Y@=A_3PVRz|}#ro;uMDMPtM!m)jLHeIL{HVBTFFZrPX!YhJ z8nmjM;|(=#NC<(hjy%E$ftFJR@%Sh*1W!Ls`tmDU7^kqB|VmTdxO>G zev!_(5~QNIec+?UL?h@<8{-o#mX=~&r-f#&V>YUZxtmgU^fGgi`;lDCYh-UJ_NKXL zO&pEbK>FG7Vim&$vI{Eie1PZ@^BfO@lY_p?w=F=d#L71WUi{Ba zjwJI5-3u-u_1gfrs#)Y9`7@?PrY8Mit%z5k}w(NHsLFG&Kbu^0+1SFl- zc@-OH)!2%m>-J-!Tie!vdz@e15#S6t#-){A%mH)H3JwZ9-rE+#a&-GHYg*wd(_pZz zAZ`b4)5FhGEP&fnrBmbN!FzjpfgJ-4z@`dhkHPI9cQqYrs`$$&sgn-%Z_r#hik=~R zQZ4}3r*CZk80CD7((%ntsDva&t9;}*Cx18{ltMnfayEJ71k`e)`TCmd-Mml9^6FBz zh2yL4%5|lof**@&YEnzGQ&>4|dCGb%u{2bP5QEOtB zXqv7}3&DV+H;6UG++bJ?3(~>F)~|)zfzey;)GFmSmtPwndsC zZCnW{*fNLh-?T2eGamiE*YUYOV=CzacNEbTUr;RTH3wR;3UqM?t*ten8(zg(;eB8E z(obbMZBW^JfjclPB^6FKO(nzN*1bRU0Z#?yniv8(( zynkOF%{WhaO=QR6=2kOw=(&-^**5oCKP-QP1vbNyOvNDoL%ZN-)n+59l4kT2aBM6( z0*qft3qU#gsAtVb7N(O>yh?s?IJ0~k8()xIS8jRJsv^@GC?c+YQHuN+FS*mEjJs>HX*NY1mgJ^kvtv2pXMy26 zm?@Z=rM~(|~d}VVPI=UeIR$>em&CuGGT3_+Ns2Kro!TxT)N^A8`KQeayooK|BzfTvVvdjb;PySoOB2#cbg&2)M;noJX!3 zV-whK{Fu0o3M}E(y0P)x!iaEj7^}~+X;;^|tHfi4%!B!%r~+e#3CNw_2J|+EmI0ne zZa+Ex;RiB5KV?ebxz#5%jb0!lq)bYGmcx80QwUbP*iHa_)6SQ1$yCss&s+uNd4d+L zVHpO+&F3cYwe3QW!V$VL#dAwv6+1_qTFh^uGpfN1h%c?KlhRNw#)SVLVYuG((>V=A& zxzo^Fx9*G4@~3W9c=MkGTC}|{ih;K zyjqpR1>fZ&H1`U9(r*H~cD^0F)P=M2OX zEnhY`h&=gPm<_B7`85FHoz@(6W&YwIaBzKGjxlzAC1jUJM)Ir02c)kIGI)AgbM?BQ z0&tNHsS~SuV?;)eEO3Oh-EQ9Wd|Cm$3#%a>Xo-)+m-XE#+e3WQUSc9KAr(W~@7BY8 z*KgNYt+%9?EY_qpzfhdrWjs2xReJCEm5bqgufb~|cwGr>W3=9=MOx`T!#sSJL?ax5Cu+0Nl@kzc_qwHY}=qScqWZ+BM#L)>+v z^Z5xX{(gXj7(NPd;;F*@j2k zy*=lXhD-LJ^33;XREB!a_nq(iY>5-3#4v3s-tL`o)nV4vpngjWsD`>?{rA*UBF-K7 z6*6FkX$R1U4$p}7cgHgd?VtVhN|#7d#9?n`2@%BLj`cTDmY9_kekXN86P?0}tRT+keHF8`X@$as#hzpS|}NI3>SXaA*flVfFe zWN2XFXZz8`&FG{oi{;(HB=0K}PzBegq1)=N+|&)j)~g=5D1t9WgQVz@qy6_e{J9xn zQ6?>N{A_Q`_GD4(@OX6nV6z-F7pa1K03%*|=|-FmVsL5%tj?u?7oshRxkf&l@vvsR zOQf#wMBSWghpg7PFX(in!3~P*`b4?xH@{JhhubTj9raC3Qv_e8r5}C#E4`SQ1|u%_ zOda=3t@+t}%D4_^s2WshIHtarLeG)NmsY%>+$yuCW_754*@}dZT2690pB@! zK!#%S(d=ZWfBW?56O+bJXs2W`J;j15R&!X9I8@>8_V$XDjD2`1$HX~AWeQd2wer}C z_-0md7{`K&D*A+T)p=d9SQmRcXwib6^U8v7p00&m46OqtWv`U9TbEa?cGpqqMZVf} zNuRUL+I58{^AjJ_gbMnP=T_g_x9}@IgQ_HCf#S{2J}fHW33|2@#TG1Y@!Z8s)_`P| zjl3dzTlokjs*`YDCVpPpiIf;kNr38%Rr&+RwdjyH$IgoTm8dat=qH71?8x;7-S{nT zP-quLqhf0dD?@fywjDIh@Lf{IA-N*jx4sd~@a0X5@42_xc!j81( zW^1ovk;&$m%l)73g53brA)ypzFYk(3f_Ptk09VL%{)#<_wcnOBI3?#qATJwVr(Bgj zR~^jqF^BNB`=`I&MWs&xY*({C&3aSu&W@)3{FNmwNwyNq>}aI`E(9zIeGJG(3P|rt4pJ}z>;gDiu zW}op^;ah8YjT;R34I-W3G*zdAI5Szn(?WlIK_}7!Z1=7b ze4lX(Hl`wlnpujLWT6=TP^~yPYtaV=FMI5vlg;aBST)CtCg@QEzNxO`Mvx+FKUM~` zx``9Qha~nwAc1rpSUXUv+5$bB^NuobAV}3eA$--~D?Xy|mk;$}( zt6QCr0^tw1KnBH}9ybT?OyJIVNj-E|IudxrdNubB-2diW@fll$DNvGd&@-JUBPBf?KHqJ$78O3AW#T=!kF9(@EDEO z&Dms(065{nUCQu&OW*aO3mW^Y0WgUR-4`+R$ZB4pMVUm2gl1kPs$fds=6qq=A|jfW zgDg14vL69fXVu)Z#0RV)HCL@DQMnXvS)@T-pVbGQb?&T0Na)}ix21+DY~+;L9N z@`B|&YYnnp+e&q!EOr@seAm02FsYXj6SEK{e1eb&aL z)83^rJGsC`)yRy0P%;>WguAa3~aR zm4qnltGACCX1Lpl7hyS<5D+Lbe)3wp|DPK~UiW|bnj40PxzCx4<(gRhiH-la)|UXc z#B7G`osX9p9;(;B*`Aj%Pb!tUI@^^bJ~cAC-yamX_is{)XZ4(Bz6gBcu3fxbXu`Z5 zyOnF4xFoz7oEc%)`USZ&)_NR6TgXOnT-JwG-*Qti500II&==O2v70E^e_7=U?cBUr)3VjR+QdGBX9XkT4MTKK-5&vrWC39w?_Vwi8wf*tV0C|F#h zM1lntEm_hIo^^c(b#Y#!82+bNuG1Aue>ifn2_slTFQlfpOqAb0y*#RjHsV6dbgB&7 znP5;IB^VgH^ns_fBO`bl!QpL4Y2Xrr-#rcbr_}y_Evwp1#EW4c49t)AUVNnAw`-_( z{O!c@9e%iOijMEg$xc^pSy3 zrl)kPgT-tQ(?e&2w@FuoK%z)rI?ugx(CUSe%t7hurIH+fRW^46#ybv21Eu z(6T+v*}Al95wD1l`q+ez7$*)vF;OuQf{HbKbM;_HKe*9zO5L{_k@{#s0K|*R;=l~H zd@-GfS?MnCQEG<79oR`=uTeI#s0%47Q(PThe4;iCmyP8U6!jE)Y8Z}vwaVu| zP|XyAFXC=jPq`VxrSAV*_w`IkYV04xlYDp|W!mjezx)G1|9>kQN$YT*a(q`uj-S

u zf-h^u{jRxkX>}P8W44#na^RJF7Y7xJC(1o-eWWNY6hAb{aCN%wb;fdFBbVs(5Qw&F z&m|zh{qODj$l8m#+Ol~59M8XMckp6G#ae7C3WePR>2@By`i2>eIt9hLY$M$RNJZfL z$p>aDDX%eA4A4%tQbpVlO|hb+O8iisqT}3P#{b=iwVTxA##@U5U5zBytCn&+#yGSz z*2v2Ji;=3c5=~om>GgZcL$!6I%76l-*G(__LlFMkU>)Fa-!OcWE6 z;Hk%Av;JFMFYhk;X6FJMovo9sm0(3j!U|HaP~d_G_@F@jBt@#Hm};Y}Pf-}CD+7>K zWwDn(#Y5WuH3IqvoGyOqM=}=bs;IW?Pz(l;K4luDGCa#vGgto z6({wDdlY5QbPLNo99^A$%bC*Byln4h^e#t`Q-2}8KrIq&1QzM0+R3bzox z*I1|^c6mZRySO{1%9*R=BYSJVSI^Dd$4Nfm@p|Z3*&R8Y4Tx`(Zw!RWV8Y&EVmK$6@(6PdpirLkYiov2G|IhfMm1|4+{^MF>)$wjf&r6P>mMTZ{bOc^d3bx`nU>qXj^^R%2Mg&ce?zHfhdhVRTR^WBA{Mq>}sB9u7 z=$zYlMQNC`j&o_wJg8}Kc*Gm|bM;|mm*5boyA-)!R50mv@-3nS+iz3lJV1PblN@)8 z!@d!F$IWxVA{%8AD!21Hibk0Oe$u7}k98Mux4h zu??vVY0u=c9#pr`KEyQj;Lb^DxU;GMHOFEs6S|HAOem<1k6nEf&R6CcK5yx2IsUW% zh_6zjJGgaB0(gY1NOEGm(O}HZ42uf!0-kFz&UKIv{j`ii&4C{mVuiLI%c3_K<0aY1 zy-nR8qwz8~(=t#y9Vs^T&QIZ{D2X$OhA~1_M%|&eh&r*3O!GUJI^1rF^TMHpSoaza zJ*49ukK827umo=hvz`BKFkod%6sf7aVid1tks$xU(Zmx#(AsK)pKyP_)Hy0i+%4;~ zuuEVaQ}~*NEZ;A@`}ZPS+cQr8d|;mnxVJp87k)ZGSZqaZdZkg$${v#?6Ak&xxz zHtnUHxjGYWsjTpU$IwWGc2QwspJzlA=}OoBuPpds_ETKEad=5zWk-TPhjcy@boaan zyC14QR9$%LyL)+Xb67|868nj3%#)-)RDy2AIXCX;KC6K(nWTgAE(r}MGmHDvOVy3O z5qT1wD-mP*cue)aO;Uwdg0RP-K_h9S&Us9*iM~DAHsj@~o%qRX=3u3x!r1u3qdL0fnwaHsj|YHVP6_BP*i@+|H- z9>3Xk*{q!yA^J-bM39;OR8wh$m*9sx%j~xrDI=K#5MaqZ=+8U2$vpba%dN7rg_pRm zZ-(R<$Jc73Dj4?d?t>^NHFbrQry@A45Rg}$eA8|hj`d9@jq*;i_x0lEZ{MeBw>O{6 zt5^51F$GcYQJn;N?IF~C95h{ze+#v6;-gbO!5prV1a->wM)yq3<&OBxcENOQUa`g6 zJxxG`anF<5Sn7-PCRn}`0=Kuy73JgV3U@8KEXFAVV82(Xw-%2nr*p#%EARgok-zao zihjD2c~^ZtC?NH1>^#&%k&#f5QQ@}+4bA50%C!8Z#m7*4c;lZmNOfr=3%)Za&1a?D zB1cO;+Ov|K-f5-8;!%1ms1p3VRFowjk%73|arw%RXP?46?3y~I9jAY2sl$`lwOr}b zT4kOtba??`+Tj9aH-sTdTXqm-%83$?IUki4p3#WG8e{{5E5)>6PH^(y6r>8A#li=p|Zv5C;JQFiVkY{EFFTi zKuwAtQr!Qsz~_*4WB1eAyZDOh)LU5br7?++>3Vn-W4nZ__^#eEufEhc-o>}bG-6}2v!S6eM>sWf3y!|b# zbh)Aw_a1q~?D*0W6C+5us-%<=hq+*L#3bmZljoEGj|-yuX* zd^{-G1T%MiAI;hoxG&1%En~+4*IDVb$_3{YnWr?S`c-Ciw@7b^$Fm$donBG<^aoGh z=V|t{b<(|orSl$t2I1t3_;GjTPR72kHNq1zozxQGL@kWt`VktuS=$+ApM@2#E?suo zX;D|veaej=v|*d=HlkA~d?LG##BANF`ve9y=+ei4)qn#=0ox6M2yP1O=j#1iEhc7? zQ5yd4>(BZ;UM69`msnpb7J%EzG`1Q&#tdIZ*~Xwh+IzGbB|$=VBE7Kp+L{O+Q2%z> zvch%MW{<47Ll^BTjR#xrGC+Hdcp^0Xz(hB(YErImjAbP>(DjYlAa4PmLT)}ncCIA4 zPsH`)Yj~CsO|f3FzreM)FYfabR}OIcDg$>;#4~9q7xQC0`lz&Hb3DL8{$v##&Cd!?QtWG4#fT!%6ONxmzj;MM(IW%XVU$t6-I0LN|@`Sdw4POu?eP- zRyxnZcUmk`XXMf6(`9HG?DkYEtOxfRC5l|ED0$srDG=(+xe9nkM>4mE4%TZ zSW}|K4zJV*zxfG*SESCy&X@e-A+XtLgp1o2x6(qCdP-P0SapUrOY%;e+n!=Wju@pr zd|e|Nc1acYw)o8DIRTj|P-$iL#_)rOelLpd+Hx*1P6MD$#qP&uEpE43ZzW0T6ZQG6W!+%-;V&O12 znYH)(V}Rm!=PS0_0R=(KRxBl4)lOQoDfOOr+09Vpo2Cd04w7Bz{$60o9V+~6bS2|tq`i~mh9ECRL@->eK&ULLJR{xQ|u<~j>Km!{7u48W3=aCG~#B#2E|6jPH z*5+s3(oj3S%EB-1v42P}q0Nl6mw*9oL_IO*e#Lxtlfv?aeUWylCy54Pue|F@3VBKL zi;S-;|5vyOKMYvaxa29kQafkjcw3PC!XWZBYsTW}{46VLUtG1ZM8QM-W`1e%(Bprz zNEOdGnPOf$R3sE66o@+7`|PwP7~8P5V%3oa-w9DcvFD}npbBH@)klj=N<_RBjhz8cf z*dl`hTcskJ1Ajk$Ip-0osOpUy(HFr>8-Dc{``yc`CIYm;Ut^50*BYm}{q(0i<-va4 zhoU2dqIRyuST~_bDaYB`D>S)XpsZq5l4G2S5dQE^ft5!SG(7~*IPZfXsPZX#_-b6I+4w#7!}rGr$twMYuc* z9Tr1}SF}S?TkD3QE3>9Ui|JO>w}dY2iE-KB9{ELUmt&DVkpS{`S+&{wQpUBVg$F6f z9YczKr|JtKO<-Q+EEhp#_2p=VkCfO1j9-`9la8`5CT1%9;L)tvrVmuN>wzY9F5k5F zp?lZzHI+DtXENnp-%3B-u}zos7_z`1EU-gojq4koh{kNwq@2s!9;0owq{=VDr-Kp9 z6zHp5^RDAvd#2;^xgCPb=unNyJHV#i1bjR!>5~RSXhL1HIuaTSFBb5xy7845N<&xB7^99i^G3cf7l32`58}Csa!VXd%I-tppcL|!u z&f1KLE9y|2Q%P$X@u7wjI@_OeIM4R-4M!@j(#}H|w*YE@znRlf>`+3x3XHP#I_kRi z(u(F(Df%T%Z_KbO?&3aj3Z7HH=~Y=rs`zYX)kk{OV6FnFc5Rzv;A&%pHGD`U3><+{~p&>WMPI#)A`k+z?!*3)HZzQj#+!{x&4@5 zcgF5Nv*`X;ETUA_3(#c$p5WU2fann_ysRF2j&?ejm{*@`CS{83!$=>ns+aLKo3+cLZsA;+6<_w{~-QG?wPuTGmAvey)k;U=d!ju;E zo~^W-F*gF@GaPqqJx+=_bYko+_QOP2^Nx>Y3iHwp?J%%h&+F0U3dvT*OHqpP;(*)C z_x%Y}>7*|}d;~d&R8PNOARB{SoY6@=cuAg?+Rda`aVyu@qWL5zt3MTUFve_;yj<{F>d2}kH-kz ztK`1td``J{g3W)vLJo0%xV)8N3;_!PKMS+XO@Co7NrjKmWigZ?1?~N^~q0FLbMf79h93MS0g=S#@YO!D}EnXv|LN@MbmzdVwfWr>Ktk+`Rq0 zM4IACeDltG?i}mPTS$g*?WkiiM+=&A3EOf9a<+y&&@NWpHSogt9#Mm`ohO@do&5!b zsl{^x`fK?(KR&2D+Cs_08qdWYX$)PAcKEX$O+~aDqP)*H4Bax5o4er^0gYZskK=mb z>Ns59QyEuvesQV?rb-Z=y!(WH>1Nt>-&Q#Tx9UP5E1Z8P0>t8VNofi-L_{|Mt zQ2WFMl%Ft5g&Y?Q+7GbXCZ5 zB$omqnts~$h0~-jx0SS}9YX|Mvcdgn4lRaJDIxb-TrEFRc@yNzTD`yKd_}_&Lq0wA zPf)fx9Q2|ThL}PS{=px>$v^qr)>g1?;f(Wj2W2B^c>7}XRHV1UT zIH!pFbFE=ri8_&$4x+f4jVp}asE*Vb`!*xVTfeYMFyp3ZPRC<_kVSrBBCh;0u}5Dx z)od6nE z=jb1W>iyKLLt<;)8Z=JG9qOd05643cyoZI!{H9LBZyZMN$Zpr^;z~ZXl?_K3e7ezl zD*PM^+I%&gaMizi3_y9^M|w{S|EaH~XU@JrZy%gKcl5WvZoS77n3gnmVn(9W*WVEw zQ>+ItGipn&HivZL(w}~Fc`5Pw5w`~lV-YtuKc;3W-Y zz*pO7;oo+FkQPA!yApp-h-pe0F#lVG*-U;}LBMt;#4K|ELW1m>WNtw%>f;Un{&021 z!F8UqIk@05@pF|P_1y0XjKwFx&2@fCCR-UMMJRQ1OEXKzHn`gG`;+#A_bwz8{5yh71;3=Kj^K3)JKF1ZylhM zJn|6HqR%x`BQ!Q6TZ0d;urIPB?5qyol+5Y(;mn>Ot`+-xOYSD81r%yE_!EMe;7s*i z!q31F1OD9avuh8uk;Mb8J3O$YHZ9fp9QQOJ$bNp+Qjf1deWuFcL}1Y?sEMh77FGCI z@zGa_19Bj(Du)lN4PblA#FUs4H}7>(7>nHTtKWsitzj?fYgN znN>&`tDpXvA<@U&WS+FHyRMe6n`Lb@2=l_A(1Hb-QwSeyw09d>&cz(@VS-=A=HXZK zMMWMj9XvCUy1NW_$EnP?4v|u)@ zm__q}TG#>Ga;=^y6>^nj^hfhuz|a`{!>OG5dUq%dqJCxUeYwR!I#Tv6pul&bKD(8) zGjw6bo>zB_*>jwN>I*I=TMuJXJ>aK^0p|+hqy_Qd-nv!oy7Z2wGvOyluZM-a6Z^J5 zExEZUK$q=$113$MRhI}U{0i;x6w zv`~iH^wm~U^R39A?GiMSjz*3fGg(&G4n~$L!uw(T#L$gx=blv_CJ3=kk!unRu{kzi zDjV`5q2=w8vhe`1meRKnRMz=@1?gK{=;S?rH``Q%$UX0MyeOgK;%he+_`{z*?Qln- zx>ERBK%U(a{AAa;1d62oS(TNW(v_t7G5j~*#CFnVK)6?VZsc0YP zZ-o;kpR2c9^f(cu0&155;R@@L4DPVC#4xQj`JbwZ3(R{baSaagFtiPVI@P zEyabvo;wn91sOhf0WWy>CgUiLVg$wrj0Iy;^t=X*N)H?`cdl6a16D-`lEUv{&0?T# zIAebB@11I*JBp`UQ}*AAAr*5N_35F#w(Pw-yQAw(A_rc(WsUFcd=GcWM7(L*bsRfK z$5|J*lG2oY529?oI*d8nQ603xwT3>|0!rFO$y{vHr}j_%y?R7k3wFT$jy^+`{#$rq z{8r-;@o<#)xXq8<#@D@`y@urPYN#+a5PwV)qD`bk)>Bf?Tm>%$HC&n;+U?Eg?j#(O zmmmHRnF|DUwZnD{e?WjeE`cj_=tQ#>h*b&Fn8nmTb*I+V6;nPr13y1-lZ&l1|4V+2 zd_@d#&+bI{V~+f8hou(roueSd=D^2p7k47_2WQ^lv+`6|)_wrr#U#Trs}`Y-4#67P z%2sxA(ixm#aQn?9)CfIwLju6ejk7wjPul2zU#BZoibXEX7A{{-QLhtL!&dSY2_wBT zu6cKujzDnVyuBo?v@&K8q`a>fPpl5DS-vyxazqJ{lg{}{4@3cYSFXt?W%7(<-c-Gb z)R?;A*%o2`LQF5LBzDc>FtKhl;iDk^BIfM!*n=*L%!EnhbD7Pd1v3@&r9kNvN5AiG zR9z)?C6M)OUWfr_Gwd5C^^$si$Ro>B;V>RAT1uBmOa4^M7Du}e{jFrSo++LeA3N9j z18`jBOX4n9omu~o+hG!$qKr3E_h4J}D^QVRR84nzYcZy=^^w2hckYZcDuRTO#ek~F zYzn0ytc0vK#OTI6#2GU!)y!lOOtR^H(-YjGLB6x^qAH1Zv6~#iem`yzfPCrr=KdTR zXev$zRMlC}FgM6I$%VS$E~QxY>@jZq+?UlEadRY)9N%Ob|Loy^jgTICFca#*EwXrd z;PMxvgF!joiJ{PNo44l=XX0l%l6L*M6}`lFZyNpAyYp9BLEAQsbJ?s}mkCws(b3Ka zdvu|AC)XX4Bi&x%=ZBef%9nDlpk_HolYyooD;7)d=fy9~EH6anEu6dgXK*`Jn4+7>aCd0YwU>#j zDazzaRYL0|gY{mRRD$}PkO(h3-Ld9CcmD@gIsc%m=PZsW<8SS4C6=9M1r?jNUd=E1 zzIkOQ0GJWt1=1E6Dc=qdOZ=+O_~dMNIFq)V3z5A|@2GJf-Z5nQnfC>1HA*{Qm$=hICb@w=-*k%t);soylKSq?E~Ad1 z^=`eYIYu?McYpUfZ)|JxKP$=KZv*2;B?mL;tlFQZo9zTPg}{kt>ZqgYNy&~}Cksxz zgw8Z3a+vK-u+Ej&!mlS-{lql)>wp~H-2&$8g3=1&5Oi%8-#Z~%5;Vf?GOegB*$(FQWzf$@ z_3Ml%h;pjHaH*Jv>@5F*!S13O)7Linj)gsGF>ozz@3v)90s;c|*2 zc?J-R^hDC~496uG6Q|7E4M^Iym)Vxx;1J}ng8EVh|0kNc(O(y$as$@Nr z)$_INuCP#F#LG+AS=0N5IE|aWJU>F+<&DcJKhB@_4|%@4k`M~t-Dpg862J)N4gYFg za$o+G0MBX)l{qYu?K0ar(qMj1fA{}*Z0K7JCoaf?M~){a*f~V=38?aywViDe$Br`@ z%f!nfU94>8?XAq!rMS%mYHb!$z`IveuAJn*#^Rpi!*e}E*y~lLtcEIQt3KG~=EnJ< zlL~$#ei0n~OFOVn2k))6&H1I!c0m3+nsg zZ*!ymq2c)D{K+eoov$8LU7-qs!h4qNhp9(vVbSnuDJ9CATGy3Sy|HJ4IbF&#Qa7d_ znEJTCz3-`-UlA#GK$iO{voN_9zzlCxvA0PSj)GC!;4zvIw$y^rsCA0B$JS>F+J81o zKsGxP4y$KsHNLpiNbno|(!mlqB>q?M@?UcoIHY;e#1lBEcU4t6FV7SjxO)`&+=!@E zoYIPJj)js=Y~jNlLZshAW6QzVj)Dw+Q}mpeN2~2T|5Y$>p4q&jk1B-<7_-qu!|KTEJ0lUCOfHiP(?j1B>hd8f}!!nsH)x1%gBs5wx8VtkeN z@L~|Rs41AsDpVk%8ZDS&l2B@u3Dp$wm#dSu?r?^!EvFYZ%LICKm~8UEnn%4S#uLaF zLbUqIiewv1RmmAXSgFEMiyBb+sru+ktK#G4RMUC17E@?%l~wV&$_BcGNI7Tpg4!>c z_c}odDWW`+8|QKSbP2#`Cl#AZzGr9SYZdf1NH$|7Ql(SIIK=>RCw5NGI!)a*#O~Tb+=)u$$x_Go zBQvGK_E*vNQhZ~wW3rPc^=0DW3ue-+iO_4luj^mw@{7r&9PJj##azJ(kOU|?*TJ}( zE#`VwS`7)oU(GL1N*`QWnvdFO|7dD%)4O3u?7H(8`2WXxbr)e7=I%f2f3izYSNrdSj{Vc@(X))ba6eaKR z)_YrT^}7m(s!LtQi9=Y7tds(_qj>Ii0v$#jZh?=pwBdf}?X_wKJH0B2hjm=bPHGW< z)HBI>9Y-cIs~vAmZ=nB(YB!w3gX`hWP&@UqfG-D9rICYN#DqJJFrE*?A1X8HnT{T( zAdzEbS>qhG)Q!7oH(53&1<`8rZbGWUfh}#CKiiirMoae|B*aJIFyeaH;|r!E^`+{| z0UZLoj<^@zRtm7}NwwEo241-4fD|Ivkj;F?;XO~0T(pyEzMdfdGK>#BYic-tX5}Kf zE0*$!l(caoLNf#3!9iTU%x%)UoHv`eI$O}%SvKJY@{F+&u_jFhDd)!)b=g`^3i-#P zIgn{^%UyN?e6Gu1GO;PYf{%8+*CI`KODPI%Y7@an%ToTMc>cNs z_MJ4_7mR$Oav$|i@cTdOPXk(~!B7~Rb-L}ub%i=g;gI$_5Q&HM{aBEE5gx2@QX)QO zv%=Ow@*0J9D{SRbQS7qmOyufCBNc}yab05I+BfR46d3QBFI-eI;$5cF-b?T8?<$#Z{7F6GFwS07Yu<9peQIqp zO&Q8ZcC51}mA?X_jhHy1U)~AYbox1jSv?YQmFfz7hcwn%-5Up2kg~P@ywx|@LE5oG zHLd4x{_l~;Oq#`G z`3(o_$aUY|xD(b>F!_^kB|*qw3Yg|_ZZjV@TKBO-w@Pqq(EBnE(BzgBWQLoB+5}kl zwmVBJixjlw2P0_c^P!wq&HXudIs?Ny;5elI#`uFAH}|GjQGiTVy9>oL|2M`m&e zPi@mM|4m%A6(TD2=@e*ERvi%(rCncr8x+){I+!9Et&kY&gy@tOIS~KcNV;?hopP&?IJr;c}L4bQt zx^v~SUZ{YtNaSObTL4tDTWKJ+k8ed6X*Wt8zCdfkFoOTH=UBlJ>aie_Ot*)nWy`bu zt3#6(76T8~n$G)cCF_u8q&!FAv3&w1=zfvj++kwCLO_WzVB<3+vX7j7V_qJQJxfRa zmO}pf#{vsm(9R;UY}NxVu;la-<BgM z#k$5#&v-v{q$&~rZ?5EhuW-ZZi?dV!G28AYeEmz#OH&hPNnml21 zQe;Y}t<*s+Hd#qO1Q7;ntPWPo^KA8UcaiY@4HGT+K_6IpUp<_6z!+vMM|r>VsFC%? zx&FRcAs#L^_`)+OYWW>ZSpw;^e7biwu`Ed#7nBUAmR4jhUnXVUH!Y*~^tY!zw2>O| zsZ~rRYtjES46v$gH>(<@p2M24_cS|>(5wRX<9Oc*>Er&y-d1TSjVw{_2|P9$J6>(M zHlMp~FEw5_yFng#VX_&#$2n!8a3C{eDD{Z&YzjdiB?h?EnGY#_UUV$ja;4@#qk-;N zk&8+prlZusGV)WD^WD)#TK(aH%J)OY<&vz`Ut;2Q-U`5c(QZKh6aj7@w0-|R!Uc#= z+ocnZNjE=&*SDS?lXme4CXteV22AdqYsYCEtV{!hvT-EW#HT{^W6~}K99jdwPqfl4 zM^}@H`U*aKH?n##n@b19-Fd!?E1Tmi${)%4o;nX1YS@3j0>Gadr1ip_WuJm)o!=BR z;_@>DTR^{gtp!<(p6~gX&EfP^TSN^bI`aoBKafn0(F|n^DkevI@huHq3|OlTg(gao zt(>uCotgvO3I@G(PC>S*tl>q1(`7<<58i=goIlFHNVhz}n_2K_sgUp^C6D&@ZZXo9 zg)r$;Y|w*h+;0m7?dP-`}%tp%BCLk673BGc@bxbl)#bpQ4%pQ%bUv^5e65k|zr z7J@%sBO}%(BG_jtL)!K#t*9fqi|OQZ9sJ5jZ+AYLN_DMMvh2!8*hUWjV&xt^|A14k z05V^=IudETsQ_HiPK7&)jaQgs7wDyX+O|JAF@QelaWG$e@M(exwNwTIkZ*FHz|&z6 zJ-j~lXZy5YA^ZZmaEEQpgQY4?p7xfT#-jNo^7F2RQ~kX?I%MNr2jvtyt)a=B=a>JE6LkA!U52f|^l1KQY}i zSFm(!KnGz(eLN*cBCNko#B#zQnl;<96dhRiDxa%m8sA>7@l(t7uAY!L-b-oY?DKq( zFY(NAYPXBjta^yExpbF;qJ34NLsktL$sJWkr>y=94SUVS%Mw~~RFy(#zk9`8)b^8% z-j=xtUtD&4fUvZZTw&Rn7D~4+VCGwt&t0`dNsm5IM?8|W4y7Qk!0+W4+O`i}sPtC5 zG!g)|Fr6NAXTxB}1L=+c&a2BKz_aY%f+LVI7H0zxPk}e}g#G_q10VvQ02Y+PQ2vgj z<`ibF(+vQ!fp1m;lW|blyEWNvKxlTLD!p!)B280U#HnwVOq&9E0gD~T;Rb{Rp1a4j z#0uyO3Gxh~lQbiaE!|@90twxEA|nHh+4TVX$30u+B?Bf{E49XurgjgOGkiN;0g_;` zoE`}FNU&3FYnZ{na>)X`H+hBv_LoJ>fUL0(`ShYj{brQnSKrF+@2z!tO4rWjL>9BH zQm}Pn`-`;T?|_5TNtlQd8C5R;Fo@Zn_ydNwT)Dw$;o~RYjxo4Ko1d^x<44m{6ChJb3DtG+KiN?vQ{^j)o%}a>W3ONL1%wv zY-+xRpFutFNc!H{ZDjL%bvRfLXdk)Dve1M1oST9YV_^8u_TSf^r#uSz>6x|B%zD$* z2Ig3&?#uI1E-uBRh$HfWy>6z*a66u0IYf>;R8<8Dd8z3ZjG0Ye-5sftCV@zW$3=L% zK`bE;#af0+{EH~FuTE@Z0d=a~I)&@7q@Mnoqjx$jyKo^&5#MN5;~s?|Uti^EuKwNo z>pgaj>v={ceZ^m1B@s-of#=J?{`Wk8w#{gXcA6rNqCKI%4Rmki6nS>OK<+1ZrHA%! zmY2zg+%v7bf(~p?tfq>mDtB2sU+7G>Ls_Zxk+{gQKJ8`4@kZDuZ2X=rLvlwz2+@}A zzYMIu7vnFF*%ZIDbWslGvJU{)-L$?|X`N=(}sGpYpu*$gq%>+iUs_~8FBljQ*Dn*<5l zqu0Z&Jo|Q}7rn65E}(N(X5U!b5G&6@Rd8c+JIleM^hM@$ zw+FFVTK!M!vLvlT2U`TY3DI>=UjR8~$Ne3iIHgYOuHSG|dV*gx=@cIv6j6feMq;9_ zpAh;Hqvz&F$bJCH6%v7KntPC-q}1xQ&Zb1rHx`YafjdHvoUFe0!6H4ZpB2wVv|A6$ z8{Oq|z+W^jSMV@LHEdiVuFRUWmJ~ZIa<3&T3GIVhaH_>o4HZH-AJvT!^Z6GJ2@AC} z3xO4##czX^9rw3N-JxQ`w$}LqfZ5PiVj{g$ev^8A>}D|St__~gf1vC*(sMQx01j=w z$EVy2O-}o-FKNRcoO@)A1BW%hj6>J-W-j(EQ&g}$Rfh+=OQ*ZuVSaV<>pKDyf`@bW zV@P`HtFH)S0HaDibUP|^Foc#6;VblnObrFZa}H9l3w8i!yn8UAhhLpmkF#twm$Ov!%x7z0ANc`Zyx8@`&DJ$!&!x{y zGI0zt2*NEgn}pL%zs^6P?Jw|VEkBDaMTk#fk-;?z zn28fmdV$9N+%&MXc{KfPG?x(>G)n*gk`Ecqwee~^qT;{W+x&jbvC*FwN{Zk(r~KyM z+T(v-v4&cn<>$4-9L0Aq29~CzK$EYk{Jg3S4r6M5rHtKr+P@5$e(AQose$9Ut%QhlrsI2`X(|+mdqb514eXpV*EMGH8PRq{Z_;%>(`$ zT4$ys`F`?}q7G73%0p?$s7xUfKK-$6`;R%37Uwk6N>A(Iox-uU{^COG!Vg5b2D!$S zL~a?R4Y$5k{Eck+%#0hWhD%*Vv;>C_R;y;>A$d#TX$NO1E}rJ|_KdJ1FjuZU{h@J( zGvxKJ3&*b|tpO18ZyeF6!>7;~Qu$4xO0rwf3eT@V%fl|?9iN2e^O~#sz+waszMcSP zJ~e|6Vc--|_1m%aF1KWvc@u4h$n(XCd4bDkPgUm@<2}9+*=%Vih#;{PUdgexGqg zX}#G~VEMvOjOYv!b7g4KQn2pos&8ZRE!e1A#H&)Mxx|=*LA<#Hk9pvKl$*a_kB%s& zuG4HH0ps_id*k#_JC8f99Bws4sP4^VHaEdkd0MqpR))^=t{rc@q5)#dF@PiEg-CPn zH2r{NPYe^dWLQX7n$@+&?qN=)Sw>AOed63Vq0NG)jiC=OaDzF3VC7n-&(57wirxwV zfxYL(E7A^9n(0C^Ge@p5tG%0EDk`ifZ7+?AHVHkRBtWh#F7DjrZW*ISY&WFbseI-C z%*~!pMPj6YBsu4^y>%a7v-~86ur2*! zq;e{7)c12^*OVM=gqSO151P51Vo+DtQ@ftHYH$}-fJt?}|FgHJ-EZ=~IS1X`PtFo= zl=aW*BYBKD#AE9`H|+NAC6)Q{V_^~*Uf(vjm{!;7;5$h0>WZTZc9p~1Ygr+5O#zhv zpAw@PvmADG?HTy)lE$!Vw_EL$L%yX%_zeeZOrA_|2>~+8^{{r6ET4#`P{{tgsU@ik zICj(CVJny24wfq#7*J(`@KO+N(;0>^BAbVA1!H>2h0n{k3IM>aAhr)z9Cq7!?ialF zm&aTal%TWgDx%?FC6*=e0fqe$IvXv9bW!BlkqBLtplJw(Za0Z|gq};g>Ao0xxGRRv zbO{78S?ykb#QNXTH{<58aiJlC8H_#AoNH7wHRx713VK9y&}JGNKd-7D-D?EOapPc3 z*y{hYL8O5oq{+bI_nRgG7-jq|UE2iAjL+zRRaS0AN`#<2x!SU&k zpfB1YbB{&#qnYa%61tR$AdHZFb7PvdEBd2skXVYg2Kg2{-RW@3G3y<}WjYp748aX9gN8NP`N58s@f`EpR8@me?Z{I+B%Y)Zz{ z&@|t;^gQ%rjx^{g7+%h6dJYgX~ z@veHJVmYJcurP8#xqU9NKEZITls0gx<5MAG!peQ6Xh-BL;D0gz?f4$=^QTRvh z$uFZ0BD;greI~|G(lZQ$b*a&|iwwj#6a-}*bMZ|xg8Vx2G;)V{?JhKQiHrf(G*y+! zB^;B8VUS6)Z>uxFS6EQ+LJ4iR*%CkxrOPo6Xv53A-yPnG)Uo$HKg?S>#H?hb(bsA@ zgcK7I=l-+C{uOMAY4GP<_CfKL$Sjuz-_IzQMp5q?0o7a)7L3qkZuZr2^#(DjL8d+W z%Pn)zZ;uWx#tVk~XjKm%7soyhJ*Qw#_OKpq-dJW*Zh2J^t`*)I%R}+uf7$R(Vfs+~ z6c&QcrXOCVH(_#4bly^WLJIeY-kJHz<-o^{(t({pWEF6E$3#8IF-Wt$Xq4w(>u+4z z)Do;Q9lv%Ia0l?wltP?tQD&5Fyn(sH+vs;x*4pSX{-GKY>FwB@-nQ}jrFE3X){4q>8FSeA2u5R#0OMZ#?W%j{xG@l>j!X@X6i{ zsjZp!zNyGS7Y0c)eXUcacmf|f_=otRi`xOiAeoE(xZ?Qj@;~Cl3;gnIb|S$`oA)Pp>|j>t@`LG zvbF4{wJR$6R;3bvnFd<>;}u^14yk!O}&9dq?K2zQqX^%(&g~2oKU~ z-vjPlb#6VX_&GJh7aQwv1BmmFclFD9I&U!VMf0Vx&6B@pbFkEKCB(QCMv30VcF2`* zvwN!|1T65l?H}AT?uIQbtEy&C6SQj*Jp1M$&ynf^{1Wu}Aj8$J2Xqvf3E78F-Rakx zW|CEY^rdRVi`DEZes^}d)_d6nf%7LfFBdmc1)C)={_V7IP7#+2D>mpM8Jc zPH!pPOI3T5e8U%s66l0v=FtizTCE67Bu(C>z+GrmuTsQ%Ghbp!w?Xdh)b@iuCD6zv z$Gd+y$gJT*Pt1C0TsKQoVj2QO|ER+We{wEM{>sQDvBm5Oc|`aw8&6&fo-L;wWJdV# z$a3slgm$l(G{V!CXn@4C(cXKu6A^lm>(t$%Kvj`tkEcyRr~Ebh;?**=Vt6(Odu$#J z9WQ{i9`u5jl)uD43yu%vBK_JWmgO25;7vmCPX|5`y@D)qhBQ;jw*<@agMsIdaZk zDKxf0WXYWvOMsK6nbdP2%}tkD-^(O#j`?eHou!JCPLZNjr@iev!(lNB%GZ~26dxkZ z+axGUSF7^D`yhi8+uFyc6Ohx|4gHZY&bH?!K&4;(;RGjV zIDt|^s$K*Xed21lj^=1Z&XjN1Cd=zueJ3iZPgnRPt0z}_OJ$gfXTx$0tn;vOZYIDD zXZjjR!xMuM=ON;fqR_shl}_0I{fxA2d3jQ|j)h_P2w*0UZ=BA8|7%Bl;@rL<*s1SC z299&>L9UjcFTVlA(d$>)hFvDTkj=Udu*+Uq;HTy`BAx8w9Pyl|+>Q%=1hFUOjzQVf zpH;6|#xBWQ$$zc)HX+uSt@^LOQfe_q@Q8+7Nacfc7!N!(Z`*j5z5lig#Qv=7Mz-4V z#>P`K+DU1GKz?ER-6^Gz9M#XoMVZ#8y?ODijZf}<>JLYHpUc_PA(>9&Br;L9LMt3w zFnF@^?6rMyzgHW{RLwXP-<~9QXLVx%;ogvjr$|xoNiHrKvxj}>wXADpKgXr)7)BMp zZcys%zkFi=b=#JDF1%hx!w(~002;%5F}L!Q)k{GvMwof66)yEm#WWaa5b zts{CP@CQnZ=$YhuJ`20EVCm>`LcBkf}L)q=Rl{+j4LuZy}4Bu9R#uJ6u)-}hAurUw_jPeHR^?5~8 zRB+-ow#y-F^0{eV;T3t&{HLUYm4?74w50O}XWsw|@yMRbYp~1#dbiMX z$LFzMW{OE{DJCD4^ssEH5LGd3PIFLs`TzMi0SxydWsPavDZ#E%{g>GM_bYshXI#fl zYWZ7Lz8-+?&gbrFki7Wo@0Ct2zm0|*$>rz1+N>20qNkRn$NU@=&4qt^U}|0AfY+*t zVy4Uv=y+sBa0_C*6bNJ|W!F2Xm@rf=?by5JH`mc~v*OTo52u7;s2qX>P(Vmw*}K5^ znXPlOtMlmeiVmN3`3(&#=Tsa?dpm-o<1Sae!=)iZTUD#ziP>ts=Vpg1M_84YAkHgC zS5Mq2Z`?U*noo0R5;92RwLR*@=3i$2dp)fV5pN}c8qwV}AFVpHJv#$Ux+C~BRpaH_ zhqJ|V?W(=*<+!nB83bXs8n%!0!QJiTXputZQNR5`W8d{Tcp*c7Pta-bH$fWh5v3yltNzbH7kr@hH!RW5ip zpy`d)?NiXT>u)m%V>;0@A#^(gKx1ls_MV^}=B9)tH--Ud;khA#!3;w<83T#iah+1+ zpS|@ui0NAmxy6eQ8OQ3lwsC8FP^Ne%ChFO3Qc|#XOM8Ram9l*QcmW5kS>^IO>O@_< zb^$~*@XZKCWWMoC2DuSLDcOm`g4tK^3=VafR|gr@alLU94w}Ou=LWB59ErkKUgEOl znUP+-+A8G)pI4BG6)XJ}^CLPr&dZ8>vR0sXuq_T9d-np%g2X?&PlIJ|!&0Wc11o+?S` z2w2DI!R}i?>h3ri+_lu*>En3gNc2OvKi>KAUoLxtJ?9i36JHb9_dlo925H?ezH^0~ zb^MbHPkue0YdQ_AfhG6l@8Y$>-P5IG755s9-<-OLmaCNVk$3r05WkwPO4EG4aZv=! z;<3;|7zHi~R9wF-w?;!6{?(0wA4(j~BW&MCo1S%8%1H%+v)>qKg81*vb zW(O(M%Rgf>^?VtYplpN~n%Fih%`>!i1Mf&`4GSMzcgT^3&yC4`v3|c)`KHqFf=pB} zKagL%vx9q?e>ko{MG8fqkDa}7FRUWwD zB^hzpVQ&r%-*H~&)Ygb8b!#w#!`C&PSb4*hPPWd>D1{3uS`y{HR(BFen zhKU}ndux5>)8~Hy%7Lxs%MPa(cNx7Z0hM66`L98Kn@u;4iTg(xd+iOktnIhz_KKyh zf17s4E{yA1?zJCBeE|LTtnBA+B0j!vPI!Taj<$M4>!}Go1y)5Uf!h0&6&txgY4H#k zi;MsjK6?W*176M152%~QX&N2=Gb(H8tOy~__i^Tz(Y&Ts-a5gYF+NH>xB2bS_u1D* zn{Hha$w8XaG0{>%N=6`C8TD(MvE{j*HeVct8dWFJ2DONseQUed8kFbm!|MVXxdAn=x{N_`v}5nBqgKMQEk)%mRa5;I7WHVo|#D_2&Y} zi}pIF2;HL4j~-aWDtpbksfgF11B`nT+573<(DsVZf;{efy4lM1D8SDG@=ViBk4K%> zHDB?4dI>gv1+H5tb;?F;x=PpYtxhu%pNuJ6FKqZk{$T+qXzWdJ3x?!4o?Bt+Fs@qo zx;6Ts*Aw_hgm|FyFb&L(7`y`}X=ccNq3WAj`#-2-Zi+Z8onyEc%|E^DW*{UaId0k9I>e3B6>e{i6)_*_UI$mb94rmg<4hqdT zYC0-Bst{A3vlW^1c$r=hTUXNQds(|~&|vBh(XRJC>vyQP!VE{MOQ=qsQ%dS(2`AQK zYkvPDnH!ec###vs2Y^_nK-e>88o;z#M)Hb>$iwA2&Taj^IEllcFOVXbn^i&ZeyW#N zA|xnd)B*XT5LUNZzWF*dT`J@qn5a01C_Y+_TfGZ+mU6ZjlUEZq_+D2CjuRo%&Vh() zmjfxLo+0pChcQ-lhLlcf&rCEq5M_jf2$d-&Yi)riV-j?&0K(d~{6&$Rj`MggLZV4! z=`cMm1(RG*euX?8^JGfarmRfO%j>)6dg82BdpA%o{3iyrz!@agX(6f>urd5w#PfRP?c zP^9W=>o1GlNEtd_U;1(DwZ61#2++3s3IiNnlInIN_h?C)IA-!pu}jDbUua1UK&{pw zlpBV&>D%-Gg4WTm?ztzyD4<$(Sh^DhsOIzdq+j*T%(0!!X~vM4X1=zPj(>QGm4ZN> z`-#I5g&%Gd*{=q%0Ymo}FV@M+9R90ITOxtXxVxV(;!z@hr`&62g@ckfttVcS_#vJd zog%^S-=He0tw~WVI!@k;W;aHynNjb5o?Ny?Mv24)zkMW(;);_we02A1$aEP8i*si0 zlmhs0y}!I!TS9n zzLEE^o1+H(RtzERn_D=7LUZVQW$JG89G)KcPu?18olQbG0R+a>-b|d&i~_tjMrQp> zvM&T1SJGRRQFXEx+TeRHbb0g+)&-_DgOk84^?F7Z7L3G|P`v@FS!!ibka@8;6?85| z68ztXA?WX-qc?P5D0(X5r#f3u)-Y?7a8uK1u+Fyq2p%YwOaoJgY+~iP|G-?X0Zj8@ zp^)tf74CZ%S8Y3PH895J(e%$?BLf-bNk$)%l#)M{YY?r40E%o9Lm?nwdKE3UpW&$~ zY`+%WOv0xeW&y^P0%JH)IwJ%Qtzig@zwpIs!!vwaOfR2+vD=f=Kxsg)j4{B2M)^)+ z{KLqUN`~D=l6MJiJi>RgJiHtnvJy-9HMxi@uv*J^p#)b+yS964?`&1IPE)x0Y&Kx~ zr9%!V5D)IVEn1QPR)W>~S{rr0L=; z{w_D`6PL!_*woOfcH4<*kNC2-x;QQ7x+6=9p{d8{A65q;=t&Vw-~~;rzP+>1S6y7H6i)ig;c7giB^kB!KMHRQ=zCT2QzG+lfYQFyt zd(iG6m{{2@xQ>GrjWyTA8Pu{3bU3nb!1H9eSA3v90<49`dg`m4f6R=NG%3RgEAMur zM4}-X_><&!)Q;7&dJQ$888@XRG8}?w@lN3>7H+>(zyaWzr9S7Q9%)>GwwBtfCH*yc zx>)Yb3S(d_C`BMVN1SjgVwG+?4a+;Ac`4^?MxaX$tDVaiI4D+a)bmwhVgm$b|n zXa&NMWU#untdZauHAbWZcrkDl8cK0ekmK+MR?v?Ozn(k4@*6;1fj04DoFO;;vO|Ms z6G|1ETN`g-J8`#~el;iUeh=wPje!4SWK{26Tu8dD=C|&fT?7 zre8lnIb=DG!rtLb4@UnPNVU81&BR!(q6V1FH3*8_JNy~E_QCSSd|X}Gz&`8Z*A7wb zOPYP0NZX{gw~|q8F^ZQ&e`|hw>I|9Zp2itDE|;e-vQ2Vy4dYnoN^E_aPba_dvpwgT zIeE#ON#~YRD`aQ;49sTpGYMgotv*sv*9F_O*jTh)Ll*=*2sB-)>N2gTKTu&}DQ)BA z56{yMJ0{Q!`b0IKx+i^2K`PWD(@aFE<)fP{i*5raKeL!ETVxDV*F7KPWdL3pd(w-$ zsqFoMYrOhwBHwqP-=0w~!TFSvzJ?{+U}lqi-!4|w{&-g`U*jAL4)9Hr0h6owVicT} z2KnWsEMsLiu?Ec+EhqK4)QTW1FfLbcZtZ@5 zt*(Z<{Q{e=#bLVE_d0${t2)~c)xYtk;;0d=VER7j+&^^8~hby6X*@F1l|{KuE>33v~7b1%Cag3z)@m*H)W#gc#v>LlEr(O z#`T2Jy<3UaQ@wz=vJB*q?s8m#-+5VlIPpic*7%LAmAFYpX2c@cTS#$Zg69s3u}rOW zIIH`9jt4>^7dE!a3)G}Gw#u^iB@stn7Bf!*zCBXqED%&nFcCPEVf?~RJYxIz0|}1~6B3;I!2^@V@m(v4S9i>E^exa_PZB%X^{v$!`v-f$ z_8ZORmo};zxguRXT9+2jQ$!n&4PQ0iO4ky`nXP|2TcRH%9q3ws7>^*#y5H@*^vbyO zO30@-{zo{9&7;*cHO?5Q`85c<Dy)@%9B#0 z=vsaTYFIiD_&~(rHW5x)yAAIN8TNB8Vx8bqGLvpfcX@&TgZh0V%#epqPhKG0mD5G! zWcI_5v{iDJxOs;6-LoS1Ez*qqH)8!iI{fF$5!YaLlE1^4iB(j=oBsRq*=0PgyM~tRugL(W5{?5zAe@*+3W!T8Y^F>_s1dQq$kwRi6+AjEunHN4b(49 zT-Sj781XbQ{=w(C zv*a1F7+%V_D0l-s%~#fMn<_~jsDoUTq6WvUv{LV7ptZKch$Wq2l*oIDOdnah<^ASA z7MulKHPqb28s8PJ^vZYY`yAJnKnU{~nKk44X2*FP{~ufL z9Zz-t|BqWmse@!?Qz+xu4wgS<;4)G3o>Jxfz8Yio^h)TWb!;;Eb?b+7PBDhjwZb#>znr`)) zx%Xb+Lbe7~E+fF# zKYu?&M8WEHL2+B9^JY9}5$$-S@?Y^W6kbodaAosS3ip;f-KYJyCRMMq@va0wsbPo? z*P(IQ%H2KaSNd_Yc?D30}_tJTg#^;36GVDkzh zrGyMFhI`m7cg+CD9ROQsV*N)S?Epl7DlG11Y~%`1`E4lt)pCR^I_aQljRfhP_A?E8 zxc8BaWqF@q5sfIWlcTye3DLXzF)d9yjaHA`&?R;GviFdWe_vo?)C?s-tu7}(c6xM) z;v;OL+x5l|4?k{1w2Y}TU{;>eT{GDIL{%L23s9R?qRtDy=pp*xjbrt&w18v$&!j=0 z3vd;4N?skBY2Eym!{rhCeD)t<0+-dE9j`fY74~oqrY+vwT6OeZan_`NPetCF99#r@ zdzWyAXE#SVz3y-@3^W!*Rd)7*32bW4+^=yBdDwEuWuXEd5I1Heuzn6jjqA{81g$IG zp}&3~Y%Dm%Z!S-;Dm`4|ZmDW6?{t_cZvKKGQ0U3IHW+G{R5RCjIid#A^Lu=@x-V`6 z&Mqw z5r>O2OSQ&Df7)QHvX3o*6gR=SHg6XB`^aTEOoS3EFA@wi!EOTH5TXTp46qD+b9Qq9Yc+XNyJI8{A>Su&g`Bhil{puH;5eqhx@M09yj%b@ zxnUlhG>T}uI_$~{U`W1N###WWg!`M>jI)FMymkq_B9T)7P&Rrec4f^CiyK321;!Wv z7fkX5fKN9=j;2gPfRbZk!w+-_U&WS1?=e8?QIwJy9JwJ-2;qH_pzmr4jZ})FxWZ>M z6WN4mGZ0pab4RS18GF(Ep#mGOR5GmP$R`BP(lTU42Ljeh7HO{6x z%K=MwdohB(D$wh`m=SC7ku|VAg&pCIx;tcsmy5GZJX&b}cs)w3VOGF4VTkNke^Ty4 z=i(^tx>^2fNO<8}kk6pHO0)QwZpQ;P;a1IThUJL_LM<;3(&h29N%79p4cP*%GnDqV zS{TkT-7hVqo?oT5mvSnB*-Qrn`N~!m#Z7KxrHzRC2>VGeT)Fp%t33(M&`$@?QA%Dd zEWW+cb&7Tc2QH7DbeidyHHz6d5cTpb9m)u^q-duK?*R1}w%+f}c>++X*N8>_h$)$~zL1Fs5frzQJq=GI1gibR2bWqInRbGl#{hH_L+^(%rn!M}Ds^%RUKBxNb>#6us+v9Nh9Nl3tiIg6e_D z?PO^X2Vf0-exTMUen7Uyt|I>}E8??~=ktUNP-DI-b9chGJ`8BlFMW=SQ%q4+yihOu=s zU^v$iuBern!0;pMHiK&vjli*Z+S|MWS5vDJ@X18`x5h37s^QlUXVx&;euA2xnR1%jZPEoP=VJH zGwA6A^VLf8=B{l1`FX*NROWJ?MS|52=`i|7EK@gy{pkCQq-)tF-^`&myHfcBpaiIl zC!%s=lDc(i07p>MDpjGqy_|dO%v^FQRo=EJgJ%ZU?&w>+zdWn0HXWWe;yYk}U???y zZ}ckvWJ%DvfPI4m;n~7<)IHz9@J%Qkp3Yj@^27~b$?LyqS24 zSN&KAaO{Z}l_wM$splX7AsRa@H-3`$0}JF(8Oa1}EbBr%j;0GBS0~HkG*f_QZ>r}W z#qWo$lgD{8UVd+V?I=KAEa~j0541&bqCd{V*7B+x-8Oqmg<$LZM zB(FAQdP!}b&QY^lj)N#g55k_(i!0EO6ULERcI+ivl|EDf5NbaV)wL!~@nQe$1%TkA zsZZcSxtzz5y5?(X1dH13J-9=8e@W3X9b7@CKa2@g$h1z$#61g9Tx`SLp*x< z%8TdJ)A&}eE42%dbhsv!8|_rw*uqVTVNF6o-X8Hy+XCJc#U6>LPk;9gtj0REH{Q1g zslR=POk7U7li1Hp3cJV0PSF2ea>2}zg_ip1lzyApIuDOB=8F@T4rzjNCYfWNsBWtNUxDfG*z3c`;>|jVrKt6p7s{cFWGNmVHZ@jNk6}kXT_5l*r+Rs#oIz!o-0Ng?IxLMj81Sa602D z-ooixhC%cCGw_RzH>d)SzGZ~jGuLk4<+UzCl#6iQEYPQH9fXuauJ(v4flfSSjBwWe z&c*i#&o{ePB_8A>edw2xlA(tsZ!|6ireK9d!pYT&X#m(>fc`GpP^k8Ez(bOsTsSfR z=;Z!<`~jeubrdHIM6W=#?wJwgPdxKB_xhw?{*6K-S-&0=9sliiOR-;%`Ec#a&Ud0e zf;y|LckWVS5)>#)fg-kK#q1uZNzf4+p-*q0!FA)ZiPVl9akLm7`6NDWhl+i%k;k2P z4>rk9%(W+*kp2TV5`;JcKqs2=Cgk&-Q6AemTetc>%fWq(-PI*QM+Grkjn@6RE;2c~sNv>Fm_H0KYQ@49}@Fi?Fp?UgjM{)MGspdrjE##1u@N|ZWTc7!__@oAN~w5D724B(EFvqCg{K40rm&VAe``z27vOL z$VRlmlS{zfJUVsX`1!@!6nE{E@ivv(Y2dvPq3;F3=8CN>ShWc%EY2+?j8-K6YLjyt z6Pim>v4<|g*4{TE)qpKPf%!|*bfJL#Szm^4;zRfk|IZbEZg_)q!S9_`WK4!Wms825VU4wWSdPL)-c1~OsgEOa%P>qY)Af60R7{9 zc}eiZdl&!_m@e$zfC@Y99jlrLvW)H)(BF391Ag=eT@tB#E}#n}qP2EX-F+U_@M|^2 zPJbZA9%zIq0vKE^DuB%|RjL=6iuhjV5*C(>wXeyQe5SH-nK#4orbl1Xlnc3 z3fv?@LRhE|fy;QgE?0Cq-$0M!mVw9luP;3mo=6%9hCRR-tAQCz?kxwCq?_C;aM}jp z>UVCM73&X@%dOf!p@XgahJVMs;c=Lq@jJTH2d^Nv7J<9HHzDu| zORkJv)i2pL8odm?hGliEoTBv|F8THSvPzAHzUA^o5jqud!0YLgXmSD%CT_O4h*$2<0PJ9R%G_gy$u3s0! zzhZ^mcH0T5u>OEz2-nfi46*%K%^ohvT_2LW0J2hU50LzU{ht?*|9|7o*qk#6f#3FLlA!*z;<3i6**U&aUE+jw%}J z-$6eRw{+FwQn~sXMF$Xmm`Q)Ra^Gx~KsysWMW zo-^FLCy?HL$1l2>^rSmIRh;lmJFQ1>wn)=>C0gL_8aU*&@DM zGL#pSevI9}1Ua21Gk2a`m_o=Nncw@@rvyA0mx$FEri@?er>`c&nQ1&O&(w1(H3oH(c>xner#PSfQWj+>R34&r<(_FaN(l|B z=zE^iY`r{@TG=-6@EJy#4i@Ovazs%Q1~JdAQ8>seJO0j-Sk7Zl^fW#0i|oS7yDgR} zvah$46XhhIUYOHkMhC3s1PHZ%E#-1Q?qg#$R|Q0C_}EXIEkHtc#oXKEgBd$US~|#I z3pUZnJmqbY&9WiS_!91|Td71&fej@$2OF-8WX``@X%zdkof}1oCx56*b1)uZAM@JOOUtK$pd;YPHY}_nbs10(;J3`Rs zwCeg@(@>6Jnga8=_*A=Ea+8GqWW)k5ar5O}u7X95*)Bh+ZQi$&cT_kb*ZB3-nuwmd zMbih>&5ZNve4D>pE>lXi%NKRzQ%}Vk;LFyDR2yv!MLZwtjd-eSmEdk9W^>si%q1d+ zbKxg+lxgL+9Rn~6&-L==mR$qp zVg|1BtBj)oPtY!bg8%m|G(PnvR2Va(uR?H;E!fHx29|?a zzM!eMbRp^`i;(b^_yqTQw{MWzOoX5cngSzPklA{d1z)yg4HkUBWBNa&JnondKtg)s zXYQSo&KM0Rk&EWaPYRh>hzS@Ftjd8Tk{>V0vRl8uJUe{oT{B4&klL#q0HsX+;K3Fl z3lJr=hE5!=-!BhWIwXVpo2uIP4I&;zPPT1<+&U%Mr+<#x8z9Hr}+1|xG75PtQZ zHNuN85H*w@xrjtwTk;+rB%YccNoP&G-*K*I67i?f&oTXD3^GU3Ucs!KdhV}vh6Vml z$fovr+}b%(X7Gk=?VT)3`eA_A9nt%p#UcN`NLC;>SLQCohrm7h_KsjxMj9-1i03$_ zsNA32?Sm6=XN%rGRs7(S&c-Rn$*JL|{zXz2cCs-f%Mo8qkuc1NhY%!sJ$ll&{FNxq z==CEd-Tg;XypF|8gp6J7UKX9H7M_A12UAvUEzTw<{FU*3ZeCj^I&TQXQ7gvO4Pvfg)Q8vd+tiU+~`KQd;f-AlrZRB2Y=}&{Lh->e{vbM z&pV>sTjqrDcT)}V|9cvmHwl&on!i%Hx<7mh#v9PT&vT8G;VGDCy^95`^dKaL(%|V) zcIIQ7Q17E#M-?)c=;UjRrF1BiRYOGVRO5_)zk1NF{`+4apT-#16PNK zD4IPo{S0eh?nzQyRd%F4%|4(E|85jr%^bC}oIedg zr@>p9htuCsp3!BLl7z}(z&eS--I9m;Z+<2}UL|WvecnNS%g-eJxtL}5$9u`c|0sgs z`z-KFLki{oqOar6&PWfaKY02q1u6(AfsxEfxm(rfN@leJ7MAK&QYRSdJ$7x+rV?j zC5xZ(s~b1tfD2{G8q4nZz~dG;=kkkzo;IS(eM$2moi=FolvTDsR4ODkp9%$O4D+KV z2bY)_l5n%OTm+cRKyhT)emvOxOs1l|BwN=25#EhreSzv-xWVJgOm@yBoUl;~K8xt5*NxrH`pE?eRmObOk%abhANiraTJ zr4i+K!%@UE_`LB0r2$GeoW7@ajaqz4cA=5vVaQ0;NT*Y=)iuWm>rE2eIfG}zsh#zv zBkEolRQkWCDqkYtN#+JG+w6%SjAM3KNQ81R#UvY5@qo)X_HcZjvA{jGYYUvy-qv}P zf6=K`Y0zZrhe`33z1XP)*`)lrwjocv&)@35CCdhY7t>n5VAE=s`zEghn%pl9rS={q zo9 zyAMS7=h^w*0sGAe(~bE8TXhfgJtAjm>VEmY+!{DU7D9 z1~cWiYKGMAxw;E~%b!|rpVv~K9yQb4JtFWHF?SbtXfeNVf={9frR$hlP(_vae_!c9 zxO7{=;6Qk3k2ygo`~`DnLI9@_K&g+%Z*y7Kkb4*C)UO_wROe~^alRJ$MU|F~wS0@H zJOh30X{-X^PQV!<7(=#>k6TK08d6$GW%y#C4D7MKCD|n7?Gc#pQ(fj5+zB2SIO1+nDj zjDaV~38^Gb`uU6D+~l>$_eIqt8)$Fk)U+kn#ez>-RL~`}5+&`P@q=9MZQ@Gs9t@c_ z`RgU7LS$9mnFGe<+?&dm0o^4xSD{kqP}PN;(PYhO8D8G`kT!}8OBJ6ZH|o(n@61s` zYHl^_-R+q}7GwM+dTlqMoBuyYWg_EkzThOzwKw|r78f`R`$bH<+vBQ184K}SCD9ut zkIEjk*rqkE7)v>!-^7L3C{RW?^^f(6I7N6*iXPYZnl*EA#~(_5zZ7o>a3xw_xeKm- z5R#BLVKtH09g1_VmIN@*lEWJcfoZx`O+jT6XpNr8;Vr{?Ag zsN8~HzpJ57(R1BitWVTZwxxCj!?AXI$b3SxCw`c2fSCij`u>aK`1S6=<+DSynqg`& z7>5qhTiu_V^WI*ulS38BayetJ(Fbb3{#h}Et8wSfs4Q=@Ad>eDEh!9cK z)gkAWVz2smi+fc#kd(sHEpm2A{=a6!r0ZERKuv>@sIMPjj`Dm&4SU-$ujS^OV%zwzCTyGsw?KA!isd+1t7p^< zwu`GFnjci95$cmDvQ6P3cIHDdbwg^eVgcgc5FhR^T;sVo_?u<2Vn<*T62t$y@%lIR zzIz#`C!~SLt4=jNZ*R>09K5qx`wRM482|gcd>#Q+_#PEb(Wpb?KPn3`X~QG~906fa z-;L-lpVu3+zBdV1iWp!ohYRxZUS#QO!D>XU(vhV)V1NWYOJt}9weB<6?Nw0N=u|EW z!wIL}f;9{B5F@EHa3AQe4^*&1OH%j@nbrt4`kij|kc!@C7I7fWqMs%dCO6aLoM1l> zX$1v-dqQ7uZNercM=@1nQ6OKmB;*DLJ{x;&U+It({#@90HWBYUqxDwry{{dwoGEQ? z)3OFy1RMrbF^4zVZ*mJ5g+P_a&3}FAaFExu2X|dqju8;kxYqM* zVf-y)vD~WoaLsCSiDaX#+__2`nnyVIT9%D=)nV$G_61h%oD65(qzu_DgW%3#S_w+~Ery5_1m0Awk4+ z6oYY_CTh)X$^OQMvXd+KE~7`$Ya zUdNe-+~?pWw24|(55MQ&In9_5mdHA!h{{btKEpYmh7NG6Wtv&*oD7s!)HNdXHrJ#m zxXj^VpvT+cr7e@D0VW?_n(S1YTV#yszy7E{^Ca4X@)m2jVaQw}5)d${TfL`p_K0lx z!H;RTspIP3?d*L;zKf4S)iDxi&5Du>SRCa1>ljdb0;Sgf9-HAQ>B1=-Py#R-3z0b^ee2}1vkK=(^E9e{ z^gwF0;yLs>3hgahZFO{QD{Ak#BzlT9JS4WA+MIo;2qjrQ%y)l&wY5oJH(-j&@gAab zHM!xJNozphcP=rXyvdPj*Qtus?tWCFa0&NX1QqUT5-(?kwm(ap zOg93Eh;|R7ROCmJTDSj)@CwBtJeYM`Va{ijjqMv(S3NDv{7vmc#^k*noLyXPnYu>N zk~>O#6B1j%38JcqCu)A0@w`H>2DglL+lx@T91?+$Kk^k!QpuVp8l{MX!B; zU)?CaG1<-%R8v|Lp~vzqCDba)wFtaT3q8A2_PFkS!R^u(k)^=dRZpaUx(Of&DjMt^yPw)gc0h5+d5h#j<*Ws^VvNLaO@y*X@BpAuC02sAFk! z>#9xQrSW4(tQ;9;6-k_b#mW9UkLitP)j zxZve*&kkxWx6l3IK2`bcrCh~GSAzq0BQW&(rvb8C4ayzq4w-S}B>+4JYBq2FVCz$( zFS@Kva)a9(e&RS^RhyEKVAJ2OnkodjKIVC6t+vbqkIRoy7pmj7PraI>NWZMxbNX&O zsXGJ*qwx`>rFe7eRdi8(S=n+;2u!etA(IWkClM29l2X-+S>JB_*F#MrTaav-i>Wtz zK49|y8ujZTI03^kcp?&f89t;V<8k>H?Vzb(8Xq4fxQs+@!i#3X`(aE;;(cOMjDcr4 zZ)Dx3sbdqk;aR=xeJ}=xJjV`GRO>B$pD-?U}aCjln04(5J~)||oFktd;Pnc!>2s@3cP(kGEya*o$1 z`66vMDRHN-AIYrue=pnNK#4aUPU^7hi~?w_=b9o_(RBUxp=R_zQr7>dXA}c{LEa;r zvMx|`g^4#0b6WvkpH+*Z%2k8IRa&pdJO znvuH^2aTkno0*4kPo%Z$Yuy!=2xxK9L8aizGlrF~VzEJ;A+D1dsV)63Tz01_1m*Of z=;VxbkMDepXNd0h0=@Y({?gee`9q*1 zd(T&%Gslz^R?@W3+0C#%qukygOP~_maAh`^hBymyR!GePFu;d~XhZl2!Sb!ZVj+&E z9}MG@VyWsswfY?;$n+g-wbnc$C!3Dwjl`~Vox_W4Gp0&P83#Y6qIr&gHJ&xhLYXqx zzBT3;=~cB=26UhQ=I9ash3^!lmj)X4 zVsonF?N`aA5A-&(Bs-h7tm136J~wAZ$b{3p$hn0(=SKsOtOI2EBo&4S^7~l=~%HpJbITCcOLs-skG9x!Kf7^Y*B(mBm~Wa(O#Sux+qnm13A-oXW>&XMOEZ z^0)PRBiULKjPa|@u#|Bf?^}X$g{_($$xcts0I8{bo`qjxafXU?=tq5M{Li zLc=cEXp56LJggFAH!$|*<4I1n|3EknqT~e1l@NqR)^2Gk-D9+R3%B~@Uxc4 z{1IKY{eG4no})Jd=xCSBNNFHlz)=%HD5GPN&Ks0SFbtqyU(U3E#_Z= zQoaonK|)9qa;Tv{b5BEc`7vP_KNlV~6(0v)2GRD;1OA3bCB)+EoSsP$l8GdjrsXHO z64IF7S<5TIjF?SJ@nZH5k+Mv)Zy0fRL!3DXnXSjFMe&e!nh{flF1AI`*u3V}Tk_M* zv#S!;=}%}O;7zVI6QC%Kkt5x0x@*bw4r*#kS~noO-uVskAQXZSdwh*GTPFre$8rIP6C)=1-GGavR&h;992ZovDBe z)RDBVa5Q?vWm%(b|3Y}F@%i_7>$*_Y{<4?=$1zGB>xy^BXVHvOF+hr`%Xi=p@V#ga zKQd>w&(XuXKkMKbWx}+q@6>U2DDPA)>5g}P=x3txJrKh%&*;YVC9%aYd`d+m& zu1j7YKX~U0Flo_a@eZr8G1tXtGP>FZL1UVEdVbaZV+>fXM3h@ac=g|bjiESECm|-Q zPgagkACoQqp+)0ao6Evovu?)aSHl%SFej9P4Wfi`hNCD)aIbO2L=>gYJjo_Q=o|G} zCRA}4U3~K`!x}KA^=uDehJa_H7R~nR>$t!y<8qY8xT1DZY)O|-j9jgFUinM!{9tc- zxEaW;pGlx^&7smMYCV7EDh;#ZIr){3y*~jK@If zb0WDYen)_ql)|*=l{cXYCvk^O_-aD!*VJ>C9?uyz0Ho|%J#-K5Kv5u#(kg39Rs&SR zEr7iLaFoRPCm|oGd{CCLkxMZHx6w*N7vH?$zKHxL|D+sR@!^8hZbrYnI9FNH!n zFo7|?2Mrx}PN&2a^y#%-j05<7=?o%!6PDo0vB6&-e1D|lvPE~Xs=OdCdsODWR|(}) z_25uWg{ftR>YH}UimimKtym~-7cx4smU9uIz_u>0$j9#<-<$f-Jq@0cJLm|lH?3bv znS@%uB`^({GGefIror!{w6Bc}1D0Zzx0Wr`gR?7LYbRcN8q zPx9x414Vro_y`8tV%~@KDTwQ-enAxi!qR#3a_MBa6+P->V6S=*{ z`;k_}R_V6MFF+Xb6+6HtxU;4WEv2v_{qnhr+F+@ZhO~jWAHjX{sj83$FIhwMEJ(y^ zsOHZrPGN-~${bgxti3I;Pg_FXh4GNH+tWwt&Sg7>N_!C*4pgoQR~zcB__)=Hhq-9E zY9(*hW@XI$qfwMrBY+%N;2WwP=#CH-L-52F7;c)29q@8!slCZ#4FJZf*y1#ibV2&Y z9XqvSm-4)C#r*p5`HeQ4&UoT`^;}cuc>T0itmIp zbG>S`Ur7iBx+ctDXjVuK)>sp!Jo6{cVD4y7T`KOOx=8%kO?o`0wH9**y>8k(MH21} zC9)n%=cP!mwtMf7gBj9(SIU~-|ks1^DlulbOsT>pF-jYr) ziqO19P;T{v>h7=e2G!o>Tp@*DPc-T*{{~QT0cH!oXK?LLS0`9E$$U;0m>Pm}-l?aL zHF1sK%?Ffre_Lu#9RB(8o@YKI_+SysNZ0i(PpopY?u#~BIg9oFyelET)yretr7!Fa z!K#l09)fE%~3hjwC0p448 zc-Cl4^CG(5nsGw34cRvqSb9-qk1QpV42Cc-R}&h7r4oW$`jQku@;3UkzSpGsZhn^% zW+kLg{g6C$(^XZQv%1jpM}hSl=ZAC25Z&)=Po?;(()-98WFaieidhvC^HQIt zUT)_Reax&>S@6|i{JK<-vZ1FvTbuyX5miv-LLt17HSwJsPH{~u@MlG={zIeyPVS&t z7_Heb>5##mJ^&L3i&%N_*b(>#OQ(!HzGve3vlMb%5X}l!1S{r*{}(xqefK;B^;B9t zL$%<8kPJ=xkwPpTp*rOc9ie}$u^91j26JHWmg++3uMEe=9LZSE!BdK#LR}W^@mu)D zhQperZC4Ml#jZxQs8JNW(b88MSfSRqOpG4@WX5qh_W8BI>^q~gUXBFiiu$)IM1mCA zN-vtzmoco~vH{i*B+tfFleOnndZQh#PJ16-dVm@7JejFbG@hKq!F1JK$#GD;-{(1m z_j)&rT%K?bfyy57dt?X-j^8u6N$NC>FwKDyWlL`8-C14_p{6y_7}cbZ1NZHUGmM+o zRhq17e|i6D_%AI0Oo_1Z<&PU9@LIQRdH(>c=K-y&U@_=^Tx;x|K$;cv$PVvTpYXy?rOr_Wycx#(*$sM(Yr zNTs}0azcyfM!F=lFE;L0MCHD*cI<>|jH}u9mlolMqer_@F(Ia+G=X)~WiyTVxT{)6 zT;|}dc)`4g8$}k6r*8WQ6&&z|2e0hw<+%l&qKl`zn{tpKPnX~>P#^N>%3#|6M>RHo|M4T#l3j2rt26}kkEy3uBKWE z_cS}czFBZXj>?0^5zAAd1~xg6>u(o3b-fGh{bJ%13Orgt@|lcVi0X%UhIbJyrq=NP zqG7&CdrLHIv<}w1Lde&D8pAY;dr%W;60?qy;1x$ad+F(=k^$3BbD47up^iEKE9~+< zB^aZTrwzW8@;V($O5Rfq941I0ru`d9q((Z*j7#YM-@mMRm{ggr33z z!4FgQ4kkAbMo!n2Mdsejx?_f{42WW@4897kL?AQMXziN$rfcayS~^9~i?`ROI0;A8 zzqK#_X-tdn;OUxA|4xbCF-wM)b`fAvq7y`1u4$d;c5~ly(oFXN0u92Fy^MUh_wT)Z z$a6!?h(4D#{xPs7{4cP*_!eHX_qePx3X@-U%bVc~88sOmU5f!#Vz_U10E1)EO_L_ih#hNWtVc;RM1nF(kqux2sBqtl! z-s+Ir;Xou=*t*HZN@%saXys&>dlK0be4Ez*mRM~bH~m_}gg|w=vXr8e`GW=nCYGgb~V4qsVc~G0BoFQ{6elt8wKzLR3Z95s@dW4=7{q{6aiHdpf;tR-t)LI+OUuoG-@-{GUL{OF&S6 zq@Ljml95IKX#@UTP%b*3i$%U}QR)x}RAu$o13!{(dI04%3J}{vSD7CB1EZw6$-SFG z_X{YF)JIu`RO8Cm82@ zL24+9x?ai3ch7hny8qb(Ez35Ii|Jrsle}-PyiL`i=6dUc8U#qPKC0w^fm4OA<|EqXMc1Z;+(Cm7qV>ftd_;FTo$f@L$*(mk^#S zDLNS}6v&VYd@)u;_L;>mom~b)(CavGVhc3^0*LAIGbfq|OR-R^$q=gv;IEE0b6MfK zQ6*f4#H6>$8@1Z!t_S~M6z@L(oOn(vMLrDLC^^R?I75oShlmNyfH9usAE^U<5fFZ1 zLx4F`tWM`aH06jg)8>_eC@ykbGap7)>L7vkU26PhR!l1HO6;3_#ON5@k14drZ#|O@=8g$3fuBZ7axEE%Ltx@ypWEBusi3hfE`G z;I}jd@x-Q9nB=m!lwnIDlaAA8CL?Pv(+R&mDs48SJ)i3SJTy>P6dE&nWz%9rU-jWkDdR#Z}YIQT0k5ZJNgGR0u&=(-ntsOdv>I$%4Ubtop zOPru=?4-Jw;)JWwxl{RYf*gEr(+c0$>CRudh3Kp;{3i8rolzXFH|%{d{(ujmTPKJ( zC_rm&X`s3Cowx_)M_*ahD-C1but7hB?QtZmI7tZC+5IPU75R9ViZtd6(F+}m5bV(O zqr|}xJ+5DWzM(<@Lv=$TQtBD~c28pEWf)D|$any$93eJDF3TC}&02Iz(M9|h()^YT zx73W!TP9Sl;Zd2T*x{eD$VZrT)aM~#o)f;4(l!&k=94TG51h^J(4TxJ=k3HImjG}* zUxm}yurN2pIQpitboTxFY>ELUWtoyr9L0btd{o6XQO{=#X&1)y-M*|ha}W4MDRwiG z$VgiaL+O{dKGPFpl4y&yd_1#R$^i|6wbXzQc!&Un|LL{AZHxqRJa_)BjFE(RUvj9P_ymh{Rl z*_JivOBp?)%dgjoXC{nMJ~$5YI~V>l(Dd;n8x@j6Rhw5vT$;`9jV{RyeGC_U8%RGI>t9rL|&%j4`svl_o;ffSJ}Ponzn0w zuLWq@j5#6nG^~N2!cv;A&d?z?wbS_y9`w9rz_zR8w)gLyk8L`$fTWI&A}2WyAn>$g z`>{!<0+4F8i@yY5935 zTj$I8?`AW29BCUUQ1+4YoGIN_3DjZ&+3oB=WU7}C)o;0NRa)Vu=0O1Wl<#c5!_;TC zS6SUWEN&K!cV+UQyPfk~8WAN-*zN&BsML4lo>VsrJ2~q7`F8kNGbNk`P1bdeE79rc zLAf){4mJ2AGr=&YeRQl=oFo#?QoDB{bzZb2!Fc&Nn(|5Nx7dtXuH19w$EPPfQu^2; zqYKNeQWjVZodcM@+@4pM3=!LvUe*XM0Aycn42W0Q{pe0;yI5HQ$iJNWhhH^HCVpdq z%gum%05>&&czSt^^^AuA3e986*7`d<9~%@MF7|q;0JY^Fu~XIkW~m+^EUh!*>gg`3 z^J_nniVxb*>#P2crU(};p4dxNCND|Me*Eo#tk3|fnfrv2bmFlQ*7w-(iqrUeBqtNc zqI#ZJs)|bXEy_3dI~ZFp>*lsr5~f14m72k{-&>AY95~RN488GvF=M^tg}TFi5tf}a zO?2+%XbUtdB8WP~bN`L&z^Nk0XG&Mcw`D(~>}*ZrEMYkwz$nF}<=|Jkq*m2_cwa<+ z5HDd!9;$e*uj5hu9Yp;;ysO=%cWPM7cE@K4fqQ5s?Qsa5Jo2S+0m|GdbaxnS2rwpx z4i!!y>a?QhApqHX=6|6MFUvnB(cS=rr%Z7~O^4tN?+ z(ZrqxzvZth~Fm>#$_!>FIJX)_`|&bH=2*ZSY56zfJ(WU%jj@+c6HolVBK;!+2wh@ zo$%r0*z5Q462Q_sNS4HU>jY!nCkpkDtM=P)d=f6n*APH6;|CN8O(bSk$>`R9O&g1i~(>wl>O%oy=*N0`tno!Ku9JKgY>^IU-Y>Zl4V zHSrZG%!k6F>dRu(h{eUE9(sf>Z>pJF`#9oEzu-A8oEbe2AEB|?e9?5~ny^Ku!yZYv z(RdH0E@t+1KRLIcTkI^yVQK3A9O@PkyM;LBtM@iWwS1~|E4>yVr1`k=q}7!Pqvg&c zOLw!P?9GPZRFaccBjQut0t&oQvWE0J?@B9VN{Ah%9~j&q&}qH(C4)F!P6OB*#gI~u zpH1BRE=0~?)}2%^6fI>ptpIuahuu`(Pt@7cz)4sCb1nILhR}~fMBKbB;%}A;H3=U2 zU=(sUt|16EJ)r+K6}FnR!L*Sy;{MLZWbOugYNY3vdA5189(q6cQH9g8b zyJ$cCm1q+F@ao9!cWwPfeo8PEV6D`J6+c-5Ttbw_&H*`o4#1Km;;z?ef|r)~UXUsm zR<@0TkkQ?~kp{)-^~dSv0&A#q_GAg`bF_p@;v($;$0fXr6*<|)srRq+%VN&Fd>|>_ zC3sNMpHiSZ?i}Esarhq)x6fm;(RS+N;G1HQ6v~2BO9vwR+iP73c%b{yi+thM_o(oo zKfVIvEWYS1%9P@pQ)R$F5Zc|Y^>i4rTY*#*xa+S*?A9~9GD;4ke_SA$MKX5j4H1(M zcaJ|%Fl)cT{)o&ugq#BOW>-v@s|g`CdQ~XQS}=!&-96x=8VsYuTMWsypgJgNaflN8 zvBA0XFuLN}cs>!Y7o_eETuzwlaX5w>HclxV_TAr_*8J@e0UfRucvsA=FP~fN9YH+4 zj~K$QRqiXgzH}uxMQsCtsp$c)sHM>3n2d}29@p%EzWm|*Vd=*_!cnZ>2B4rO`RW3( zZEzDM9bATLSrsZ%_${hDCw2fl-ZqyFMFVYd6f=o}!T04g7>U%y?ot=n)8VXP0joy@ z0d#~Wlf?kbjc9i&wjNK>oRSYkXJ@#3nct5tdSZDr+0A@s}}Gk{&O|fMGZ^X zibVp>5R6*x2RV$&!#)uFO{P%3g;19tujDA&exJ8x^e^C?N=VcAO0{v4FicR?EXKlt~w~{b?;LGDiVTpDIlGa>rx^hEg~Sbw7{WLI;5mqx?|~X zDW#Y0Zk7gN>3)Cfx#!$>?mx~r>)`U#w>~j0GV34S81Pn6lp7*lss#<*LX!F$#(5_FyO~1IJ9{eJ>FWK0qbMOIBs%lN#<_b!m>P$SCw;xgn z%)N=_2n+yNMrDs$_PR;V04QL+QxD8%;!i@?dV3ahT-7n;c~%ty5c*!{ZgUYg3yy!# z$n^p|f#g~2zjb+aZgeQ!e=tEp8p`D_3y0LPmH-DfdK1YMETJu;?G=?HkhXx2%S=-V zSn)E?%%vF&soUq+tr@AE?Qx;h)u6uox4twK^Om3#LjCzH782Lp{JlUILlC9-(R_NO zdgt}TeDjbmih0aw_pBdlNB!hJ`1V9Y9hc8$Pq2&OykM#FWZ;->D62BFr{X!VndR2f zP5uf{}%sXOgXc>8X_XJWH2sXjegKm56mIRi}=R|zTPWeMaJQnSMz@;^}K_s+98sKxZ+JbN47$9I(k)8KYP{4EUOk*7Y1;TA1pc zvO16DG2u_7nBLDJyyN30ckW1Eo^NMSD<^WxCLWg!db5CbUDe|jQ*7Idm#eg)BKd*t zK=}Akzma4f3@Fgtb&{E+D5p5hnsoyIFy<P?vlecqV0@AopI5`p zu6sMhT-U?16?Yz368X}NFv86EbSDpft331%H!gz%O*iN^0@q8?y5 zq-z8gxeLtJPlymK=YoA4wHRUqKL6@+xl1JK0=Hythqq(a(sPh)QmC|I+YClxm4tZm zX&o%W{fLF^p4kq-iy70-^GhkwG}IM>&K^dYs+-4xZ!uUmrBX;_Xq+JewJHqu_gLE^ z<@@y~B%AX}=h-_&vI5X0ACxKCjdL+XF}_3f21N$8ga$Enrz7$rqr zfsRDHQB-#0Rj-K`uOrsL<;g*a+>Q{=mip;0OCDoN{W+MoCUAl_ z5$q_0pbMba!^x`heX^R`Qv7)Qcac@;mvK6nLf?P3nn}Xj?ePKG#!BNC7V{;uTiP}c z1C8aR58o9vJ)5_s?ayZ!&|L5EH_5W)4?YA+R)-1q`8-VPmZ@s4dreqLM+rLFV^K^i zBivsVHaDMcO1a6BPb$#DsKyn9MzvEnh4Ezybw0;4C>g<1RJAMp3j{jrnOED2;Qdxs zW}BC{i_PP<&z9ry$^Cm{1>2`bbKhQVZ_z`mJ~75KQ=ngRLl~4;NsbxX1ErFzg)W1J zUJJx@F<LX-O-Tn$2x_l$}!&a-%RKjO9TQd_rOi=H1Bf;U2LaU3 zAG86P{Q|I8Z0&cOA7a9|8ka)4ch|9J7qc`TD?KFJKwOqYR@jVG`R(!7ri7cXJ!{Xq zD6H9rlN;T`e%EsIeGhAU`n=7PkXgjxV{dF3tVlNfqsfP~ zS;7co6NTg&ut(ybktjJwcQ6RdxfZy$GQ@vPdd>eFCR+rY(?mWCLR4Xz@-7#}_up(w z?)k6xv)Lz6hqz+iG~JzNuJN2PQ?%FGR0mO#UGtoVpiH3)jhBniw>Q6ROsAN9=O7X@ ze?A*Y2ar$tPJTKG;R z9@jUQ3GVRBa3QwlS^82vIiNeke&5p%HfK9N3GgtC$LQMku%+hJ`6o3O6UK08eg^$+ zgw(`?hs47bp7|H8N1shUbQU)8Q=$~jhHvAP@8R=)Pn1lnYj)XH z8jY((`joHZe?zAwx^B;mykB1gxJjCwM$w9YQ#^49P|u{IN=<|qTL*Y+R3{H|n)qR3 z0cLQ^YyEiZ{idIZO>4;tYu-B$6taAVlSEe|ur@LILoM1t{e9>|Lz2+RY zLA<-i6BE`WjVm!x;O6-LC;$M&d0$(Od_|7E>-wU22qg9B` zulFKQ)nekG5P+N=mlE`(&#Zbl$Bxi-WrLZ^tQur2q+aam zFtNM&x7Vs>Fp5XkR~7+xf3hZ@wk#@?OO(aN%Ew1s%hTGAsa&nn#@LJ(&o)X#Wz1(6 z?T3WNNGJrsz6M_IiIKjXFGv3T-+%zy>SDdDpv~X@j81V{x)`(6&=R z1E#)--AJLWdPY&(QGk!Jd{y9rX*lpB4T?}J96h$v>8Rj*FdNSX`7V_3i0xS8C;t`~f6_K$U?~k78MU;_ z4hnEkg-O1}$Mufwe5cZE;PKeW^ATUD^oY$Ks`M;6NsCpa{y4En)i8Dpt@%lvA+qX8 z>!K;LUsUX{^LjBr(b&?|ef3f>j{35iXAEIx4p^&&blUwrI*FwZX;1;<_q(8 zcF)_=z8<5yJrS>Tme;DiW&ERT zP6-(jvuCI)8T`p~@z$|XfgLq83!U|h!9ZMM)Z~F>IVCJ{Mqec%Cb6W|(#$I4z=tt(A?vOn_ZPj>7RQ z)yy~#o^4HGyi?8nB-!OyXzjZ)_w-NAjQp`a;T$ZKrhwJw-~D(EFT_e z(aDkFAP@CgdMKxJdhOdKfYEOJXt~>w8-($qN}yL?wyD=sa<0&;kG(EudldvMli&p# z1$UR=QV+(M1-4$1U@6Sy8KW2=N;5U~vZKX^{b{ubkd>#PR&?_Ron6fS0ZKU|auLv8ZTyPq-x+f%QQv-_+4u$RhDHd;OGun@9 zg|d&rOhxTM;P|*6AwU%GD(C130WBdB8^yw{c{Vf$n;SX~N^X?&j<23p7&O#O zuASI7{cKG|VI+(|OL@=Gr)Ki|C*Q1?*jd8sbNxCDUHdt)hFktmDt0?DzTZci@@Bh# zScMeRy>Ymp=mY)ow1bva2-d-;C=REnyWGo%Y9y;W)fcewJ^I8I!+i`jeUrX~p(-X7 zcwn5sTT1E{K{G}-AU#3~_!S%01acz2V50A9hfTQ)uJjD34RGMZP;YxfAF^rYl49qg z2_QSU6;5UT7is+Wul-39L-3U;e92bHb1M1suiuVd_l*Ovy4#G!&u&H2YRQI+7L}=g z4kf7|+MW-0hE*ut@rxIE10Oywx7WF%17Zly%Z^m%Ojrds!w`H0Up4c;G#e!0kEO3d zj-$H$k8^l{9wuZ06W;Ykkf>ZBSXwBL&n4|I5bS{tUnv8bXDA?Mx9i30Hil*rY&t=IQ=rH^mtH=fdE_(qsX6swe$@NDr|Yq{J4zr8esBO-pHuW$XR z5@c-VxHG}5GU-HNX6!LmYF2~ahHIdA5-hD@7R9Oy-hlHM(<0?cs4?p6J}fg3`_x0| zyXr?(AxAD(UlyG3Nnwx4XC7wj!0{PyMU=zg@yeJ` zmcW*TmQX{0)>J-JNoyji`#VsmYBt1XSP$^Hl%GeYGc-bpk|MF@oAiF=+(Pb>BE)mi zDh&0kyq&4$$&{IGp-wjSSvoq`jQT=y)cn2&BfXJ#aC~eNkw*Hfv6Q|UJS%bzwCM-> zMJ2VeX8;J+S>;DO6^uLXtOnNtuf4{pWQ8@i!fp=$|JvkoW zXKz||H8Gv5mX_igUnvXV#rh&nDle!Qpz*41QAMQ<1Y1v0b-pamGr?0@`bL?dVl6lWU^Ikmu0CJ2O8p0@@+S)a z`HKPpX?KCi*hoVfP-?UKh|TD0Xv`-3WO{AUhqv@O%r|hhq_BvG;B|S}MOPz>`geZj zvf`U!GneZQ6H5tr(a!hNOy1)={7B(nF5+|)VRit}WkkDjcE_6cUM^4lh*CW1E%=03_YL_u=k>ar}-S z@~CSWue0$=Q0sfK0W45sKro)!o;Ih$tr<)ARj2~M>96O^JmccUX2I_k-nyhJ9%{#rE=Pq3 z7zrm(ERP3{c0eaogpMi^f4$!5tj??4^Cz+I@nd3TS1Mx}`-lfj;3eLA^D%F}yr{;X zI>hfDd&?LmqitEvxQyKf$+#7>6*>+o3O{Vu*MHjSVUkqb0*&iCW;|{E%cY;4bLL@x z*m{rplbDKdlGpLRG`o00h>CFY#M93M%snHa&Mlj`1n7HD0x9>{?9Sj7_}3{Z^Bpo2 z=nuWhN%UWnW8wN(M);pu0W&G0EgF-y)7>fSr(L}FzXqCgFv_2JG}|FzYi=pPk}I4G z%$xzxtJpM+>{qnckw`-XPkg-r!Y-*j$=L`1Q(!$Q=$x-}l+aaG@n4MP->c~Zf9q|M z<7nvqUUP8!Fh8{IuNRRtM&55`P$lQ!g;&ZT0LYp59KS46IZe&F|I%2IGgO*$p-o)> z1IT!_G0&pqEN+qkY6%E0kfT~8$4}V^We0Wh#yz);d*_CUE3wC5x6B&yCFpVemoSeR z>cXSC&L177-)k#uEkKwlZ;8(Ppc=z_b!<*WHNA}=Bgbff^Y)_6HO5|TUFWx>Wya4m zCW?@7&*`OvclCWi*ew><=^HYleXt8sXvF<}`^YQBHDn<QKUOhoag;)?KQ1l9FbkvW^XiW%>dv^3GpFQ;lD_#}iJB{#ldbo}9>7{|Y z+CHso(?eT&NVq3UE41UzDjrNS3_d+%ijitDo5n6*v@!kpTo%{lu%cMPLJ=1dg-3C! zP_w4VxtI9ZU$qHq8AK0d%EaKe!}gP&Gm6MmOmRC)Ydpu&V{5W*x!f3M;0H3gSxNp9 zezcL76{3`v>qMmcw#2dZzmJ9<41G=L1E68RI;`4@#dhCTuniIk z*8Lgb?g8clojSi=rxXy^!8UgrODtQDJIZ;~8<aRbt%ublKEI3=)};YAvk*aAs#o}`PWXKH0d=z1P! zKTLW?i3JZOG_ay0gBKCUL3uq=eGjQ&te`ovqbiN;?k+FAlCJi6GOHO==~izh+70@FyH2K?l8Y+H8lMRCZ|Ox5TdZpYd(j59}gewZ}yZ*BG%}(!|V{R+18x& z%t(LSHMs8l?Kbi6h@(e|y|wt!ol>(d+qel+mKciFyDA~eI#y8XS;67>FMJx=`cV{I z@mt^e0%L|2tNsD9AV;Oh2AL{S1<`z4SrE}f7lL%Vo(87mhKSDyq?Z+PuFirhv>rKt zD17P(_>~hgraFAE1w0w@+~P@^wbN>(!O1-ZexjsIxACezvf?w+<7u@hX zx^n@=n0_JkM zF(D~@AWVb(-j&Bt5~KRbbk_ z>zwKg)TA{rP-fiUAqnXE&r5kBA{%DMgBrSP4y|x((}A8uJjJBd5vM*aQmMbRvn|d2 z29zEZ6XeSz);g-XZGDBcM^||BN&lI@b83s!(y)45aYeZ2RviRu6&mDq;F3q)Rx<9bgO(POWkWE;0x8aYA>F z0q_Xm1!qTyGzuQnuLVbV>=`~df{Na5n=9OmiQcN%p|}J4q3#a+gm;5}O0J_Z!GKXu z*7B~KJA)12e<(9FG~~8;{Nc4px`$KY{g$+7If=P4K4BpYc2; z;mA3F7Ey5>^a8##1UYS|_Cu#VXWw-P0Pd4s|9J-=p*2F<7#}1h#+EgK;CcD$7C6|;-6lYm740zP>8Ky#Em%^c3VTqRQERBpLkU|84GhEyV}-%8(?c zF{%UB`>_}mC^ejT??#*i&`K&eT=f**Xhvw(Io79Kt-p;s<~VUo4v`QG_Y|a*`Me*l zKVZ;WIkdGE7#1GYrmYBUeIMyYmUYw2-1-)YW#;4Na!xeSiAT0!A%J&-PO`n!wcW|6 z^Oo+QHvsZz>M|60jkf1;IrIRjD%-2JQ6Vmr@uy~{rn&w_fZr`KH?w!VPX6}^0J7sY z#Pf{fIKj%w>ICZs4~!iV`w8~I0LQ*F<$|2aud>e5pps=a9lB%jw^gJd;&~z?*e0hpJsVS-Or*_;8dgqN>_X_}DWp6KiqR4mFvujZ5PdPUs3>a(leGNp7! zZC-g%z&|0HaqAWUa1JzX5Qg74IZjxsS*!hgh+H(E0PHOv%u+EGV*n#swbAYQv2KM+ zdjR-=fUXTaC?$B=SlrkELV?Fa7rAO*KVL0L~{z&JSA7e?V=AL%BP2Ki>jv^KgLAqX7;9n z3L;;QXSjRyCd0D$VPWZiV48mx;0!!*g_D zq59?S_^6v7(FBG}cC(!xpAkh^#-3){#W-6Seh`Akoo4f2uuyxd+<*^|z`NC)7)2HU zvth!7 z0*k96zH1h$mFY82Qp;Kg zyo`#+bTT12C?RD3_2P2HZDAJ@M7RV1!=?shx0OCcP6<4jMa6HsG+IXnwx+S` znl0Ct_$!!?lnSlFmw$@>tivmSkaqI1x%8m7&E$78K775qY!#L>5-POh1JUaCL_ySO zI$Fx)|0fN~5zlxM~no zBzSY!9uqP7JdpB(JCG^ps`z;%izS1+BwsS=_{;qG?eSa7qHe*dH+{Dv!o%Xo`u@zi z6Nnm-!TIbdC?lz5cPkw|O1Qd!7J8=^0M+CT*ZH!*=P?r@3s_5;*aFy7V#YLQ16Qfd zBL8k`H;ca_%Fq$^9TK!%{7hJ4VByRDHEOrLC@{B>-}>?%V63Tp1RTRUXTV#a@@%d* zws}2TE_lKEymVasss#}4Ci&xzhSZAgE$2!=V#YcFSH?II)?Ru;d3S?oC4R(188G0- z0!o9P$WZHE02R%`Xy#&F%W27p+c+t!PkS>u9=QAhsraquj;t<$ccPEZIhL7fO<8Rb zY^E@mk@hEj0Vrb#iWz4W3gd;X4KoSKF?`3>S&3!O_ve|>ef7GBlgW}Rnq>~Jb5HQq zJGXOL%Gh+wWnDTXdoZZM_NBZ#iQmyt_X+IM46K(gBL#hKD)60x!fF-tvK%>^kja1s zywhP_L?+zG(D4LaROnoCQK0m=pShMI^>+Pmv)Kt$Sj;Un&}(Lycwj=>_g%{>5zVwm zElWkODL_fu!tP;B#1oiCNDYI<%JP$gJh(Kk+$^;yFE6<$kuH_1MbovIUGJ$s<( z0=)L>;Jc#;PLZpTh`depn;P2E&I>cZKk7FN5-Yr3=ifR>n0DyrkXMED0=yaj?N0(p zfNgp?4E}m$H`748e>{|Y;nuvPTE>!ge*;9HRHxaO1BWr4NcUpIW#iRO(PKdW{S{>k z3ii)3z-TOo5%(*n*JWP_(yDb^BgN^DOuL=U$0qALtvWoiPOoW5`wQY9EfYlWAj(DZ5B@-6_~UkLLtzpjZDvv%>F9elKMIF50-g zcj>yoSr}8bQ;;-!8$^^F(hoxVTjn?2xA~dcvcXIrnc`C%ve)$9 z=1VXz=eTX&9N&6SzZ&{dHe^xKJhkYy5Bza0)SYmBnpAd~RJWO8UMe)@y5BGglyGq+ zte3%R@=x)TCcSv~V*l62vKYKd z5?Ju^zL{I0&GyN(j5>+H2Pf7nMa= z`6=x#0%h|hVGDSyeHvFsq)3ibM{&(CJ#>U>`@){_$o)*+E8p;UNNAZD{yhLecG*^P zKD6sleu)KJxzOgS1g1u|1=@)9w_-biKVQIyaUERJq%?QB3j8|kd7j(4mUbQU@AR)Y zF>=;DU)!x**J$(-J`vtlfqFC%oj*}g4`nELd;qvhZzp0J%23{xhjudIW#{d49b5yz z+9;=a+it%j8t7ebD&QdNV*isn0_nj50wC4L1kowjJu>c}EDcbwQQVcTp5wW^!z2Z= z+n9BVA=fRLG%(Kde%h>drsH7=yn)nr7w4__9wIXMI&7js;)g&EHUvd6$nU93uK+az z2)y?d=HA|z(2r{g$&P9K1y3BS z;%RNzkGs$T({-$&>GB_q0T?nqj2x1ANZQ3K)+4Q0xUb zSr41O=~=atM%rQ-4^nj5(yVFlQ+Hpe359Ym)CKg;nT56$KW}qqWo%()&{0|NbX7pG z>=xL_vnIlJpK&APADlJ7UDu?$Vz~y+XD($pbTLhlrhHUz*MTm;T z`@r+pW6_-ys4LHZ)w7hw#_7+F%H;_v*EF-}5T+J7pgsVFHu zKEYom|J*!yN@cbl)IJovmPC3!D{r66H2v+jB-TVS#zvCFG;}3VDWeIWL^Q~?K6Lv< zTRpt7JL84x!u4EnYN@hvm&Q7?+!_z8GybEa#Nc$?^Y|?=spE*~$TVJ>=Nqdj8q_Y7 zHE}~U*ry5#T+~uNGI?gKFeS1FxaKLjH?puBBARz9SXbQS4;q!^#y42|o4*ob9||5Q zU5#_DSxs|8-KaSwNY(pELRq`&pP-q1!Rfrtc2?G8Xvwkcc*qCWAqh8rLi3SI>mMzE zER{SJp)77l<-5)zd6oa2xHSg0(f&5X9HFYottjqc)lI`gvKw@H1Y@;{lKDp~B7T2e za~b=1hpSz~y~f*$M6q91Hv<5A@GXFv?oB^=z~rLm3NfM~y>$|Y-XIC1<$fr5WVTsy1i}e`aLDeH4?}<_;)vOgesKiF+d3I7 z*;F2Fbx5+SnoXc__O15};+ z;`T(W3{|kLlP_HxpO&L26&Sk6%-b50#SZbHY(K0QJSY6@`IGC-soEzZsvn(V-rT>m z+w4)Od%jI2Bw{=wuWJ9E>-L#s3YVs(wG|!eiNZAWNaWu}D1#g4$_5bcCP-7%;pzRG* zrhk3NPRezgege&|UGpJ)(hy&TvE2uSjbDK>9J&W?O( zZuD|@TCNY0pF9FA;Un^p*mjC@kJBPu1yTyct*Ig<7~nnFiBN9#$avQSz9(!{K(%yP z)rO?6xB`s1^b1cuJPiRLx)-;fr=$ZtJ(ERIpdc*%=1pxBR7$!VN$?2=TF zE3!qQS^4;bYw0m0&9zD_I-J@_y%#1VVH_E#&CJ@q6{e*Y=cU@9To$hW|7RuG3u0v8?@dSF^1Pg=-goT*x?sV|cOURR@ z;HTw^6ao~y^%KdsiY0oO6WJ|6hL;+|1-_R%AQ(SF9V=9*Anhr_P_FRSgq3p=Oqv11 zjyuXAM9cIYEa@K+g#X-(jpY9QRzjr zz&z-M;K<8uKwhUr;GJ;t_l2D1u49FZovSXao%^<2c6_PR@SNQYP zMlkT9aok7a&<{#*jVavD2;UDIJp4Vg^AXvRc`j%h4@k5lTTual{UZ3#9m2+!@alwp zBSuL?Nb3_qZy%r+DNe$dqCV*u@P1RKUi#}LHPR?8_x5a#bNBgLwHP9YjVb*E?kBqH z9NBH=14daR>Wyix;A!!5{IJUxOMUo-G87GW zMH}8P&91jit=NA1D08&zv=j=Ch+mb>Sfp0 zMp`i0E-!2=VC=@=BC12gz0uy+*+fsL1Zx5WN*Q4H0bn4&5~gFT=?q`aE!};0>^YX8 zxq8pfrY&pK^`9>%7J^-#-@cn$@+-R2hDu4ZQ<=~C$2MI?O+3V1qLYxZMX~fmLGw-e zZl^n;$L>6!Oc$P@6b(H~ zupmw?eYmrD7Gkpim27ab0cbmhP0psO?t>ZaqT|U8Zvs4dUic0s88-Ea8O#QFuKiNH z_2VXVpOaE(Y~l5KTPRo(=Gqxnbep{{=aKlzi0P%Q+ujBq>J`ZIR~#`+Ois$D-@_7w zNZw%f>E9F_>A_>6P)m1^oeNQ4kk#=IO6Ox-8v>56NAF=QcegXeWSn8Jt%6X$?r`mt zc>Cno_J7*DP1vSX5YySdoe zXp_XUb>Cr@1*Q;pHo}Hf0Dmzi;eGnf1PA4TFT|N?J_I-GwW)}CNsXE&z$`fSHdh(Y&RX>{jF$C|7L z8HXb<-6qx@)gJzO6C2=R>@2QlT9O#b5C*N~nP_@eX4&hYa~{IF7Fa-Gk10Za@zcEd zm3-QmY`d`M;*;Agd$Nn5@5gt5TJbc!HH>0%Tl4E*3NJOEF_M7<{UxzVu(pjbSR_Q5 z?0$4gieYNL@#f~p7E`ob(+rtHY0_SoOI&>s2EOt;$|Yb~`*@VaZMoiep^?&RZ%DE% zeT>WaVKk5wm8UC%+@!s&Eg?)QuRoEsq(@9(?VV0cts3 zu-+~Z-IggY1RH^AS5Rf+)==N+XvcXn7qL{0o4o>WBl53v`ifc06c=B%4E6!rYxSb- zlt#EFbOqEh9>UodCk1-{V@Avy%NcRxU+%2=975#boZ_6HO0dO))~}LO_o?bvJjBi+ z&@nrQsaH{ zc4Uf?^gM&_%{a%@<7_`e&7-VXW(Y5AcHIzCQ4CbC@veURe_xvnDvW9oKdH3&z_^+? zC5=EQ%{}cupBx9G93SxTAAoZxJ3ZFDN+}r{#>jmdq{Tu~AW!y_a2@y6p`J&7S*tX^ zG{#LbnKKp`u!fvmoyw@bPVN~F;`EKnie?!aCz*n|Ava8wo|pL6>Jcc@ug>-IUvNsaCIoZ)=@YdCu}<0Ni03u1 zjq-5pFsHvX&Z0vXRVHie;GX0y_#IX05NO``LGK%?`iH!e2m$z@?CO-SBa7JShu39GB9m3N29%tBEzqIgu2y zBGV$Zl?MS^(~+MrM6a^z3x!60oIWLr|fYq6+USC7s0nGY+D z=%cJg>mip}>?V_05rS6q-@MS)3pl)MFs^qmVj@<01Y`nR{@f0LX3mecC;q*HyACM~ z(uCdd#7jp;!C_}q(+Xqutyr?Zasgyo4Nkb*H$8PhbC#C#M85wEkw(9?yQ*i$L~op8 zbr6Iw)ht#+y$sZFMN}x_FUyg*npscE*}|5MGqu892vNP#mH{;5i3#-5*n~NkNxgbj z*0g60Mh5>H#2dC&rnO}K&iHF6XNRi~ z&V@@x0s1LhO_@&qy84;OX!KB2pO#i9XS#LfXS<)Ut@W3ppYWtt@EWgn0dT`q2Er%KGx4!`hilLEVlvZ?+UqXJ?8-&(a z;`e+qpzP&DJoK?`@$U|jYKbpa*H)z-B`Fy#h?cx1#fxBFONJ%o_4!|P{}cKF2r{HN zhv=CxH>Hb%tN(T9z-m>%yxxCQaoXf*+tP~S2vJP3YD#ndyknU7`Ey3^6bU=jE%^(L zjFVWX;6mv5s%j&61g!dIY~y`mHlg4ZmPCiQa;aSphRU-_>5_*1tP2H#ngR;s5(Bt6 zChJ(h`NjPaCsphDt)c8jP15xz9V?%ko2$kJs@d54Xf+@*#QIPqgrExID13>)ve9uc84ZhX2TUW+%y&P9BTcgfX5!_1W~t?%)b zh#g|(&@m+u>~oeq4Vt>ESA6)9j78-%Qk<43abb_H%`xI9GqtbSBsz~E)dX#%P%QXR zFpMs2I}g)mTzOlr&=xz)om03|bz8kRhsJEZkCz*F@xQrszO>kNoj$YBx!e!zA(tHj zr_jGcE!!isz0G(t-$te|r`LZ_@{ZUY3TPm199Z+tikM(onAqN#+UpFjsPNE|IOE*Nq(q8~<)K`$BpG0)Um?=e?T2io@YpmiLkx6u z_gFa+a`@T?pf)PpuSD!4$e$2@M9#V@+$~?Lv{AsjLGQA`aUmEupsCeB%taJAqLx$r z7%;<|QjmG3E1m602nl$+bv6c-KX_)>f9jX%2(}yzk|({~@_U*4yevB6l8})~xi^Q= z#%345EV`_0B*N|}mYG`k z$0D{VTek{YA}6o3*j&p&PZ{s}s%9{zPZFWq)M7<8ZJXoQkg1@HXsKo%#T5uBcny+U z@$3nj8$7c*>X)>%Yu7U(zH`&A_5Am|e`iS#5oY0EA(GZ zS^p!nV)XAvgfyUjS-6f89hWt>4g2a_I!+C${oFau@C$KDyp4QmRUP`VzC3Y6O({VF z5!(E*?9#5Wa={!uxS+P4lCFFU+opAU(`AO4Y5_`2RabasZdDYl#hcg_#I#Ss`3S=E zZk75)1P*m@_=%im9Mqw3H3^28%MR4tlpq->%S&z>a-mj%`BFF`XnMjl_SL3Gg z9h2U=4iq>-F?hOn$iq*SDf}PXHX9j(^$`;KkujiD&#;s<8K;_hR-I#xe+ z|7%7rPlB3`BB*mil68W}RKgj0p#)W((Bc`%XG~L&GZzW`?m6TmXQD|%YQ=$OHuAB47@?JC_o8IY&|}!FuOMt)y;E9) zO~r1c@kAN2!-EAIR>JgrBv(gM0&A5g+!cP9w(D;$-KDwOSCe>v8k0Yk1N(N%1i9rd z0P(U*n;%x{CgpLSnwo5nFdK9ClhRV2@_Ocz7`{8^0{uEkb35oYOS=15!#%jWD%mT5 zV37vCJc+TgE&@~tzUq`ya!H<~%i#8}*w5!Ud%PCKm1Y}6NW%;^ROCg-#t_~Md!(|# zdLQe;h{A27IM*6BjLIK$GQD?(_iNo`vX!dhbQ<9>`HHPf8Q2c~^%a||i;Fj`D-ZM> zAEqt5SN|e4^szNCBLW|giSl3?;h2RgmG^S|;gjlNd?rObVE1+1S4qlC)_2x+{I?^f z7eJslP+1&yMRR!*h6UT`;k^AzCjEhm=+xUl@a#Cn7l`VmKos9d!7qcS+l8>Lsk(4r za;OKCSRs$7OL2GI;M}j@)#m|^+KGfWAL`=)@un$)+xLz)`2dkLdIkGD8{|o#`(pHS z>Cbm`cxR1Ll0`%gG=IJ_zY61UKHHMZ5ie zzt)n?fF#KgNk9|}q%wgY5nPrPEu*V%y8aALRX7_L!Q#;ibMBQ3rp9$4Q)gs7wa4@t z$_`aY8zqb7+29pmVt0-#%Xf}rZ^&Pz^Y6x6>#0l0Hpa&8RN3UG5@fQ5_tkJ3ijC;c z<$x8SfhZp8)D;5@Q%$i{}&bg_o2sX{|C|M+ndg$OrBec91!Lj3e_ zk96XJ*DXul5o>VEMYJMIX<-UW;tn9)56j}P4O!;JeA_+qnao9R9*dJsbVFGvOg)rJ z%5g_Y84k6fk}TsWjrFCNiY;Yj3ZUyT7W<_Hpvow1jLe*0k>F_8-+6WsiHYjgKZ0h z)>t8y4PE30e_Nz&)amX%MHUbI_}nH$SCY1I)!Bcd>SJdoE(N{LO0ySPQ;tP~H7YC7Pjz8e{&7-Loc&&hq^w&l@2Fu|{r*2T2~T@GJ{rFJ zxV9q_=2Sn8w-W(Mlnr&wmDe{dfZBwPCEznCHg);`;>Jy+bmg}0-AsLiKB8e-c2y9} zA6JkH%z-<)ZPpD_5X{E8o@6v*l4%Y+I7D&sj3u^ZhF0Uk5kgN>S!j-}V6TXKXgH;H zjHVfa`>oFK^-_9hZNm5_d!7t$d-LE>I*HJxBI*Im03u<%h0ain>Hnzu?m())|9?rS zNRnJTNoKCSvdYT5W=33u5VFg?nGuOg_8!@LD`byrC)Z5&HM&N*Ms|MZ_I`i9pU>}K zz0>>N*Lj`uJde5j{ht7n)+TZRXJyv?dr>N!!R+Q}`p5q~s2`C;4!SEtcSKgk)y3 zm+)?#Zflz-c)Hh_k$A~ghkscko+s$HObGHBJGLsIrxiZkPH52p#>?ePuATuv5Op`d zFf8E&Z0p8Hw%ljWmgjQJTnW)+n4h12rL)$>c}js4tr0-prLrR9T(2+&7%TZpKHSlU zrvFA=W9|QuU|4vsc(P#@yrTnj#3kQ%-&68H5#=s17myRfJm0m03;pzrzl{r-QB2@% z5S~cSZKDvWG}oSfWH(ad^+Z!@!beEZd{n3kzqtXcfF*k%xQ1etZ$^mIOUdK$M@%ne z&{1*|%+M~I_p~^PK1Put)zWU9%8?$x2cPMjU1AR>C3Vd)WXYjnU5@VZ=HRtYQZY!D zVqVS_3Bp@2*)P3m!N_yapoCVOEZ?|Zac@Yke%SI(*?7rncJWnLwlI~JPe<|<6^N`Z z?(KNj(0LX!r}L9o#m^=~zIzzaH*nfHz%O!1=nsDA6cm^5azLp-@;xnLC-5bw0@q`# zzC;OTYF*Y(#W}Lw43r;ZU~jg&F0oOYSIKRqi)O|K<8Nilqf`FDp~kOT_{KmdUZ1w- z#9V3o&(FTXUn7MhyIkVJ4+=QpHTK8DTFO^c){a7(u)bF zG&`bT#Fqa|UeRJU9vAj}NM>IHC@M^lfH+NmeE50plpjdrp=tz`3x*&Z`67W0_rQ12 zt&*t#$&lIIYh+9=@_+NHq}EIHiWNu>eLxD8DrjP{CrGtN;IB60W`Ue>Uqd*wXAX$z zao~48M}Bcn*E`hRDfc^d0DrF<0~811p=)*F^!IPdmS9B9RP1%lQkOJ6zQ+0HXnHL} z&He$o1B@TseD0FGRdaU@zpX~~y^|(bg$kZt;Sn2Rm*kw**!fYbTniqbeUjy?mnPK& zcZ;)~@&Rr~L;7viyn~;gIz>%ylO*;h+Ty|XJ7g-|uQkC0{ak;=CX)mE&iMIv0nY^d zQxSdp;>g-h)VaQk>P2;2k&RO~Yy!9XT5|dn4#hv1vI6>VB}COvoMkNW^&?!o&@#0> zGwgH?^J&akQy?+tice7Fs}sw9vU@Jqf={^`hSmKhx6{gw$-j2HMI|_D_f2+HTpIF; zt-n$J%7M{FJKrnq2XffwE(1v)mjDBOG`vBr_~dN|b%e;a&Zni^33V3F`OPBguM)Z? zT6;9izgb!#qjexF@}B!|*8z*`ljxITWD2mlSdbK5{d!cAGoX`O!aqGOQ(NR44|a(P zBXYaVdv?VsIaXynd$+@O60Iw*cnWzunJRMOj%l_Fj}%(!kXqJL2^pOD2c!I4q%`T6{Lf-Fgm@p+vkpy;zPz$=_Q3j!^dH4g;4nPSA|ymvf0D8+LK z1a2uHYVAXqU-XDm%XqKkG!mq4ElW4==951s=y49VZZ4SrSvJRf2w_PckpYS+7(v2; z>||%rN%}?5{O^7Zkrg>TS(Sr9cpt@~zGSoP73>ny)?|KHMrx)wR;ltw!mgMiak+=q z?I8Q~4v5t;lg!AM-7A!2u~K1?l?E)MRt5<8!2n(GVlLe&!64B7BeKD{qK(uZXe~8C z>dW_bMj~{Gpez0=q7}8Eisw-g8CR&1no|G7b0vxJI*kAr@yICVPSqW$CF8ZY1)j8z zHX~fsMU*i`Dh86HoKhaUH?^cK5guSn6oDE9)<6dJIBIVvBv6>?Qfyn#_X@Foz)7G z*_{?0g^u@zM!H6^=E`W@drU6ydkZ8@x|K;_AE5oS!cmr{Oo68uVRM{uc2gyTG9Y^0mbNn*Bzm)(tvQ^sXhsxe^WLx~@KO@fE8v4# zExWJnaX=d>i0Pp`RBDrdM+PEdS5)?%rkqI}+72woib?X$+Ut$&58fcy`^Xct@bB39 z;58cTh*2e#ZQU00I{OBmCLQb3wPi2Dnbn%bPFQsy(Qs%8?Ic8WvPjpG_ywe1Ir8m; z5In&Q5-;$O@1BP3erh>&LM~i3PoyM;$(`}W>dx&z}qzC?7R<=zAb}O zg{l4W>b~bz4=M_Mr+^utYWQZbulW%s0n;t}yV9eFEhGns%lQ~DG@qURf)N}aBRs_= z(Y{9`87ox>-3#mR{f6m3r5~qwGPK`TvS+BS1kgtO#@%84KBhPxe`Wc1KQB76y<6-H zgL!Qt_xG=g`Mu{$?;MH@fABrsJ?bh(Y3?P*O4_A6CCrGlAch?%HgC4Tf znHl(G!w<^}p9|P$9m)xr$jkd#op2w>|P6h15n^<653++x1ll)pn3-;>m0 zJLn4TL7>?9~x7i`tw@zU1A$f%3ORJ5A{8 z`o2n~&x4E5_TmmwMx~Isfy^p0VUV;qG-5wX}1jxfE#(`6nWC_FrNG63{8T${%XPk$w6Q78Qv;tHc9{ ziq{QON5(yal&-K_bmkbitK{%Dt7TWUMTm>}%$LdO$%YVIBsk9b z(N-jQJ%s~^c3!U_bV_+|p&EA;ZL>c71vX`GZ?Sb=GaQfb80B zO{VPo6vVxOdS||8$8-4!jY}Lj5H>iNf8FiLyshj`k_SM8@(GQbEiP~5{v2T4?+HFw zis?#y($gEl=u`tXMr}?Cv{=%A0l@O+n15Pl(Zm|IiiNpy)CRnnqTHs5w&XN;B>PBAJ$ z+LHb-j?|Bub-DhbKH4Wr=*kAEg1;boGI}AbjN1Ce%me@CckYUxo*9}mmR7T)q`9pm zwrx_|J*7(U*(xwgFM5BKq^0Ql23k)~g#X^u$m|-K_TA|%+9Ive^M?@6!@3Oy!?M4G zh6{T?*4nu*r-qoHQZFC1FOFEsUN0P!CL0VpX0Rf&l1E=VomMvY=e|<(Dxa`@y-U!d zWmXb%{L_(_UG_pbQ=Qy}S-3m4rmzHs$W-^L<5#wTU!9j7>zt_lv^G^19mRscB`Gvi z_!c^3EGt{+lkD>8IJEo5ZmXqn6=UP>pbdMUcrP6T8U{uW?jTWD=S^?}^uPC~-C^x- zf2PcxJZ=8gzJ8w~Eemyz@mV;wtZ}<oRC{xdPdm47t^ zKmZ{=zUA1cUNq{&D<}eeQM{PtuVg~c9&1LD?)JX@sUpfp|c(h^Z>vcb&$7<&X1 ze!Q}M9K~7K0*4rB(TJR4!5kXm5PF4nRtCz4+~=hixdUS5pjxMDujbiN-t>mSuW-NL zG2YJ5HC%un(%6dAlhnPa`@OZO1Q^3w!o5_8d~{}&JFUKUH*mD5G5%tIf_uP{P4fM| z>N0A(*Z$)aPI`T1eR@?52VzT0mO#4Fy2#c#`$_Nfur2WS;6QSqbMrUW7n7Fz>(n=x zU(O%hlrm<7AvxY#THhE87+Zbi&95 ze)xU_qBYRrF*K8XOjMT2i7A(Ix0XUmk)8?;r3m4{s)2jgqa@DrK?~ZhJnUbY&XsmQ zY!ekJ8aXX_Hh0}7s$5OMJ54W@3%RIdHN`~(HA#4c_tZb6G0&v|f5;0v48=gDY&3<& zDH@#ZWIiMum2>w46kuT_~!{=>(u5S>e>df7+x9wiw3oKJu9W;K1}^8~XOs)Va`1F)ZC2!Wpm zH9vvAJzjhiWRSnmAB4vIp7T4~jf(WX?RUxRO-~D1gv{^B6P%gYDiFiEfB8k_cc3uj zt=h0Z`)Yqo2SLc69ywgi$dKFXKhG0><~8Ym|5iRzx?3N&Jr*VY#qx%>cU2;cY)AEp z7U?vwyKmTLjOHzQ=b+>^B%)DlBMR+3-#_xA-zx;w znsuu+kX+iq=*A@S$BmAyOY3bHygUP(G0sK#b|s&!HSJ?+G}o-tfZRkzS=nb}y;A|d zvk2MWu-aKcN{(HA$~DnA(}d>QFx#s#ixqq#ErBi}6+Ho_g-1|NdbF_Giw%mkrSIWx0#3i&#qD`k@B%__~XUMAY`~_Z63I!0r29 zprr?(q{Pcjg%rZdYOGO(3ru0{gxa z$Z-NX*?X%sd7d@?>_kp;UV+Za|HeV!3KXyEFt{;+rcj@&gcgEvi-5>b6Hni(vClVNthI-BoG!VOpi+cLz}!!VsCu_>$2&=REYzMX zPSw0d6|U*gQ$EVuVIZXwEDN>w@_L;hNF}`uA%oKOhc+&1qFSZdZdx_l&2y$7k4a0G z?buCo%EHffMrSRj+LMEPM6KkNeHo}amR3df87^<}5e+5Mh>HFF&MdET?MJvebHlHj z;j?taG!A1l6w++`In++OH)}$^dYA6Ku83S#G1imK3o%SlEVh0}vzbeSou?Zd_VcR1 zAn%mfT&LpSLJpTzEPn8;VyZkLW2Ps0K+mhj$*jlz_42tD$-$w;et~1ruD!l~*O>d6 zz}Z?Ck^j`wRddPaXcCjytX3XwIK_iVsVZvJf05&fG}z_}oT!#P$b-B4HfErYT4^Tw z*BMGjxOd#WfL#*%)}*`QfCmvrae9Br_!|L$Cai2jatpa&O~^XcG)ICdrrPHyna@xD zIAr-UYBJ4bmfQI3q@I2Pg;bBXVd*<4<9wU`ZbQUvfi~dnswf0)yK&{M&l$M zr;0p^+;ehSERW3hbRY~(=!daw-5&3XqvIMRKm6;=ZY8e?YONsk_kQzYg_3Appz~Jr zpiuq_Zl~U}!7Qfy=x!@PcQ42i(7WOtM-_#`EvU*omjneznG?M~!me4UU}9?Ux%+N$-`VU`$-!fQG2CBkdGh57eLydJ>p|B}CTXN)#DSEF z=nAK~dx9NFLnzU{DE!HjMhcjQFU0?-YU8X(pw_VnI+Hjg@tA&nEQblVKeAZ_A3+wf9@VPiD4yU>V}tj91R4E6?MovIp`Q z%l!r^W4c8~1nkyL%j+6M;$fr#zT#}pDtn~X-rRwRbUkB21}+tfB}D}Li>*gJt7fs` zm{SZ936wy=hVNrcK$2}h8}+>9T&C}qZfZL@A3yOlZ!yL3B=2BK0Bh5LcUD5KfrqQg z1J8a@u^PV>4l!vg`nd>+k$PX90`%txxK%`Ou^z}46=)$Bkn3wLEPWI|L~-<{9=fgK z_A^BK$Apj05Urw0DeIwybnrwN#kfF&a}U?mq?v=O76X*8%;EOAaog(9u?Sh4M#b#b zkLb%b@9D6@W!xY~Ypom^tAAKTnS~iaYFpZMYS=iW@|fml@5buo2J*!ljOT~5^CfwF z&sr)zBxY#p11gY@2F*Kygu!qki)+GtsZfO?!P`Kle_drFyr7J3xWGIv!v?D$fBG#_ zH_%tAvp=pOdPvHqWKq$!Nv5pj@8W}yOzt?=FUR9RS;a%B4^G{o`k<9e_C4q^_%w>9 zx<+up<_-D$bXQm9Z`^9%#Yg%{%ltwNJ&smd0pOoCa>I%>g_2}g2E@Bq?aH4JMk9fn zjSInyj~?z9!hCp2fl=2U+-KCCUw!|MaqVXQ^q`S=2zk6dG9-6Qd!bvaUZwU4(Sfsh z)6ylWU5vw+0*Ml%hY`o4_Nn{+DoN3u@~5*}i^x1pRO7 z;8kB~=Lp1|^m#hO;>OQNe+po%Ij97hNOY#wP7I#Z@?U)%uu3?UTsG3{tx#8f)w3uZ zc~GJteOEH{jQ!b#f8*`zLrqH-Yg=Ux1T&zNK_)^kU>(r^7e4MoVUr^6Q^ z3JigLA%YL;88{8rsZjJfxoIT1U+V5S8J=biyEAuJcx}^Cp)B#*X0J@G8d%uxBFyG; zP!UKwEuGKT1&>ZkN&f{BkW^ zuoj!;=|^Lq45TVs;*?Lz>TsXFi2gn9ApypU^Pqpm64_3Z{=RuP7kZbTXx3vy6F21d z#~t8Ze_~3a=$Hg_<JYH%e?Fm@ z2NgLU?O&PzIlJiw$f_b zxkdwJ>1(t$L=j;X^J$f=iVd!1PksQBtB8^oDDbo7t6NT^r$|oobEY-Fa(id46*>u1qa*U8rvuWqDvP zVact=49;7mk}-|!iZ}&pKg*rd0#CkvzIEY<3iNBVhN|fEwWT5&r-TV3I;FBz(&$j- zf!G9`9eDw4>}6CDYq7_ZRE#zazVlxa67aVS#J1^(r^t!VvvCoC9@t$sKa-MK5sd5& z_93|M+MR2Ll@DP~qsQQuI&?fi=)OqAqv*52Bd2F?j<%56@PH}et zS-~KH+m9>P09T?3-4Vgb4tF&e za99$n_=p^IfXB$~e+B_xxinf5Y)xRc8al23FRCQ|+X6p?sLIy}{Tdb)$#H$~AE-p0 zoup>CEZbU_F2p872-Wj6Ice3lE>Rnzj5kg7y@g23@xpU*^9dGXW!yDTj z;j6=f-9O$SMYd>uNftCu&-?2rH`GDE zQgj6Kpq|mUZ3LPZO@c)=6UK!{4Y8<73TU~+n_SL~sm#~_8YlK4Uqt6Rym2Lt5~-tJo^Y{N^YwsZ!J02N7(NGOLn9 zssxaJ8|Rk+eDVpq0)1hm{_P1v8&4AV3Gpk8vr!qp#Tq_k7oSp{u>RQ!sR7h9hwO5H z@9U+({6Ox6V~PAfGhSHKH~B3P)&?h-%tr{Qt(FunXBTX2B2HDkcMc^_z|RvX8CpOU z+p;K?@*6kErE6!POtPU0Huvd=g9Mz>2z@cCDaJ$KJjm z$Axyxi@tWiSbgrz3t_r>ta~nG+?1^eZn*`;W|yd+D#+NoJHCXeL*Qyg$6f61Vf1tUW>fBuE4$ow*GrVQSdm{x5jgtlSoIFqEnGHb_Rfb-fgsUdC)k5vBrjx?w}1l8z7#R)_Bim8!`5kQ&UrRW@d3^&GBz>_}2asy?AO zt{CE7yo>Bj8ODhMJc_ETTFgFl_`-#vHCD0O)q}nGCaFGFk?t+Qjk@h5OX}H`|OfLI`80ec-E& zN)_4RU+tHn0~w?%dM&9W2mc3nmhS*aRXSrsr8&j98Sz(jLao9S1@kx?|I{1@$Gv%YbmLa*9o=+Pv6oJ1>C;bdnjF3`J+he(8Q6L3yzzt&`v$MaaExEWFp6XRv!hW)-$BN64rpOy!y>p&LC zP9XwptDBfVfkXIh`uWrSIXSs7Gv}r~x8Q-X&cEXxNM2MGE5Pc@gl`;#Zy5sG%mc_q z?&h{Vd~aKyV-8aJc4QS1QEa5IT242eM+i3JT+ii&5Y}`#g~p!^>v^2A_|R{eRA!i{ zgx@n8HXkRSgn4XKcIPQvk-Uvhnj_42IQk(kWw<&aV96dx5C#qj0XQ)8gLb-Y&In_L zs@V_C8({|r4l6j~&Dn z_NmGQ+eVS)7a`RjDP?P%)NQ>WM5%&vUmHafrX`cf5P!s`Ao?+$A;4H!9!lbcH&|Wu zIYp=TppDWsot9+wDaG3N+{~c1wCVeL(|@u6ct;3iB}nAp-0OQfFQLV53Z^#qZ@0_t zEojqQIfi0`I&ECrxZd2yx$h%-D9QHiG_0uzQL!n8->!*e6Tc z(ZUpAE&_?ytf9{iH*9jeu{Y`TI^FXh9jV0)>h}v~CHEYzz$L(SSt`OaWP3t2U^HMa zd+qi+$0%k8NAK@i$}sIC)8nKbc6N<3tT7G#n?=*xC#iNN^21%K=2+z#RmK7;<=`W! z@bq2*mKw3ZsXJ+Vl?AM!_!{;41) zJmRM`=WZ|}xkUNK6f(Xkw$_3_0XD`THb%_LZ#r&C0B_meeFQ#FKzgTq#)Z7XY$f+1 zv>mwr>Ti6+bQnB~&=ZWOuRb^sLJU(ZRU&9ZQSfSC z0BWurWN|0D|LQx@9It}*sEF+QOXc`BFSZib2Qu@tH=t@ho!N{X(Fz=q$JFS+30L~$ z$`cpL`1CyyxEo$yUg)TJW8dh0xO6VNOu<3H=N|7)6N$jerD&xh3EW*F-|Uaf6t3<1 zja+W{EKM89g>6 z#qyZhtlap^ZyepcYW}({!8ZynYjzT*~&98)KKBRhqcJ#mR(>q-1_+AS_d6Hdu z+2!ad<^OLBhXkwu$u{4wj2NLy)-o)xpzQH!tfzT`>&@$jjgnJCCe%}5oLZ!ABkl6X zq*@m{by!g}T<8F-(Z24LX48GV>BahrC>H%a-opZ#!&Hl4KM$uL^r>9i@J37_Za2ea zxUO@O##mA|?x{{;1r0iDo)TJKVaIK*f6{HeuLxSh6SJaz^YhF{505Tf4Tcg^eaT=4LX0AoW z2D1OM<9nAnWZUH`*mtjJlGUKmI!YJ%hF~C3`t)U(U)m{D79hlrY9C`Gq~5MJyp9y= zVtcGe7F_pMRC@bu`C_&=HQp`7mh9Z1T(utdg+q};oaw@HXM)I@XlOAevMSG8PiCxe z6xyK)ywm@eByj26PmcZ~F^EAQK_>U_Hn_>wstGZdO^j|dG}T~l`u%Jr_3yrs&DI*& z2RaM69(U4RADBT7(^DQ>Z*+S<7ndRqzY8~theO>}ply<)Y?*>(3B6gKNAdstUSbsd zJ-KA1=+~5_$~+s!pL$0>jHnG9zno>moO|z5TvY6Mpy%}_r5C$o$_OvX3d}(!)xviO z`w}%JF*j@(Z1pV3Kcl`n6cvBm%j%1iGC*|2gm~=+QR=f$WJhADH7+Jt>Du?7Ajc5# z*dR8MMdl6zX(0j-1^nxS0Xq{_FCKR?K~~#(^mHkx`u5f4c=_W;^qmt$G}M>PkMSQ?gblRQ3VKscLlDG4?9{6M>O|U zMA8}c)YDDYhWNv8sTAQS;oM&eube~Izz5#l0?L8^tU~~F%yd|xY&K{M!{2AY8KsYd zG34Rig7SJZnI-xCwIZI;!}@XO48j==PrtWjt|Ka%JJv_qMh%1gwYB0%*2M(!Sbu?r zW{!uue43mW-tlvbSjWOd_=>UF&kcUp#2_D`BzxJFTCOl!J~q&jOWHP=&B_ULjZUb7 z7qzy;kQ@ zx%Ar@-gW-^9|ia&;b$zTnBuZL00v&QaQ-+z*LBOI_PZV7=uGq@sms{gmbPA_WyKbG ztg_z!^c8$dz;-~U;&^pwkC@2ROR)Pgi*?vq9jpgrc8PMp8qL6jc0>KV+?(%rd+W#8 zTGd}Rr3S6pNIP)+iysoe`af5{HpzYr-%Fq%?yA;e3}K%Gx=M!&SlQnX*miHw=if}7p*`o%ZmB z^6VVk>ugQW&1c!ZeXN(_dDI&hY4bFCTOYF2ds*hYluO_+RR6b;-U*d&glSfpCWs

{ABpj@V_`UOKu%D#gPqkZoRnwccHvd_CMB4n>e4{ zaKKeQVrJ(u^H=F7PetP8(iSGdlCEO1jqI6aFRn)|%66P`Fi6Eo<6msDXS()Y{)B;U zCpxm?j+p4bBcmQ;j`Sn`=^-Y(sW&d1E2Vm5Y!jDzr#d)BCzI!*v~#eRU%lcw>i!9Z`@5~KkOkD;Nz8au z&k+_ughkTJ@%{YwP_U>3{a@P4h%G&i@|ZCkRIK5769N9t^dT|oUSC%7c74=$k|}k! zX=A?Y`hH>bk&@*xwtApaE=Y3N#q&c@@MLQnwAE5iKS_t++}-* zr+sxzcA-zzCG{qMQG$b$Rhv}RNsz!^r~@Z6Jk?8*^TNgFS- z#(#bbB;buaQAb0A%-ittHGhl2Hv0f6LG5(~t)Re=2nVN!%@CHwWiMa!(L)DzV2?kg zCc3v-pLyzC&LsLmk^ZvxV#1$jJeO6F*K`Fo$RpR4Ywyuq80MqBB-&h#uiq>3oBiJS zyJZZO+1h3F=d!rJAG)SYL%c|1V(VSc|5zKr=M>N)F$%oPXnXFa~DW~+w zso@S*Z#MLfk?{B@F9L9D)bkuB-cdIYH&!nV*{+j0)9xF=`*yc~OlpGtun@i|1n*MV zyJhKCYdjt8;nY(Hg9>_H8(O@D{hy=4fV{Jb)InnNo|;s|Ug|h+DmiDqXHGb=vHC5> zkZetpYV^rBD?7>yW9jT`1B~Vjz&s~=bW zZz89fLR90V+{vonAKNKFOY3)O9PV6m_(Na>2&!-#h~(?Jgde$7ai`|oe9hOF^y^ETHv8i`k1Tt_`b^DcMNQ`jCM8s z!2$o~tT|SZJuQAeIz) zkOYaKt$~9ZS)pi6pCSt@UEdo8}1IY4Lfrfd%f0Ymk+pyJs z`LVqoBZ82uHK-hEqXd&ZT$+vtQg?oz9}28$KpkybC=@5KewUF*>jIMQ8A$^;wksUi z!4;RHS+!0rUybyZ;LuODC5C_fMYnFXhFxgddh5;oSkdvQHmUXkR26yXi;(d3Cby&5 zc@kO4%l@# zj~Rk==*FKJ1P4^6JK^W$5!<;7#uOdmuHo#Z`=WW7Q4T)OXx?0mcJyEYRnjnAIKD8ahs%dc4a_el+47qIXx^yzJ@2A^Cci_E@j2uIi;30>m_b6`QFQC=Tmlej>g+`ZxLJMY62@ z48Sc@YBEtN9*w8RpB~)tTyDDJG?5sf47Z{%7D=>=ewpAI#0;J84CHnit-E@~saC-# zP#o>=7R#G0KV0-Dq4}@s$>Pz~w?n(P`0L^F6{zdbge<%P|B4n*Fb){V&P^pdS2q zyZ7~^y{pV?{vu#$_3YL2q%_`D$(f8VGm6he`4aN|nvZ9GMO>hfI{uLzsYT%U zoc>5FT2#0-vIkaq7_|%CtdoCgp3|>C?HVXY*s#!4~PW82z6a zE1dCM>EL9_D58953U^`Cz=QZgebt1Oj48u~2pacI@v;5bL5BBq#-n#JdF%3ntz#hefD z$_PDi9Bodi4d0)YN>85X8tZ+Pi=&<3Z4qL&G}Uk~&oL2u`+P*Y`-5Rz{*=v>42cAH zB+an^2=NMo$2Q|p&N&KHG9HCl;QIvuw!a9n@>v=)fTrRIfU7L`R?Uc(dO1)Nj}VG% z9PRc&fc&j>H(eIa^H1dW3HA2hNPh@t9BQzvr{?Sfil6Ni)KO?bE`#`Q=Rup^AMZs{ zl^>cJ9X}!JMnckQb_9NOJgi8_{t`1L@pzm;)7lmkb}Bz8Gc40U!dIRcbd0uB?L8J#LAi`(E$TFXxqTto5e?%nroeY z#-kalE7>*)M4YfJp_%6h>}JVwF9*^Wfm%_0vWy^&?uS^~-U<0|6QBBI@`89=n#v62 z4xx9u0sPCD8;>2MSdJ&VS5Vhd?%`-cR*%OCZZ&k|FQu~$TeDGdtoub9^bncZTHTWI!py7pdjQ zH7Me5EQ`HV9J9T_6huz7`KPiTXNL6*SFyTL*TL|f z8Y4yxrDH4p-k}F*h+WIpAAD|jlyQw_qFm9sVygykCD$%v<{&&XM;pBOc`Xk$m*esw z4X+1V6aOt(@+a%VeohuXqr7qu!M6GG}jy&(IR zLlV)+rL@7ksb%K65Y z2VYeEP4cU$=4n2EJ`kn3j+1?Y13u_z*YjCb<?L^T_NO^r{lwIb3k{4IM6L+ z^nX1iRLvmAis5mkrj?`C^6QgoB@YhMMWsL=z*+p6XGqj+%h_>DK7j=L=0fE#ahbpF z=IUwMFKbR$~l+jJ6@eGy8P1v zPZ)6tcVRH^0&)iVftvgBi*aPn{D+|8@vs@EZ@bUX=m${P?zKxi^eI#am;k63b~7*1 z|A@E`9ls8=waqv3=Hwc_ctP6tNa7-Evxm0+@5&$E>d2=36J`|CwO7?%J;pO3b}Ms_ z2^|1=-6cyGDT{Oq4}d$|!`P?F$H7!XI}9cJtcGq(B$+rr!ok#)Wef~AMf)q@-f_pU zP0u`oec$DxW}x{e{;-RcNZX15d!fX?EU^E-_q}s$E?|gR9{T0Rz!Cx5b2DLCSN%;7 zsdSZM&8Q-5YdZ<*lH{8}&#os01V7uhgmh`;cX~e$8n~4Myg6t;b8R9NYcsyN_1|@u zCiNAc@z7hemhbzHTrVBD445E~#Zbm9d}-K$xwMIz>?1yD76|8sSuroZ zNUmBl3R3*67+Iw08bW+=-_8#aJm`pegsAR*Z?*ZQ&=115t-{2-8|<4z=Ls|iav7)_ zf{r9AvT_6GTp?tCy8TX6p3@y#UGMGnU8+7@Nm)I=5FxSn*3li!=H?xrDB^A>5hFQujdy}lZ1N`SxkKO!LKcn? zTIn5q3axkriL9}mL~~D5oT@BY=VE7p!mT=0sAr$j*-N3UgC^sk(>v#O2J6aOjb=_% zaS=PIzftP8neBfeC6If;K#ZhEw==?1&fjRZfh3s;lNc4OcU{uA>^1tnc0s+GsZ>;x z#6!kKQ?Z<~vK(dq`%9CJJ8GG7dk%G6H@{~4A0a`L#$Ac-n988*rqrw z-MYB5v=bV(9fR@op_&yUz3x;!w_5sZn&m$eBUtSN!o=Qvzd^l2FR2}5VL{?U1Z?<8 ze^7_l=BBgXtB*eI5$|A_C+~xY=PW~*<0;1k7O~nQ;ba68@YBa!zago_MynEun?PoS z{KLn8_=N5acBgTv7F@z?&dd5-s^ib#vMmJ`vedfn1*e|Odz&?SVpZ(n_!(fXNkI|A zB$s0oyG{=X?is6g&S>MI?vc9w#xFAa)25^uM>bT!cvh#+U~sR#3$ei)7`Kl`#SjB3 zbiC%z(YkGTTab_hrOwaAFz6T3xJ$5@s_v9NPXA(t(>G9WHiBcmH-Fo8Ad639*d}{W zCknDvDfmR-C!oU*mt?dSt>&M!^@-==w8PI;&Mq+yJB}|`T*-|Qpjdjs&74nmt%VT8 z`%9k}j@T_?xA@cFN+x`~zw+T8F$weMwySTU#$In>kUnL_>$491Z7!Ry zKbBGd78KeDTtng>Fq?(Ef_YlX6($s9HL#$f{_~E41{L5W?=n$eFV$T+?Q`6waDGP>)4cYLhkF60&Z*$@+%* zfE=0|rLmA$et67I1joB$>kQ~lQ~Tqb z#&qADeQ{}ELAb54N{!NE@2|)ZX%q49X8%D1jJZ=`tWBOG(L7syrkED5FnPUfgW39dkzZVR15ri9+s1(Eg!DWL+f z8IO_F@qY~n|C}C|h)e8u-GIz@QH|d1a~rkAW&s^fqa9lb9e-A*14`2CqSo*QlF9JC z>ZHY^fLlAYBX3w5K3!?}ljWM>Ri^OYM+D^V6=hF+zzT6FkKUgu{_bNm9V= zMphRHm6bvT+Ce}*^A`!c=w+8`M(Yngl`l>zMy*Drr1U~rF9w`HDrP)TpK?tXdMEd1Ct9o*^; zKWNWX#xd!H+Hf;gf?xai+gv0@$1U}?7f`aUunVuRy|q~-NS-XZ+Us>X6DBIy5@O~ebKS=jTcpL zQ%kvY6E|hIY04tP{!7U)+|>SQQ-j9LDzEdG*E9Hkqoy6V_8NaOB|i3stkjK()z$Ig zm9!Nj%gxZph@&1z_|cve+m*(pH^#I|`Ju%^SJ_J`Ix2AK9 zOooZk7{At8_Vj^D8c&mIZ$ZBt-`Sy`BcF=voW!r~fN1BPgOFKR#?Y_rOn+2ZbM*i| z5}=UqkBf?EPH)NQ(ZBAV0n}pOEa{+~e$2|$aWH;G)0-f0x| zX7>Q8x_qtyr)#iGQN_K=6OsK_s2V`ju+=Ioj5**6*4DWbDy=E>fBZ)YFE zIeYv+*Z6*a|HtDV_duOS!1pZ>YJ(54lN9BZyfkLz>b2xYgM1S(FSD(l}OKnJ2 zHK39DrDSV;-$E3|i8Qc!tBp>X`f$;eliVl`}9T9HSPBHj!fVEs`UddUziM%laJmT0Xe-ugBQqZL9u$!8{B zGTRo!J*TBX|EQiGzRq)Fp?{uP5&W6@vBzNjSS)X%z$>HL$xu0j(5uw29hol=e_=X% zZ(vE1jN#+$>i}`s`*6MUlwt$o1Ta=xs)!N>j{(_jZBNNb-Tgk-sp?^J7!~sl(CNEd-dTUROF49>S#n%`!bNBo?me#3CX$`0bTQqU*ZU3O`2hv_ zwXOu1Z%~p2^Nj1xQdYD-GPUM>OjuKmXq@+tNrPwMzPjWt@FRPU;P1NCSNGcYF|}ml zYqjh%e%;!&=b}q2hTVAT!$GV+nQMr0G^wc6hc~BfE&_uH+A91xRMX}>NieP_ulL|I zH$Eb5pX$hX?4QvSZ}pD>Em2!mjmpGi_jJqCBD2Rz;kOM%d2Fnz}2oE-Cy~t#3YRy>FXCsZpHo~REtArJZqJpfDb$Dsc61X z`}w@-N6PxtDl1ZND1f(=<74{7b1?iKYI`Z2=>HZ8V0kR>rDd_n^obK`i(EdZdF$-= zZ6LJEHCZSjVj>8a2Imu1{16V%=Y$D&QH3*1f#~4MetoEZ6j{uTO~?Ns^Z%p8&sq6< z*8{$}zGk;3lo7rGd#O{A2*hONv9i9nimOOx+)7OBlZrP zerTBVloqQr+Fn}KKK(+C9i`u70RNu*{Rs6B->^kVM&(i69~hiDcI_KkW1}`!lI-n$!$bTz3wzgdgOS0Ya@EtI=7Rczo3WV9+ZxP}&*n$kW)tiUgxxW7L)TNm@ zL5uaKVfM>g_5b#He!IpQ+)0cQ`^V{-|3<4ogOd4P22O})$axq0e!RD_iO{P#oH0hd zh8|xpjc=6q2Zxa`>U4kNpSFw{(#75y#)*i%ZC2R;1e*r=&Q@~f%@rlTAi-DOCU9@f zJW2`U<6gYOFQ!5tP6!PtHp8AiRIRn>PI>;(VtV9@kec&rZO36RVXoqcXCwZpil(D| zzu>`4gN`PvD2V7;egOqERsZ4GX)e-7XTEdNtgTYSia9hQlMRRA$NgP-m8q{aeWdKc zqdJJgS9jYlnrp%pgJ?K{SEQi4R(sc1Kd;N;yavMx_=ISp6{L`TcNRb9KO`9QSF;N8 z`ed&(e=_WR*?vX-6H)WH`~QVSn5=_DSlfh&rPOJMpHlzVNKZWq46BxGZFA)V$R1Bk z7t$N8(>p%#o%xsTj?N2j(qI|S?5|pOoli5G#l=M0>v*qWnfC4-Ja8n6N*dEC9P-P$ z-!BM6V8C`6DN)j$iD7ncH8;*YGz1`=9Os(sHPTRc?x7d7tJ`#(l&& z`AjAlQa4XD@lzW6(B(rC&z|?4&t5CiYOcSUlHS-t*HPnOar90Ml0ie|TO`|+pH$BU za)jxz3o^pMeQ%v_?+b23S*%Qp5?h7m?UQt1Aq!SkYrYoPkhO+Boe(1Q_AF#;R+>~u zSt)I-AdVY)H_NnoUyV;Z<|0>sb`2n9nYn^S1fEu6ln$q{xythw# z^u1v>7)mPQH0P)tzeJsWa2A+-&X=2lli?oBCM|wObD;ts{5nEW?5uMnd#?f=U%{>LT(9c0BvNTebNA`y2UIFF3 zPnR$7F``mIFR>>v_-!__>o|$emv&wP*d*HedrM!QZbtl>hDQs0T22@bD>-|6nnX`|AkhtZKn8%{&H=TiNEMs+3Guk zH#8XDER)A3P_#OYvN}u?Yvc#dE#mHFg@x!GUcLxUTE458C5olPN1x&t4;L0i^|zIp zJPtg4%qM<`^|r*960t4*x4o6DfUPglhI&7)`Dwlw@=ecx(pKto1JzO|@wt|CRl-#q z{L>1a$DFMcS1*g=o${jU7E~$^7BC~2NxD`WFE@ETFrOl8r1gAL+!!G!FQ<94jY%sK zcakNTASCeX&2Yk)NT%D9L=UVA~43`RyA@#_+mHjkYJ)8RBTuvympZ^ z%EAoMgZ!QzMmGNzNnPX7eRfgncMi#3uk3cnVEZ_tW6l>#G45M-wU-$MIF}@_{aeI5 zCxMa93}I*aM%KVPKmPSdyn{RO+}$rk5?Xq}V*{ENH|X zEEu{GEaZ1XMa}(M2#FMe~l}jgk5( zHn=YHb(1(2=x8q+D^|s~BoK6Xn}rGM_oIPq@hqaB)!0UEpne{#*HwVKcNP=NWQ@Zc z1FH6)X)&Zh_FrNQx2H@vNVvIwf}wj zpKze7LA81S5u-z#)X@{Z(co2J^mZ2@hrsp#C^_qZvR9Yiew3LTKmJj^50y%JJ&9akGY?u5 z1hL24vdjBmvqbP5DrrYuPwWG|()|g!Qk5N!bw>?0fEcd7ZmezDz`ce`-e;QNJ}s(_ z0jnk@6ow;}Fo;w%N-WA?$k)9C5YfhZ`DS&}jwg)Hz%2s`^WBng_iD(c@5+AcDCI@z z`Q7>b34`RAVZzUAa^Endj_b}PKZS;D+{LsV(Z|=+Q*ivQ?!;Jfe(N~m{&apVQ4h9c zXIuL>2W?aYjEM~9cxys?@`r2~|C5VdF6U>j?q@wXVznDNqHlu4=5$+GOs?^N@gj_= zuARhKOLj!ujRf`I(7~w+Ir>4uFtBs+>F0kZ4V*WM6lT_tR?$^x(l zc@ZG4bgu?-yc#u;sa6k`TpytAJ_j`@_oWBq9Mu%&3=E#spw}z|i`&oVJWl~2$N+u7 z#s1aVB`}DbkSgt0YZeD=V_t9rs4ff5!anT)45?kmm>1A+>=iv=pleaj=Uu@h=lp=Q%2U&n9X({>gH z&yx!M(a8CEGAq`ih>S;g5V=KU&Z4XV^OlNj(RxO+7f$DClqe>hRa}-oi3gpQ`*YW>NTM zP#V(|p}A}ri(mq5rFBhX)%guLU@NVq^f?UKS025LFVO_FU`uJt|B3 z4jkR!4zO)-7bw9x>lr(TNT5cw=YX~9H*vz609aJN-`YbUNC37+`kn#km#=jIzs@W` zbGri&6iYVwn_0yOUp#b#$`Q+4I92zHBWm)s^J;+s<&JcKLy0Jpa~{CB&7wEA(&p7pGF;kb@2iZ7UDRHsi)G!A0L*gG5#F9*tm6%fIL|)yya&n~sYD=(mtr`} z`Ymw$_8*|nk3=x(f}3$?y19!bo8w&xBap4$q8AWF%*Ygm{`Q0)t^kj+Ct*e>USVx(!GoqtBO! z12$$_ID)AcRl_fUb=eUGVcH6(zNH@Yz!x)#|D@u z3>1WkgXj38F_8&^v3F@D)~$E6&xhP-)egV$d@DO@h|bOzbmtLqSU1?;C4tdR3F}fR z8D{%o8N+3#AtZ1eGcue)D%!VeZ@s+dH=y3-+oFkJ7{`oCjjUSa$Mwj0PW6XhyFCNU4R{uKEOzH9y?r^ z8LNmg!+VRX8mJ8_O{|tQi^qVEypU_()kNn;#r?dN*0H>i!6X|exoYdH#5bkRm`QmGff?02-r5NTUZTH!jGIIk7<*e%65RR6%IVy zb*IhS)9P-AV}?)E5m_Sri@Jgijn+4(rS8stPmU4>HHlNmEm3WD@)Dm*&75}VcR37> zZE7-Gr*R<`sTR^B`R>eQJss@^jBGRS?gH=OfCZ8s8hZWZgYHLueL7;#G=ljw@TE?c8V_IQPT4mO{2jXfKpn9#4PmNauVfHa%~I1qjU z`{0?5fyU)bal;-(^V9e2UT=>Y701QFIWZy!jz{uA0DtW)y7PVYzPTntRsx_w@$ohB zJYN8iR!eaE>-`B_gB!2wTwj}ED+XckZ3*a_8n?etP10d+6RF^8@!nr<Km_d*>BDOARs^nbm}}Fb z^dnoxx8*WRHX?`(J>7HzyUFTWkTojoY!sZRS&eJo zq!pj*aU4aI)hkM)MAs|`?h86Cxj!L25YdLFV2r;&(*w)`ONEU*4!St%(16QbK-_ib z01^m6e}k`fgXP+GQRDcZr21A9*FIqf02}G%>BA&*Gp+kIz`VbQ0;q*Y=9l&WE`W*| zz{1my#{rAY_dSJyCA2vbz&Lrwh`nNosb4b^zx!HZTU27^Ewt*K6&6*X@o8LxG8bug zaG8N68)b!b;dFd+i!N`d{->2pJp;=SOj(|?DFJ9Y~45t9<&WQzjbJW zZ{CVx`eGe|%6WtK1#U4I{iTxQk6c>zDZET>OHzh(=;!)~_2(Axg*L}pm8|PYEDW}U zx+}PveW70?qFwt38R5z~QpE?jl*z1q%`)1m(%jgNU5Uu3U%n+=)(AUSHD&o469_{y zoVD|wp%4t9m8FpeB}>T(FcYVo!OH8e)@bCF+?+{3A2TOZpv%@@^M4( z>BUWaijy;7o-#FHOh->_h29B<=>QNkA7&EW!iNHM3dS1%=$0Z(6dy23`9lGtr=4Ay zu9TTx?zX-~`T9O_3`_?(j9ap%N$S%VS#pS8tBP8UL%-Z!aKp@S!#BrqCMGsl41Kby{^RsDpZf>BzZ%x=%!<9D2|-;b372!7$tpiT>^8yvEs$waHAmX?xMc(BYerxpOz(dR2)hmnpb|4 zZ3fzzfCg{f3xqSI-LN8h0~#NBir?YW&OB2~*@9I6K=cN3uwywVc^#5@SJ7+-kg51O z9mMtecFNMc)R}6S?2}w(^2bae7)Lo}*-3)dGw}^*Rj825>J@8C zqu~Pl46<$j}%FlDOiD&M?fQ;p!`N6$@`TsJBQ|ubu7V%yOPuwU7t(*NELGRDg zy>-J5p}KjCdS07KsNauWMw3~N4HENHej)?P;-v*L;YWat&}56>q6-(k(SOHn@imgJ z+2@%H$+ms*s#kfZ*kHfc99UsneYX0C=29c^IRSOi%`ENT5(b`=pJesAz89<_Gtw-0 zfK?i>{d7}>FY{V0U$p1ec_JM~`Nk7I^e4i@c4AtM__$yD;*rM-7AzM2ENst*!U;7? zlp};2{9`+bczZ$B4e!8^n*4A7M&BB$Fmd~~W&I{DA$5$2W%qQlDCAkg)_fb?#R*g9 z`I9mGBcn_c97A2-!MNX=KmyFhF4m-?)?_nOHY(a&30|c32W7RK>$;HeG!^e%(7F;; zxb}Lj`)jPHt&8DUzFqQ;LfneI`M;+8ah4aBK6k8zfZjx3@mknt3ZQhO7=jUJ6hF)F z-p(a<@{o|Lbo|#BlUCj0j0moAFE+-L-4F%lHfnv>rn2)KcL9BU+yYKmQ$=j*-D= zVu@?Bv8}UI1HXJwU0twz=(n2cO7%h`WIcK)c}^&yDp5u zyHg5K*j@5s&o-A;2ERj1QJa3cYV?3X5=RJbG%4o%z1;WoajZJIQCoCu^EBUZ!)4F0 zJMUhnnmyFYprB!uPFZ`9dx zrk?GERg(!rsUa zq&5h%zZ*<4w ze5X1t?h3%7ExoNiU;41g4J~;y(2AKDZnIzNf_=t&AqXy@+^uvG^P?E%QdY4m~) zJ~tjZM~!!lxRbOW>oxG%XmBB_>#JWTS3V4MTQw+2(8W)MGp_>aVZPFKJI=C3Zm`z*t62HGr0%Un0iW#s=ZCqy=1^HWJLUpDYFw zd(Q!oG=#lX=5Yw-yg06s%;6G1)#eV+kAzGFC432-0WNE<>h|2H-g`=|6Sc%df z`C!*vl@cp@{Uj7b(KJo~W7*jwvNevf3tuj8@N63n3Fb8F5#8tbBd%asjP*xp*ClO_ zw(A`RkS5~AIoBJ+B9c+Pw~e{?jR(%<0R!R9_03UjLwjwuXldJ5d*QLnP-w$ZI3Ru} z=@tmxNU6*4@3IftOvp~)nG;f&WmgX(0S2vM^hz8eA|C=L7)HV8ze`qmFoD*HrlErUim{?sitXfUDpr4%a^D)M=QyqM=s zADyKaAJ^^*$ItOu62Gmhhzq=#tS|jSacbAD0P^=mBA96^`;UklM_IQY?23gVFjIGj zYXkk1GD|)!_jo_yoEDfm<;Wn*aETd;*H(y@&R{X=(WIOrEWFR$hSd>&iQQ}Fzk0>bgl97 z>~M3po!0TNEdVG8gI3@%Z_u@afDtA^v@t#&PHG&T*#L?xfw=lu0%2UK#h&NbP8JS) z{o2bLkVKQ@jys0uJSLH9uP>qa=S&XCPG&Erw=_70b(JJK6GWymM2W5@>!F^yv~oJi zk+Qs)vu^NEg$Is5+Fm#)bOU9O-zFuT6y`l7{k7bA&H0k5P_|nU*J-B(=e>Y|hf&Ap#sU*i)A(+#Wnm5}~+WL)7{K$+R!6Ac4gd&6kMTzUKj}jY zB1VI&;V5n({sm$Yeh{`(KVrBgFP`o!^!>y0F3>%=NSyeSx4dx==)gSmMM?+5A_MK3 z#cFx2ErL~!3aeJ9qnguag|RytN(L_mTv|ArVe_(ZwesYO`p|bF?f3p@IfN8;_Kvu! z>_z&{w_P{jG^HhFD-k(|)86H+kWfD=Fnlt=Q)<{~XDpG05;1B+&GJ9OuETa}vZ12K zmZJl++qsaY)m&TjZ!*;kD|4&$`bvjXauyr>fN&1mLlwl5UdeFpN#~hFvGk+vqYPHR zQ2sq;k|06sk|R%`PLXF_e9omUSR!>a&W)w!e-MtUilTv)Is!W*k(v7;@CAS$OO(;f zxs7}-flK=Wi#Wag(a+7iO!pJ)J)rs4j(Ft7L$p6G^Q69!8lQ8tEwF8}-V5pLcapFK zEI?Y$@I82XJ!PJ73XCyt%UqtTUmvPF41a#Qi!K%byncy}$m}qN1ENqp6Uc|3l!%1C zH<1JuBJ=FqBo(oBa>vE(XB{8k-eC%akjV37AEVb);x<-lC zZ{M)nu=C0sWc+iLNrJSnOB!3-!(IsFSK4rWuB7f5SMnu!VqgT%U3Owzx(Yf6ceY%O zevswWHMEhbW8YEE93%es#cR0c^j7JucJ*MyN7H;K!l$+q3ej=HA8}p#QFfq(CmyyB zXj73+3@t)yZV77-+n<$;am!!!b<;~XmPG{?uex_*#y7Ofe$HvhZ+<&)VMq%iVhI4&E;xJ+SYhV+bw4y1G1DBK-ZRa9&3jBt6;>EiZs*tVypLs2 z3ae)SEg_g~HvG*BMecdS)3z$vAc$t})+==b>+kjADN5AfiM9U}^8j~oN%zD&inyMs zR77Xv7A7LOJ*-q$r1TaH5;Z4dx$iJ&Sk(`o7HlUH|DUSmMwo1k$nA--PR3C4pF>lq zycp3^iXAK!V@78}yY*{z-**?!Zb@o_Qqbc7VJ)L2TQ1lw^$oYhuoVIjga(#yZav z{UsZGf6rt7xsQEyu-Ky7q{b)5?AXovddbv;{W3{wWuuBOJlgDE)XUIbH^|~%peqBz zvwe%ZjL$V))Fh&<7(WXv&9sC2hA@i@Xlq9e_t zU-V$wH!-kEAV~q)CD!_Jrrn-K^6tM8WM~}5IDWbth8>x(G;JDrB!!;A>Swh21!-~w zl0CD7hdey6ORLfi17RJYUp0)km`mhv;?zQzsR8hK$L9gvG=>{9KcCh~!KAwgrEDKE zR4wpQSMma8=___gM%zn9_WdAIh%u6Aiz@8P!cvPM(4H^MFLhbSYHi?ZXRdSeOK6LU z_5YuG3rP?!8LR)S2IAJ3b!|~I8D1k$}dAaEF*PQJpoiQe? z9g@ZGU(&~;jM^LtFq>qqM3Bpu(-a?wk?(YY*Sw0A9hYZSC_Qzo+py>j6+Ppb*3REDUbeEj5s0)1>PN1ubX?%9k4od1Q_<OGdcJ$Uz0DZq$V8$1WsOE~g;7C&qJ<<6J}>UCWBQtbF>by@NX|2u1&_I#%UsF@ zNYE`9qd&!jSn}nbcU%YEp+xlhrlnWavo-B$Nm?6eHe|dm`dq`MD)1OLUNy|dP`5hY z?2wjDi+ajHo4VzE(%#5F_gqG6nz?SUl$b7i#91Qblih!kF{j2XQb2mXogf$KL&VF~ zj>8&A?G&yttS5GigI{(@`kVB$P?cEwS<+mxx~lkvK>KU)Ay5k z3RF&iXI#MZf%u;VX?f3}Qj?QzJQ;1ol!j`z z4*PDU_D34|w3cs|9qJ^M`fXIxk6R=1A5q4>hbX%zVf0H(dbY5t7^d#VDoO!Dr*gy6 zHzQD9JPSb_C##NG^p@QO7t+&m4%n_ZDEh5}}Dw%qrK;l>>${fGehvy@Y<-BwKnf;f$SjAlo zJz~P9L)>wyUl+)9?ymPurQ~K@D}ScE!u^>H=Kldzf-U-3i&5)yHD`}Mo_Qa+>$6}6 z+5pkZs`rM{vjLBknt18!V4-Pcpxjs^_kRx(_+kz~UziRVty&UU>e~E_bGLD8lUjn9 zyki@0sn;{Q7q#_Tg3+oOqM7lY>Fm`y<^JB;%}w4upDklcyCq-CXFK(w&Iy{o<4VBp zu?M>5&|xTf5f4r(WZlH7ne3b%OovKtMcG-N4swOVN=RWLPE?k)gD*#i>*ZjyII7yA z-nmvh&XShqk_nx}5Y>(YEeK>n^SR8~@A6)bDu(E{83GWWv%Cz|!28STO0jupm2=~A zFwuEnu=5kKW9uQXdR&;EzRx};`KjKei@^@?$ zG;}YplBw!0hn%Ts2-tk1l<;-QM1+DB6CX5>I*eeXm%E_2(ASi5r83SdwoeRV(%{bf zFq104?rPhYCDw<-By*(x`}2%H{$=G3?7J%ghiBS0+Q^5dV)*Y(l9VlTVN>v#N}6V% z-l7Z}V)91$XwmnklLdzFrfdq8r`Rmp+E$3GqOwb?wMabjjB^&WhmUHCd`OyjW@>iO z&8+^*2-iNw=y>R(3NCO2EjY`kMk6IVEBbmODL*tz-HyKV5ABGr?AWav(MrtEFu(%R zo@LJ8f*WH3{X+YFX9J+y!;N>#a;!2}CvVAD{ti5T*Wvi-uuJB743n(F)Q;$MH802YX0%1W!3UBQNtX|l$zM&mj0fLy#6 z?bc>n;_8nwg-e=oU+Lauc~I{p*z_r&e zuuNsbveY>{b5iTZ`Uj}i=c0YB&F*Kv3MW~&vHa=t-m8;y>L@W_@*Z$n^muY24_>9( zjDSs>ATtLW!N2kz0trkgW62c$9DhuGvcu%mE8iA*jB~!=7;RBiQZRLq&A~ENd1cfW zt6I~ZNf(jW)EIR1G{+pca8Go8dtDwgwdhmFk-mn4pEY9f6hNA35tHK-%Dkhz_H5x?*u@H?}p%V#UbkzbgcKRWs=mSPS> z@l|~a1cQoptcJHd8}y#rBq0TE88?}$;H`8b@KR$pwYH!=wElZ1&yv9M+0@Zc1q`vC?5`vyw8Ll1^BuLce2v$&ig&@_Y>*abh!wP*ze(l2hIN+Zxpr)EA#XrsuOCzRapc$ zH9Wry3v{JxZ*fa1B8zGdX*@Wq>P$Q}a3#n{wpp7Iv-#sLf`-@wk^NN9(m=L{WYX80 z^2+3y=O^$l+txgP5=IvOcNn~fo~23=NHKVV06kDC{+fU`F$C~uo$xdLWX4#h ziuKdMISivktTnb>657@USjfOjQ?0EwHNwtp8QZ7a4I<89vHiY@58&2_gkOEL-x)jl zSyl|Irl>h_S*RTm#UiG9EWKUnbKAgpJwCrdH+nOELDxJ$Yq~?2O|;*SC#GI5?YzvD z;H7(pPeh(mQp_(CDSM2!NXH-gzTn?lfNVLoC$nM%QUIRa+YPSjc&+wZtthcTxmF05 zMM6z(dpbX*+Sev2-p&v8n1M*r0eK6=HYwn+F3=>hTJ@g!C*?=su4z>G?Mf!Sv0iF%dOQCW&VWw``%1b< z!w5dFhoR~lMAn4MKD2l?^p;wssU-3CwX7hWOoGAB>OyMy31`@g!zQbLG)p=L?$3br z+L14m8%)Yek+UiboumEsuj{+M|FpKd|EnmP16b`9<#`N{@}Q9JpKt*K({$3%7G8>f z?;DxIpPy~q1}<~XwsKS1SrBb1uI{vFu~!vQYtLtAWK9tahxAoTdrD}mU*R|6f@*Or zW8wn~a>t6W%`L<2LiTO#$?CEy-W^x-$UZ@WbPlTy(bz)KNBdg!4CX%Dv~0=(8P;SY zVXI|!%MJhN{;Bt(RTXHYC1H&Bb2;9{&K?vXm>|gR&#iMadEQ1agMbO5f&@fN7Y=GU z`P67lXA*4r#wZ|2@zPGs1jzKMT14UIop(|nm2QXoYN!;i0D>I1#dJ2IMITyLaMu8R}C zEs#?h&pqrJj?*6Q!4?Qk#XQ%m$(G6HW$UxTw{P)I4t@Tg<@;s)m|)7F+ z_Q_fuRi)IDqWihYCyZzYTiR?yJlfuuo>l)dyt+fdQ>u4o`--Gp?0A;Za(Js?BW?>V+aM9##We9y z8PHdBF7wM~G+{eQFHsmhpHXmAu3xzNTL0_Ezy9<%hP@XDdOsub*N??Y_M#E)Xx7 zf0P9+nD^hpRI$Sq=@(){TK*%FAV9nI>P8AVER5J5I+7A&^TA$zAnegnW2aF2{1@5A zqZq%Hxt`#Ecw^N33>32p!bBSOTCds&SE-8)%XUrAz6$<7n~s5{pL1Zy`2qK*u*QLR z{HBKcjyFFf&3ebPFK$1HuwHlZA&+U&BS(mr?Fx!)@@0xRXoAXzhsZ?NG{ybaP*vt7 zGacV?(5KTpdh1t=F%aE>u?SU7DZJ)8 zC`WN@?g4-1ib}h0)=jCjtGkc0o?)Fx387M9SFtd-J`ep!h_$7NfQhcy#(%(mw1x*` z6q{%(MO6wE({J9Rcwb@r`a$~iHT$+Fp{ARC#snpFq13|v?46hWTQBS5n??*)Z8hKd zCOu6V45n&BrRE3xEvB3p^B=j$;3LX*xPXcT6BfvS^BMH$2HJFOFk$j8mr+1^!+6T@ zkP;wc$kYD}%zb7UZF2p)n5cPuI`PT-1iY9~=hC*cekc)QwFrCLb@I4I{l^e7Ed(&J zC{g%#N0;a)9U`&Zi5=p04BPoOQ+J<+NxKlrjHPOY**yljf_umEQ{&0omlzE2g3-3E zR{beOBe-tef6`^bYQqNle8@O`{h8(Yr!{Sdqf0dIqc`;9m1KBME{YgahNZWZU=Y74 zn6nFGsV%LNe=_gPIR8}e&8pfP$MmMfD3*et8J6WNfF-Y8|OZ(;AUA&xf=Bj2A5uhai^u z+dvEyO%#7Z8N%|>FRVO}C21<_d{exO09`rp~oFv)f_Xe_srL&8O! zS-%A>6+~RC2NF|Q{zGJA5~6@3vz=*C|LvSBrQK6|f^FWH1eSU((g!;0srg`iI9~P5 zHB9<`xU3%DUiC!V)qbNo=g@>km|LPWAD?c!-e;Nrj+CDTyQb)eu+3Cp<-MQB8liv# zme{zP$brFIqdug*=xg{i@5g<&hHVaaNh;Rk2uXv4FyD{^?9P}nU&4l@l5;bj*pF7; z8_7xrpSin_$+zNg-I6i-Z(5yXyV&eHnT-hpW($)EW`*}+J~aDziVVi9(Z5a^xZhNG zSZ=d9t%Qc{Twbmo_X48_nKoZhD2u0PF6i8=5q;w)Gz-l!qde~)*#F0s(<^4Yt+C^* z71641IwwBg8CpR*UAf6q$KwZ;M6~Q{0LPbQh;`woV+1y|2TknP^uP@`HMPH5iE|HZ zA(uP<1M-jr6_LTF!zPNX&%1q;6ZFpoo>6a28#5--n;9KJ1eV1q{{gSEm}ERSqFp(r z0X^I1o>jLL%(bE(9|AKzFQNbpT6`X1w` zPoVAw!8*V2eKaJNi-Y>ApV^-7#Um4*bQ>j~af#Uq;lCwrv>2p*i=h~LvN1$`bq8>a zqHA3F!0)I8H5K=`sLem9Q>-t7jbzRwy%(<}&U3@)8$vXu7Om$6tbwlPNmxHi z4?s#0(t`c)OtmoaY3=EYGVw{Y?fLzProe>Kdp4N3ujrfY(}eG{;GGZV+SQc}u$!;5 z&J_4QoA*xe`uf3{Ti$P6oXYx<(UxyJG62+=yy8#ZQ(h}6QN_xIHK~3#G%8RUQ_>ij zB1DrB{RqCdLYiw~rfai9P5h-$wAV^}fSs?{&TI6`@2bGx=itW}lv|T4gQ8E$b(qg6p=?2T=1qvL#ts~ECcqdaS9y6k zC%|L22V=d>zM~_MQRS?jrs}G&1^pupjQgoXPI<<$nbt#gsrJ(0?U8#j1@~> z5u@o?R~plfSSXR}5TBCwus}!GpiO1ku+9_?#U~}c4Un(cnvDTu2`d1rr7&Yh&@1$4 zLeGQhUi+6<7pD$rUSpZ~D}W2p9ldqVrI=pP2;i@~hNBTkb>zDG%@bYzlefIbtUp9; z+WY8q)D1H{6TwZ;i7nq0(49JF-@4zzF;?|JVo%w=4zKFW zk|#q!IhFV$^&I5Wizm70OG%Fve305oL)^^3;Z;unF*wsd<(auFg$VvH)_uYXa~0oZ zJ0vR!ZLU=0w5lBpwKXk293lQtn253dwrqMj3LI6)-2$=T>2oQStay!DnzW;FSCtQ? z-MS4Cq3;QMCWoyUf#I)I=&OB8MiE;X|FV-c?vPlh@wCs;5VE=t5U7nb0P7oF04m5r zpOtq?vuOMknn&_GFv#(Jz<2H8FWM`6B(V3s7+~X)UKg2qL3W(fbkPO?JY-yfiJdTD z(yGg4b*(oA4s8B>R1M&4excmLVXw#*e}vurA%tpB9-H9)ZoR5?(q^aS#wD_$5Q*-9 zn*g=bC=zghL4Rvgi>0XZbM)Y7xU2x?tZ})~1}sc+%M<>oM%RGg24))VW=#^?=87mM zyiMAU{>*ZJvLTQ+CeGD@{$gVm$7Jhcu8NhG8K9ks9Dxfiv`scXMK)<23aLf%9pXuO z%s>_kxU5CP?4F>00?CYND{%4t{jDv*hp69aE85V(Tpe{{-yYFmBZyKJ>!XV@o%8ML zut7g}j+PkVvhHg1I2G|PY$%gHYhMnRo^zektWdDDuCi& zs^%BpPaQq7bhkgmywo5GqQaa#DWgq4h7vh%I6F+7N+~+mKtL}II0W0RTn3-C(w#wx z&i#{atlbjrLRL+R<@H!;EdU2~D+(i4z@E{jMtX^+EteWcr`^iEH;=&nYhJ~vu9B1+ zx{MaAl+^>4fi`&6S9Z+v6K##hmXmCGtCjmF+K4NZ@%{5B*w7HhOyl@Qq(9@@LKEcN zeb=Nncgw$^ahGi4fTo<$TOU0y1&5L~k>6^yXjvfc5n%jBeELxb=pGj0>yL50S63wu zzD6!v1gHu}>ORIet~2twVXk`}04zzZMe}%H$x%zE`T%_0wdi@)$Z9?hazhaN4QFLfM;8teiq zbdv{#+y(2?iDTcO$7U~SFZOKB#Bca7;hXKnhIwi;(UCF_#Q_)MM>DYT2%vFy&%<6(8$xWWMD4&xBr1*kio>?;E!W6gx1cp zVw&=3uIXrhAe{bMP*lS3Ip1nF*UJ83rWNss$$c^eKIDWR9{Q+e$(wbQxo|<+##e~# z$kq5sSgdb$QEcj0wL8S=r}6DFYtG;T#)b;A{u{%(9iDuUZ_((tBOt>}8?Xddi*$5? zxzRNrxOlW+E_XNg{a|#(K4^(45K(P`dz9Q~<&jb0#RG;YsJ_CNXqTt%iQ3fX6()B7 z({2t1FiCz4=g$8}*L8qXz5oAX6f%xT$gV3DSqa&4OUg`kRzmjPn~>eGvLz#Xuk2a& zUdNWb9rJMbf4-Xc{_g*IJf6et9_RZR@BMnc;3?@^>t8J^<3F?(d`dS<7sqZPo;;3L zJ-bW5P1`$sq%4pkCdlC;*~~5WUN`y!eED2&npg9MmPkSy4k|>5vQSHS-eM}zZUL*G z&UmD^qLSFBo#Q>XlM?><7%PEEK)&0@51X#Vi)>s{bgPcsb>*j36i^l;aFH3zJaJ3G@-oI`oieRy)U5_Kn2u1ig3Y zm?{&$7E8GKRqJQfgJ3kr&hW0hlOO*Dl0)MThEwrK`kt%(XYWyz`Zsa)2QrB)8Djf3tS zaO6akM_=XCH zaqYRY+F6}7NKD4Yu}Gv1Ktqw?&yaKZkx&uWNz4t_7uv_ zo+}e`2QQ&8;sQzTGUsvd);Y4_3$saUm!m%%h4O7Rb;;1$2dTV3KN&Dn;cXiOOrYam ziEJQFbV~>(YF8D_XOkYyV;1M;UII{_j=qA_3lS_T)6tEo9zaMgHO{fcjfBE%Tg_oM zYz*$YfprR$24k8oYX$M>OoUaZAd@u`P;Gi00yW`pbkxp(pWPNP6~8@%H05f~Iv?+r zRF^@l-)mryHqnigo`8GYUdPP$$Qz#=g>zk&+GV?Y4S>Jw5N6eto%3|tCBg4Vmm_|T z!F+F`XkT{6zb-(WOhB_|9A$f*T=P?4{!&X%>Hza+W6a$Vu8SdyvQ#^vU7INUw?xmZ zFBS034LcHx*F5c&Gkrft=VqeTm$IZ4Z%9>u-&zoVB-QmM#3fZ|)J}eBZYb(?BzfEU zIqc}5E3y3(S5ZIP=J$PJ9Xuv6R_1YYXGa3%Rn^BV3ct7AOh{^Ow_Q8yM3g^&x#@bX zk3~x)D;t~ML<*6~X_zYV4%n_nXz#MPG=J55=D5;FVlrW+5 zls7qw=j?R5OClCMQLy=nSWW$T=bmc>T~{VeywZ#SdSh1?=!ZJyhzn}~&U7axQC=Di z$Hn9|z}Sg2yB5Qt<5N;g$9t>C!2Ic_S3k|#sc{R$#o%M`CNbKI4Cdjyj~(K3Ab`O? z&n?Ycx?86gd-ytqG`{}|Eq+==utnZc_F|V3LjPEi_v}ZV%q41^NZ-T<;`2femM5F< zBze&-O-0(fbd(qTg?bxCzJxTsb2a?yLj+&TP(w>z#ZPNGc|;oLHIm|a`|KcPVCu(= zVuNU}k<%fO6xMBI zq3eWLg^_0K%y-TE)}Ky6jejg%-bovva5hM{cFflScfUGJnt26D2H&oSQwh_TE!I?~P@>dLlVy#s#m>i5cy z#7z&xg*4XbrD*25J7(zv#a&jeVywt(PYUVpC@d<&8Ao0TP|fdb_c z?t{t;G}ECfo64?|N-5557mXhO@_rt*^zKFyPJ&-G2D2Vz(V-=ZR{OIDEmd4vdzwc) zZr;@x=L~Q=k}5K3&0euT=HP66;BI96F<=@!r(YzeTY1lBcBeMnJN7QOM-KI|wq46? z#B-|W-zLx40aav|UX=npU1e5zWNYDu-*N*L%mM=ik|N0kuJJf=nwTTvjmd_l_i!9R z-W3d3%ofGQ^>Nu|0RmbzmR!PzBhULwy6;n!q4y>x-0o6H2fpa#qmqsn8Uw@ZrsLP& z*hu(`&|v495Z4K_vV=hRYpGmQ&ZklOlY%3j;IHThX7j z@oas-@a&@4Un;XWUs;U^I9$N*t_D~rb4#X%w}LJLWk*C0V0sYfrVO!2(Rs@^WEhNy zuNGa!7*HGlicW$<&&ow%EYLtkVxfgGT7;t-6uls z-oe}tdM1`oxG6w`e5`9&Il#)?!ENFy84(8mp2%Cr!h(%h|ngLUtp=Rf%u3|N+UAz)KAooOQ#YyEb*ZE<*j_zIce)K|7 zl}4*YmRYl|&2@{=BHLnCDz^OVEhRR(jq*#^z^hKf&vb%u{eg*78cg}|P0QD)KFbC! zq73*I>BjeYKr(W|E%MEmdq0%J7}U&7L3Ojlg8h@zQ}xXF=63Bj>d!NUlFv^Hy~xtx zi_R&<-4>&_jg#n39!k(mRF4cE{Coh)C`7!bpUGS2#@Rt=4gZW=J7o?e%z8gpmRu-QKA%U5^wQP zO%T>6gh1DLwt^<8J) zl^H2Yx{mWhb+V})1MR$ws9Ij#bk_bjoub`h!nwi30vl$~c96Y^ah!5JBdc;`#GbX~ z?TeetIe3C2xvT6_>P1{nqW|IBUx^^FH%604K}&|}<*AQ_C=s-jkE6+W&N&q12D0ej zKT}y9ax@lD^f_qN6LA9uWrSX2UT_pwU!ea@PSA1jnBGb$S#Qx;r}m2pK3fh`nwqqv zZ;=PGz9f!KVg^WMp+^zP`my(B#Xe^i_TQ5Bo4CsQd~plW~1kfX=4z#b@KDRM4Xiq54* zBo-cU#I}ord0C$@*q&R9Hg&3C2|Mk>K>j`s8j$#GtsuA3Gh2(j9RJbu`CF?CcT&Tf z8ir#s(5Lx5s=SL?YH;fXkEwb6YaYTZNk6Oy%H!e8Eo}L^4iCpDZQEi89wO6Fl4oqi&~RCtYumIu7dN z&jKZsxwA)N=Y~_)6XFXzfBfCAgYPdwVA3tSS7m_tH>Kn`wnye%i_=%q)vz@S>$Slg zn#dzs6}Lt6^zPh|JwY+K&rrscE)?qPN_Ng@p4^1rC)^LxyR++!$_~0_l%3ElVCBkC zFs)A2o3`B?M@7N*R!fYk^*TU7YqSnKg5r(M++w6SC9^*ATjn!;MT_qXeid)h`FW>= zRB=}E=u3U!r!$W|;WLSU@h=th6cYjesx5e(yE0Vb6Hnn{a;!#6g1Yw_>E!5R+v?{2 zoi5r8%GAG^1LbWc$d}~}PmEQ5jn+YiL6ni5+dp7PG?ryY?MF*yjnhwoR87H79t$IqUYv1zxs-|NrQERmS+kgZb+vicw#{@-{^m-Q>}QfTvK*uj(aVijD(^dK;}RF1OkH6jgQQYN=b& z28TK}UbfU%#D5M#(qqK7KKP(UvfO8$rK&qJ(8$a9fL+?aMq3~}TUGziqx?KnxBFys za9{igkTlLDy=>IL*Ba$EnW>y3d`2^+{N_Gx^J7JF=5^+Q--G>exLW9|I5lxypMLV6 zfhruX^{gKEMa8MJ_ra*R22yrBe8C|13}wT)~L(a-gMc+9a%x%)K9+^=^5fUtM_a>cIWD@eX}O6{B~mN;I|# zgTu@A$d=hzm0$%#n6}Pgx1xY;pd@fBT9T)3#af*QSLfBv`KLdwZ8`(HqSWp5d%yc@ z_W?DTMn;AEJ$36QES09L@nXN|ySv&ZL*(OJ`6L#mFT5<66UB4+8p!@mxQ=N({-NQx z**?5asNC4rz9QiYUGRBs+LY+1-Wwx%hoTE#$Ww}hpNKxS0a2N|yI|bVer|8xato+| zV396e&+2G$`f8R=fHD{ydhomVKp>2fd3(iCXJ0(?yuZjL3=JH)xiCQ0cW`61Tve%m z=5Lo$wkVRvkxz8%k3aU|AR^V5@=6|lR#sn7lcejl7*#CR^%dIgwS72zIXY!4U?dNj zSc)5niE(tb$()pV2b<%ZEZ6M3s4r;xcp^3jo&NK zK`^ra-S|=bjmay1Ylzn->6$}uGB#Gzb*JvI>sV^oh;dFSG+(iV!Qq?a0>@qX$G_Jf zC{N*Uk1$2NNYt3%mJTGNjAxfXLVl9blEtpxO7M(k(K zA7If+1x3T77y`Jb+45V{-_0@>;JK^E)zi|HUdwW^%}R={p9!%U#h;Nx@^ZZ>EQM>u zE%6_q}+1iMP=CzN-%9P$j zhyKhL@p3{Xre!?EG{raFNKrYr4nK=;%%@%=8va5 zUzy1ptS^v4HaH}O|2clh#ovOtXx-Y1sGs@gY4n%^oo}~B>-t9tV{aO>0kVI%9PmfI z0(P)+suEQneQJtff_^L^BXL!J9_ySPfs=!dAU=i+ltMb@LlY@ zGzo|@Kj;>$KmaY}IGC#yZs-6e0Hf84hP<^%VKB)(uoe=l>Q9m!&|9SlkwDC`H+)SG zr2b}Mb&`mroui1BLhRFrD7ZkigI`}B<+v{gbzU%~WMrV_(s?TMhCj<=oOI37vNhg~{o4o0vUYR;bV zS@OT!@x-U1&RS1Y_xtGzz3Q%$Ii`$3Ek|su0gbo|u7cGU!H&A(ZgZQPPr3Vxw@AiM zE`ayJd=KZ6^tm+@WE=!Or3>-1Hr=@h=1N(xqBRvJT6j0Ri_?n{nZAdFA8p0~6w%|C zwf%LJK~%eO>8d2Wh1VxBzI2en2Lz(p6B!_z%j5p6s+5_du$>shdHrvh?!3Ac+tnZvwXv_1tdls~#5V=k-UAg5wsr^)ajZ<{7^sKgikyy?tKv93H$B3Y+_utbM@9rs86mR zNpl0lsl*9SlfrC#8*^us9=>Y{)#pg@lucjVB4m!%i#f>SYL)G|$`&MmJ@A<&xf<$< zfP5F9&&!+SV90;li+7iGm$F$Q2J8Cy`S9fBJ!R@Otf`n=6Kgn$gvujFu0tPam%r%G z48iNIg-t$X{xjVE-Ysy)eIfWMUHzdtgw(S>_^vRY&A>D+^R-t5g&R!nD zddW)8^&vli8pVbP`6Q;bcU47f=N5Li4mn-##b(>i$hKWdFClo8+4v2n>=S7WY{^yP zTykq}q1p$2v$z}BUEv?aF2{0q!HpF|q!C;uFZ6qJ??rJ2_Hu`><3q$`?zjRn?(=WD z?t~b8)6uy^LWOCQScU5&AD;Q_lK%Ea`bX_2e3Fzkh&!GK(qO&IW+G&|;P ze^V1#4W+@CpXNW8NW0|EwZ|W`oXckE6=(5n*xGpV8w9;CdC~nL(r+-2_5oE%HE@Di zu|fLW9|-<1BEPH&c;WGezGj|*A~VR>0N_&`Y`=)%u4W7Tfuh*= zbwM$@&p*-O?Vc2Ud+Lk`NHAF6>f{^JlqPg!I%dUL@X7|>ojmZeeBInu*7}xOjYT-c z%OzYWhc&$QMl?vH-d>gshq+ddE|~xQK3QDIvxpgyy>bZRa*Ph1*MYqrA@dB;#l`57 z@%BqALILP^RNDhz;RbWJ?~t^Sveu>J?(QjF2$@w=Cqv88{Bq3yctK23Q%QW5%C{rM zM1ZYxWY}KgeT9IUZ_s?$;_aMxRDJ%BLH+MG=QS+uAaH8xdoA%JNez{|d17qp*cul7R`xhXlVMk5W$qlI=u~|6F3^pgx(aJ1 z#BN~?x!BBZx}a|_iI0qwsg$yc$rCVYK2mndOfvK3fb4;jS${>^IiaAPt=)lh~pGWuUAlogw|9|2ZYY*7E zcjy~kV~(Z^hh0|{SsyhU8GQR4kV1O{GW#0`*c*AxYl?={&2%To$Gi2<+XC?TQy`y! z$Y4!=3T6W6>dNb^b9TOk1r%X8eRm2Trljsk;t?@|=0df4hq^N#@o*T=B2K!(9Wu#v z0q$k_opC%2beGM=@Mw10@NHK2st#zighTGE%8~OOY}$1@%k)1R!`}G3(Ck5sp0T)a z7y_=wnw?7iT3)*Azc0}XPgr&V@m@X(9iAJ-5gX_D03W zw=yud4%DiW+Lss7v0QBt&=>gLZktX@%+AYUij~z+X$U zzvG7Q-boU(qF7`dIo;Ozu1!pkAaIaDg$eQYA8jefBwWU{Pw9Ry17ibZFniGS0PXr7 zAnlofCTV!wj-B_ws6y#tZ|djNc-Ij~Vt9u8Ra=}O0#ax^zV(ch(~_i9a4$wI8Z>-R zrDWsEc~H-}>5@_4^44V_kmMfCW$-pf=j&TWQ$P*Zek){d49NGH0%Be3-V`Y{))TX5#r|}_m|Yc;W)IK?_GJOZaaSQC z{uTDUvQC!{!K01(HJ}t`PIvG{d{k|G>*8L3=Fzh{=UD3=i8b)@17Mf|EE{8BzGD~f zAg@6uL~w3*%udq|5FLf+p8$z6^HbIXKc+a|?ACy%pw(p>aI@IlT=le(50IyYwAB65c&S)y<02iw^n0$r$8@jKkybFt9G3x5fqf9qhp-&y#>$0zY?e8$I~ikOfMt1-_!SN-q=#FX6vuXSL^otsbuhQK zSA9|mNkmqn0LRa^v~#I@B@pXFtKsOkMRFY53JnPXo<~z(8e4O~b!Y>~x)R)|rdCTP z*1Ft$KyR#78{vAmItH2@z`V|k3k8qj->18wwSYV6tgv_X#S7q6pJAJe(Jj+Iq>|~bO^J|=z@kFw z@PIsx83fzDCTRi5Dok6(lVVz5JKF)h>)IU4~B#z$B9y>32F%X7^k^VL^b z-|s9}KbMyjZvzE*Kz(objMiDl2{ncfe#}+)mo0#P1k8hzd{zYH8R`z#HvD9q>(nj3 zGLEYd1>4~+OaQ*)xryApybzbR=)R%JWfGX)`e28!WIh*YXD(7TravKLngm)OdesY|H0AyC9I6@m3@ge&{hbuFP zDlmvL3O2{31Y#L_12tUMst(uQVbNne zg0L$97-1fq`An=t4o=&4i*{i77O0IsB$@QW$z^CGrh^3=+9%T3_ru%;iME#>e3>B$Q4p}*R1Rm%LZ@vFmj23%QA{;E z1b=x-HCrbJN!_At$&Jf&e){ewXyRu$4cWFDb`ZE6a~v%`tf4xYeBvvEJpRTcOu2q> zd$-*UYPVK;jme4A%mV6-ubVtrb=S?iKipfBGIM+v0oW)k&CedSY93GPfo7uy-4%$) z#*Vgu3I`e&I-Mmlt@S*!22lmWE!WHDY1et}!7Pc>d>up|#1E?iDuR1No(qHLm@{jZ zDg>rpl2+8yZ%uCR0jBIPV5Vt(OWNHW)}ZeLO-S}40B>96_YZsQp%0#wE6+ENAru$+UOq?PNxp=*BwX;gT*=60-807 z>!uftOQ(F+>gkSi4*|^?9gTp{n*hT24sT>Uq4~HN4a9hAzf3p2!DG<#*ZKply<4~B{*HckbpV>THweGt2SqR_kLW91`CZ49xm0;dF{xx)K^-;k33{TZ2p&Q%kZyqO~83E>K4r+)NL*3jwW#`uUsCP05a8;I+xF}s^m9BG*b~&qxy4DGUUR5 z;q#cZ`Ae7Bg%EK!@QkYTS|P&LA4GX5d|4ju8O=}uVw4493_caPXJh9Bz%euN_30nZ z-moX-ixP=XUtXCc;cMEg4?Q7xC(AO3eW+c^Ho~S%ngYVSa1_82h7sdk*-0258`KLeRupw?(IuYu}`C|3vS(9UN!6EHg5aP6HY_?9zg zCMtYo4k5wBaRc4a{ev%H24@~(PPSXgYc2;8QNz$oV%1S{gvJsuv>Yel!mj=lW?TG| z?pUK7&D1l%o3gFbzgWIYZFYnJe`pL+M~8GJ%9eAg$FqR0E>XNuoxay*Q=Qvki5r#O zalP`!;uJ)ET}?@BCfuNQ0`#Xt0{6(wPP_^rR^6Hnk_0bsmN|pY--?S+@*Ugg8TfMF zTV&tdTUttyZmEvTSZpk9Bf~I;*Vem(PS;vsSD~M-9~pu?HF=2n{Ix(y7R>G6yuO>hUTSXVREJ;g$Ac1 zY<64FER~lIJ@t{XyVw-aRh$u?{S1sY%Hpa0l=tye^YP*OdI#?e!TRejoD>saJ-1gUwvR~Ca0082G%wBqr%}AY2QSM;pkD%IZlYLp@v}f^vspdB2Lv~YLP_Ryvq!cu9 zVfb&m01N~L(}w4~S>ZOCFX#lhXVt_k>DT6By*}FprZR4xypD|DX^cirzVPK5U%}C@ zfS4CGAimC%rYfLPix323oscSl>3D_1u?I#Ai$=u|{{%xYr%`rUO7(C^$*MP;%zVZT z$OT(<3LR!Tfq2cOT9(M{asw55QeYo&TOL%^`Vrf9B?)d*#&4XtyA9|2mMF-}QAdmR zBBaLD4&lMy>Xk6(ZeRj|-y|j^jv_4G=gl|y?G+ErbU_0cZ(G_=^^Cv*+@af|?(kOG zGjz^!VEXP+m^KXy`BH{t5-UiRJW#lk0{sE+q%_JmE}@eIEu z-L0YRq(~(QQM-qd(@wGQ2M%iT&Uwy2@ovdhV`YTC( z@m(i7N}=dt+E+*}Z06ip_LNZ^!*JCHW6!aGX>jgkN)5qYAW#)xU7z=6Yj3JkB09t0HuI$o`3mb!t31i=l1l?*K``Dm9nkR)44w~0 zIi^Q}xZoLfoldSG?Jeo|`mHlInS_u9E*FQ*8-Cu5QUAC+{-Ey)*qbMq0!m_+hpdIm zL(5-aeIL;KhqXP8^gdw&nH#aaAT#Is)z9lyRbPM+SgLMA(VcA49W#H@1e0=O2lMgk z8awGq2RV5c`wyFVs=5&~jV9VPt;0fKShYNWt;+uHMbySdJ=}?Aq>fzSlrH^l5uOhv zUPUe;o5~>&s(+)+As57!JuS2f@nLdYVc0>hyGjnV|hh)(Uf|+w~nBg z#af$BA>uJ>W6I#JV-|lAHR?M+MV*0Ih&Y{jENnCQv9P&DUETSfCa3No5}qIhY@au< zV2pjHgd3Qhq}PeOIH)qj)ScjPNk4&t+yo*_fwP;QrSaVM!AbJ;V9b|X zVxU___z8fYPtsxUN7tFi0v#Ch3YUu7bugh}1JE?Qtbz_c)1|&oYcWiHIL6GdycAzi<>YbPqnHh9MRzM!uJ5C;6y5ow|(sMOq2M_r5wStle{%SM%PoWX9;CUFCrM+XRcv}Gy_0Z_{~;j zzpc@d%s9S(73Tg(S_p(Nc)!%%0o0?=knbAd7QQ&osJYi65Jpx^4fRM92#Em?>K zRILzXWe0dp#7-s_J@p)s(7x7Qml*t@1XgDOH|r9M&4c+?4OYzEJ~r`!P{L*y&VVoz zpe1C_n$4UR2Ob=u2CLU5=5XePD|SLK%-{_!nZid?@HIdayJ3bk3i3r~VObn%t$bF#1-fqugvn2RNfzjuEOl1!(Gp_t;uWvbJUb7(>X62om1ArNV= z_}E?e`&b%8=-Echn6x^ji4kY2vVuRUpH}@m^zbHMx@*zxzubhz(C<=Bq#YA*VF*1* zX^8T)XpvP4z5heJ&cpP4Zu7RQqfZemTU7peoRqMOtO_J(w@2OHm*t)gC>r1*;QE=z`@~n zp56uInp z9FdAFwt)_)a1~1#0^EY;TI{-WDx25thN39;_8Fp}*SHJJ>uy^@pE?(6;yG+Hv0UWH zA#epeOG+{9*_WjZ^U3JZQB_%@ z8>j*TlC6-fdk*6wQ10~W>zK~T4pU!2<;3C$=N2wz{%4&v|A%_Bp8ivDsvp81m8z+h zev5mAryKC;<0k_Z!^EqT+1JZbJHPeddHmkh(*?Bi2cOt<5=Z zaJl2o{BY-C2aVnudQ$+G-W`Vd{Tr8@2b6>C4(Y)8zr7YyEcXJa072}rSo1nOZGMfc zxYUG7U=Rc{UqjQPD$(2~OCu1auPp#84uMH|6W~-;)y&+)`6s?2-3?V9FEP?R74I7g$h4t)ZkdiWU^g#djuP?2l)=IkE^ z5)yR|I&R&E>DVyt-GZ8ZTzc8JB1g4>e6Mn5tk^PSG*sytx0PCabuR_<<)1mik~^8? zT8+$*&URi;(rvZeJ6(x(36YbzMLENsbpw=(bn|{iq(J}X#FF?ErUM*s(Cay1Qo|MNr%)Pe} zHMNimgHy2AGzC%VLq5X5rUHWTn3gZ55I?m88uC60dlf;LXW$^e>@{?8td_>`Jm>-H zgkpKc-&QB8-G%_K;DG`DIOC)6my^OKg3dq!&ey`1kV?;Lt_`pDRT)ZweFhwTJ>Xl! zaQAz|8yru}frTrA>Qi&=(fR;7B#ry+QE*Z}Mp00nC&g_$b>?fb%(R9@_24RwbqJq0 zLnsq5*d!<^YVrhe=rj#|Wr}}lH#WqS6<%G7YIHL*U3t$ggo(>l?ta4C#W7fR^*u{x zR76o#r?)%b*Eg3`YlM|q=H>QY4`_-_P|YxWKW-uhm?(5%=%gr|dwas{5ml^WyE^^B zp5hVut>WCQfv#zoqM|YXdQw!#@#muPp#ZulE9We${+$ZLDyBJ~|5{ymc>K8UV8v~oEbO#0=13WvCl^DK4r~^C82kDky2srQ zrlv&o-|&m&x64-Bkwx|h2Ga?T7xqlPW)NE{#@{XG)3BKSWV={t*PwDQ)`-aRO+?R7 zo({6O_o4qo&KE`0FW&RCA<{k0-M*gu^iYbYfveEBC{D}e3KwAa^!)I&sMw zVJk9&MX73J{s#UTXHggiHn9l!*20o7F}SngUh_CaH%>8Nr6V4H%-K1vBDl4uBW(U_ zC$j%hI3ogeySe4u{sf7mCX4GeVF#Ld1zW}K;Q!nu8@z=|&dQKjX0jWd{n@gv;jRUJ zH@Qp^#U}N(4XO(4H!DH?R69ef|Is5t8dzFFgDOTJ58)+ytrXU?>NaW}sF8@Wl$0hP zI9*(Ubn>~vm$N_{kLI!h8Frd3Ee@RaX^;7_kJ~8kd!+?c6a46-v z(lXX*tW+bs+=74o(W=BvwVGS?DAwZ?QiRz>^=3z+ z`snNC`zSX76IDYr=y@T4UI5AXzT14PU0uG(#n>wA(tHt;p&Z?MO{><6Abm#@~6x}Of8gA$vc4=w=@V=ui3^W>w0eyH> z(-^R|D*%5(gpB01_y8vh_dWtSLlhsILJ!G1RUZo=u9sKkatK5qs5({la@)BPk-IO) zK~}JY8u$v3g_KQCL4aGIA4=!CpJ)Qg42i+v-<*#R=B4&Be0zFjglWPj`_lbJGSpf% zZ>R$VT&)DMTWn@oV#4M?Xi#-&s|E|swd$SaQY3*(-^GP}B*rLgIN%y5Fn8lot4yoB zn}uuXB>oTAM(-}7fV;o$dT<=KC&B}_^v~HT$)|OcsR`ke(P-XzGUtL<+d1QE6{~dS z(mRm^NlcW`-r)Uk)|na+;X$L}PV|DqF^@_(M_Ryt9cmcn4r!Oulz%}>Sm70;+l-Y; zDr!<@SA~G?f_A^4)C%APC&F+T2{`!2mpe#>!csOxl3gwV1z#|Z@;t)`1 zxdCMZN%5ea=!v7I-3hmo+`SzDE7gTk$$k3!hgQo%g<1J)QXZ}&8HXSf&jnG1&`Nby zWtPHtq#Em?i`Cx=4A@8d8dW+~ob4Q}A)W7ZUDYLHpDm};;{Cb}365v{@%)i|F0`~mskuCg zJX-H$(fRM&g@GNF?LOX|lMbr^!Z-Z@@l?Kn@yZi*VKjw@W4`S@spo*9k&y|AZHnGp z5g!4?M4fk6rSdmtnh`*V!{po-{)GfDucoiW8nV_IH>cZp-qZKq_qU!X`;*aCh!o|r2Y#$Oa)>`Yc)=#T{n`x)%l z0frjaYJD(Ab3vYY?bDcy>(;y6y7=OY1C<6v)d2C%k!hY*u39OBRMLuj+p#0tBlr{u z@@Y5L$V=a0%w)ZGr5y=cwYI!B%q;;Vdn%Fae9_xPMWH_bmm^3OX67Z=(83GJYX?B0 zmnehDN3x1uR-{H)@{3)YQp#FyOYfFt|IQJMNk#U^+u!C0tQAg4WApw~yNjC5^9L={ zj(nY#V{bi21s1L?+r1W;{Z>qi8`g$&jLW%#=Ml~_nG?ox_Lwgy1QpF&HBh$oR5z??UWR$rn@WwAs|4A>?^{(kw#HL^AayuceXI+m%CR6Fw*}Eo z)&{`~gyxy)MhRwu8&LguDveEby1TMaRm)N%J;{}ejuH54WU&dt=cWg6=30%s2Cm@x z#i5V)s!POa9sXifoq8(juS#YPI0X{f#!B2MKJ?>-Z8v3M9v7y^uir+4?ev~(mX77~ z&U)#hD^j$>Jiu>{0_1j$!CY&7i_dJ3f?KPg#&O6Y|z9D`gfq8uCt7WHSP!k+Zyq(l? zB>0!`UeXq@e!n94KVEg&V=5G2CMYh;Z^RW3DfARqsxHbfV*r}R?r~77$$dDM6e4uV zbLj1J)y;Jy%|lgXZ8Ol+F9DMheAr(d1Wity_NRP(>F;Jy*aH{hD95|IHpZX%cCYPL z=MYFeFL<7F+fJezZY0;NxC_%7y=J0uPPva4)Y1cQT)s9U;$I6CIIgPGT zF1@B6;%Ly;P#u%O@Q&M_dbM=TfUE}8l~|3*odE=N7??L>AQLU8yeK<6LZY0315r*e ze3A3;9Vu4!@XY}@Pi1Xc#1$jj{C@wJj$+__1V^?{NTATYBmRT z!&p?Iyc)?LXO4B!L!w+dE_K!+;jwd@P27OcO*Yf@S(KXlOA-0=T@#LueN11zV13wj z+^`a!YcaYZ)JE!IiMwDkd7=^8qj|&^gmng8wCr%K5s|HM|L)KD==U{&;sO*1Zr0d# z!?QGnx$VE~ymzFq?l)oX8>9-X_l1{0#0hKdO&C~bUy2BvSvJ7eG&#uV-{V%U(p?1+ z;N6-i#nRqr?;A?xpye-BYjLi$H{I@>{qjJT+BK*leV0DIxMgS6z9=fLdN^$cSZVTN z0!896AilIZFMQ&le23k7<3v+Y(`ARfSaA#KU{F*E(qdt3_U!zY8`{aDSwd-KctvEn ziXaoHSJH6MRqmE$-d_yrSR4C{07lvrkaw6U=gAxL(K(aVonQ zoe|l&d4+KgpZs@BNi33phu46CPh0xB?UBav?puei%TH~3w%$3Q!~$NzF#8!NYsFRc z!q`M)(e`C@Gx1kW?MJ(4P4!8^5mnmW^ik2G`0QdOb9Hb?d##@n6sZ1VBcLA>Mll^_ zvii4p7SbKrs7}VA-UKO(KxT-iO)p`9mg(NOq5D;yxdnE&IesNmDwkOMrjV5 zM3_=?^pxxigRX&IX9Q)$|2(+AuQ%a!^G3hak>tJcHx6HbWQf{ew)*mPe^W3`bV-Pi z3+koiDz{oV84MY$=yUhX=S9NL&pc1Kp1OFR^npNZm+yk)!CW$|&XUn&=kxQgZ_C&G z@s8znvwZf4bVB-zdU+&b2Gs4;5g@LcX6ZQpYf+CiYrHs)(u{>=XpvzKW2k{o&z*Lp zPH#Y3wfp$zt|eWScgI;>^;npKBJA(y2Q{)SLdEYUFybyasjr7LuD>C(nxG|E!0>d< zD*h@Eb|9w9$Mo9{{$aCZ+c6!hGr^VxMQrLR-WHZ>BmB9sFPuGiOKUpciMMltzd)!E zZWiNgB?0L0e=Jo$Yl7Q^fO%IZns0T%A)V?3p}e~H^@#Wd3r3~cQms2Jd!#MR{HBJH zj-XoLe*|${RU%?Ju(ViHBL}hc&3JpgpEgxWyhixiUXtG0ysKnutCHd(N~KvkI%Mm? zkA0DK!F1G24OPIgVM5W=Vqw|@(rKie2reCrr=TuGp2rL$?J{CmPWOtEqGnl`r&*bhTc6zE4dN2)YH80)j0CB2rV+=W>YRGXj zva#52xxb`xFKk8p07O6E_bQTs>7I;{h%-J(IcYbg2&&%U6$0(b@qwDZ4%|Y{SAE$Dt--rxK7xG9+sQ%XW) zVqMjOy*1s-(M$;rCkXVyvRON@OzSi%y?%#N>E|>2zR3y6xC<@}#W2(Bo!ooFX_$@9 z%5o+)*x^RfDqFovit-xShb*?iWh2bO?|V>^1Kl<{{Q=FawnPR zcS;u*Ssx75#h;{eIZ2bWv*~`#%-Nr6*-L(HPHNAx|BYf9YRmO#XgeNn?e}ARf8~+F zH0kj`MSso11NC{l6P;FdBp{?mza&rj=ZF0Hk3BLmXC^2lb{!^?3d8kqH;XZ+`@LTrr6LFEh{ z&Heb$n^TK(wEue^LH*(8q=&f5VGuYiwBRFm+N_3T&dAbvq_9rbda0~mQ8G`c)|;{1 zSv^o`P6Sn`Kdi~=4epA>bPP!fpR8_jK1V_I9}AVhcy2-P4qT+g^2DIlla_ix*EK6? zYnz^-s|S1AMRO_LtLkk6&KXouwH`V$e4;0&O!I%v>Wilsl`?XOQh6=w|M_pHSM!>p zh}nYf*mxfn7CMrEI4}qA63g!U1P!Z;UCSb$LT?5LzF?>PYoXRtgH|(x^>+^R>lj^A zaEH}Ao_*o_bMgLsP7zY#Ym6^sOJ+ShEG84;nL;eau8P!MIG+~jUVSm2h!_VO?1d_^ z_S2A>T5#yaf4%DuvvGj8;KGLqi=IHX(EPVuHp775Z5erknXuiIgoEcO5~_LFjjVJX zut*CLu_6J_a2;4Ui`QT)!sZVQ=Hup@=e~WZ{PQzHpSz3X3@*&ruaj>t@IN2J8~eh` zCl9wk%I)Z$0R06#YLJd^0(g+Z<-*lJVnRbqKa-l)uw|gMW7e%3}+A}4(U?Vzi)}iBhs#k8Dosmo;aJe|1K0= zWKE#|fXd{iz`0r;gpxvx12zW?(?V&@jjrd2Bwzqx(n{x%ut{a+^m{EkZ> z;|x|-kChm#IpAlv z-D~!hCEw$aIl58*Yh|Kx+>fRIg5@3vRgY5sul0dUP$50W8T4~?!8R<6cg?(!U9AGA z@!b6NaE4RAQccGA***;3>DSKIVQ z|FZqpLfntnGWw(0QDTAr-nri!CBlJuZs9tkdDr1?&0m^QU068iG)v)-knuQBJ>|-D zuhwhjFr+4iBAw2Ro~_=IFdd<7kKg}4w7t3*=&UEpfSPWWN_*{pto9$l7nVbQmGG*E zhwBsbgm@?S$DExx3eb^jBHtJ^yrcIap79~y(!Ae^v0qL>xG&McKfM*#c?>kV{+zqi zx3ABl{+~Z1@`$`EdSpk%fn7r?x9bju*G=^2phL5Bn8$qmayg5Zf1zXKTF7LTA3EHb zyGQqrHxY@d0ocjRffHGY3=!J@-hXJD+V6>A2&yR3fUVKYYs;|#Vk!GykVivLM@!+@ zs&Cyjsvn|=vz|)CH!{+Fh1F-y&PGw8fA24-UiMz>atj3q%|O_T`e3eW;&AFM z_uA*K;}UY9t=6tsgLMwA>_^p3_B<`#<IVDLBU}Hnl&ap<OBA!v=i0oG_t+2Wz{0U;Y1on5Z%l7f+mfNj0=^ioj06&JY9LmR>LDeudtc z6c#&DHWY3w?Dfrh=#6=%Qb1)8`A$4%UYu?4_8;!cLs_JnnFK}=VE&(Nl^>UDhosB* z|Ju9mc&hjR&xwYjiL%u#AxUM=(xr?jdn+jpM@hD0wA=<-$R^q2P|mSQg_|9Yy&|Kd zV;%Ew_`N>5-P`SZyW{^~fAlbq&v}p6d_JGA&En$Xd*?nJ^JUth+xY&*TG~zs%@ie0 zG-$n(R8?l!Vxx~*rc?w4mq^pR^rn1Yd8y)(;!%Y(nG8*1Bb!(3zrPRg2!<~%+#!~=o!wY8?Y^^!8R`ZkC4ygA&<`tGM zou^o>zdf(IF=t@H8ATeepH6dXfMXrkvPj^dpC1v@N5hgKk?x7Q$D1F<9B%D zTgB3T%HhpAS+Xk!OjLc6AI2<-)lr-jJ-E3ayL(5;RX8i}DdB5CA*oZ5o{#+7h1$JZ zb5C4ECyUukDm!;HS%fIuBjX~mY#c932Fq*kTvqc=uebW$X3LxIk>&6>30cb($F<*H ztUnIs>PLR5Yz^xIJ&x+~#a*{sdbsJZea)+gRPRSmxxitpr(fAMQdyDrNosw0J*{^` zZDqYv@Iqv%5T@=MfjqPcdaF5bY>Y&^$Ue~>D z_2k65ug_w`bk1!3ArMB zH~e%XV#=)2dNO}4a@B^@LLss>}r=2>Rz%DQ4 zlm&Slq%GVJDvz{#G3dY>G)j|{Vp_9%)xBub)6+N&O#Hk*ZTsIv`P~pf)KkSUU8W7o z<7~pi_f#&CaZ`o!qro17;4@-TD%&RLxS7X?y$?v#k1mQiBspoBN6V!aDY)lT>xsD5 z;a`-Twok;+h+Y=U#;sg5Id=`=G_B}4ix`E*j*vzim4DTzAI18C1nYJ>NF7!FCw2VO z!`T?pGjhBwTRX!o&$4Mcb}RRQSCnx1f@6Me+PG=>RZpMOOAC|nWABiLmA7t*0Mu-X z9lY+>7njUE)@N(IQ$HXPxd+}pGk>g)q+I!MbZb}8@)0TbX6$gay6_y;)|wcepuCi-I@3qxl< z8yZnhwun^XseJstN(+qBbLyC-O0@<5Sk=Ed$?(u2Lb02l)A(;(ja>Ban)#IWWLP;S zWzge!#);&D{Hfxa8_eoqGn`ucziI(&#(ec?a;W8TPj`>_(DMxay#qxhD}e@?qoy=B zZ!e8N?$A#~C4++_@g+|Cn@hNy4jAx~k_3|g7UtLOG+xW>veCoA?w*m?Z9JRTYs2XW8t$Fm}-E4#Wq|bm6+qo0zvY?m-JY)Wo#-_ob#5kaRqDG}O5cupI?nKS3&QvFtK5WuHKqM@xu(OUR4z>aveQi1 zVa$v0SfwS}SxE68%1&5<3s^a%;9kS(X87<*#x$?9mF3UFm|B`_U^81l;Nfs3`_ z)=f!Txy%x6e3wc=qdf;vUIiFb5G!dUdOCLwnuONSanA*O;H6tr6-D#C+!8|v{<56# zCH?c;2j`HkUrB27FL#0p@bfYX^?Gc>!?Fgu5W{Sf2_E>xqdE3RSzUL3ozb};qqp%ows z<{+|~gjRD;A`uG`r5e~3ASh;3t4DW1lb{9AD;EgN+%nB@OA#c{4KweB!P&xInnQw2 zz{9w?pU^YvY2J?eewbhIL1A`M$Tj>D*N#WyMcUoR2W_#H?tmUa-mOgmDq!+Raoelc zdC@lqMvo6W(h?UxANrb@CqMUHi89mCu|kNpGoRa=ph+5V=;NLsxo!Go5pwQV_d}Bd zP`*y{CZ@-(aXRG1$wQlgP|jx32EFZ~*GCNioM{5$olNcpNH`+6{xWbr2q0700jjNN z;O(YrMoML8B2ujwG!CHS0H!jIy8@)84bFO}W`M3oK9pGzjr7@}pOHwUO>3KF$hASY zuOX--c^No-NZTU=1E8yO4kVK~Do7?9$0%UaRfBj^(XghN^$N|Yjc5wYPrn^M6tCbp zLt^dF2s>Qf0sw}F#O%^$61(yLq@REIxdjF}sn@+afjF8enbs9#>%NLlpcKT>{{|?r z*@v1eea!5r{?77PfRRQt={d1BI7VOc+V8zrUIGPpCqA50c=QK2(`*E) zURFOq*KL4s?XjMv2p;ZuX@_y1MXC2J0Lk@!ri&eV$M$O3-swldQ>x*sHH=HiDb#%4 z(Z@*&kH`a~+P{7cT5TSoCdZkWv1{_LGdd00c?+p4y!cq@jMON;xSiK1rc?yFe4Gvu z%)U^`>r;ds6t>tPJgW(Emj%~8od*y@o*thNDTzoYz79iK1Y-4Wd)ai60D0SlfN7Z_CPW;#{x>qU#mn{Fte#NXJ=-rf>1v}GE&GY603I^t0Q8RMw zJvim0-}-j3OpG~)V+g(2PYI!)K{0xB8a;ZZ$ZyTM4v9{CtL_5ljV2$c&G@~UgkjAg zPxq;LfC)a;r!)!qs2SU89lOO;fb^3wz+yi4vl3fjZs@E{!$+$-Z z))tpC86jKo6D!z?HS3j+?eTd(z7u~px%v5@JkF}y`SPCICc>*CQYfobX+t%<(xyZ#6LIoPY&52w)qFA!Ha%H z>y5`sL$C6*lOj6AcDTG*EA4TM#en*uHQ)S7mB5x29@7%@(cZ`FjNStLW6y|8UA41$ zvO~+&%_MEvRg0%&%}AXYWxw9Fb~6tbVHPl{rQBCu(Id#FrrCV6LKhJa6d+qqwVgL$ z&e0nK4)i&MG(AJ^l(er*!(3xM;abMXev`PAL(5vmoXPqCS0P3w5Ve3%Dx2>5P$iw? z10>WkhQo~cue|c5mvHEkYVta&Qu17H1M49vkW)2*p6(kjMCNh}-Ol<`X{_jI;)9P3 zXz4C?JN8y9mGL6EgF2%u@*WpcwXDB+dw)c<#sx};`9zEIbC;ME_9CU==+HqKYRg`s z@0ygv{jn2kH#?q;wkvs8l;>caKMbIluFg>5AWM}~fR~94XPEYWfSOFy2-`+@MU=?z z@G$6{K#MD6pxRP&1R~@;UD`4Ygg{lwBH$4)BqZLIi5OHrdeT`RtH#>Z3 zzCFs?Q&ugrBH7+tJ9oxZ?=A_frvwkCh1!R@M1`==?TSevYE}z3Cu+MJ$5h?Zc@dOX zQ@@cYXy6{_?WMrQsiIVND<4|^HyAgv9j!TWm+NOmMI-vGlLu1&BD4$O?^|(i>RsO| zHkv;~CNDh=3H{tU>fvqGrD*lLXZX6sux}OziTT~QMcN@0Y8@wY ztLomT_yoC2!Flz7LpKyr+$3~NJ?5L={8GqlFlFe+T$)k1TN>IwoF7=QavqRm@0`qj zRA=;i?pvxARuhD~m$JQexM&%LoKBkTy9Q-VO=f!ET}8=v|K~6=^YziYa#XDu5yFlQ z%)hT}didjXOi7%{hOxpJ%ir;mBZg<_`yT|$Cf8OQ#8F#MD@K(DR$q+@ZQR!$n)w5` z)fdMQNoPXGK72yt`vXw5`vIZA?C_O??iqgaTSSdQiVhMSj2v#C{_duH`)TZZ29Qlb z;{zmQ)*vbSw+pI$j-Iho(0w8jntku4&Sw^H#-&+Bvd4HDCI22XexIj(N!B)@s7roy zuMR6UGXCim8@4etqPnm3Mnf@UIyQHHXUuS#LGNMwcQ)|bhk$3QHByQx)HS2)yUw`r zx;uD5(&tNck+ulFy)d)+9RQOuG$SO~f-$xP!2g0<710% zu*m!hQLvAmm&c_jXZ`ilg`ZzklPaiSMlldEB0$jFmW2op>uBZKp=m%_wjdH7kQ&aA zsI8|;y=k9;^KZBh7q`W>c?JQ^ z@resFG4aWQcypK9uB0c6w!caw+}FFS45RD^y{q0QK3QzagJPvk2TBjmIB{dMf=wlt zlSW&~EC9g~xSTy>H^g%7kUrQU3MkwlPQss;0T3-Dt7=CQ6UD{$dV3_^v1>_hm^JC8 zh9IR`%2L$M((1XIdbxDE#&?{)Z=>UfQyQ&UJAIszTe->LILL&6#LKT@5mN zw|uvlx7P1>bAI$^ztj%YJX8npkd7T(djHIlaDlE5QnsC_i5yff6cIQGdGZiIDi?#u zgxJTK@m@uScI-@=1+#FUZJ079JB_%!LpR3wp_WrhO3H?`O{r1( zdz$6F2RM7R4H7M1Lo(dq(r(QzJNQILLzrD?iz5Jg0Mxw#_+zB1V>DbBMOIu{2+9DY zdR!EWmP3lw*ZvF@t~^ynItbFl81~#&`vYL9wqt_tuwp7FRDY|D^y^BzqRVGfJbuTC zfhXWlkA1Q^Mcp=jQ8>X2j;*yfvv}^=3OK$TshUoOy!nE`S1p*~B#Wgkc%xi)6{$N< zPX3WFu3jgRGfFWx^lRB(8K-DBcXIjc(oYPn* zfGXg#2xrHZi7yffhB@Y-!gCaVrcuq8iB1k!e4~Un#M!(m^+bfF6-)uJ^>+Q8OGKQv zHKGsnnD`n`ZPkCnG!`LdrPB7ut^e(vmN?~gh+GyKZ0`$%`;fkNr)#WCN2&n^G~n)g1!#K6!$<#Sai-ggrT?zC1eWN>rGc7Xa z%&LmA9vM#xrWs7i`qI9W;Dwjq&3!u<0d|fuzSKNpyG{3>s@{ zSqbJjz(mft@%eiwR8H`Cy3xzyN^u<@)f5Dvpkb-RxD(B3F=*HMVIWZUp#8ef!dpEQ zA(jY1GU7x%rdEg*3Y1g^?SfweY+aCH6Z&W6jHx6&lw|vb$6M}?q_GLDpMnA!0SdfL zIpl0A_|p?@iFtOdndg!brP!7vEneObz*z8UpP{EMX{^<=eWf-`%w75D-%~!zZdHMq zidr;ujCy{?G?9`c;TUB~E;f^^KR#^Kc_9Ngsc!qY<=~irLeRLFfw!rEySZ9K&cfbB zP4m#hr&m1}#|uqCEAc00f&)*kDh}G{kW9rTzG*@K`N)1f#;MA&K*uu{S(&(OgiWH< zRSz6?ejle=&q@-Ox}1S;d12d>NW@qGzprykTap|mbP)zR`tudmMala2DKeJZ-jZcp zgAOjzQdoGqV?Hs%(K%D1K%DWEjc^yp&K$V!{%&vR)9b@5aucsuhhp!Ew;1kWShh_l zhPX`@mXNatSV{5>+tAVBp}-N=yCNsRxIOhV(8rjMdA`b7LK^UBSqHFb6pNP`{JTq1X;#+jJc*d2xGMAyMDt7Ap&$<_O^T*YiLp z-B9LtW8G#J_qa2Dq6uEY7ch_U-hP5fY?pTJQ8(jFZy_3=|!=dh6Uooh!=ma5OJ*gq&x&C<+j)?z97yIo|z_MYtRfbGFucgZk0Y zWDm!}yaVn$b7445Q_-4HQ07K+A?6)EyIL90F+$S6`c%a&2{_J%+y)Uzn*Mks{z$vA?(x4}zju4&7(skpxfORtyMc+!kCbLB7HpmN>qfVz2rpP`8fPC8VQn&iYQ}4|o0Iw#J*U%lBNBS& z29N(WjQ9Jf;`c-;JjnNaSBc?~tc5O}9@+PVybhkcr^GcB7u~(LjOh=tEUTYZSC{+h zAiiFJAHE!X!!Kkv0CvweRk^7hxl~tkt+PXK77U9?N-ZQz8rjA`X}PNze$+N$s-ni` z483o_hT67&x5>Zlc`&ORDJ2Oqm4h$aUta1wwvPLWZ5EVFWSx%@fS?@i5<_lId5rXX z!l>yNfMya=vSPniHR1B_A=sbR|M#W%?oC}?uK`6XByT%%v1UEJJIH#*YH+L&T+G(jxC0I|pPC`ZoFi-SNOzZ%Z;=Cuh7Iu)mV|uK+Gv{ny zp5uT$o~Zv45lWhcC?_L;L%a%7UJm^wPvFPB$%fQ1uGhTSqkWYvY5{D~QL01Dc6~Z~ z4Qz$VzghpUZ_e;oAMn0jApE0Q$OxJsvanjScGG%4x2o1uZ^$Qd^)rmyp|^4l*kNfj zU_27mut@QxQ*)22uJ|BIOC1opJHWIu0wg@W(6u<&Pe1gd;`DcS`Rk`g3BLN{ zi!hhfzSY!8bA5#@sC8-{^Qu|_2@^YZvuh6pcTPaR@P$B(S7XQlqiv|U91jbixa)^6 z0|V$X(EI}-5O*OootABnJ$Az$CVb+-l)eEPwFAu8V;0^3U7m__)~{fm=OG3Lbf-mm zt1a=EmR$xuB*j#iMPE5}?>vqpdhO76&i3y=?L&3+%ptZo6ybGAjkjgT0~x*?6g7_^ z%DNa3hHx%(PsL%pn^1JTUK7V6fJ%%^4(8X^KIW???=hVPe&MG`;k*f0_szo`u8`AZ za9UtJBct@YDHQGet)(|#RjP2s#Qje>?QhSA;i*38LS;dn>?V=M<2`?_6NYmtb7%aT zl!J60p@oG3rI8QDos2$6duaHbD44?>=;TyDjJ1|i&Mga5N}GKrgv>9#6-?7@NHf4~ zIgnh!dI3?3Arf0jK1?UmZ$Gzu+d@3F^}j8eukXfdkjg>|tcbL#(tTEz3GenrxSf&8 z1X(g?lZDj4s#4G~P1mjzg)xZ-khB{L>aEZ~yKL8#7{E^CCidRnb;KTKSGVLlT|#JE z7)X#eZ8r1dICAkmqNC#7o~+8KOfcq&%hP!1J9BEaN>2H^D*1O7;p?Y`$IWE)PPxZHQCG{Q7|3xs%b;gxj@y z@Q|nH>k+B6WXf+es5i53(TD!1R^yMK!{9%wM!Ht)_^VRp51nVudcW{+jwUPUF!S$s zfFXt-N`MY*<>7YrfyLo+=N(H)IbhC_aB@W0VPLSQ5)oGB6i4gkHLLl)du}L;;a>tF zGl6>?gT@qox~wxJyLNoFK|ivsUsn?OIX+EGp!nxvXWJ_gW6QG?sxNA19#_I9Vtj$a zM3^Bl0YokQ7j)I3O==h+ac1h)m)_>Gpudiki8o{TO*L)XUs$&k zXzN;uYd+2Y;llj&H+;hY=vFLu?$6lXzy8^;mJIo>=1D$~XQ^QF<#_sgp8n6jwdxoX zHXH!uQRD6R4LEo*{^PqpfNU;`j&oy-S}uB7gq^ zzipBAwqI_j_EXLmiY5e54}Mo3NX^y^@vJ{9Ia+-2Q{gse;rB_Ko(K_L6R5QFEnWeP zJ<*3`O48@^qfJnB!gV!t{}9X^74@4}{y1OMR%o9r5w4{i5bPY8KGRJ=>T7GqMWYjj zhPu1PlA6bxs5y)MvNv#FUhj#8B}lUC7uvVKc;x*_WaZk(7gX6_t+5}8ePVMN&CsTz zHX(ZKWt-<6O*Px52UdENCABaUK}>kOs~`(h^90>0ap(R-x&i?L<6H*}K4D2rr`M$M zDU;my!2B;~R{J!)53V}Mh+5aN&kkFgz~AxYuuAsn^4ac3BuAXE&)iY=$HC+Gx&Alg zils7eSZ}nvzLGl5>9_0K9*xor(_7wWmBZzAc~2xrh5uV`G^!LT=d+7QpLyAHgCuOt zozy`Jy}CIe=bcAy!u>djZ)-h&x+Q+BjL4rX7S!0%cUOzS;LCy@m4tohgMC>1Py6u0 zJG(b9})%aXlUnJ8C?2Y6yCdK;mO(3ClU!={uTT{ z;bOdN_MXt@A=3SL@q60SUzy9;6s9FooB3?oxpDEfX?9WbE-89nka?^1e9Um_<6zvD zzdB(*`daL*jm^!?hNEl_ugL%M>iu?I8a^`QY_YJ@@3gyG&$<0SMY=X0P#MPf7cJD| z-%&gM$^8KGe+zKH>aK|%RE&k&hc}jMQAO5m2{H@LZ&Jt)UBPLaa&a%lZ^Xc9{ zwW5Atmi~AF;q_bS@=)v8|M<$k`AD}^35t9W&(i<(HU9CRRchL1)19i9mMCs@@h_;u zf2_p+e5AB-6+)lQcxB_tzP{Oi?{#9W*CT?wAkk$YE}2G002GXP%_WfZ2pgX1U0H60 zc3Sq?LP+^c1Z4*fn&vft)^v6-)g;T(4+|1Hvw$;cdw%M%zR>0L`+xzrVy?^kH*56! z8(H(Jma0_DqLi1umfWjMFtAyAMUfkDm`8)Olbv3QFTc=^FI^ zW&4$ywriUr0XiA*Vt7HF#G9NpzvY=9Ow-1ZLOud+Yi2AKRBAA2_5@7}%kcEq@sVcmM&Xe=3ssF!RW@N@bWt?VU*CwY1#~lC|vVKepE(~atccQPP1*W zFsxO^xo9H#)~17hGT=Y5HebzIIN=qd#e?WgKy2KZTS|jT;si$-kSP5OLN`qGm_>4~ z#r_ol6MlwbBMZHsBK3WchM!HxvpEpg-As#wc3WI60(nJHOCzWK+zrf}^~@~K$=xO~ z?>xlEFooo9qcqul!MNB|BGX={-i)kT?Uw$860!w$|VupEi~RTP9Qyw}TU=o|1otCAC( z#f8u*4fp*Q`*d`(AJUQ8!I>DF2bGRJ>DPu`B^>)t=im_ z>H?~%5hw~kgQK>jyMnf28-Ot{f$Y*_W{#u#8Q73oR3hJNe;$wh3?FTsta%F=dft@1_$SCQ-vB%cfs;Z-d3 zNBIw3!4=OkmE9p(9CJjJZdSW(fDF8y2~D<_S9F&V9C<~n(2ej?)b1b|(tc0KxsU?D zMiw9FULl?87f6eDhh?(kHQtU{gEHTa)V9a^BDxEp*EC{etw@SSQrn3l8%E^>0IYSW zaAI_G)t+rO9u@eT*7RREnQ(vK>$5rBnW(u(r-W(6(BI(C)3P>RIBilbG(z_5T{K@` zR5%oF2SR;l6Ent5=m?9=&vXujqndXdb%moEL735NBweEn)ovUn>71gWZz=4}*q`jg zxGxd2D_zET;c~5b>n4+}c7^9N`XvG1v{<&;d&_lZ)^yN10S(aSB!1hp57F|TEdc{* z9|hYkLzybMx);QVHRm!jZm|BE{M=HhJL&ocXWhnGq%dzQMKZ?DANM>w;S{VVB)DGN zgk@Nr>9jq!Xv|aGVe9{7Mv0=adY{L@OnMH5U3xu>XWDVEDa6FV)_CtYf&>J#a~Yzx ziGT=>$993WWh2nfiny+0;acq&Cri3bSat${Svj_At18#l>mxfXSAomL`qQHz|LArN z)CoT>w8(z19y@6mJESOQwQxH#4r`u?~?rGQE%|^AcM14s}m_c=Lwj#ctu1 z9R-Q63)C}j6UUL+39_TKJmZF8U zmKaiAoP0v66YoK!%K%my1oGatvmlBi&G5mtF2erB9d3`2IA`G*_dSd}Xuac@Pif@q zc$v6I1VKRJ;gy$R7)-{NwmcXZ&;g!HSv!8{Lzx%J2MUGf%8zB}?&qCJ^UstI*mYzd zgBI1hjctNW>Wr=*Z!j@NV`ZWh)3QY2A(APN@m0+^+}Z9 zvBfK39FeMtcA}+Jy$&blMp{9~Z3Am#Mx05mgELm})n#xi5+J3IAhI!Cd5$5Q4ow>g z?XO=PgBrdGg8=8yjZ z+F5ZXUg-p*B1r*H%<4!9q(1S>yi^;`+JUw^PyDJ`IX1Uik?}#tKPUTHi+L}(Ap%d# zyAEGS;5$KFY--1n4c_!L*(N@hwkdy`?&#%l&{(82-B?<%P3i&dmy$2)45wpwJEoiXcNPMuD;NS4r>87j8wS&W{!na_%f@XFVfq=mAm3 z{?7#z*Iq1OSkgUgQ{vXpu{Ic*^!AnJaPqI>1tH-I5i!xnK0dGVPtOA_w*KQnRqDg* zb&Hka`V_mB#GD_^{@;Z)f8f;px=;yTEho87IcTYEQR>?W+;Kgj37l8} ztI|{9c`eFR#xI;sf$y@7=no0L7Kgy8K?#+x^sHM+$7<*QCYWD?f-ir}rVM4IQ2mo( z7Nbx6l~MTB5fqbQI%eAF5E4|x-r8_s!8-re`zwY2v^EWI8I~woDEf%D?UmQID#aSD zAGst-?_1lS@^K%$5C`KDdpA0g)9Ig6kSam#&%^NyOd&c^CGeXG8@JH4HI}Z)^8u<~ zp%PFlLa=?iWz%ODSlu~wzX;;|`8|E>ar}1pfo?9l&&Bo~LlLhv*pNo4I7&U_K(;Z> za&R4*C?}x70NQ`QC@rbmT505GS6;sFc~Rq^mhD$jZPM^B0j!e!6ioUg#qKPEJNzx1 z2;qH3Tsa{IwA$o@^ecOF;w};!2E?1OVt>m}MN$o2f2FwkpM<$ho=pnNt2m!Lf^alr zmr(e@kttV$C4L=AcIN!uI#?kjbtLj>kA(J6YqB3?LzOn z#Z_qv8x~XsbXE%P^G4TxNI~d)AS*PusZm(f0$6R07pH@MWTd`kMMM$p%%>00qe%tC zFm71btY@zPC7c1h=|mCi%jXM};g^h8*bb3)`zlrM*smIEhFwg)ycn`jfNK2LViqhMCeeU0zpeZP|fzLcGn%yL-Gp|?khD65zOn9=$h+WnVv4Dt!m*GKy6xn52ipk-UrFME#rBTe*%NJchMa}7F6C-3X|l5pV6 z_?E2?+ld-=Tud3I>`w^D{gI6GANNBf=Rgc$?1!t)((>(5^8N+%DLNFn8h z@W)E4Sk>B38wSl`&E-(fD&cPehI_OhT(hp)Ki3N zJ<*nFiSp)v&V~Gr(2kbH)xh@XIzLDlj71c8M7+^?Cl5s0e1;|V+VhLT1aGmKyD8i2 zh3#m0BEx??_g{b1aG2pP%`{tE$?Qdp>CAvfJ}IxLRYbRbv{Dr>l3>=g4tWJ79rlL$ zvT%z7TEr`RAS`!%t(9~FQ(FA!Q$!r{ZK<|1M3udGd(+ELUH^>}Swqw=b5i@7 zMPlZQF&=^P*T}5GIT_PG8=>fY+-l|y*E&H6B1|;Uj?`~1geSEJ0Z!&YNcYlR)eS6(GKYHBG0$zY+Dkp)Vo&67(M{6MpL;A9(s8hTGnh#2C=^ma0DSSjw+0wEioyqu4j<9yI$=YSdG>9ECig?foCk|-%${uxT*rOw0q8WQ zJ6wno);d)!3>c>pekBLsU%6BXCN&eA-px-AC`UgK(0FR8%I*B*(}vb6KcUr}8qVFO zv<1*9c03meuxI0yWf~dg$1u@*mjd+DV5CVDBnMZ!&dB^we)Y1uJu$eUBuj+^07%x> z!Bsvq`^s;P@|I^)dOiU9ZnS;c1VO&?p?;`Fkgr_sST-#Cg<7$DmD)|({xOlT(7|pJ z-Qu4sqB}Ih^v4J;WDDd_>}+8=rtJV5if~0a%7w9o$bytL2IiW~CMnY1Ezjnb63P7} zM~j!2>Inb<5@BE^UAn_|OUEm$zXUM1jfeEbE8K#u8DAAW+QkGhsTPv8A|zb|1=qGw zw8&DC_DM@Q*0eH^Thq_&c(X(q)Z|pn>hF@8r|#iOmp$4x8Xdc4tf%F?<7Se;N0K2ojY zg#itb%XWE1zYfvjw?{C12*)s;A(^;oCLo?jr>8pCjNq-9J!Xgo{(776Q+=?TFhys= z4@ynd$b7IxsP$lA2G4XHpgNUyF{sfPaoUAvt10+ugMD|859a5oVM5asr~avA8T)Dd z^NhQp<44|JpyfHd&|or;#@BQv;wgu@or2R2h}bF^bv~a_1}Ci>>Z{)uA$d6(sb(Nj zp<7-h1ydbh$Wp-_Z$P(g9)9O`xZGNhO|#6~tl(NU)Qh^Hs`#9>za*zK-6fnTb@#(r zr9L=a+r2AnxwMEo1`@a(?9x8@M^EIu=mh}zE4)8w_4@XObvBIK4sJ=#^Uk&vmj!}VT4cefX7Az zVem+Ou%0Lod;F{)lw)9emS#l5Sa`xzt%T;oZL&;D6rn zR&mG7i^GNwPjBqkG}*dU1;@a6i)Br`zO2k%)^*dTMewz6pShlwy>aBpr<+@u#U8$w z*-Z$4zK}uQ-dk*HN5($d?(|5G*R98u<~=^=Z708Y!QIj)eFhB5wat&J?QYb=q$GzJ z1D{?4p7oW>*`GxUp%_~zfANzm_qzf_a9tEZ8=}C__q|0fJ1FUIOf(~W_WE8^` zX$da1~FQh3Xc1v;4HSCseFE;FFG- z6vbIMblm$I)2JT-Epuj0-4WbO?$Wl8OQc`@W0<{nE$pq%Y?PZB2N>IDYT_IWb8P_l z>>H#S;S#|x2fX#-Kq>iM?b06L0v&Ch_%g6Ort@LQB-B20w_1Ln@O;p#MQz-o)3DdO z+n9$pK)iKoiM*G7!$~F7wg#Xnao^|U znqSjgUrvrA$S8z}TnIyuJCMO5@&qxh!D>4{ortK|U12e;%a57_WNxgad~|-hs~(2? zxH>G{`XxjWbP}1EEx;k zGkgUqt*%(4UNbiNWQI$MJ)YsGD>e-C1CHV^D~Hlm##?8Dq+?2Y6nWB=1p;f|@ZR-5 zd?`67{D-Um$DU+)ug_{d7j`5(m7Ag;KWDdCDP74=)}D?V9Y%?(`X) zZ)Y^rpJ(Bm2)2{{^lI$`n$wP!g=6N7k#mPcogUY1pPMT4Vciqt zQ&aI)skVvk27dXvHKi9uhBd&@#FVjXR~?Y)I}+Peep!KD1OZ@QQ)yCWtq)#pP_`7$ z&CBlK$L`9oXi}{6Z)l}fDvNu$DdXLADCmIvf3X->&TlcysXt%C+Zg(uDt<4JA~wH}3N#5q=1l|C zF#Y9Ss^}n)DK0FaoTr@6ZQG`Z%&&53^xN23+HpsHnsCAQ6@sXP&P0Jjyk)O9ZKk!b zmF-e)vNCF$p)Bg2#5?PRlyH6=j{DO;|WX1d*w4d;!rQ&&Q%K6(}B;LRCf zFcUt^L2|53DoQC>F5EBwIFAF8*xZy(Y$ragh{tT_;maMs( z3d)trJ=$h1r$l&0S!}KFH7Snv-Wg({V5+Vh*k&lrrq$sT0-t2DG0d-dIcuzvCN8cO z<5AR6r05ei5EpK9--E?Q<)6&k>K{+ze^Bc-Q&_p|NY;~=n6!4N)hX5Bb{er*jNKdf z315Lc|EozAzPXa6+!p{wbO9s9bGdbR25hH*lwVh0V#R~f`}%f)C!(ErMmy*7N{oY4 zBBzq2&raH8VxMQeEBQMM``i0BMbR`d{+;7#(H4?p)?|+&`)KzCg{^El(sn;xoRI&? z=@Q9H@hBJ+KEHveSBM^Z)mXP@7oA&?PosGZ!|6k)r7}sEcID;W8RM-3KQHW79M0F! zEDan@b9!HaPE!mCn&C_{K$h=^=kYgJ5F(JJ-&;8&`AReDSc|D9mK^MEHEdc;{1~Y& zkIR-DJ}X~`F$(bb@+bjHV+mk&9ZVL2cvtewye5H75wPsGW&l~7N(sNEhq)bMN0ZQ2 z?k#0|^fQK(mmla4PgAS|b>1Y?-lI)Fwi-MAi%qh`ZtYuCkZ8&XeQA=j^~$fq99K1! z-7_;hbXsNfBnLHTnja%)foB6uCW$LvXo?{OlRGVJdy59^TLz>(P9q!85~;KtYjPA_ z8#h$5=c4A-{@oeWcSlHFaRZq)u`F2h=`W}2pOz<(88st&LZY5I%AIt3Ner8;I`p!~ zu&#)oS48Xveo;OxrocnPd|1F0Sx%|<>pyoF<`&PA(lCRnorEZx*0C8iQUafI&*yBxHTxghl$|iq9{G#eI1O-1`vESBbP5>$&Vd~z>vU}%jJeeGX zn~%4mpylk4olxJ_7j&X3DKJg)iI#1(Tk{sD(;wfaY7S2BbUtX@|J2;BMMjasE^|h_ zP3xcUwVkG~CdRHiw>M~f^mfZV$>&Y+Fp9}yM^l9R9c8QJM0A*PP)gDGpj3Qal%T~@ z-vH>x1%57J=ylgm)t1;DrLTqkv|5xkzp8o-}Q}z99AQJ|$ z56-`hT(=Nc5yzt!QMK69;Fu#LRi7FLMsJ%G2Lp7Bhy?rgMhQ4^%~Sb4v}In?F7IUv zMLSNIOrCRjyys;Stv~c~u43_rf_K~v>dv_*5j!HL6kFP!;S;Hz%;B!sQcxo4d|y($ z)I-O;x7ty~%qp>F<-WyAgsc3ce{oY=Wc{dE-h~F~AousJ=&UdYe0HXygrqnYhn|kT z4Mxzq*!U8-!;^g{x6cCuvm;S6f~9A)qNN&paBfU-Q6G4>!An-0vv=cr&r;$Iu^eK$G_Cc~F+h=Y}o*WO^Cq`=F$?Qxo&BJrrvlruYwVZX` z;?)~J@f_7a%v|~!l#b+UejNJV)ulIg{Ex5~P5(M+*fV!~=$Xh}=YXH_)PHly43sU= zl%BR-n`%GV&A6QjEDg^K?=Y==Mm0VXTv;94`+4MEj1`rg_2TL>$@#68lAs3SgrlpY z39i5>thv65QiEk0(Pc5Wga}>hj24B(Qeex`^cSO?>TEe7=B&BwLvXb)Y0vxMyIe{( zfQhrw%VmlLA2;ecvZBw_Mb8z3@!WuQMfYnXBwYhuiv6^4%fQ}Yx84%0lY$3nr_G7! z6FG0E&(HPi_BJ2BoEqqyy!TmQ$P1fMiBOS79=2$BgBzcYRA;Am@!t@7rp zUb_`0DG7zXDqNHi)|g1?w{Gty@65k#7HjxrAR=RX(PMvE~yUs@}+AzvS*VF zDs~UF=pnACK}qIPa(%T3o4VGw)K4;f&=xeipdI@=$vIgxC4sw&etT3Fke1 z-&j^p61@AG0K>&5VIKFHQ7*53Q2Nq3&Fn$!e=jT}?et4cnLz(`ol2!W&jgugXG_%BE^R^?#eDc>ci{ZZQ{v^vT)W}@{{Pnf|JMDx zTjw2^a;L(_Cd$bBkm$p4E~~l4v+SbM3Su0+%2V9u(MO)DN2X8|2eR+%r8G%;9;=`} z(;_q!mZe{3lXD@bGmh7Y8R+_=(Hr4wn oY2~8q%q!s=TC!O&<16&Jk4zOTcn1pU;NPjAPoKyBNqz1>4Bkvhv><(+=LiC+Z;Re0AeHKMB=0q1!OrrcICYHfgGwyu`#{}X}7v=8JmI>h{n%44icShVukL>&g#=n;lCdBCF z;>~@9uEaKf} zy%hc3rHkeuLm@hg{<*jHr;y*DXooZJD?f1%c_CkkUO4(l0e);=JzLtH1g3q zvf1`rOCQ@=mowK>6BED7)>2x?^UA*2Nf6^X;u30Ywt2w8SF4{H_0X19;7#+32VJv| zabV_-Gx*PI-xuhX@}t#rT0Y;phWTO@!mg7d^3X3z`lQP*|Dom$2A(bvTH)7GNBUZp`ut*1_5FAsX@*68Z(Fyra*vIXQzFx`nA}H( zwE|7wz{czkWfHkQAG0B%%%8LF646R@*F1VCC#W6H)LBq_quVOs2kH9+b8=F(K!SJp z%p?I6_ktcp;x7c?p9GS01`y)D6Ro_Cx*FHn@xjdMWt^OJtA=1u$g~iC5X;g;uko`< zVIhJi9-$AzJL$29Q^70Rd8QM~c+OnnL9GQe2*QA9WxR~0hL?aRZ+w*@kYkIm={8<)fOBOOVHx?>fDiYe997VwF=;1pi1DrQ z%iS$ZWn>J5p)INdBuat1vo~ul3w>jBX5M>)zeR8@$kUkO&25qAf%%cw%}Ej^_#;`` zgBvpOSg(BuzL|0F1#FUcvRRk-k3*Y*`RgzrJPSO#lB0JmCiyg%e^C8?f8|HJu~&lqPdSAf zxeTu4!vQwmf3@zJynSl)m7@q5F(fIdQ$#qkEF*t=Uo-cv#4PRp607z)t+{&}7 zDfEwN;sfN{INK=aJm%J3FFU8!`2Qkdia@j=XB)TXydE2d_jhc5V{T3VKGRM%3ntef zc}plOdHc@i+YgHG$K13fu_6(;qI@O$xqBAvU=WSc!maq5AKxFp?`@}V$De0?8DuW8 zb5}Zg=!M-2i!VY$(nFVq=qV{F%_u!%YuWF{j@%W6C>+AHw8XBSn{WxwNB{-)ux zzOGwXbR_gNIASn&37#A(^@WsX~0y#1=A&BSrVhp10;yf5!Rvn!J4dXgcZ>NeLf3`w+%kBX3(W8iS$SqQ=%Um8gUu>FshrSndz=u z6dyj&yUO5mxiP^%W?K4;R}o5Qmt;Ko*2aCU&+iJJ;HE#XQA>l)hxcC%Bb8U!StHD#I_++>H<)1V(0$bT4S{ zK2~^-YlVzJ+S0*tvzPPd z@1KZdfh2In0DSh_#lk7<&mkOM@QXpEzAYHcskfNDz))?!?w8wAyemh|6`4IaBL+oiK)21jKut;_}$#>%_H$oL7aYKapN1oo4veJGi{{Y&2z# zygHKdPD$p%K>onaK!?eg$xzwS#MXM`gq|fh!Hi#4B=0Fhuxo2|F!)p1yUy@Ki^q1d|>-$Z~R?&FMYx$HyG-VAoxfmPo=MuU;o5xZdKa$W6j6#`zF!z zQMOTHu5GokzdoBQ?er~}FeO)~9=zG?ndxSx@VX*LsI179IH*9ZAg4g; zE<9go)hhBT1;Z_&o4itO%-3))$sM)wtK0S*De#qwc5MiIUOhZ zh%}ZdQlgxw{DB~;LtH7xBgz>ztV@F{yNlSPa05U4_djlKN1^0P;vHXMc?F&v}$kkNl~jh~BBDder{PCfl#wRoTG^QQx3_evKsUxQ#Y+ z&%}W_bo~^bo&eTl9uQUWnw5{ODLgP2YHGGn?ZI8$rgp6Eb8u*;@!8(V z`6f?2`O3kH%hy26{rm6kuA-ksVF-N~pT1|U|Gaj*k#c1{T;P{;{ou&l^x!YSqTQd0 z>B-AZM>BP9$*#wfse+zD^`><`b<3Uy`&+(gQ@u6EYleM|gU3b8#>^BV8qinR7n`b! zPFl^Q2X*^t8@rqJr_a{5asGy9A7?C@p-i2=lHO|Aku4hxPlr|BfKA*uy;G#k_x8pNE) zj1>D0@e!&!Q^p^78-0vljcq|v*(;|ElDc_i>{NzjpnU1ggY?IXAl$7Ms*F!se51zJV z8a=&FB}DRhYC9vB`K0Xu7TdgjoX( zai`l{WhCrs&%e_Z-^l(d&GLhlPEVsfu6-Q(;|*U@{(Cy#YhSC!`Q#LeS)<+&QC?*= zy_X;G_4Z|YMe^qwLeRA6Gf&rqRK0`g#)A$2K2(*0u|OM8Gx zL*|d~;IG>aEC>Be2hWHd&R$C9u~K@b6ndS}K8TdR4DNs&W3JXL(i%&`$3uxGNL=Xe zW@HgurG8xb)vRN(Sf@f^7dv#4@YJle!Bc4hOb}4oLUFOR23_?#oC}2;x4gOBI3VP( zds7DQ(OzRq6OZ~uwhVYr){+33;~tH9hdX!)1$R+5{(QgBFF3=7Xh`O4m#G~|#bsz; z0=ab2x6daHd_s_+7C>>q%g*Nv9Fv!Kz$8rHy-Ze>_peWY92scT0g4!TH417~+6P8_9GqC2V;vC4U|mvZXgZXod5 zp8w975;!b-u~6M7jC+fxhcJY#KE2qR$`i?i(P>7u#T)tCY&IP`N*rsLElPi+qFrsa zZQ$O!a*#=vb*S`bzErWFb~-ml{6d&UC8cF8F>A4VB?{6^>6t z-i!A#wgsERdCFvcXIMn@O{+l>D%o%b+CHbrN}Jx_!D@YyGuhWg_f&+B*DKWFS_g%? z^=Gyv%@d{mg6Uxg)4u(uC(B@DthS*|zu2+8qCa!LWe5L%K3sv5w+2?AOa#9?XCR!M z9Io5nsgvPsC!R=$>WLrpPWmJ79dT2e!8ku%4VL?p1&Yc1--+-j*XtMK^dvz$GiRqx zS%b&C?rR#yxJ@WCy!mkcW0TNt4g4oWn=T=GO^5SWlLcLKZFIYL4u3}=St3MCK@;}P zxcyv{8}gbIkt4NXv^=>@UCEAhUD$(7qvl^RJUJT?ee%nmn~euless`dq*x$|7> zn}d?Q>tU@?{Ey8daIGx+Do&vdYAN1-Y8|UQ8EC;%$ zW#e1vlfSYNWaxGEaI)b_GhR$xj(7DBqUlp$vmXyHD`XScerf57euxy_Y%nu|;@oj! z%+QmsYz^6Evk%k#Zb}kXSoL4u-Cgb%AOe9;wxp1DutxjYZ-MYpExie#vb63yH61;u z%vOor*PK$2cJGK_ZdQ09VbW@q=KWyZWp7Y6lw4}^6r>K>Y9Zz~X$>Z)*b}1T>^?4b zcU|ry?XlHf-s*>*@}wYZUjLT#8a&u7=!5+DMB7yJb8ojFOt#Dj(;$D*p?=K$sV!c2 ztn<@Vsz>iU3(}nwpbZX;$d9GHNT+7TK4lg~i*kfJ!oO}mQEsxJ3PkEsFHEbcSW<}?H3GSD>sMzs3kMGf9yy8LZ z8*P_&AJd`_W}wY2KBHZgoM@dre%C}xR77~5Q>9RU&C1E1e!)tuvLnK0>USLi=VND} zUun?qX6)7?He?|)YTH(KnW|I}BT}D|x|KS%Zzylp zE~h=2)Gb|BLOh=-s&s-!?JE>DL6wIaz_*f=<9M^}5qif~bsY`pFUeyaxIypx>BKVv zXz%y1Xhd{vX^5N|HhERQ93z0(orh>3>!5rLuXSHm9Qf3xk&eo{?&J7I8;ic#=$kLi zc$PNq>kgkWc>(+2aJV%GU(HRPM8TRJ%{s!vJWZJXAXf%HpjJtH9yGbPHrg}x$}G$x z9Oj_s)aXug-#I&szmWNzO`;rMO@_W+YAb7rZWVjoF`dQAFW3I6SS?+)U?nD>mCVZy zY$5~IPBU1Qz}8E2d@J8BkzF&IlxT^=?IZh{{0~ihce)<7p@^Cf8aG4cXgtSFh>_;e zimgRrT+Jst2<)e!cDp zfdMx+E%S-y#+Z@kl>7e4Zhx}jd;0izLm#$v*l9s=Su4ndq;u16S=a-+nioo*vX2`7 zYUbs+XOov|8OrGKisq%t90_G3m+qZfddHeOkNT4Zxb`$9u2GVL3+_D7q_`=#lwcEfA(( zkcG-ttvTJx0~6_y;c&oD(gSpLGY+kyJjW&~>(5*yI_1eu_gO95+uK?JQ(C)3S5#DV z6(I$~eoB`X^*u`XftWtt-AzAhXxa9du%M1^PE$;s5keK;!sgu0{;)s#!hzifvY!)g zNQW)7qea>r2jE{?FYkFgY|jd;b>4(Vb2W)$z!arB9xe83;GS8I2d|wbwX_d2sKF+&6XBU z_;_ow$%j8cPvX&6t3)|zhF|| z`W_@!;Iy3L8qW48-&jxNZgeZub{Eln>i!HGaz^n(WlPcIR!EyUb-XDyTWl4A0GW5G zN^pi{<$cqMU=m$xOXeQodqEY_N%3~b+q%A0L6@4E=n7?$Dm~~qdwW(Cx(?m2+m{(G z84ChoIpX-RsUOCQ)@y!;&tdx3DmG}3R-S=;eO_>A=woGpGa&|?iFI75hdUSR$wRxu zEMGq!QdE$8F4~9Yg`21=COq` z>g`Isy>=xT_snJ{eWk7GgMmx^&!dSTUF*A5Z>DV@0(; zt7)dtyrLgtzkn+K%^FacWnBN^G);5RH5$r%Td-WJJrntt#b*B%VI*Fqy!iPUv=%p;6}ZkXsbFY9v2{6m?2#V~VN zDZlMV{*!WjB-{ZNku$g;?YWtA`ZST)IV2iiWg`UJBeW);`o#{8o=vTe!w0YMO-^ln! ztR{ssi3+*Nvc%YYkqx_?r;2IF7sVJX=lV)?AV_>Wn0$-1uMAgO-I=lCa%ASnT=8nnCEV8Fx0Jy4NlC3WyMzH+(G=t>cU zf^KA+ZzqR=LM==lZY5Nj=liN-%w4-{`B;4tv(L{n1+D$%2Mw`3+dMnv@^jjAPtro| z-aJ6S+)J0grwfL-^4Vsd_C+<9+LRvi04Wz%q2O*$Np|Xes1BX!ke*Wu(<(Bqmf6#M zm+d-WLLquM8+6A$>q&wkJq6QtD@eahV(DSL?h;Z0e4nL;5+gx=^CX*}7G)J4Y&<~|I15!uF7gTHsK@6(Ls)THWHVMHlhxZ3Jvo}(CDc2@9HK;_ zropP3Gq8&&5~S1^A@RTtxvP~u^$*DlmH8!+S*?#yyu%t|N^!c2pyOc8{!9z06d+T# z8DS48r4Sje(s=1B?Q+NAtxVkM=P^$Qj(vYb6KH3JZ%E7~S<-;LHZ4|#t%td&1g?wI z3VV}(@14`SpXEbn(m1lh-~&$q4mQgnsLhwbNc8EA*lMYD8-FlK>WM1569u{xW{S~c0NeoQO7AA|%T zMt%S&P=rxvLoE(lhs2BHD0MXlS#u1{GsMlrzRQwn&MNoo>r-3{mv8E?oI(1B)2Wux zuz)>EkGZ0gH(|c~B^u{ZATR^O04bD;a^ti)CsUR(q)w9?H#Iwx%p*!)yU~BR31T#2 ziLf1!nbN}=W&~7+_KuQ4plUOF2T_vCUD}03*NOh5<&^_jlXH`{E=fDp4kME6N>__M zq^(so$NFPno`#88V_~FJj4qvbGDXA2bF#Fhen^`<4jJU7OEC<|R|z!SOng+R{bI1w z5X8s2m+d?NWVT^rPa8 zAAp6-<4;^=8sgKYk%tTy>1k4&sHw2&buHIn$?!c}ww) z;hD2>jT4_xqTx|Wfmibf@BMjx$`@LLcA>ZiUYGngc_8g0lqvwgr?+r&*-5q<;vwo7 z(U&a5KI1%&0>}oir`TGyq^`d`uF0r9vsL>eisrSx8rDfr~_qJ8rrp;76orMNWu+JhLHNiSEA&W0^cCs53|K`+i z&$B>gPUBFSW$S_L_@0l#pRhaDw)hr&@CnQ0tsMO9`K%dFlw!$bseram+@n(UqTLdD z9)b)dA9NK0-ph}Cl+xVlVZ-9UR{2G>l!ChsKs@nR(1&)ye0A}L$tx|n_8A}-YjZ(L zqAA{P1ZVXxQ}sM0hq7cDU-dx_&CNPg>DHtQ=lSX+nR7C2sRyXAD6aCgz1~Jdu_CSOV_`o)lrb*m4Cn|9=Vzn3NraQI*fXQP$H zCd9BS**?u)6nx;Ah_zTJs7T}&J0AP-C-r+n@{I*gFlMawG1nN_Q@&`|us_TCHkBgi zD!KJCGex1Lyk=V;(Q8ytcr4x07|tmC_I5_x;^M)V4S}#{`M1MlOm|h$QP~=6<417X z10IK_rw8>ju5x!!$x+Q($cXG4O%z_5c+2t4Cl(}rNBoMAdt6~}S6W8Ad|jr4qLG8m)ZR?8jz zTnEt6(Sg7y0+(x7Fh4h3)2p!CwXAedE;&jkt^AnIMkN=^lsAZ+k3uDeqaG~pK9w`v zG^g)c)OYN{b}icXe+-?bC?6rotfSi4L{m)8xQKCe!AGsaN{gV*3*|ryMX^>4UCC$t zl)q^OD)v!j)oF2_k_WM^O^n7z{8+;z{0XZ(FhbC6Wc`ghcHsqdyZrL1j!jPNhmGlT z1Q0*vRd1uan)OM`e=|X0kC&EiPqmvu?!(7c2o^wH6_NwecLc)%a&92!vz|nLcw+++ zvq?99A5q{OW{CD(_$;e?urVb?qau-1Nj?a)?G%Q}`GX%Q^3_E{P#)tZq0S^u0_X(L zsI{9Y(X88b(71PijIiAT%cv@eBy4Bog)94XF2OUqlBzg>S9{QqC1C$^yQ5cN#-8rA zN~RDI6~^`#`0S`3W74{?0kUKOh=9z93;Q{b=vB@fZlN74H}tO8sGY4@2NA=uEe&})yieEI%qq;%>HVlm*n}&qZE}JkZyF9 zEYB;z=@?xNb2^JrSGoezf~*zft+!s~x`vQzc{(M~73GvXfh~0incjR29x-a{Q$JX{ zAFpr9#ElWs&2!SIOLqsa(G_^5Q_M#9oU;p0Y;!i_Z3+>6wlQ2Ger}sKyh1st%3$1H z*~e=7O`~@<``2G~Ln?21KQNVP`vP$+c-Vn?=ks@i<@I$Jq;*eGhI$*V?0P z9&|d|d(bt*SMgx~-3^x311_Rs%@aQm+DR~-+DQ-JM$qnyya5*xwPyLmvJbzDVCbdi z&=sAXO;N1ylj{!0MKzSU8Z?OJC|FZ?k{Q*X)X4Iy)&Z$akmI+%Swp}6ptIwBmRrs+ zx!*b@@SfdF&7M|tf5t>RoeHVMwy%CUYjjBbSCsypk7^7GE%mfKmcfxqliQUB+3hx8 zvRx=bhIu7lOT2eFr zp#V9o)vdi8zeJDVssy_XAF(CGX;JV;Il<9F>87O)W~F++v>T%|SOl@1PTO#op8`M+ zWaq_KuE)^u{x}hpHq1Ai|E_i;q^4K$fUTXk#GzzZG#Wke{+PPoZl+R|p)q{pb=dXz zB6OuyW`b3JL{UML(Ml8N!bI3Xzo!_wcKF~Lgt-?3QSzTiTh)hjTEuuXx>illR0C;Y zLtQi?G3`F4DB+ZL(U%?;q{*DBA3AN#w#;dM`%Z|@x0Buq)q@@g!)0O2aSJh+xQ=HBv39qan0e&| z1GMCW*yZJhuF_plM!Z6__d}=g#vTym*XNr3uA^JP=}Q&Lf+${gyc#!+tox@z?Jgcs z@INV+{{Rq;bplN<1^5&>*j>icr;5J+pB00D48%eMCC}MM?Z4otq`5e{z`q`8LJR#oAH-*cIlB z31e3PmD3nC)*yqw!c0xm(XQKM$6sq zF$3|G-=!D!D0T-h)|kF)Z}~yQrHl9$Mp?JMQyaa?Q%dFba2PArQA{H08UK9NY%<;8 zy+0(+HBkzHe}&On*X6e<D(GL{MiJ81^ zqmIOOI$8K#7CMXF*@Z^B*JR zKiDO#HwcXUk8&8Dzv1gBy8;EbudXqM9@enaozg1`G=BR)iRR5mG((~Eyw)l-FG`bK zOz0X)x7*J6o=web{qo&x^!Pz*H4fAf$xB8MOswe9LiGNKx^m*p7Fi$^)lC_I30stv~0Wv-ck;r`eygeOGQh7}?~3p8C;?`PKZz&g1gZ{dJo;(+Y}; zcpyh#tI)tber+QIi!i9eRJ4SlXU8)RYa<0F2qsaVaX^()Z)T|O5V=4M5BwsEskI%> zRm9n>l@y+fo(h5`PtAS=-@u%o#dN=&mz^Fqc!FHrZyNU}?fBhRRF1JI7{onjwTG+O z9?+LaP&3do+yIQpVZE{y25;b}=FJ~fkBd3!SkbOLn$hWh|vI>7;JFwIMGksHsA%!mttc~cUh z9c$(;tCMp%FgK zEB1@*D$u-A3TkJ3-F_n%LB@gPvmId_E`n%|gv;~-XW)1}#kKEBpai4#u{BbTql5y^ zVdOFY{^8{B2ryH*skZ)W>G5VdR}*(RRV^t&fWegVQleJC7jInCHujk+ZC2MQP|0B< z6I!`_JQ(3`UN-=6nO^}~Z!8W7?-7VwOKII2JD@~iBq!A~C9ARr3Ea(AO?y)Wc`A#p z7HA(A7Q-iP=ayd6gW++plA2lStAq4T4TI+SgsVX0+24AobnRvTsJ4-4@O7XC^>uo~ zc9D3SC)@xwk94u)O z$L8fx6OSEsRWC!@C8{P$^r|BTh3A5ANs1|Ss4=Z4n1x>nYys3@!D|e8KNF{~6%C%o zTtONKkt(nd+Lj4cWg|}6jrbv0H@+Kke2cAar|xuu$~>`;=?Ya}$(e&T&vlVag=qlP zqoQM{JFYVdC5$z1|DfkL{ubaINIBnA86g=|VcA;_NcV#Jk46C!*}YemQ6FIPq~5^* zO>pk=@M|~q9Lf+Jg~G_zd-mp|UZ>W`7(hBfbB|i+a?W*I2>l?q$T~eI0FBsW@P(|$ z+|aX#)m)1xzHz`yo?LElPR`O?Q8B3x6PT_MbIVIXj1;J&k7v&O$L!;mYk7^4+a1hK zfS_zSt_;IWS!ai}K8;lzwho{ZQK}VW)NT=vJ#P)gYz=F<(Z7$lF)bCwqj6XDCeyn^ z&w_LB<&t&e5AL~1U_KZ&7HS8I6KfbuT>})pgjDG1QSNN2;~V?q6;p3jVIRomRjvkd zwPd=r*MP=tva4%XLODHGeDKVQu2%Wjj#t9Kc`7ABWanKN&C^!ipV&vc);m@|R-jLyFD)Z8riYr8xkyL@0(#WJ}y@ zcl}&(lQ|vk26AbzDv0ux2GX`i9tf9_YN5?nIeF(&8(^*AyjI}_s0bn6AzQpY=}~E2 zLq*y!m4mF0k3m;dN%0GC73kj%=X&WmZJtQBT>~&4A&MZXH4Z)11eJ~f(z2e%^}z?y zsLSdkgEN#RI3w>xN@1W>RPcTO124$m3tEQGRaUn=#7UaZT0w3x03*J0?u0U!f(Wq24pa4>)#-rJ zrgV6-94iH@j_b`CwG#bM;{4Rh_MG3#vwNJ3) z{4OJ}swW#3vQbmXu;>ffPMy6i*)^nkrg>NznRUSP@S_k*`+0$%qh!?BbQ{a}rSnj= zQOTjAtV|a1cB$zEED@G8-u(v%>30W%L_7C#_q7MhqcQ;vR;9+|`m8FxLS(Y5W90``pA9H?h$cCM)BZ1swHjp1Ql*GGD~UH4sI(C-chU8*g{| zjwxtn71IptZK`4-_x<1#3aq-HqLUdK5V|{UYy$W#s$z)My;Kq(EmjY8yYz5UPm2_p z6|88siFK!k7x6LPjRyeX(+0h<3$c&~)1Kc)Q% zCG}svq}%uwXEgYczW9G%lt1wR2GZS9!7EdO^6v9}Loxd*NaUR^RUs%f(z`E+AS%Ay z08bzzsccoq1 z2x2iAR{)e$0XSd`Q%FW3`qM;_50p|~HK%O;1k6xz1ww!^+PT4WlAhcA9j9S~SK_{( z$@=(Lew(3tM&^H_@Bghz+*ZjDZ}Kj$v|R2@wvr0H?ojsfsOo5US+B~Xf%nh|r4U$o z*-CYZ^3vsNWdI^O}U$zP6s-F>DyjkmZjEW+yW4QzbfjAV#3sG2Gs8WRiy3 zE+7ezO4Ob=w{N^8M^J7+6N|t;wtEgHtvCdB6M2pg$g@ymC3^YUz5fzyfV<%rk+Fs z2j2m(kXD?_CS$-$o^;Qegfdb&@-lx`ad%Ua3$#50P{q@P=Cjk|eIT-s?eA|%3Aq85 zs_JuK2jC`OFz-deDdYRqNBT7bO{ZA9<2f>@b@M3(CKcd>-COkl7Ih4HGEd!1vfCw+ z=%Uk|WfNK5mnw_|Jd;ijQ{9HRYq?1G{_>Xz{A6#-5zjC@E?Q3v|DBV>ZAeG-wa>^E z#aO(cLH&~DSQOwN=Bk<`sWQ5MCi0;FLJq208GXZXleyxl9}I64N2U|F0d|OnRPUw4 zx!Ag{%4PPVVpAT=pCoPK9&00Pg!qe*enigJZmZ0I;i!0ZY5?@GaH_`IToR)y5WlZI z;{*WR;}$Z1^2ZH*FY)Dxs73j($ulPpK3%Ip!;wD}L!3>kf}h5((pA+bdIt(_OIs%(Zc3QVl?5woyS=Kkh>HrWc^l8q#T*I%!8W-1BtG5g!lF_- z!%-yQUW_br>%_lwd7RPnmq`bsr#{Jq&2#W5BariL_L>0OEH{P%CpwViGw`Hj9&j@= zID#6Z4uO8uDu})A5Oi3u@<6?C4{)JfWg#ex!0H0P)gIue?%XMi%uvgvqvIAUjmq98 zotPCb{s+qRKh{&-jBfyLa`;vA+5=z*LhUC~J*REZ$m(h_2zCT2D9^3T?{~QMWIo-+ zz~@%+a^L(f#5q2go-PDPoRu4w!-g}QfrKQ;%QYIeU08>gp>iC)yu3pSYj^U%Qw4)^*`^@@d3LMLOUh=h=q)ws9Aab8)2!a6H^ z?bS2v*73W-?lvk$n_6SmUPgY~A*+;)Yr}xMlgjX@D3g}YCvD7*U4=9!Un6gSc;KVY za`|Q+Kd;8`bJOVU6i|&ArOy1>^G+5^uWL zbf*=3Vi4p;iHZ8|Uqf09*sgXx+GecSh_x_dBM8t*GO~=uuE918ETy{Nyz#1g70cS? zzn1F@z%_dSeUaj~EO<1|ziNBI{^G@j1Pg1{$e#v$#h<+uOChmSbo}-S4nWZ;2wUj( z;&aDS2^t<<{X(brRv)4$p{Vb3L`gL2q+pT5Rg`J{{2MdC6`I6|_de_aP&nnAFgpK> z_0I#91U`1>bSCXIHpycFs9Yi1cwD5}dzud27t8$!-{`(8H#V zyrwbogUyG=Irb57Q=O#?X+24D;xN2Yy#}wZHn~qN&W6KTtq)d*^QZwZ!>N7yy3UW? zFv@}-aN9=ZkUj8BU2I|AF-N*t-e^I4onJ)aYQxSHxWv#bU3&jdr>9sAGfKbCHP4gv zrjaaAy)#l}>|dUarX#rx7zIEixgmr-jitZPJx>u&=g}d350PApr-~3VXOTbB%dCY0 zfyP+JV3;wSZ&afoqr7R# zMcMBEnOVqSBLZfkr0~kvS_G;77NaUX1RyYc0nZ$67q9%{$@LpQc5MZaE^vn~$j5?Y zbi;o>2cw)Sc?Wpv|ET0-cm>AY4F3f6V7+I*QXaCAjolXe-gms6_#YtoKnfw%WBpeU z7CfL66rcebm8O@)z}STRtzfzM#|B9Y2|}HbcckorvR-`{pYOi~)U7ceptpBG!ZDiv zSYz{RzRiFC$T1yiDm;6(;b@<)(!rG*W`9iStF->V>w*L2M0>(zLd??Hx~xh7mK&~r z541O9DF$^w>xC|^=e|GP7H;VQeYibP`aG5rttJDBnGzi0Wt`~2{tf3A zU^p;dhRl-wNLP6PU=<`V^yPWnrR+asOwaL8tK}SMOUCsBUpm!*+dKi@2aWrfm$>wC z0$dvr_aDM>^qlVSF-Sa%${#XN-$DZo(^HJi($7Sb=VD=f2f$p?Fi;Ar%1W&I)8O6g zMdLsR&IKqzsP+08Tq)4GI?gf3L@+r3Gg5GL^^6!HHdis(^1LrbRr&*6Oyxr78=z4F zG@EkZ$$f4or2OOGfcD+Y-Q9hEqkeNT_{yzy-lh(k5Cin@vJa3VPKXLcGgh?4!NS=1v7`Q|#aGMdj*cyNv0k{M z`~N!0E$&LC(pOq2gXec{usO%#Ej56qH+fy`&^TGZDUA`3VeiBss_e9ne>0z1w}q0C zq1<2ZfsAlN%i7J@Sb*Ymfdg9Rq(!`eZk-nr}~`E2`3v=V4eEQDb6ZJ#3kH-r&%>T%Pd=`bq%0AP@v9fcx7nf=EB{YFX8 z|7fud1*N<@(i!~E$smJ{a8K$>lPzGvhwCmyUH$KH;a08~z|&MijyJ+(H_{9Q6Gt%=m^niOV}n(Iqmktk$m{3Rj`8!U zwy|@;=#EG}gtu_In*(MK+*1AdbZYMBef}x8;meLbM4Lc|&M1SXe`@-i<|_;^40`l% zR*wTSdQ`1m$Tf#+GJpFl0Q(9>5=^S^My<&gFcF4Vau$#_EC5(qQvx`4-~eJpdM~G1 z3BhG>$-r3qxsZ20F@=0wzivvGs?wplqKtDwlp2sc3TF@;q7Wak3*=i%|3L5#Ra#%F z2sE>wUW{i9|9803DW5?ntG$=*>Q2DKu$SJ$!x^9LXc(aJ_w4+S=lK6-?7%{*{G=Or zfB*}j=d;!D;wx_xuhava+2ozF zJh9>??3xKL823h|W8}5Q2sP#<3r!lJ4w9`UW+$s2H<%dL9+#%jUTjOvzh}=KNFs2qR+EEFDT=~n*e}4Fr-OTMpdOt zet_aoMh*ZOe&yJ1PonYdt3a_ZLNCBa7pi^iQ+A%ZD*$aRSuz5^8-;&)1^^t};nwHx zc_mB9bH<$r5!SvADd zLlNGrB?GDQI!4X*HNXj0fNiA$w4F8fd{7Y9cW(LRVOl#MbURO|hF)_aPll!eE{0O3 zci1g2)q|U8Z-)(Zll^d6`hgGSSD7t!nlim=$8)~qtq`5+7QmU2un?94?0^R_avX}l zN4@eHg}r$TIk5?f->w{{>N+tA=j(2Z}s2* z`$s1PT5_rsh?_FdP?T}S`--8T>B|1k4a;q#KrMN&zSABQ)4XMLu;AOE0^6jk=TtX> z)s3RuFlz>l545G{ssRl4);&oHI9bKoc*-WCA*+>!fkKU?)xeON2{X~wp2^WCTApHn z88%5>!qEV35)k^3*;W~LV_zF|>_c=2jG<73XE)`Oj-r6^ZW>yZ4t=25N}i1nZ+?8^ zcRqW|a#BzNWY^Fpn|tD1KB8NT@kWjaL*J5;@^1Fb((oLywbAb&DFBOp2UeYICgfj5 z@ygry5SdvMaWc?@y!jWUALDr(4lrV)Vq(n7ZF2wfJP>PrK)FTt$JGjxSb&(c6tx6s z^fnKR>9v@%T{54F4b2~6pn>~A-n&bHaeR3W?A{_vGSWbNT-i)aTs8fXz!$6bQuysy z2ij)hBf(qj%|e-l-SYVl9B(7j#(a^JxlLbUbt)|RUNr*wVqDpOCwjn&=yXJjfm|Qp z2TQbJhId|!8(V4pw?hk{kP7O7Wt!N>iLJqF@zA&hcRzz%%(lnOLmdI65Bh)ZLSJc# z*H#pg;5|l`w8nMg0=<6upF`(5HIqg!h>0kid>!eIFhChyNwJrQ-5g_m^Uw9=uf&(3 zMiA?B-!*r4lvs!TwMtnedFE!k6a8z^-hXNh|JK$5rzp_&mdZwPA6TRW-lu5>rF!(L z?B@$e*lM=nos@!mMzSddO#kNNH3H-AMxKzxC_<*O*ErQ@RUq2|naaasM{vJfMTkEG z+20Q1-^rA}&a16@Mk#;2BJAG+A^wG5YdH-Cvk$o5?(mxw|eQ{KKTf=!qX5%h(~O2hVYA^UBm56Lg+@gFvK5n>|yue zn%%CB&R=x`{~e|kPb0vl6taPfy7$*C+y!YFep(qcxobrHTj@=_i2z$%sKji|2XFB1 z=LvIogS(MW&ALcjmt$L`fnK>MKdMzQX`_(7gO+N-fL7(39G3LBX3VT?xf~#_y73;5 zxwHR!*be+6ESwwdVd9P{=|8SaK_wxp7X1Eo2LhKK1I=zP?w(*X@YxGtI)!T8aJkWk zlM)%Vpf4eW(Wt1N^870IuG1eq09fpA{(7vv6R**WXU+me&vZW&<&Z~tX}H8cr_`hhalW%SU8*@9| zXE9&B(5%R5x_O~8NmhmpH<~B?k9U;+BOQP5sOijpt>v47x1koXB}+&KYV?4T+{paKg@Xv3q%P;hCz2o2@C^KQboE&Kw_k$2h0E!38g_8 zf`D{)3`H885+g>ZbPm|4-+Lg&{oL>GudCiYwpW~Zo^zh(y!>`nOp3l4%fi3BO+7X^ z(Ox2x`M=V5@9^r1BzbkXJ6_W6;OEGYZ>c+$1>#TfzOH#5HF22UPDxKyPit0o$!p1z z&ATym<*)DY(J9<6mu9KF(^X&d3`hrzVQg!$&7G>f&UobCmjT#puEV7K51c3;z00a= zGm5_AKK{hHa&U$<;ZVU)FjIwZdoqBg)c%G@R+&Z7b7 zcnUuI`UMu6}vfBjy*p+Pd#x9!~JiYfdrEqd)#8MivetYKcXjo2(fijy6dm?%V-Kr}^ zIT<7;dH5~bm2Wcl(OIMwE$7%F$?TZ)g-%U@Oei6vIVF&HIWw9VoCV$0qB4L*#g2z&%!}dVkr?NmtgA z;Qqt?wkpL-dUeOnJ+;}5y3l@KOT4{kjCilBHASZLJuBeV+Tt=5QHc~sJAAQiTF z578>MnCX(zz)_Pgu4#yo@d83!M`I_)zgjp7FAPBiIu*TR%217{=c6e9Q8zq~Vc=vw zthv~*NbD`HEj;Fd!LmM}>$#h$h@XpVy&BKYQky9Gj%M<29CM22+ev5+mg2+Cc}qvb zz%cy?X|LR_ux*+KHk!f7(&=z3kFD*x$@XL^!(R7H9&^3!m;VM%(aP1{$<}#?9ODxX z3nOIX@s0PaXMibA%eJp%IkI`00dFP8M-9}PgK5dEmp*^D_(N|2MY<0vTsvPNyRjm` zv_XMf0urWJu6i-(&E%rO*e7!L1-|B@M! zOSxl7zAIVSwmr1bd2tBUzaff>8`mrDz;_hiD~w8yt+U0XWG6}7l&!xTtIVba5uRhn z&h`~cdw)WhOCI_FXBCHjxZXP3#xQ7|$8WWWE(-_uJ`{Ywp61Zk9!V4l@`YVDhT1) z+LBnF$|x*4rK3pBQ{XZ_%C-7VJ>Cs9QLc(#$(+Gk5j?I-==>ic6Y>@H52xEV$ufSh z4GsMpSpdxw%umwh<93Q&8o$#QiXCS$*A1NcvSJ85o>GhRkJTy8~ry6YdCSS z5z5KuuqpaCF8FCi<#Y~Pv1yN=FSfb*$yHBUpxRY3ZXH&Nwe;xK(h*QSEj~}L(l3AF z#B_-)ivA35y_8*kDkf_57kd0d5JI)$0~d|lke28E>pIC#cyvZT^LA#WI6CSUt5(5` z;V33J7D-v+M&QwE{OSF=Tn@WDofc>Y85%}`$#PYM8n(5FKWhfRRic~xr_n#agr@}V zq@2**x}HB0W1{w#t@!OzZ9#<+pC2&it+?%OiQpWwq{@n_(3nlp0j-&|(l3*D__kY4 z(cIr1fw7!vDfn8v=~Xq`t~^sI155djUo5;7zgw+DmJ^=Eh>-vF8^13Y@X7yx9IDel z)lp~OIxBP6;N>A#2YbOdyC(P3BRUf~jZQ}9rgZ!ph%xnZy_1&%BK%`9E@)k73W8dh}LHiNNKH~*<$?P{8M}qH4T|f zs#e^}|23V|N;#hX$t#Cj-=6{o{`e|e*No$(;)pa=E&+I7%Dp07WYJsXwL*<~IE&ot z6w8%D`$4g~IK;oSiJK!H^?hcw=HqUTVQH5A!~Yfy(zRI`0orXy%}HDUi?)hp^L&ql zEmaL@_LeE*3aWw=iYm9Styxc(S(1NJo*BVUr-YBfqb81|&cN<(B5)mDFaMS^xE`WI;e zza6de4;cRKp>|I9ctO$Dhla0>Qr(rqL+rVGQb|<%8d5d$Qvb$Sh?)&R*C7pExctcO zdxAWve+vin0-doj{XY^xKczr!g@N@;7~#xJ#0v+(oNsQ0vzr?oLG%KXB9P)Ru?5(M zXr^j!?x?%vrnw%2o%)p?knhr(4L%QM-6XawsoAI1`G;k?uNj|p=Ck(&MnPjA1iL%r=3_52@q{&?dzJ>Q8iQq)-izqdiyU+~=UsPWc4oMN9>1K0ReF3oO0S zqLyz-eRNo$;D-bFke_q#?#Q^Sp#?Kw(L6Z66zG?WNKBn9rnA?mL;)Z`Vx-4CsyLBqAX;+XzkcHYO}x z#(O`moe^%GBr04x4>ObU&DPQ@4V;8KsT`TolC9_cP^k}xSLnkZ1C45pDTZ7@DYE0s zO~rx4xJ*9iZgN_PGxNezVsz%1o*L}bJ#n{|0$KK%N0Ni?rPJnC9594wSJE8H**4=p zK-g~qveJ-4pBQ$z5QcUqA`E&LW}6DX;_#Nd!I(17>5r}Y_{R%uGoc=_g9}Rz&AI%U z+uc1uI%#k#4wTqRYKWa)&veo`1w1J|Am>D*4YY7}PMtN_YOQMHAWez%jUNQapEeN0 z>YKT$&3SC9I!8IsNp>AdD$;oXohvoBkg(nYDx5&skAy{{@m%@}0ieIaG7GWPE7V6x zqoiIlcb7U@c+$O z#m}*86JQpTI{0$WU-y0XW{WgFS1-pHxef4(^MOXxZqocgWd?v*aG5lu7NFc0c_9o8 zV!8T1CTLNjfZ2j3U(i5XUe5#u1;JW>_yiPYWLs7!fbq;#K07Ce1>QCpC>-ND8?mtv z?m1CtCyO7EBFu-_RcQoHCIU#VN7Vb zha)Sb$lI~}{{YKBwH5i`@W`OdmzCP0dapePGfoSni!^EYkIRHk-g#bF22|&Ifi_W7 zQiB9gaK-|(2~$#Q9Ad4yX+CsYpzkY#Dtln_712L&z*(#fwM; zTA&38f(6Q~lh{g+b-|zP@&LNUh1vk?r180PyW2BYKaJhzZiH98E8-i3_w+UZ|5+P# zUvjgONwNfwK;knmU3U&t2@{AFRK0?YA%aZ{q_bu0E#M88u}CRotRs z-?P0gRV%TA4~#a~=H^~p3>2*VS`6?s&x0e6xdZIsBA~ym#S18u?#)ytalnDj?P^hB zV}MojISc6aO0K>7*=8>H*q(JmI#L#NRKEfFvIqk&&maa;RC2SvnYwLzri_=^EyLP3 z8Lv!~3*dKng6S1k;j$VkD2!;UCI?Q;PCzf#2DjV;%}Ol#Gw=8>7_SnZlBfa2_@MQK+yl`Q5ka$|G3Ss)X*&-bmC?a4hrJAoBxZf4;-+3o#PjIX8S zwC}igO@VsYjV7Bbd0gtIf2kxL$8?VfgkVE`QZ*ms?S9T)-&5!O)#tnSNoJeD$kRbv^U)n6 zsUBcrvEQ7cEDJDs>pbU2*sppTP-3FKblLCD+s}3^K1yGk`AYfK`&|)S>pb;^)|Rlk zt{lmC9$S3`Psg+slZj2<#GE2%T#0tkln0> zCTbW4v;3zp93(3WIpUf1RSuIKIghn|mVf%QchBSB4783oxyGKjq=jH6trr3x>Bt0Tt4ka4P&*h4#%b-2j=RdDV9&AZVj^=|)`J$0Wq%#=f5cyp z20(Dxdx|TvDqoNP_VCE*lXt2cuHWqJ%wHYgygy--HoU?jwz8KI@myaL8%^!~APPlp z#C;6QUzrrg6?DUdzg6|DtxL(aYD6q1eE7eWK!r=tW6g#{AIr$f%F6IF0G!s8HaDr0 zsJ^2D3#cfL1Q9KTObH|Y0vP}?#$>a?ZDkgiIGaamb&eteVt#pib^sVOyQtZFxl7+E zYA#Gy8))!fIY*vz?;|tAAN!c#r15*hjqq|{6>P5{Mq0*XV>CDhvfQXaxjyq3PowH^ zRwlAu$xhEHDsA-za6sNSZ)t6W;A+6P@m=Bn3WAWcAQFEI;3s3ffYDezZxmRfDjhEt z2Lf{+fTnyX4xqij0B#)6x}LsA<)@xA@LZ}THE?Gr{OC-Ns05tw`>!M|Jty_}(H*q+ z?hdKH0Y_?c1@;p_qL3V$zW?C)wU_WhV)H9S9|z?>7xS?~F6qfNA~`o|G5%SFp`-4P zq9*J1rxV;m>*j@VM$8xPLl8hp0v?t8ydwa|=^S^Lo=x@tGORm1f}R`pIIHx`S%9gP zae_=WMy%DmCDMd6TRw3LKiks|K+AE#2ZLfb8+EmmyKAJ*ufffM^$dWYVN!W>io{{k z9ss~;%}&*IVCDr3fRQ;sTWAZwx-mF1v9@>(>}T|?zs&g#H|m^du$GDzDrw)LwJ-8&O7Z)daEhPfldbdfG@FarriptXbWM)k|HM9&g zWbmnKnO=(&-Z>=NX1aLglu9Qs_jYlbO~CE;^J^?-kfj$2_n|wZP(z-z%H8dS>^MS; zMI83@!+6wY-z4{`!-QD@Jj)g5**i>&gWKiaTKxc5FH4R3#?nG524Rc;z#)@L?Aj$7 zL9k9h2bI}mh`Cpg3=8<{)c!a?L1tvO=QvL%1$5)nwBdT07XBi1`C{k7 z_P~P1?y}KN^YqYAyylq4@DU?z2jbw2(JqllSNV>9!&kiGy(ZaKr_pw2*(@*u+`xgG zb!iL=I9fO`QL!RzL3W;_XVWpB6C#BV z&Y^nc-^@#5+*A#QG>gkL@Z$pBrE7!B;l&x-4LoZ?rOnt`vU%mPUv-4o%{!;(Eb8#;lFywuZa3@RgtoqwrcMz-At^Zkm z$OWI1s;3<4<{AB0#bOS7>gkRU$C%1@u>wegOToLjsX=_0)a9to(^_>RS+Ztt1#EQ` zrqtZ~KBQ()(*C#TRF_e04MTQJ@Kx)T*|ML4iX@{JfZLPf^ADP(E}oj~K-=w1*k#GC zzCWR{UQ#cU=29_w4t@?W-70vX-LK?YZg9w9?+FX9iKB<*rptgBd**$Das!u_%5~|9 zf%PgEJal(*HzN+v2~9ws%TMI@F9o_}sI6Tb0p<{8;uKykHoUpmi36tTO>?QaDaJA+ z-INuQK3r3BBKaHb=AYNLf0^Bx1-7*#<-w_vb zu$UaMv5JHHm#sz%8_Zv@nuo9;wL;N0;Il#PT+c%)X5b3z-?girZ?H-iz?oI`rWY?g?v%`^L4} z?8+bTBje)v|6vaDsXF|(0^5&<+&s2~acpGJxvC|*{D@?1d6kUAM_A$EQyZJlHpg!F zo5gao^Lw1;@od`@iEy`%Nx=QK&B=gm{rq(I<0yrv=pCd7i*k(>%~ckBg&t@?Bo0q6kW2x z^pHB*6BOzG=z_WQoL3CvLa^?j)ioyzeRJ}q$S1d9L zKG`(sk}9oDoH7>~lxl0Q0}J*X{^k0ApDFtdGPas`%H7yZ0QeNj+wlhO5UGX&hKo)9 ze5(Vxu&nEJgC=Un`ZDL6}?Rz13viA-?;I7&!a;4(cc5_0OcIRMm)q6Ej z#;37x#s57}!DAqbm>)M|hWtvBE&cr$lUi`$*>?ImCw64VdzVFLrqa^+j6||vuUn-N zHP^B7G0oVnl)gJ|#5JckYs9fR50#exIo{3dKKM({yz<8%p6ZrAbyS^wJ|l0_DTz51 zd?_az>`MP&MD@*eu9(;8skQ}keUXY=appb3`b2dLJ^iTg$0MTu#z(**9aS>ENbSSB zw(jr=a{it^zuA_Gi=yR}o*t@qq-pDeb=7&#nA!x)?iZ<*Z(Uhy^e^q2uOyXZIjl{Y zMy0HZ&+xV@dM{rdebMzVVg3Ou73|D=_&QNg=_(BqS&>>?Q3ayz5ew(<9#A-j8_azLED_Gs$6A5h!WW&?0?^qhxjjyL7q<~kH=mDMO^VVaAe~3$l z63W}?pc5YNWcSS9pZhk>2W#)9l9lvW=d@=kRMWT^JPjb@qK2E5Or@x`mIzneB9_%z zT-m(pvza5>Y9E!1h9Z~UI_lM05zFv#okCBYg)}&yUR4jp^!LBMmlI^l$(G~A9rlls zcj<%?7TVFBsj#OxMNj3%936@iyXGoizoT%8L!D`oHY-^b2uV>Rbl}2WUK0+@{d=7@ z7s;{d6qHvlpX_D>QU0SiTwPXI2fdF=z9y2zWXlsFJu_WCcgxs_GW1Dr8(Lr#V)brZ zL~N&|fY!UOs6@vSM(_*NOF|G6{s5to{Nxkv-rHT?GFBA%wiX88yTZ-Nwi2Cto7hOI zXuLpmgN5Ej%jd>dx(J6_^v80=o`TEi$J+$W28!(U^!1|5naBT@&uTAP;{cZ&qwCHE zCmDmtj?BFmXUuzf$M)#sT70Vr3%PZ;4|wgl9M>iO?M>kwN=-m5YfKszgW+KaT*S<{ z#NO|c^s<@}Tqoi75msrGg&z%4+SwRow|y<7m$L8WjMKmuP_M3ZdSxXVB{5TZi}axNQArWRVj94%SE`D+$p0D!XuuoIbw$y*PdJ*+Y~! zA2f@OG}%XZ+ws3R6b69wML39-y}th@C#^mINVuC4bd*EC4o=vdI7h534(Mu-JwT>8 z-eBRBGgkax&dAPel;~`(!*NY>gJBL|_O%u{cb>*gqi=4CAnLB$RM3_v>(F>TYGZUQ zKu>h{8~z^Z=5I@Ce1%GI7b^Ghg`gVA51l~17o%gXdwS_Cs3Zw$33Iqz)43;J)$UYd zJ!f4iGX=1^&rl_Zt&Rb4JDtIV@q5m>-y`*YePKornf31ZuH7j~r&~5yB9(Z*SzEhr z)-RaQ4JMxui7bivQIm|b32~cK(sz0X2%%Fzll>Gb3@dJ=cdj*W?6*iG-&T`K1nXA_mJyzpw!YdP?@Iex9?8AiFt`Z+7lfo9li z%9)_^QLJ6_jL-9Zxz47#L3YN)N^m(Y$!KF#Y`em5MOIZewsa%JwJ#aXWNU2^=7RWs ztn$G0&PpJ2CypjblX+WSa?0wJwKaNEu6*UWjLS%i)R1JTF9q?#aY*0doPq+Nif?rwkR&J|tBztu#7R zK2UWBDquNg`?T$~wQuT!o)WX>W1Z%`c4cPUx2^q^bo+{=zmzso9S#6b&<%ONq!_ty zmYjAdT~q> z5idd&pCmz@QZ@Qz5rRR#r1^IeVVjcmV)YHpZ;vBx^E7{fVpO0FDLoLtulKt0&=%w0 z^JVcSdOLs!96?IQmg-@oHR`xUTZY~)-PkPYNPJTrN@WiL7C6rHaPa2}=fHV)2jZj* zZJdrPE44~|_Wy!uPG2?&Xovn(^Gvp!Ko8q_ENGU* z``@?UgJj(V&9M6@Pmo+0PLy`8oM!ljlF} z@6?I`1ztUtl9e0E_|fRVhs$|`-2B!Bz6)4PTvn;A{vt3M4#&?tFa6U{1I}lb1DS!h zpXB<1R~b}$8>kDaIVqPiX57u4sRF_&v4TFs1L6tyc#V9P>iJfRYBSR@MItazl)XdH zs!-ob{wZ*bC1P6iw|qQ0Y5W!W5~mEg_|EM+x2@y?|IE~tt}~|kSg!p8>2?9RaRd=$ z`A>r`3h)|aU1hQ*S|k-G(WV?6=ui73NKcNv`v(5isrj}7w3Pu^i0LL|ZypdS8TJt08!nGe>pLD-+Z{HGD%hER7+=?{ipX}Kv zw~^GcYiI46SotAnEXl@#GLn0hS`Xynb;e;tG^dC zAsP_tS^V$^?r;w^=uzGXjF4H)l5t)Rb88ax+H)Ch;BS?8$n9C!+Szgxz!x&~G~VaH zp0tVc)4iAZTorPCT*_dn|Nm&6|a_5ZPI#^)A>T1@JkyVNlBzd!_lH3HA}k zp247P-|rZ%h@UXx^wyh5zIIEbPczx>9URYw5AgICoC$IHD7|X`v23EVJ$wK#Z#ZJH za-6K0ua&?1M3l5vxXf^-_2&+;msb8x!j~7QRW8QPbWGBH=~Dy5uN32mQ89hEcJ0@N z*+9s#r(iR$a&Lat&uMKloY-hx&2n94JEnD@8r9~MWBoGDGf~B_=K^Jp&qVxA7iynC zHUtKr#TSO|Ipl775cAi9LJl%9NJ;Brkv@2n3u(YgW1{E83p@M5^vhifb@GgN72mV7 zUK)%{%3Pnomn$zUw+ETEbNX{N@A=Z~BeqK@eI-L(1K*`^l1Zh0635;xk8%Bd=3bzd z!Mb<(!e{|LrzRjIu}CT$*Msuuh5lSu{H5S^Cnj-U^ty!3pQ?c2J-Au|r&`}ZMpP$7ubzdQHY$P)m;UdmAc0yk$aimzuBBqd=)w>ozI zdG_ai1qRXPCCWH0@A*UXhj|5>lhF_9K3R`6xmZD<WpqK-5@>5Sm3QlcY?gxJkX5KIz0cZq4r;8s|HV$^Rc0WG&rC>QD zP>YCg*AG4OH=a6LEso8FO&OjBmj0{t6$f6=CXj4lD>bqOIZ?XmVi+e}$+LUE4hH>c z;?+5-dr^|tQ(L;mQEhz>ruFAgBv5&tYB580=!F0IePaW_S4_WI=$A+dQ#a>`keM$L z{BEK(bUr-vSn>YH>eh#7g>Q1@NXrM8n1$ZdJKNjVAlTP7FVHd9l1nT>EPvaeA<$=< zzP}oy^HT$Xq>nL6aA`H;xmNw`=2mz1sg=)T7lVGgF81KkPf`j|Zg=&2ZC5PjJHmk~ zNLY3h_Gq?KUlH5BmoSbEfX!bHAKq|N*bgm?iC}4Rdtz4u3kjG>T)M*@*gryFEb-cI zHr3i1luOeCEMv-s#-R+p55s`Gu>+GtbvIZULYA(5v0u?CIJMf1KaI%C5t8X@xJ^Ne0~m5Tl9;PKukOemm;zi`83TI+rGJuy?pHJyG}TT=b6H)XH?UR#lX$ zHc7+d{pHr$?3J2{g2OOZdY1r_k&Ta)dz>$fZFeL(K!) zYhT{p1X}Pf@T`cj=T?XDZ+R3G7$df?XaAZXuR) zeL1^Kb^OBkr_4%PQBuSL%f0N>o=W8SgM}5MiQSRv4@Sawib{Y5dm=`Cv$E~V&18x3 zZ{#2#oyi-i0{lx!Xf=ZgU?1jm4amdN*}49{>iqnfVLLsUG2FC1dcQ{ z8JX|;VaLMfWUSoRHb^fnP)Vv|>B%5g3d3R*Ez zP|)o8T&$8%SX`seUs^CX(t@6ua6yPQu| z?GXONIF$x4N+uIT zX|=el0wJRHUcU9G{Ip-Jen&&+;x41_9Te^4`dJ9EH8vNL&4WoBZ*~`%v=~4k5@+ey z0_=6zO#}3j;e;53s*VAb@*BC(rD*;)=N4vpio!liZGgU)Vd~e(aTtQCMU=HaL5F9=^4(it?DDCT<*SF7H@!C6H zMnAa$Bh!w7B*v0ko>2_HVwOF_j*~ME$xD+b2x96SBdjChqr|ex2eSm z_tg&;EBAN}3Y%@_H?=5QWACmdD$gv7&+s=d#VCSf6&$$-{ zES07qB&39_WT#;()ax6ek7d>tH7A#>tu?i%7muudQZ_RtN6&tj`r#+!17p<_mooNX zI1+zSJ4H_X3H!idf8CF(eEEh?iuMEcaz*0|Jk^3BHqwnE!dA~yPhW=}KQ|4x*FW$G zfa%{<@+q}-GV&z|IlSb+gsS9ww=RZm%8pl^=ipT#hV)w-w zvsNZIOC|+fk;;}X&B+Xj9gCcf9&`EtwQ~mJ#N-gWPN|77HKWIDIVZpy-V#G>qtH58 zecSr|P_mgO(&a7d!gpTLW_gP#m#gc?mUDh2GBb`VhaEzHQr+|IFbp}KPhYYZN$gVI5aQb8Qt7G;+b-uI29~(l ziPj)ybS@cjiM*Z_vTdW};J1wuOP8su-t{r}1nJW$_hr#*sTgUUHP{yS$_K~oETYl| zJn?6{uvBs9xyiDQ6(8HHqg3;sG+q{0yDjLi-N$PjvjgWf+{S!%r3Mf%1i+7rWEC+isLh_rHl4)8&1jCVBsj%)?a&|*?){9`v6M-B4xZu zl>@mq|NNo%rKPW%MZ0J0W+=6uM*%K^sDpJa8P3q{84Y6kF zM7}DXES_DdI^(nZOFs)<4rFo?%^)x5T^JC~RC6_0m=TK7F5b6Br10?!^4@y^52lia zWf7woUDO=Lmev^dq)}F1e`GXBuWpY1umWkLLlZSr$HC|K6a~o4PEb80cl6imODGg+ ztrdx3;QwHE^EvA!Yk}h%!i|U*tNQYD&xBS*YdY67Q80PBC=B@ z29{=99X-9!J++KvK~`A%&g-b_1g~2&X%S;A-5BbV>>5QLLo?NW2T##tEAO%pR3puj z-;+13Hp{QFT9Z9gObXaaWs2D7XRJ&l`H7q}6aza9|S{kAa4a98Sn&sj}qs0%N5a++Tn6z z5-n&YB~PY0l!X31s^wMivG_qQ;K-8kD|Ri4AXBvN=0E(2cZ^_W*e=Y_tx!{v%SkDA z%qfxcpETy~4|`cT()J{f@K*5)5;uovmFz~1uzWKHo~z+|4ibrA4R&cMtLwebZR6{< zE;brHtmB5gd>prDQAqMA4XW78hXl(!xz`i)H~>sMXG^ex3tFFnIy zU;R|)AS$W0rIU-wY`#J1GlbUPsz(Q1Bh&749Sz8A(5|>CN%OpUL#Ny7KxVsn$p?mK z1Zf==dJ>sbJtJ@}s6{EF?VrMEE!qImEL;tcTK2NVMr`EP8UIYN0?V)7JayJhmfCx1 zGppb4igBTkaLNsKWWWW8!_p{wg|1RK2_~{4g{X1{=CS`U=9|sz{R3%h!eH$R`Kyb- zur{egBW1=WBdYKn#3|N?TPk<%9(`tJ8AI5YMA8v%gX^LX$u9<;Jv&})uFEtSRU=9S#Q^_fTLR7Q!%t{LlVQr+FY`?^8~s%TcMt!nsqcXhNf^^M$ZR^caZiikHn zp=SGmXSMUmp{Oj-kh9?xkDrw=?X%?c{&yc;0bUiboGap@n4if$a`AB~z}-F0WcI%G zHd5vAryAv72syZVkhY=E>J}@d_l1riVTGx$iaaZY1i#r7_`bo~IoWW<+y<1+P2Ywt zk}|Iil40LZ5CE{AhTs^J09k;vdeIqIg#u5zOM$n-lcp!^Z?}*)*#LEB<3xQe&+7c*GH6oe~$4j8HfC9_%f{%(0gru zUBs(rF_?LT$ToK|AdOz23E<$(|5Rq~O%8G0xN&Ke0M<7n+!e{?g9 zSHFh*lZ0eC;!!f3#;D!)UEshPbx__QH)-(_f?D;S4?U`0ABiZo-G@_h112-9Z`K_l z;;+~Z+@|c>qP?2-r8Pc=41+<0BV0my(4h^RF-@Ks3e6?k7e`<0MY+b0Koxq&wKInQ zD1N|cjfZhb%nf~)QYOOCPc9uFZsb2(#~|O}_>T6}GadbB;9m+&%5W%-KNYHDs(sA~ zW`B(%rH+JD#wsH(eb7XURDV`l*kxq zbZ@@Qk@ecV&MV~jZKR}3PUBY+C^|TqB6Fpklfzxo# zD6MzZ`Wnp!erDk37I6oi=!d_+qhO(nH)Mw-}=+-$}vKSmpt8 zN|!L{0jXyAm01L7cAB;2itkz5j{rkw{r=!)<)DYI$Lm;^I^fmy@6*7$oEfCy^zCip zr{mSQ$!uR$%^mOu1UZqs-*CX;Q!);#=s!y==lo)#Ov+Kflo}`{6BrrAuLTLC_Ce^r zv9>(PXzhe5eSC`3p2{?O$t-RR$WYa^U$V^pc?~tR4dr$vXCkt>9Qpg&?wPhtbjCSy zg*XC4z(LKb%s%av7`IVT#4e@%?s$C%fm~Af{=%m@G((=)%ZwPfa=RMKzWyG)KGcKp za^8NoU1+pf7T4b=|iwCgZJ(Auu@DNWnK;|{91ei!!^A15C zEj&f{hzmRO3!1=FYy(Vi;5qrzoiJOcm}%MdQZ@W5t%?M9LbH+A%{Y}Cf*8r6Ab|>% zp<20 zwZy>LCClgHPq%gSI_oCF!7mk#C}b`FJVc7=gezCGqs7O|`wHaC%%j1PxSMr)+*<*X zldBjI0lDLJ1+z zR9g=m4r>X};C1#0oWS!1X81I%$~0oQ9U*)gV5u2xd+oGCx7()yc2D{|F#J*zU^=P7 zfoIY*z42P!8uhfhJ`~dN+-meKB0I4Qo)*U@fx+u(hRbgw?sJYTRoSHn$$E|R7^MQU z(R7MKh``Iknuwu8kdOQ&LrtKv8r8DG7IubC?8Li&ri;6?RN9v!J?>q5BBnnLupt50mnW&T*-qqnL8L{F3s z5Apq}7Z1*xmIM7HM&4GDE+=8KqfD5OIQi5ipr<5zvp$k#HJ(D_Cx4eR=#bQYY_@;O z5+P(wwlHctTHfLLvInea^G|iGHe9DccaPp+O{8Ie_B&Y$Xu(yVfPNuym&8+y5d>RH zhL>ssaQob2zm!CQbvdVG#@cNoTsS*A;q18V!`ucgi$>G--C-)7QEqVL2?ZcwAw2OV z`(ClxzI&6+ez2ydPD-7dhJgww#|bB__a zEDj6qUFFj*l4cSnL6|AoS(6h$liCPF7ixVwaA6}5OAZhew>1$DAgBp>okbB8LAn2? z9%7IXJE*hWl)a_c9|76Sg`}T80x+-Dw4kfkq4z5np9^FRLf_rmAYikm=TkYF@ z-QIGL*%G_P=u`Rj2oz-YsJqkYs7_ zv5E64&qi9pl%T`iq{8X-HB8xouj^I=y&0|T2S4Xr3xihGVJcpGop<{3Rpc&c)=|BR zr;LW-(X?&O^(lht`lz64dy`+d~}+=iIX8HPsmg79m8(;6PZ3fY<>R@&OmcRabHU zBkUGroK^ASF7@jpkgeaa1i%2ZJV;LZ1=+MA>mow=kO|}|kNdLjU}sy4J`mVh zZt@9^qy(c=J9&-h^idX5{gzBYhx$i}n#>#J5c|X&WJ}TG`bSP-QX;!af%`-zGtWVw z8AGEv0^YieQRLaYL*yFWg`5I#R<$tLHLha#nB7OUfpQYjnww=Mvc3cLRbDc?wGW@= zlG(7?ks@2DSvC5ha$$F8Av>(W0U+X2V4L8;gMAL3zQv(#@g-dvgTMnM+A<@wK_9(s zsx69NL;&Jfhr*fh{9o}qT+l5)Qhp-GGT)G^@Yppj`n z`@Qhr{A3F>$tv1o=O1?$u2(J~I`iZ388wUSGWsa=Iv(m>ZpE=fT*=n^_{w@`V)!qz zayJJ^QsqR%ZO$jc^b?1aaqKT?KWU`WFxFPzcE+k6IHG;{`jl2%9x15kn?KE%0jSOa zq*Vspo%i>b&hPtx(&_$ENXx0V$a8vYF(_5Pq=>xtTN}8k(yvYJIlyfK4sj)Hr^*Z) z6;veH>NUXZF)>(0sQc&?Rm8yMH1R0lO|>{>&Kjx9(8&9=LmU~PVULKckmmK@>>HP#oJn+Xn)3PM7oBcu*4q>Oi9%2+f0E`A%xqh zV-a*)!Iwul3@#$YK;xBniMZu6#ZWL&8aFxKgMU2ls}kgZJtvuMm*2#mnemivsXM!- z?$U9yGN9wWI8fGW%Qdsw_nv_)d5eX&h}|unk>Dfocl;nVjUF4#&Uc5bimsOLI*=Ce#i@?)r*Er zPJFsAb+URt|Gh6mbKuB_;USo@re~#(iw%T=sUFvbIE(<(_a?Li&1AO^)b;0nSp}2d zp3lR}&m}1O8p{6+-qEyaN{WoFtd|JYZCBFwuY`Rq$Uc#D=%2(^BL6%J==_@kooA| zUBjXYAXOvXRH+GRPku(TIii>8v=k3 zVOy4^R#1r1RP}k=n1r#W_)Jo$`O{sJ{zw{FvDqENkl_N5ch(xUCb)bPwDvx-7D~2| zd+(FLD=!rSE^ef!JI8HTVd89URa?Jyy$#9hR{K&t+}6md)ePWi7O9YhyM*1|@)eqc zuZ&wF(tYH}bCd`{?u7rx)^*2I{r!KWP%9@J|tyB5*>5l{Y2w`mkMfz_m7He!xW#P zA*^jo+lh0!o`ps`&#W6nf~kz3IdS6BNI@g{30C}y{W8sUFMZxM`Gna){7PoD^h6Fjd9Aa6lZ(Mxqo!sh{n>28ORxXa!0Wa>e3F}F`Lsg}NDu6H?8 z-z>p$nd}qfKrPo_?fIRk!i1gb4RtP!Chy6%LAAzbEpDA6Ont;x3>W~Iw?p-HkEAqa%G+{)X9rrRxUELV0x3lZ~e}da~0v zByJZOt2Ps_auaEC1uocDeMRSHzhNsRb>J|jCVu<`i&ekndjBABs!epdT0NPzn`T&Z z7VzI;RM|Azq>o@(PcoiFR(hK!c@SA&3gLEl z_kovthXsU7mJGW7ZHm!nM#38X+f%F;z&|O|4R10Xtc2j(M zt@=CKWMlM@uxHhEg5z`3on*Zd>vSi;$?t_`dz?9Ug8^s9GG1G?9a0Ddqw+q#2}q+-K4E%16@%|5hOmp zMhICTe&V~2g2^a(Se|@?kj*kh1_OIlr&HGKIoK$$UbG9jXywdX-!Ac1m-0{MMAK_O z`$)K4u;=VwM-P6^n~vIjygs~oK9A0p_n^(&>|-Mb#?ZCdu`k={$LxEb{#D%AJC51B zKGsjN?zARNO!3grgb!-hCE+hrk~=+@-8Aim;5aqggKl_Fsd)|eKVA+xndN8|gB$31;BqS&vJ%A{2X_g1ZNIiZ#TrX+E8#i272~Ulu~GK;q-<6|Aov-#UNhT=2bN zD3!vr6?qxU39<%=Y;7#Q6!D@nxgg8!u3@5ES^~;CwttdF_QqC<}a2^ z+}`~!gygT$_?9Y*X4Kp$i#^7Rh5MC$6jb$|B7ru7GRO8fU%O`rf9v2&qN`kcPO^-9 z%71>%ayaBKjGxJ6%4~;G^oJpORaZ8kkA zD8|42JzAI?DljB_-N#asPRXO66CVEBn zqDTmR>dk@rygwti%_Z%vi4`OtIJer@BVtj_2N-JMwE`y-v~*%SJGI3;-%GWK@b{_c z7blMWoc;VQ@(!`h_uCF@&w0P0-8mNKbotjdPKsAWS1U2Baxf>gcE93Q?JU2a#6~)$ zXYpDKhadTRP3X|a_pM%hH8a>63!l zMm7bLUh8Jl3#lf>Ju_$+=YkKst3@<#x zKaJej%!-hgcXGsxH+&jN!LQs^k5+*;Mc5_7Hfw|Q-`QYnsvfmI=9&9c!%aAkeTW2L zTG9lr*=)^IrJleV0l)>{B^<|^Vaizop#9v!KluRrFy`!+1>{9sQOKbC+1m4|)I_{PC8LbUV5THK}*ArAds>aHqlP$AK1W z8mQ#FdB^|H@k6W7n_Njyjl6F{S%gtsD(d4<8oyAD(di62$|YMH-Ag-UC<%cf2C`7Y#de9)W>lf)d}A)oCJY&sLLKde zi5QW`=QT$EYHjT5m=rZ@EWsAQ(qEgOHuaxISR!AzH~H4h=j}Af0RCrTaDVSAgB+??R)B42+)RogMl+9Zf^W92M zms`qcQr_UqntmCz@{azx_Urt2-0>dYI5ID^2zQAUH2YV^f}<&}+PIo#8!n0K6uPlh43iZ}V5ozaL` zFDiUn6kX)&Fq9rtuxV8gDlHrR&hqW!t{_zxBW&)ew`Fw%zw2b_bi+n9Uw(bC>R567 z^nH#{<|V#d`0V4@W8S94qAq6yV)c1w+0@F5>(ong(!-?Bcl!H(?%WK366J;ZZ90*6 zs2`3mL5Ii;fvaYfu2+RBC|knx$6m{6-(Gf~Ae~N@iN5$&e_E5OHp>FDSx>xqtGGR~ zX#POxmC|yR`5wreo$8Gop`|Z1Ziqs$3r-SbQVjQ8X~tQ?Ev~Xe=Z~wp6bxFdXeH8W{!^r*R2-#X+;Y+FH z<{p2idcXGtbe}MFFU1F4Og}PSt|ntiW;|dkWS0>ND;ru$)9BT)zbl%kp14o&PzZ$(LKODe#EnJohPgT z8*!AZ74TI%qP)%+ytq0HhJ`%jT#3Nh?(;}~uZS4D(~{n&B)Zlg2+CqMQT&}NlP}x1 zM$hQC@vFW``*EU1X7ugwjZ>ZOj;>#EcU}_olp-*3g3};;o#T>moa2+L+if%l(huWD z<{pWdPyPwrLHG(GPhyN|q!&EHIHj|wP7|Mt4zF11F`3ITt$X+k9Zv(iYS8}^3<5*s zQuA6RDX#s27?()q`#cGc*c!FZMWwL&Kz`N0d6&7Q%vkg*!{e;@$<)gN<6Hm8YR^ zuqk0uLUOPV_WSwgNhhHcTT5I&uI+;tc5Ma8g+6t<=8`q_)y6Fiwi7B}}EQyR`89&c-jR1k~} zUy`pr1)4Nqcelj^VlDPC&P=9;eid8^C%=kUU_Tyd6ntn^-vzE17(x!?<=?MiM5w|?_ z07Tf+bGhYVX@+fphuP@~=8~;(8n#oIwmDwTy{Gh?{hu-qn0Ue{0 zT&^`-oFfl5+?4HZzTNTxW~dz@T@jPBa18-lhrxZWPrR3d|JnMMxF9n zQRbie!SD5h{CSH~=r95Tw`)VCAW7`0R958J=#wioDh3SgFKdMH-6$5aiNGE^ugZfM z3<>XPk|FPILfflGJ87S?{2jwURlS`HL4Rgjvikrr!E;w8A+3^$;ZI<(VyclVN^^c+ zV!kTWmBzUEg*{q|$fiR^c19t77HRdS3Z0~yBO~?gNG3DnS-CL`tC1Vf%~!;25` zKS-ix{)`heV6FSxigxn{=BKg|_|YiQT#??sNy+fS>z!e(8)y8xwW~jc9AmdZ4&mIfdlONLp;l zRgTsd8k}^b<^NAEpGB_Q?l1!r)b3Hkz4n&kRPeeqLd%r~7cuC9K9d z;^Rgt?wKA493cphZGlF|kL|S8p6!3{ZKAONiG+*poeSZW)LI0=A5Sw#VsDHe3`Z!r zTGyL|HC;;-%53!1ta!MLt1Z{%T}sUF3gT;@>Uu!4Ujtc-_2wp-U+!?8+ciC()91at z9cHeokV`V}VwXK*MLlQV%`3-ZB%$~ffY!4b6!(giq|2Rudu7;V<>&WaKUUVmJ-20< zPj73E$RXP5fWQtrFS)$}$6+7J&A$tYdp`l^dQ&yD#pf?|*7!(3+t8NAY}0oqdt*l2 zqtx?OdhD^ktf^f2)(w`$O3hBWN+sG}&dMs^yBk5Gu zC7mU6K0{eC*4&`V(9&=z!b&mytos&E(>f^p@%NBJvW)dl&619eomb7vdtL{}GJqP8 zSATp(bSvikN4@owsuga}x3n37%yNumhTQO(#_m7t@V!EUtn9POc^pYB^!PrP1%82>DbTHuapH zMTWY%@|%s?mgKe7mp*VaG&zbm^Bx9fzRq^$=7hf@qkj_$w}+3Vl>7D>V4s^ga3ZOT z#%uB;Z=F2MUau@GX=j`s1Mb$4E#1|E&nKnq$qg8b{Gf|Ms*8cBjf_=)n|3S{ zH(jwV@(la>!6+%y{S({$jK;_q<0f*>CGXP8jFdW9VxoMcZrI;)!?d;fy^T%h$~al3KOzu^%-&!fG< zP$ct{rZ4ubfw=TFKawYlj?CnRF_f;Q2IK@tM3A*#mAu)eHEwD0^R?7E_&XWf%JPi`&{81NIMg4+R ztQoh$;9ril_Uh!ZkQq!tmgEz&s9$B?8#2!f{`Y|R^g9nI|sT~?0@enu=|MG0S>i>C=Foijh=`0-)e zkk;(9X2NXFi{xO}iMt>Fb0Z6b9>d1i4eKqF467E&nu^d4?#^nx_Nj=<3ixNC!=CU= z58TUBg2wl$)n}|q)G>k7W<^mSZ~aw*Vj?|GV)0!B00j3ACUJ*is7Zb6)IYq{q{Q~z zetwmMRe*lGPProI5WSBR`LLlN^3*gAz}&2!p+$wrFrudjx_GwqLB}F=CB6$>n};3Tnqw}Cp7xxvsQRPHSEEBJ4#+;K3043Oj7 z8zQ|Gga-ki9?DFyKC3*h)v1vD(l`S5Xyir--~)sUT#L=Ry%LD}dp$&YPdwHert-63 zVYJ8ek6Q)bI{cU{!z%8u&!y9J(ZO*%%$+6>-g_XpH_EEZRj=aliI1X_Gn!*YDlAqN z@SMmz7G1IMl2v>5q-Li3w7F?U5OM+)32Nurr~m#|P(+*T(9^-(8WQO=@2d)Mq;E9U zt4!Ls!Cc)-{!E5>f)Qq5-6_x~`V~HB(YcmvZybRQs1Ds#``9anf_&;Jh02E0@~9;5 z-`_oVnUqOtb54*y*Y4@)Y2-0$6{XMFELRcPV4!1qB<81c1WR6`=wstNDzWiHOwE1# zvtP}(&e+KT-HFv_9lH4DPoYiHbcpD_NbV)M!5gN*eZMKCL_-q}C0QQHU`bt(%XP)G zoqg?i%e90ZrfoWiV%VzU=H3kcPBEK7)71vnyv4ij>i^dc$$A&##6zOOp8xFvfPsMk zp{j>yl|*&zE+mt+?tKRf-or}P60ZR+v%r9c9IJll%WF(%QIZsN1S&PU)(>_HW< zTn7KB*_~`q{U;y(OODf?n#%6`1prmj`?I_xU(ei+`)9q`dE|6SVA^cYC$lh*xc4B* z+k9}?@8}7rem1j;VSR(2_1C$l&(Cy!Gc^-S?(e@-qiPsu8?QpzB%(C`tS$L9K+(d4 zY&EIuf<$!pv}<jcN3&(PnulsCD} zcFxUCCrFX}COo?5{>N%J?x^AmR>vd0N*E>x`;}#gTnuFJ=9Dt6KG8$wmGMt>Jj?ND zc--N13|#J?Rdz$kl%Jt7d1@Oy=X&xJX(y(hOpCEz@|bZkR-x!$xp)(25ZKQk2RUx; zK;x_hn*cvRiK|gu;|q6B2^Ad?b;Ye#{oy(R35@Z_uudCDp;=~y)!SfucGDpC=%rkl z`-TA%EACwf1C8;JmBe8k?At}|_L|r`f5`YZO%{g@rSoE>*{EJl>H^NKpQdVs6iAmjGDX@;+uZIiBEUhW!S zSLUsD=~nbU=2fC}W@5EIjbfmba3lv8w}_o48g zntjvW>WBLu6RZ*CX&A(m!ox!q`$5<`u4P|_$w9UI~!^l+$*^qC@f?GGQ+gE`*p}=*^boKS88xh#v z{+!-VyN&rhW6XF(V={P?Tr?hvvh%Xj=%sxfhc^lGsSH$|;{j-OiA<5|zZq5!o z(D3X_r^A$!AO~{klfuzw$a|rjCwNe0z)~QYkv5Ennu<1>KvS)ji)0~DnUU20aWSk3 z>j%opzt`(UR@RqSdv(e7uvLZ`L|B#Q_!cw8i0iV|WJyL}(p()6;wvIO?h5jr@#y{h zX2XY|>Aklm(Be7=Ox2lH2^2$xqOi=ke*>V2puXc8Rzm{-_ z@Z{}k<+av=b$=4p*UQk$Clb|p$)WgqSO9W@_pF}pny~9#8R$CbXW%|kkJ$orH{3N~ zpYd=`SkMP)Q!@SN^s|@!1!ah7ZYwloy(YBJnYuYtH#HKqdfNT#h4~JHOsCrBQ*2BV z7H(tQ%KM}o;)|3Zo$T5|PsTfZ)hk=1_)O zQtV@IwRS2@ilEPLp!;rMkCXLoIn30ir=$u$8(4FZ`#@Ccfk|IkY-TvRfKIm7$FPRu z>}T`5oE9Lh!_t-=H%$l0YEWyv09{b*yJ7iAJ3(uqZ~9vS_pWvLOi(S#G}ER&!tpi< z42?eIdfVM4_1!jhU2^&OBoAaue%jV}GNSn{i#q@fYc68 z2N5b+z?CIEA85RFZ*>_)SngQZ`nqsi)uoDIN2If)1#Z(`%5Lk{j{M->KgQOv?m4vn zT6;3K)x&&s^h9M*NfSV--q|Wc69Mm?FP^q|=or|^4`cy(V402`bypw;>nhe%TScLi z@jqEf1|3NK%U-(`>j3qP4kX6WsP-vG)#N1xEn(g{q8?B?34EXOEpM4gZsJkMNxWF+r{@}+>n2;o3eb6Z_dVFd`wv?ZFTh;i$DkUAc>B|z z)vU5CT4!45u&o{|=0Jww>va*q$s+Z+YJp#wA?}gx)w#bLyDmMnEZsp=`$c)xS$f33Uk?xUc$ZL^bj`WtV%Kf< ztzn9EB8+v_(lY`#2^3&wfZ8d1V1g+A+VK?p*{wn+WTe+B>*MxJ*7gURr^TH3H>#)N z7FD+!0K6XHD|Lk2z6Z{@o?X7r@3*$SPS_sm*ipF!gRW5-63-ThsLXaCLO(O3U%9Gp zmHb(M5EJFtuLBfOXFM0&jXg7{YwP#503_L0vnzcrInC*=`p|m3k>Ob+Lg+m?s!)VY zlnzmLc;@kpx#v~=$~CJ>?Q-0}f=qv9;whX&^H5so+Q?PHaIt0e#j@D$6!&|$->+)d zoB#|eZ~)`*`H~yzs4x=};$_{I{mAE3a$ur=jbx-`U^{OXRAx{;d|7A zk@*p~Tmrmw>o&+6t)%W-sh_cSaLB6VQk@oRqIOiJy#$@z>23S`I)g6t{X~PbC@Ql^ zG5M(zfs1?LCAUhOEZ(WEjx|jv;5+Z2-Wi}iU){=VC_dk)<#cL)dDeVBQ{cd^_TzTc zoDaYj53DvICUg^kZE8;?xZ8aEvQ9+9W7!E~h}%S|(GSMhWaAHB^HW}#*ypH?=Pwx6 zWVS|Vby{7&I(*=kS$f`qz^W3`+!{3#G`et^&gJPzG>oHO06Jdmwo%P)LTR=O!u)RoW(3TzeEIc)6#sSiR=3W)nKh&3xUL`b&WLqq z9m3kscHay@Gb&0W5WVdQ+W0KOYUPI=xcc^Y81_a7Y`cz7@VB@@}s*Bjg=ccNeM+A`C7|7 zpQ8H3#34jklHV`*wAZiN^Gs37>_p;|ln=!8qlVa34r{0da$B^&d5o~eJD#c@WhF80 zIG(_N`~Y%Wt1b0Pv(e7Qf(|23dE98sh*vO!L01FVd$WyR#n7GQ0>#qAiz#0>*o{Rw-uM` zahGyA;D%IEg4}2Y5x}oXa0f#|j1idVR=T%A>j1~Q60|9bB28JKbwMBafWoGsKFlKs zIGr8k^aWK9Y&$Ab257BoLFnxu?no3W_0z6x;Sj7q-{zR}{< z24t4)v4npK0-g^!Q&}qJ9kbGb$KKA^zAtQbsQ2zE$f+}g(TFe1Dgx*5Kz`ab2|C$U zcwn2Fhoryz!43lP_p1U)I?b^F`_po!nhzCH&9cLUR4JO5)>mQ=7J?8ohK4kOJ>A+m zFD-D3l2i|w%VC*wOI2dx(Z$j$-d0jkBQkJa#;B1@AidO>X}=J9QDv6cZ32`%mw6kv zfvP(PS>-E>%1+dZma*#{0k;opvsbE&a;V$<ls&Q>c(keXfHx%CKqv>P*b^aVl_HKBSH1(@~zQ)IhZg6H?TR25>VMHMBoa$QcuZ@ zP^rE@&eHHyl!HPhUu&#=}0*EL^nOXgo;g8oO#)dWMX ztCm2_fs*sA@0ppx+fhf%;SEU=M@+Gesljj2KT8}ih0&)Y&`M;6Pl{rnNgg|UDzIsp zR8hii*#O0J`jIn7oE6M(M7qPgl#v4WC0$pdHZqr}(ydQrJrhw0!G8DJDw0~RgO=qL zD363*x*dV*HJH*h9lF^0huhGp2jgMu*3emnlpHn@14kAAfvArgxBX5S2FPmBC1!4f8P zNJM< za84MzhMonB7exMoD#d*VYElwUqNWv2kwZZFc#6Gkoy`zNG$yQ=ow5#*X}t$#F7M6sB%D(2+XoXa)rL z5&PE1tggAVQfkEZ1?>L4ynN;Fp%wqo@<@D$66rErw9mx7n4j3Q<_eB4gBh<+jtENJ z_ES7%)@{rogc^|-31QzFnuHc!$~eMRX<8SewIjw)yg0E$n+G%A;6FBRKbo18J%Gjh zk!jC84fmRNK&8(=^1u88e>LQaJ466%`uK&<#JL))C7h_q^2Mqnv)lG1Al!1gOZ&tQ zm~>;fzvNxn4I&S+v@!L0L(?s95nIn${n;fMz!pu1=_s*KJJ{+Pfcb)8?0|tfP-+*t zEJqdYv5u%81MHx2X_phCDbwf7h9(MM^Its ze$2Kb>&YQd&=y>~P^}S+6%?9Rl z7GsBriGtZmBEgg$r;AU!kNFhq$VZa&&BgnTpM3B?#S!9jIYpDO>TO1zoRwuH?P;zK z$$=~=zO2ZWs4=e=X!N#uu$ufp2CRI5+{K1wY3>@c9xrvlW|I_r+c9)ttpG`~&g?|Z z04k{iB3PT(;6;d2j{D1HFsL=_cPv)COX!YYtjoLC?l;A0MZ>tAv{INRlPsh@&v$&g z*3HKDVaDIjne9d9Ij##N)0RcqQ9tk-buc3odA+5#X-sZ;#j22I3ZN9zqmIsFBbJEA zIqm?CUsE_Ei^f1gX+)!&;!i5QfTixH}kA?nVTJw>ePZOzYG`LIIL@?5kHeDMk z;R5yZEf}?3BQVCy`UWrcm8Mb~UehT&vl*=Oq*Q6A<%7!G^MO?;nAe>tt$(-?iIi_v z#G^)s)I1GR*g8cF8xcS1d=3HBt^`nRQ_%TR@n4rq;$toQ8ZgluXCWgKkI`BY8rr81 z!%+ZTWS68^o~={KnNST&^;vZ|RCMN*}`bg?zLq$7PI?L-fq1c&MVL z)!*}u^={?s)TZ`lc*PVT$T4Tx841|IvqX?J9=drTu|nsld})TI#I1P3tO)f&u|_d~ z8YE1LAA-;RdwM}ll7X3!s$%$&_0L7VS7Cg`9d`61kpncS`?Z!+!N*ga#-Ht6U{AbS zy*Zm_kSNEFa46s-`N=YdX?w!*d1wl1GstBbG77Yogv z9qnALETc*(_!mD7s(hX2Dfs8;UjCNk`4g+>QDz4OC7s^v<`^fZk62o?t;*(UXz=X@ z{FZjiYF0ahQRfgsc)43M-$iRFXLhTi5 zfDA<9&l9DM4tDMQE=+G1>!=V#O>=NNgoa?FXVE2*M~3V3DVKVG6MJ{vIg*@4m%KE) zrzLRq{a>|a;>V0Y5>`tKl&2HiQtX8{8hyaa=G+D0!N6_^GGZHa!IO8`nU8_c^!?KWZn%Q46npPzbnq;jnfu{i;}I|e zEW^RdBB|2)!WO^Tz$Zp3o7F>IYEe@*jCfs&FO?y<6kAx!U(y7`{LqAwY!&M-|K>;n z=cB)2w!8U=uPk)tE1;VK7B(^>UM=@|%gx?vJoN5hs7Dv8H-rp~@5*0G9omB>`fyUd z31i@@GMfp+H1cMO4w=>KUnffi(~BO=5x6GPmib?J5(ldgH3CqNTjEcGo>oNa`ieNr z>6Vn>d7INQ!zhnV@zkvrQ&l%cyxL-5a)PTFw}eOOJ{fW^OETdhtA7LNGa5hn|MM4F zGEb^(Qs=LirhY^z*I|er?ku)oC`t&K3}6>o;zvRAC8_AA`CV3#-3GhKYjPK3L3gYP zwgH!sf9V;}+uTai0X;wFDA{e*raP{OgjH*jR{TLa*rnk^%XMi^)Xz-;DcM+*uT^r3 z8?lvqIX&TBq@ZbNF1z4i5BT|30Zev5K$5Y}c7bHj9_VG?3xhPQS4Xo{>R#Qy3gzJM ztY^`ah7rGhlC#EoB%Me`Po_+kNIpMy2Khl3>wnAy8}MXbNCe3ICq;MZ)QQ$Tx5(oJ7hf|PvKcwlKiB|jICrY@>;sB)Zqb?KKRx5?WTe?*s3`O=DW>XwKt z08iyv$yaJL*uH+@W#}0;GSq-S-51S+N7@y4v0=KBqKdAYPrzBRv2gsRzzpmfz z3`oe|*9;GqhrVo7{$b7kF^`bna7Zw!=HiPN@sD1%#(m5a+82}!L)(+PpiFhtWq^Lf zn1u};g59}zQ2!wVxqY1RbU};!^RYU<9Jnj=J3*_jx!jY;y>l2^{(3p<>PV>O_JuxB z)d+ZgxyERUa1e~R>5?Dq#T0fFD#+|f9qqL_vMtG)7iggaRDFMmO7AB)SG{YjRfdRT z3Ta@+n-5tn=SRxV7(#`RHzDWl%|2V}g1Iu8$uge!U9g%H^nuW3N>beYik3=JzAJ18 zpEm2C*DxZ~#|#1V{iOk+Mmf0js8K)_`*M|@YhR&Ob#&eiiE4(Rj11|8L4B>PM3M3&V@>o{dvp0YaK!8n;H%sAid%}b+B zJrT==&0`x8w(QSXGW9R5&uN-Jj>Att0M=4li=p2fHRM|D07%}F}>mf`{FJeF+x(U(7J zU;BPKl`8wz^NGhVdE%?8CPmgZmc~~7cpJSv&IHT+7FG)b)x;QFkXg498hV_Z$TWj1 z=pDKaU?NQhaC$kfh;5bEl3K0HOgyF2Wd^S>w=0`pw+m$|$}9x7p;GnygE9elT{dB= zPbNu&MR;2#y5u|`P4{C(LEvB}c-KV)67uLqbs{vUhJq&F!K%stJN=%r%uIXDdQ z+H}%4hKLM~s>n%uarI4VTE;SOH_++bt*AE9naum1RuJJl46%!4tqBD(QeZO6WKoPe zpjyu7atna_qKe(AM=}o2Dn`u>O5u%OU&N0B%gH%eD4?1bbeHG`kmuE8(iI3PxQmZ* z;f5Csdo61npW(&@mNRD9OSaS;XtBwAI2)q40x8?2>~x(jyZxTv?n*$Iz)1=v$83t8 zAu=JWU+F_Rl~P(D2U2X+5A{#91XdqO+SB+W>U5$fuQ49?Bk_5{_QEyCH?1;F3aFwL zC{>5@w5y#AY8yH~8R%K`&6sh+{-trhEBAULb4;B%OH$IYdVv7I5IHI)Nka`%UsI@~7y6pB$p*ctd@- zk%O0)`G%}{+`P;>+s~it6H{qx<>fYxzluLNbquyDbQ+m{97%KSDD4`R41Iq2tF}$5 z)`Y_sewY9Yt=oa>Rwfo)4!eCDdu>{7~Meq-Id*FM*rl?5|gDMDv6eL@45}FBUZy5ao7>KNl2E zvR)I@hLRZQpd2su|EPcvE3>{6o3+x)XRE*YfFVVXpq10LWe7E9H_Oyq=j}7dE35_P zpvj3wjnGJpl=G;VMfdC}sqLL3;N^bCk%%P|rM(KuAQovRGQ70XH?MofTKYLV%KW%a z{nNiUbG>Y0-#Uby$dbE-o>U5=S|RDmD?0YmxiXC{>&*SSqi31V35snl@dtbpx>q^g z>cH8W`Mg@-sM+GF1|`H_^e*8*+09exSc?g9FBm2N@~JXNYi6p$IqTBU<&au=IIFdE3<~_c&Q1r_;m8|v%HvQ>Od(!>XpNob@x9av`t0~}#r=`X|35*~YpyOBG1Q_1$ zPKlog6~@b*bMf%-NKgyQ*LZYOqCmh)eIVs8KFu5(X#(wuSThVi?^}V_Q^PS(#OQmN z7p4^>-d;dIpY*jrG{7qErJPvsV$uruU;4G8o4vgc`X7?JgkCmzy}qSy{i&tpKgD9O z6F}^nnuD{b`kv@zYO$3iLFspQ-}urNb-#W4*5RIPZwm9Hdg;4{`%nJ(${R+UV|kFR zKF-CdiT(0>TUPeO8Cp@)rW^%g1|pAu|1yMzOm#Ff1?zixPZa0y$rb0o$v(dmu(yEs zXg`05edar(e%b|e%RLdOPYQ5uHsvNT%z!KGAX`^P-?jVe4F!3@K0LA z<`q#_tUdjP{CM<)dnuMlu2`=(I@ez~`$(lsf@jxp8o|;5x}c1GAdJrvJp^@ka?{c^ z!LfEY<>=Vihk>9Rl5=cWx<%-K>A>ZJfB2ik*K}QxByoU9*=%aMcu!VF*w*wq%gh%r zu)bjXTfE+p;|D_47IW_IzEGdnTxbXSR<7pwpuk1jdfJ#D4PtsT3#=E&@+t8yYcIGe z6y=KP@xMmJ*y?Vg>KgbHIdJdeBf}Om%9?650fTJkYqCSh?S)!B%US$xwJ8wFJeJ#{ zc%JTp=s^2_{p3L~3oZ56F1mg{b_Yj$<+^uh1u=nv7nhx4PY_e<(Vqx@viUQFP8h74 z(RlVw*9X{j@!XStssya`0o_7*xT6AVieqWwNwD4@(P^;vjAvlJE>aY0cj=HKrN-9~ zhAnK}H}q50%z2k|*~UY%)i#kYjAVHWo_TCrXcBito5_8T><+A{fS0>wJgbuEB~;6NotvP zwZGL)Qt>8ba%Nj&ZmsfAC(-$I)A$(R9PmQbOYONEiiH9d0wM6;(b=1bT~E9~^+T-P zA&(g=Z~qh9C|#~HP@EVwF2iy3$M+z{z2p3&Hk|7P#26jU3Ja6AQAu#M{*QVfP6lEqf@S@K)O^@{o%>)Jm z4Mp!5^cC4h2&G%<8Ce60`6MzjwOCoK|Gx62z|qbY%}zsXU!x+nEPwp708s9cx6Etg zs2Eqn>L^>;=y#_5^MHYr!tT^-E7R?7=?`F4Qg_4XJ(&I*snI9U< zi3!+#pn0MR*O_Lq#S6byNystZBxSX}{I6KU4kSXYqO)+nlga6pO*Y>#usuW=oXmQ>XAv)F9<)r;YOv3xM& zT;~3R`FNgJdqL`Yi2%f$v|7#EYy5v{6=Aln59lVaY|_2fJEFt$XX=L1QTGIr|71E=6es&NwaAsN{!-M2nSlblX=?pVy6fy-vc97H#*5JM)O;vXk67FB zfiu(}sW0dgDZn-%E=?uSMHkevi`)4!6ao)yhC0_je$}IG47&VI;8tDD7XiFyjk-;v zjRS*@c=PX}aOb{V^`@O-vsFD9mVFE;)L@QG6jk|N1*r? z;e)HOw&rrmx=4~b!}h3q%tlv#Z|In@SnZBzYcqmRC_9fY>z%WrnwvhDLL zYRms*8u=I#k{BcpXI8vC^=WKqaM}DNh1e&OFxD;6#Y0w+$9JL%^c!-gC1c(GE_+-3 z+SifXPxJEHp4%$&jVU1dq@!2LtXmzXlI_qLuPVMmpR&bk?^QUu<;Vg^RY3sX=RtXc zIskFA+TrRYvEfgdjvo&z;(k~2f`I*A4#10+8b>XkHdf5CQBQExrpJqfJn7Dg@37@> z<<1w#<2k$C6!Wb^8mLKuVG#OcrYHMaFVZ?EAdF5v&IOxW!X;CBVK5O{pzP7bROgg8 zdTR%grt6Ou9g^S?@9G2726R%Qw+qVPb`#KKg!-UkJK*g5c%DIvRbO$f3QAn|=teIl zopRWmh}Jn_m478!7X1@sZnyX7XhFX3LT-(iQ&EC=Ux1l7Fc4igh-vO+^kL#P0QQH&cF^kA>dy8aedQ}z>Nth5Q7d`CJ zs;-GWP(Px&?CK3)BEH8_lWt-F2#uu~5wKnBgZXu^?WIrO7Jz{3T5-wF)NOzwFxD%_ zL}q2wIPK^>ey?@xUD+8pu8G3_m`MSH?~5@@{S$)DX`pTD{Q5+p((n$#jo1zRsK;XM zQp0DYJ}{$Q_(6uS46D2t8ouM3h1tkje4Z}l&&@}VZ7gU+O(1=6gbl|*9zvU_F8k}1*+kBMGI2awW}nVX~g-J>ApWBeZZFo#Jc zxa-2WsW^0fC73UpWIyD}yy6{;$)?<<`)UVc!)zmh4}E05;_mA7`D`7E}2cQBdHHkEKfV|iA-nZdMb!k3!lNYe^L)YwS=CxJ4hQ+@a9OO>_@Jai%- zb=WrEj7{!aG5++d^4J6JFjJm71F>f9rgx69S>?&K#7a@jc^~x_lKMF}(8_Y3@y{aS zh&qUE;>wk1AuQ++yYtQc0B7{U}m$S3<$zH!gv4->;ACLxXH}F`9x^riGzE{m1jfvfuWBHEAO~F9A7O`AQmYc zB^^1)uc_53dgdt;uvvsVo;5li@{Uhg``@?`8|DM!=f%@9fudno-}W?W+tLw?8UKg{ zKe{wyj*68IBMn`zB;Xw@valbrB-a<^Cqq5P&vsk}s)i+OGfA&KeiQ?sqh|E#T47~5 z(+a)6#1a5>rnSRBlU$n!;`%UaFeTa1<^w@bBh!+ZUE zb#-+whV$uqbu)F0`*p-8V?%rM(^u*0-E;4u(|dmI4=qsic6aslD(<3jMHm+jwEUWU zzU_jVboEE6^?@T?r;rF&xMI74eE=~kJ7Uv+T!ebp!3YQ zE=S}~zVkL+6SP`v>n$-GNFqJnx38^HVoRhaW%}h{kYbU*$Dry^y^91b{orMBFmal@ zzb&N_J`L{RjY+4xw!IO^vM_QNFXoP+b|WDEyEFUxp-;aR0J6=LR?L=uo!#64lLC|z z;`RVgPtK|syIk>>sc`B07y$n~Se?9H*6kczzM6j^r768q9c0_wkM|XS*@q?Tz5?Y@kfL}g)Gm$P^P^e+Asd~ls=;q97_CfoP^Q8_y z7<|Y;ZZK(~eahm{9I|3!w+r=yO8Bf%RCUXom*y``m*oLHo; zoLk=&zy<2NZfoT`m`4i*i2hUbJdl*N7Pxj!s7m^fpzx2rE$1O20!FAyY~eebIpwu? z8xg5e$(aWdIjXKTQd<;QL|rK`ggl`klGG2Ey*1;2Db~V5X=r_Hq zgM7d;aJ%ScqO2!=3bAdmT8zRKuR2lUV;sX+7==oHCY6Lzk`UvKK>Asw-;CgImq`y@ zD7g6k^HW31X=|1cgtpW)91I<1k-d4Rd_)#@_taNUzouN*sGWd;`7&u{YZ>xhvEPzl z1FF`K>3{}16znQE^)3I$)|JOY*>+*eRHGC_St4TzDZ4_#*rGxzE%r4e21B-(VeDm@ zHpUjEP}#R3`<9Wh6j^7i6SD8?pzj$iZ@u69-}5W!e(vYo=bZbT>s(jQ***Zc061Fe z_u;UMy2Hc1U_Jofd{;2%>V2!pv3?md4heE2eHYkDXp*6sxh=@M)99ZHOp8-%JOb6o zUulDSZzGz3Dcwz;0C@opg0`s4N?y4bzY?PiDKB$2(%F}+(CiO} zG66iLcAZRFj6Att!~zctE{Y9a+iq^#(h2sT|Mu;T+0ivQ6j^Sv=CG`-kczUhFM!e; ze?RI1*~ij-x*NbjO~ow%Smoj`GK^F-qvOifIIJoz>rjVK%+ax$bt)St_V^Ap+j*|& zoxk}+V&loA>D>KWE0n75lodyR8``gyZ_RLLk(3GP8b#22oRTacY^0iY;9~+Ols9RCLeG-@^l|?vuA~VvHvAN4K&_Lm~1_&!ge@v z`uC{Aywbw_AvI7xO}pm%ch|i7revDLxuaKA>B`jo9Yoh28Pq-igmbr25A10_&BAdG zttc_nTCi;0TVAjj`TRq6u|#*i>D;*NCe}^ZeVqHwft)8@IONEsVnf@w5WlON(%x** zQme}%%ZM;pas7SXrxRG^hf>PE)%TRge-JFE@&tcCV!WHJC!2R6%eN{QqgV2A8oakD zi@;I2|B*xuzn$h=AI!f78`$I(QGeed z3-3WHBl{8U^+j}`Y7)oh!e)0>rP&Bp@PT!sHfP~;&TS{uQbPCl7>%K@Fi0NI%-?G3 zo^5gRDA@m)Kg8yN;Xv$YAcVW}tDwYNavWry8jZJqA3I)jC75x8rERJAT?}emPa?yS z*l>zioa)ZWp0;E%sjU!LVI#jlC|L=i{*!aM{tmZ9_I`bSMrOt3IY;dkJ&mwJVQtV- z&GX!!mu!H{Z*&?(a$sO`Pd*lKY|UZw87y_lQL59&E9ty+$KZ8}R6VK~2&ool=r}_^ z${%zR`E;sdp*S_`=BxLDpTuZqA1sL$305o%XScu!T8Y4~?TJvC6-kI5>DoQfmFuc{ z0G_(kd@5tz;Tgg6(^W9dd-=S3vs?r7A(OA5+jsgu0W{>5E7GN+$>FN8_{HjGlBbuK zLECcZrx_awA48=C9&H++zH)09vn=H>I0Zzy$wQtc>PmrTF?2%DdzarFK=h{rIOEdk z#n64MF^d%=4^Q1y!H8f| zgEz3F)I4x}g9v~RC!;*et!LK$C@#{W`ONPU_hvT9^P~~x6?P0M$8B0J&7D+MB-{Fz z8lu|EToXpQfEq189kwC}Eqi+=MG=HfBm*bzrLtq+(>U7DH-ieh6$7BR!7^mlz#(Z} zR3=*nb@b}Cyjj(@nc4OZpFVMFZhyUX`kf}bL*M-+KPcTv5R_!3&7OmdM(@;rfNKjJ z%6P0Y2r5apbRjkfyX6Q~6tELG#;gJ0N4?C>Ee-arTyh<z{g@d5oRJ9hStVa5#DfezquxOo6#gu6d8%z7; zzF_%BsR0C8KB40?tjX&dN+)WEzyT_Y`S+wkq%Wo`6*3|CW`yVV0lBGBIi97`*I)HY z(E>A48EKPo<7}`y(c{IFX_h%TAB4y#aamsI!he zP`ZdC7E%lNmRPk~#K*G8HQvciKuLX8or8o!D2_b2oYVub&pbn{X#PI#l6UE*T4~mY znfyw+Do4)_X94g@63X_P{IcCEW|~$wT-$2LyXVP0v%@N0E*Nj>Bt~eQB}O$uT(=9hk0KP) zpA?^ldlQ!PHf3_FA`(19apkW^)m4l)4JgiK$8*JjmV>uv>ovcrn4(q>lfZRr?f?}K z!_VkaeFfyzMaV=!wnopoypdhL!L)X3N5%XhP)$zv1Mif~Eip=t?zC%tzw)UYa{E0C zN0Lgtw+S|vVfeECr~lLE6u@#!VkOD zTtClL*~J&Wn$!8AqyBbVqrAe178hd^89glgb!K7Klq+FzkFP0*1Gir!7+(|qyfBl}Iq&T31Io@A#4;T|s9Gz+}`8H?kP|{}E zgh*{qUU(s$y^NhGyVz*dk41hEyn`x`}AOXMQyFMQH=x6l% zbiBGiCkRynls3M15RwhzS5}XWapu8A1`Su*JwblW-_Cte^0{d&`#@`}%`L2e&ROj< zT+rnVZT4gDacxIikgkzQ_A&iR)N8-eDb^F`y%zAPU)BjyM@A%VOpXoz(brc=2TdXZ z6n))|@`HbU$)6o(4hD3k^7X=lPT0V{@ODBuQRV!0PqwbtxcaqPfPL7uL`)AiSNc4r zn3Fc@dDvW!mhTA-gX&DRpis#$&RkJbgNL*f`Rved828;Qx_S)6ygs^9t@L&Z7UMWj zyK?CSb`iK2g_|2q*m2DD_38Us&|)^!CYo@69)3jXm!PJ#~AM9&d0qQe*er4 zj9q&^$y3;00CtJ8TH<-^lAxXR*1X#m(bMgfy%bySLvGMJDm!dwPibmJ{n3CG1;~J# z=*^_P-u%)b{OQR6pSa90AJrZGF*MBg*_{wpSeqa`GX%Hrx#AQ(#kY`HpDbN^w;R_q z>@_T)%*LZ0&N|d6dcmMC<&A~0R=^7N@EKrew0mqlU?lNmM3=+oF=GU*32@@uSxcGq zN{4FBt(CmgSWQ^li5L93)2wt<9AJ?}(^%bZRn5MbDn^!egKqPg$(Yiayvj4l=_8#- zy#8xGK@xw&s>Ps%(BjI4OEs@%UR_Xr(X0(@M0v4C?`7q~4f#;n6}h#U zPnJF0)IK*;3WT12mZP?Meto?JCw@vEgxD1g0yG?F-N3#4>k0X9#FWN4LwmRQ*X#P1 zG5g{S+1YLNPrD4a(LUTsD_Qf%X-U-b?uxpDaTyuGoaDVfM)oDc8SG9XaH+%j-mzKb2cg$PCBL|>PWSflH-~|1 zn0(bgQL}6nx?7MTqliX{6j&|EC(W@0Bbi|?vObjplN?e0xb6Bb!YfU6J~a5TCLI!T zBKZD7o9j|J@s_b@u@9ZMR&AsOl>_jiHpkyQ5T-)aGP!isrUP}We}xCYY@(Ql==;;? zqyS?5=NISb%5aGiYY&=dv8ldg(o&%V8w(a4& z)ciqFuVIu^y6xG5aI;Lh?G~s+*+SrpL#0p4Lp!d&QS2_dbmd&f=GDHcFP$0Y_`g}q zvh)6HgT9Bq278%4#oRzF)x5hmzR{iK89}V!wcbCqAKD{4s%2a&3U7Q;(d-oj=BGsY za*!PLOQ)n$d}E&1^MkMF=89h|rj{CAIx`F2r%C&mW|Q@8>ySzXl=Ji5?tWgf!qze3#Yw)0Vox-GZa z`1&F*@LN^mF!*H&QP~M$Yr$twZ4OJ#`h#HAC&J?1WK7vbF49eEI%mKz-RD|K(h^uE zOrwy45|B%-dUf!!=h}J4!SZW4v-2RYv5b;wvo~E=mZ6blH@5~kAci9g{Q2lcY9sM8{PebPSkdp1uLBihFQfet3}kW zUboW)N-mB3B#~0(-Mt`AVWraDuR4vOef5JGAU;z%gN# zMlMV>UAYPJS`Ia&%>!Q#k%ZK6A3N5`|D-UrSF6^PL3^DBlUkJuqsZ7_lprrb{J^qZ zfI2TA1lb*RZ65FFXJVdqx8#u4H}=owOGf>6gC#q*V|{PU!U=IZUVA@9JOzMD)!xku zF?Q%6H^$2wi9EvFqsAXr_9cwlC$SP-MhGRN+H!x;)d1_v%>j8~uh5j__zeCT!hU{( zF5X9&j}DGT;loW#@r*jn2ZRrL#=MN6PN2xD9k88HFKjO!rdJ@ySxHoFE#AL*yxzY# z>Prz?)hkoqBZwB!iwWPd*;H^Z1)K%&IZaLp+||@LhW@l}JTOo@HjQdFkSiSS5N&KZi3 zP9;k%$jzj=m;TGspvA%SJg;bS4?KzarOf=>oPdNb2ekOu_xUB}Qr@OUmYNMOq7ZMy z1#$1Lw-_81ID`v3!A#G?!{cSkuHc7)qcDjF^1%FVfpi4C&>SxyexBr|zi3V$a8+<2oTFQxK!FYo& zF8>Le9P|<4bUh!?YA&(~dDqNXGDnuujgd6&F+oE$&ANvHr1Lru;QAdMSA;gO9%h$UfTPJK3+ zVs0#8YrATY&HA7&*)G36M?aMK%g6j&W9(7dh8NFiUGDm{tp2{?{Nw&}WzhgA_PT*` zyTDL&Yr>ImCJKl0Y>xG%m&P0Dqr-a&=&Y6?W%oX2#Ob}beGWFh^(9Q6pIL{)(~Qcw z>iCo{ZX?U=-`z?Hx&)rQ~U|#)E$Ty~QX;VY_YZ1$Kj^d*4*AR0xohIjBls9PA zhid!Eh!*UCU3|wl--wK8`;w$g**68OKK^UJZvsI64zz?$hN{dz^56fspAO+_uM4(h z^>5L=iOgT_*)u~lEUv`(`1+cd-k(S4ZQs0YBKaI&aVb|? zhEv?O(f&QGw`SMUfEKf7=lzzdDv)%6xk=Q^w?|vcnrB1L#q@5+D?)B+iYlKOAhBC; zeIBgr7cS{);2%4+2U)N6t(!C+QU-9^;=gi+l&mA3+6_jvIDw7-;tm83QwD=%>jT6Qw?p_jzkMUsgGIeBrT-NPh`PrQxQdtm5&N@4x@fPBbm-`>AM7cir( z=I?1?M?=C^z+wMJ-%@k+3pB&K0Kf2WLNfG+_>QZT`+oyzpJS%u9~mNVN(-rh`zjuO z#8Z5Ggh^w`M~W|*j#UVubw99hCqOEj7oVD?34T9gY~73l=j!pV6I?6RO*}WhN#IX+ z=jMrz6T9pT1nS4)t0vDBW|hArU-~=m_GrmoGLMaRi(Tn)i=nwy{MYzS$0pPbe+@L<)RA00AGHy4lw-k-4tgNm+#s?w0Z5}#t2>zW9vqj z_|#ywhMTL0bczK#KUQQI@GH9hT6_fBUxp^iQljVIpNIMbnz}>v4_J?|A=F~Pnk5~@ z2Ixh|8;cDCXx={lguGaA<4rhIK#(C_jjeAGV@8+d&IX8|rdLC;@PcUUy{OT3ZT9cV z92ym=L(WaxV_pwFF+D!5Rw%-`dft|j(1-Fz&eB_t9MW>E_k|Bv&SCWx_IAS^W`iLM z(Q8d(O78s%q`z;lz>3ne{5@ywBU^Pad1C)Se}BEq1&dJd{KE58?>sO!Kfmgg_F_xk z(9K-mWi61}xrXC(hQX>q{27k=50CY2rONY9)Z1?g(AYH#@vB9_MaRG7JH@eo@6oYc z4cD$pyCd&c(!jeM{!N17dubc(mz}I`l2dkDf4~!ms2DeDIV($OdEaA`yq1+oDvRf@ zwDo#8K2SNMP5EP5n(84fxImK&TxOq;2-05EzeYiKGaZX25Qp9_k85bs)rSj-WHPyr zAFK)v<8|rAdwrAhIKpbV$MF&jC>(Tv9YXkMrsmCyH9= zvW%M3o$SHG2@bmT$oEiY@n96C5wHA5Az?R4ReONBC7E+4ENVUCo&HWfeG_$h7}NCBH^QFe0!&vQ-U|lUV{F~&AZ?kIHk~KABUE1B(_Uy)SSish z25|`3B%eMQ@V0rH9jmCDCx+M2xRDA&04m`J{bWq+CMQC7_{0hB@3N|tZuC3kNp3Bt z!%V|-AeMR`GiAaZ>V3KOAFB(+TvlOYZ4&m?M`G22B70E6;PzTn#*4a=t*j+t_qmq5G|qFPSSDTbV@z zzeZCcw)q#M>Cfm~-O8$RC-8jOV?)|d=JeOflNu%dR~{Iq&O2-hGrwf#Pqfhu5+syD zGDM9lH-MQV8Y%h;Miy&qKL&C@%v|5+a_gWh0TC>i)uPKi!O<63!U({CH^}RKf^|E? z3izE%Rq3B#tLoW=(ai8=K7Qq>y6L6*;mdE6t$W$e%4zdKccjZ5#v2TJh`VKh-=|oG zj>zxtKd0+_Uw?(l+fdquoYk+`S;;Xk6_N1iCA@O!jqGdbo$4BfY%e z>XxLhG-cRUs+k1WkVFl!R1L+JQG z8eVD%r^$TGFWXLvSwS@=OQQUuOFhm?q>z9sL}gE!rFpg+%2o846|7!2?1R+AEK+nF}~p*7HH;Nv#FZ$G?rrfw=xk5F8j_-h%z z(x^}DXbeC#*O`hnGunj>e-c% z@+efwVCPxOIeEWwvXWE?9N?%Z?j(|CJ&H31@l+61sRDBIp8HsH+{2mVwes;BKzpBS z3b7BU4O4&nO0ov0=SvciG$z|gWymYt;UW#~6}g`=TXWN)*!5qbdUwgWb|4Oy5$J0J z5H!pnR~-yvRVSwlt#DJaUT;lr=q z)31G^%5YR)E2HjJ29;f=Xnu~}Z-G4nTe+ON!cxxU(^GeHS`ce;4G;^(WDHgqK3TKZ z^D$Zp^;-|ky0R0C@+CE?P+?b%yMO0l2M3RSn9NA5gh z<2hk#Cu-**V+uegbOPGV{gZnnyi`{r>=a+|5Gnv>ylVg*%eHYUP41DRABe%fllBW1 ze&Bhg>wBN&hN&@^46TY?co-Y9$!rK*w{a#9UHw*TcUBKSZZ91X-K(n2m3Mh0ez=}V zH^KV3+es@JC>NP*j$C=XBohD&DhqqDDLY*(&e(jm>!y3J^A-hl;_-5MWjku`4$)*J zS8DpyDx|wJn*s`kbf7CX+mh+-kC!o0GgV-c&wE|!=$=QRJ@Y^6vZN|AvjsZgBmgf~@_Bs*g z4vP40qVj4=uN{`xs)oAALB+s7b=QCJf`?1o>5_ZWr5|qw4j_@rmSV2gg+(nR1Nd zbiu#VrA)0M+hoImHEQp+Vxh$uwba+vbdsumyYf*FmA?r{eH-)xYR=hMkPz#J;aIUQ z%{su-_HL63c&PwTDp~k+W?HM*{y8co%FOulq&z{!l>q>Js&ETtvy>ff$<4c?U=5n)$oGU@`CFJi(ekHB`N?eOiVfD7}%&mI=0jEyI7Su9iyge zM3$?lOzcB1upE8H0{NN{Kg@QuE_GUjC#UeNT2+buG+s#SiiRI!et~(yULJ#0O1==D zJ4HDvl3M`r0Iyg1#jvX;k6qPbky(5xzlwjz3}>oOxRsBIl^C~d)iYWA4<-mxmUSxC zaiU}9p*04_(cw>&%Zp~HnXnOgx&kQ-up8x%yxS| zb$xJIckp`E6pRWgbON;CZ-Qy84bJjvZ!8`nJgM}oTzc*>Y_?cWMJJde*F60^R|43} zUnF%>A*^`t zR^n(wBDFYn=0g3DsVLvC-}~KCw`~6-mj>=&CGj-1!XTLf+one40zekyTpR(%i7DZw@H}t~ahdfuUgIBE+His9aeu<#*4X5P zp5NyCBZ3C2SeW(_zFhR6Z_OU|FP06=UQi)r?_GrHgFz~8W5tV6n|?{yVPL^CY(zXq?)%MdBB@8*UHotQPNbalWIn3a)tFz=cyjFx|hl@}PU$ zC_uXm3Rq`GxI|?M3S;+*$~fp&TtPlIXOJ$_>MALh{@~xi{KvXq2|S8RXAY43R{x?? zS}k7pTtH(GI?wfih3&rE7P)pFgDVYkm3fDxSuHvw&E~1V%NtwW&F3^?SGnius#=F?nGY8JJ7fE?5daGU2kES$ zVy0LCXT85cgU_ZcxG%a}w9$hpWL~%(MZ8%Q zZ83s4J7yQ0VE=U&hWWP#u!XDxsnF&mSJeVnhPFgSVaG!2dF%9ut5@ax0-R(39Qn-T zswALT?Dk^aS+uOyF+O*m&k+M6EtbKx^D&Hq;F@RK-bC_jGtPrfOBo~e1>+nWHXM&SXIix zyjT0HUpO2(Ylg7aMq2=s07kgMwe8#A|81rgBoGJTDvkUs^7hp4gy4?~Zgn4s`6jPZ z4Zo`jxDmjrdTz>od7KI(jGm@scP|!e`CBtW3_yPB+;+(27}XB&wu`5|ECC$fbBlv? z?MpY~Q<#ud7I=>U$;!_S&gUGr&#yj6#F|ljMr}G&WdkU6I52$b10ij=x4VELNeUME zIr`&S{nA8Xs)X%2uH@_7yU3XT>(5bi{qvsFo5^~M{D=FZzQ(jTEl4CZG_>e^Y_wni zM0nYr!^i65t!aP|4&ijOTHKB}S+E_W1k7L^8p#ozB3O_QAGCT2-jNsgyf}|nwuBA4 z4&e2?I=7E}z6OM7nQ-aVh{zL%*b_?H%^Td2qIt|2N}Y(Ujp9=~ZOpo~*u6T6f}aEo7rY+JsyGkofinXlUv zB`?XJX;WuY8Mea@@P8qr#DY-^=gfFEOOY&%QvF)TM$fW;SH!xxncbJk!pIt@{a?I9~X zCy|&ECc=(hQVF+!W8jUg;{EWYsA!e?V5I4k>$d%V`h;Pdr99KcsOYydyOhTN)V+ae z`hCa>P*epO_gAm+_u_}{(t-EJK4uCZI-w0(6WbiAgB37a*{i$ZvNMd|rt$VM2$F53X zOMSUF_@j3<${7EtwY|MTI-*L<>3JzhldeqD-;JJ#9abogGQD|E9yfL0oLlS z9PG3WM-S6PIVAV|0f+hHdz_?$+i*0h}JE1}SEcW6CQLA3vc6}m61M=2WtwsA!Ewg!&o{)nBW~b0pTxQUjf)}+ zS5xX~VaTfa2xzd_uX*afHl4*lgO^`X`MTZBYChZj1eO{gn2rbeo3^YLy=l|GY&ZB! zW=raj{Fy-TLuBNT!<`yvxrgddH?Ga?kQ+}2UR>v_8KLE5)O&WyM00}#oe~TXwP2givkTvxJ9p?XrM{_yCip~ z$pF}@4S@P^8+27XKNL#HYnI}MO_bRJRZO>N!|yjI%t1EOOsz`ODWl$H7FreDOo`*7 zD;f5})rhi>D{>TeEHc8lA+HKZdLHb`mefIp5{YBp`0ZDP00Rvkb}DJv&^~-*>frX@ zUCwOPA%Av{|4+z&?0OfUKR9d@od;oGsd#o`Mq~PkU_jilL+DSHVvvZiQvooN?-g@9 zn3DrnRPwd??iB+Y{V(ij^B_OsRNs`fJj{<7MJ|Ndt=U00m>@ngqC+Zg39ttCG_LZ;S z7x*<#xO)sG)#%Cz=jeyPo)NnL@-{!*w*K*IMpFajOxv3(8h3%hR`u*AOU*I_VY8XQ z)Pr1_dFKnF2NRgLKCl!&P%XS}1VS<1XSubXM)s}sJEGK)$#-EVQ)?0r4(Sd*zY=TM|wQJ2B;!u)ca zulwk4bT2jfDbWsKm~bmE%GZ|n{0uMuI2G!Pc|4;b>l*TN@O3RLEY$YmOI2z@Kaa80 zH1o@zmx3TwOFlHC3^d>y^Ta(Fd`(00{>XY%^OurIgX4HT+nGSsiBSBP*yRt$vKd!n z@CVhZfb!sTrdCi!FlUFKxFM2=P6dZ^7EP%6c>H%*2Jo$0(_p%3)_T`LT3UEn~pDWZom3#CyV zl_k{HH`<2bYiW_=4&P#QH+kI+c{$djaHFiT`(^|4HtwGcJr@AK)1Rdul3nw6|96F) z>fO;wK*er|8N?5Yuz2eX3QNfqg+xdm9EQMq(^N^C3H+Ptxlfp4h>qPS-IqRdMnSfU zS_&_~5>hHfKW9|W-R;ZOPzmA?PpM2xR!kjU|9Fk4FYj^ZyFqK!lUFE)AewPVj6;V| zh_%xAvh-}GVnSgJoa#~>!&S1;#CKI$115t@<XZBeSihKB{i&RC*H@|H z*cs>U4GXp!sCWQO*~kjA@>ybQ`3!$mHc_&3(c)HX&#i})3w7ew@*!uc&OF5;noRZE0XbURTCK8%xZdvECT(F-`d$zT^`eN?~8Ts4(b&&Zfm zUWnU*BSe6Uik}--yKF`G)!*-eb^^=OTgY*X{Sv$U!+L;6i-UCFmsI8|BA?x9vh|H* zz?NE1?(x&#k`HkKy@Wb}gu^JYMa;BEcYMJn*4=YogytW-$!X~?EU=7fblu;)(a7`I znb`a&VbR5hVVa({)jhi)As%e*dJJfiI)^)q3H}hJL!-c7k8%|!8*Pde`Cg8MiJzfL zux4WlargU1#yb0(&Bp|PsjKt=0DkR#M92R&9$?kHAcIS6KxhI#XO81bnWB&{1>H{o zOlXMdF`JF<`PX|qgC2l&T!w)^;6gxUgzmn(-xVusX+E;Keubd?(b3N|%s-?DlUN1h z$Sc>XkdmA!OkI3vQ-SKDW5i(Q=SvD6oHau!ie(LbyN;#DlWug$d}45)xVK%Tb}|`5 zWRsJ~68oNyn0jU$qtrGSbF)jU^aqk6M6epU=8si2tNkx*7xfTx(9`bCGKgmnpJ#v^ zRx7MLf5o@(jyR-Q)HbXg1Wkb;2{o&#X^@m}zY0cCI!y|KP0;h4`!|p1sl(;8fdg zrUK1HorLO|M}fCGG;OB-3e3MPDMZNs#T`b#a-yuc#R~RB*?;0jLuE&R(}`Q;+aCko zz6_{0l~Im&rF@@;tqN>AZ5AtOHdPWF{idt(xV-qe&+`%%UQ?rH?jhP)JyS5l+k~lW zqTS2uWLYv>a6!Mqwky}K+%cQKdi!+RvhZb35-5gq&ET0`cSmoa^_(?3oI54rJa_1w zI%)T3-ZAzv8==zsPB(_nTq()=V?O1NWXHmkF04BjDFJErG7LgB*M`XRS>FkJo*R}N zA`MxOu2t@P=0fxrAS1*2`Bwvp;uzX?xq9x@YNnUwK?$!TKT8q^7Zwo6@}e1hUJ)k% z#mTj{QINI80sk9E2u=4V6D}|gToh~4 zr_w#Zi6de8}p`KS2HrPgcPb1?LVp`NIy=|`=kHZrRO-}|YM{kR4Ps|yZ z4k!WHr~6!w`0wBOXRrfwD1B8VuwdI>Gh035wLlpCeQGy9d??58>_!vNXH;ibJJA#w zD^aT)Gb|SD*Md7;C;31{+DWGES=}6@GV3zTaGro&5eCd{=!no41LZ@92=vuTds^gyU-{jHYXtuzuW3h#=m z5?Kx=@Ba){>o&x%9Nx6#qMCBV1Ddr!@3s?DxFQj5$B_3?a$@dh$nnF^3Y^M*LIFbf z9{Ff@q^?TgQ{gt}wEc_~5H^ZG1#z$3=-}bJ$MhcHKYq9!1YnMe ztQ|!uq}?h}&tA0qj!y2Sb8gVN+Q>J#hFvFTq;jay)El%&s>s&vZ2?T^NxB#u7~p97 zMv2!W{(y!{D$(tyNiS}5{4ERnM=%4RqTF4tmdea zmxt=%VA?pPNRQ0B0G}Da7;h!2{Iz4QU2|G4cr~XKV>>!-(3t-v>@wb|$2sg0P!{MR zI#71MA|UQYBF48Vm=>pqfTtD8TbB&Fk%fu?!d<0xMYdQhN;!+Qs#SJr^4ej!;8z6M z-E9QMmAyQahsGXi{a@AN$D+|9EEX|mG*Dq#v5^wcr|npP?qI*RLapfHX&7_f!IVlc zWu%p5J$2epoM?B}s@@q)3o&dt8mfN!R_nZ$m!lALWI`{Y(RmP-sRzfwR&Uu-xX25# zIP&1?u76ai0(r4HDA&20S|`;l>J&t1bH#k@ZhksbIvY4Tvlq=hHHxUpt|H`n1ZG$K z%qpj$v`PC8)65Q{it&GK?a!O6MLURj@%(%!3|cYfzMl^YwJlJTP^$)T7p2@m#z@XS zO|HDSP9~`#DywP@)T*@bZH{c_HSq}k`BxRX-%r}isGF@iN2MxbUS%7MajP&B5a~bf z7}Kn1MVW?UDRa7(Ln)GQqi5-2O2mVo)8BU1>_nx`WURYo0wobcEjLO~dc&vRa5+gQ zIz=eYGzI2{uLyU1{_g_F?iqKc^F40epe)L-pm7rpZI#xCihH{dIJf(7s$rqvlLd57 zs+j6{xn(l>(yU`(4dgp|Ahdc^+tpjPbEmc+yvmCI0Fq~usk0{L6_o^8y5;%-#_Vn>yY>!sd-`Cn-o34YQsjUdxc7cp8@Z}s{Ki?B-+>p~@Tz_Shr6$hsk!$xE#!}!r zR5K5oE61ei^r3)Rf%|M~8tAwqk`q&O16P~l5xJsRv`Uai+_3YJyd&;5A?i)4 zxxuNKvg%1;2cFmDL|$1#=qJoztc44w1O zQh>taZjBx8r~`vXvKob{aP!jIAtFS$$9ro4sn2%2WaBw|K$(w^H95qE`3d$Eh)x zMyRn;&=5lxELXTfYmg`ZFv*luN~bN$&>_OUh#P}BBlgH*923p#9p zu;KLtOF)U6l+Br}Sb&o#h7+FNGi6vl|Kdx+GrYxQ!X=Y}ilC0wv?_q6;&B@F*h!v7 z2QFGE-W%Ozc={us`1zR21zP`hvz5dS3y=-EZrCLU9Zzd_XlMn}YfgPxZ8m=Ge9M{s6^ZH*2Od^tYEt)k?hfrBN2Jg3Xn$o9Iu`D~KYU1HL141snyzXXp3yffgSr1qu;;Q&t>&irQRVntCAunV zi|z)ayA(ZN3>j#wxkh*H~w%tQaT-;hB zNt;h37ajBla_DLokUYB-A~a(o{>=G1xQK+cQzU7>+Kke|5V%)y6Rv(#DQiMVZz5*n z0bbPgA3BZuK%%HkYoX^NMjeRw>5BjPS~?F@VumA2n2JT_f%5geAXL~8^S17d`A`+6 z8fZ$XXfbixq1Ly@^2!XrX1B{trek846! zde4ddu(~vz7b5RoNz{*ei||o-XvP@Q7UVRRsbZ(}vPxe%l*N=LUV=|3({yx&Y>(d_8Q%CfF9iT@lP!I?G$qz_f z*hDy^T2=hA6RU~XD~2|w3e54t@v{LachY# z`Ha**SlN#_wR?sP9;j$=dUpX{TAPn2SU4*}&x90x)Y2C4XxwN;47qiwIQ8lUfSKlU zlC&}ckzh}4W}rRj-uvOwXFWx`pP1d>D)`zt4RRyXFIsPUoHM1oUPM_j04E^R-;um62oA_vVO)#N%0FUE{o4>gv6cR zA4KfduFKST+jz^`lTE+j^Vbr;CUlPLc8GX(NXz!4!em7OZV-)U=+3yZ2{ z?m+YM%D^x3yuUx65rgE3hzk!epHts3!EjTm)=l9(Of}|{ci^N(e$KXsu}qdwtEy}B z#dm1#HO_vj>`2$^_IP9XE#LVq6B{J@a|}{a9WfgqNZI#ThyosdN4-@5;{`*Nhlh`C zpQ>9YNKSkt>Ig(UFy6QIxcU&4$8|v>e_}JmSETq}fo{{MpF2N&0IjIq(2qqC+kgqx zl37&5=FBRp(DT1sfj#Pu-NtWfy6{lE*3P`O(EGSOmY>a}e(q6jz3mXlNZ+1yYnH2A9ED zyM&Q)>AB3pjWh8l(F-hhSesiO8lOor(+XnuEI;n}M^vniWst4-#aE=E&b%Rd=c~4fNuSK}(j_dTs6!RF znU!?@3Ol>>x_7INNr!HdG^#4B&-kC4Ci*SFl{@PX<+1($`O{rnp8>5)@^Gm_huf=K zn8v>JAKro&C_xs6HGA;WZZ^JL_6=_uLwlIeN5zTL`7f&s`PZ9HiZ`P^3K0 zzWBIp7khzij;3d7%6nt228#Pb_w(5BnoMc9&=`zz67x1o?~>7eneISbAx&5$Bqc?%GYy;j`&Cm(m^^t zk5Mnn8?2h`WA8^jKJGHU{Bf!{LaJ4HhC<)A8vVaZ0S-qlh#9|7qZClt4l8tV>Q54& zJ7u^2svjADtq~hXVGfO6nEKMTZx{d+W$U+QJJXL-E8Sk97O(Z{5bo4z+5ACB4&0|p zp%v&REJMHdv>22`K7Ts%wE7c>zvPr@wg>}Io+yzjQFiEctb2iOn9aG` zRM^B%U6r57YPA!y1*wOV#?#uo@4!Fy^6PKSScU|YQUsNoWj;Y*PFwmQ{fFDJ!}JEOnXs68Q-B}o6O(5|L6iIEHf4f;?${B(z+UY zr^DL!jX?eV)kvB(%C#V@%ZO`oy%?E)3~184I6&vE`S3;dJ%MV6F%dFhOV^Kmrr>Wd zg9ShY*@}3?{&;(;Qt3ZXNdVdKx&>6qJfZ9Wba_JkeWFBdhWO?Ygn1AW{FH<7;f(d5 zK0AM2*bdmCR3JA?Fu*X1gOC@#9Rd36RQYCQ;k^<5R+pcS3rP{7DOJ=$c9xr^)9^9j_HNP0X-B_f zbwwptS;KfSlp9;R#c?sq?z4zjA(rI(K%InPO@6DVJ-MXB<|s-hu1i}$?E2rCS2dVn zeqe^tUw|Yfd-3nT0^Tn70EojYtb_@ovQ?5t;oT2#zZY!PSUnzA_izLufwm#1Y5LOD zW1rb&Wws#cz2dYQcZdf3@^~2nz;eKoA|DlLb8x)onJs1N+I)$qz|zGR+H@r6Jb<&u zX{j>kM6u3X{;`-MW6&iCFml}D0`j&oT2!IVWc>S7St!w!&jD?(`_rUL|G&yLP<()L zJso6qcs8&k$D8@nIQW$d^YUZTA zD_;0d+6r9F$-G01{^TAeoZ9~_PVy)ncn?4Ki>roF#fR5Kgi7vu*3hcz+irxGL-+T*__kzUkC|& zJ{5iJKiY>DLTK;-0DDajV$cebEbv#-Q~4GetRJM~drwRB*2C?^8dPS<9jH}3FrG1F z-xUG3o(Y^WimH~40guH8NUl<+q%qK+kfk9BB9K9y}q_OUUE z+E^#EgmwbM+1Vng@cqj%x&ptr*8a%NBr!B|QTrG#d`-4ml1})K9&>dG1Mda0QdNuK zWvSYcZM+2o`FPx0rXjD2cf`uxexzG*Hu78xpi$A2MIyC1f-;u z5DBRvrIALYMMNYANonauNokOfM!LI^M!FlMn}Hercf9f6`{`Y4mTN?WGw<1FKYKrW zpZ#EwP~+CdeAcC_Dk8tA>Q_F^hQK%9(rNQQWTRQfl?NMlKD^ugxY;7GOt!QZT;*>3 zSR?^4Lt)y!$4=}rb}!=cMKYRA0^${aOT&udyHbnRU*Z+O2T+EX6c-FA{i=Pd_1oh4;}Yv|#(Bqopp?Hq;8$=RV?YIh{kiatRfSi#cRP{US# z%G51WLg%8j^quDp8dw>6#Lhdvd8|6C#k5{%u;Yhp+*Vl{^gC(=s>@k(if+~8v}3K3 zWu$!L2R=|!s_@yeF4Yc6nzM_5L%CRo7$B>j4#|O3{91J?zVCJ8rl?c*_n&mzzoTS8 z;aN2w4{ET~v%mi!K|eIadS5Js92O6PECsURa9`4DzQ}s0>qDo;bIieg$@O`X$do0K z^)jb2VR;ofI4k^G!xU}hdHY4HgKkj_k45KW|J4a8Z~$5H-g3-a@M20S{6TFtth*Bt z&RImX#ZdT3-|y1wLuQzlXoHH4V$)i37)33!B{sYm=hw%chPmI6k7DzB>({j3MIx1m zSMr;*?xyras&*I(2u0F0z=v$KiJC&CDgWcbH*C{ysw9FMr;8Dv;CMDX?V+XoWE@`F zV19qVEL!UBFHZx>{3K$k!aR66o@|k}lWR%n{?2BLO#=;jvBzu77Nyt0=Rh1>W zMSW+E#4Y+4fH~-*b+C>by!a4DII*{vCcxq9fo)l%%wmUwI_SK6mvfC@Pgr6{UboE>ElvnLe=Wff6AAjyK6L4Nw;Pm%r^4T(IF>G z;SGE&IqANk}j&esKeyF#m+=hwP%hb#&{mUsYfEWZ@qj~zbA8~CTu z?n@ZP<6PU}NYd{LpHJh4KhT4BsZU~`n`q&94G@s{$+r4j-meNh+{V9M^g4%R;`9Q> zjA54Y>N0qcQa{tF=s4EQ1krGowu<|d=6zjNqTs|UeyF13If>4%4@(rJhp&t>K`h(* zC;$4l5qk|01~_H1Y)cS*;BO;%*7pXCM{rELYi^Eq$@(5Sy=K+zW+es!Plx5)eELTR zj^YAv)1bw+!H8c8IR{jyUod88RcQNiW*L^q@cuC$&z`3pPpw#SiK0eEPd7;}{F-KB1L_^e<_PJSiLoPxBa&-Gi{qA6f zc%iJ5y%3CnB{6YI6bG&L(RJVXYZid+J&qW2%XvTZ=1<(FTPE*wu9t#23fX~qrz(N3 z==;45fX;en>-f$Z>#Uf@$6l&&q>yn~NHyPb0LSvud>uo~KfLl^Z<|C23@v0)7escQ zFYo^&6awBF(t+`0_7Cr*N$R!{WqQ!flACbTOTNZ}(H%>~c~bQh8_*w5`%c0a5+>nG zx7&x`0kgPJ3%`Cc!u6VE^)tW0y=U~;TAPuqm#2oq+bQM~qF9#$IW+y(0hE{Q;`#lWtu<$cR5mN?k!LU%@+P(pn4oC*N8Vz3G41CxrYN1@B^TGG?I^3ns_>kFcVlDW;@4K0dWo8(AQI$ajk;G%x(K z;=*mf_{KzMJT84-BLEG9^JV_{5~F#bBi^wG{&fdX)c)}(9n$6`p`Ts-ii@76^pI{6KNBa)pRD+?%R;V6yoWQWI(D^zIM(S3nt)g} z=lu7yj=wAz1(?A5*w7}_i{zIkux(P*IrM}IqEm(kbfNI{k*C)5Km;6abCuatlrnYs zfcPslz~~`t@1N+HH)fVT1q?eR!|98`T+Ey>YMTSb240Kh8(WU9dauJJYpnAgr(wBP z=k^mXUT9xHcZgf!TV0CUhh=Ds^t|>o79=Y z=0YP;o#20)1fZNrQx>p~#YY!X6&M_09KgFHPZdB#x3;WDF^#>~Bgo&W;^U5w;E@th zo8H@ow21-cTP|$s0uWM866gMneRupW1tiE@0I-N2Sl&IWX5kX7Z8tiy#M_TUZKDU zXD^f+Izi{a#dHIftcfqGd?MkZlK7daxE|zyu+3`S6C9$2IM#OeZ&Tkh$zG>g>3+U* zst5F8+B-Cw(>#2!0_MkkJKLEuiHO@|H-mPQLUt=~)+u^H^q_C$k>wAxPe0K*-&QSa0}djHBxweAP9I}#Ihuj_@N4e z_sNuRH{UHUNgr3Jvv$}Z$7bZ-1{Rg!SkbWU{oVIsHbl+TrB`3Wd60c1g-6w6Kpzd! zInVNI!A%T=2z>MD_*p2|v8g6SRdPS!6x992&oR>|o1{z24T@y9ze;jq7vqO_S^|@U z=i!yEzsVv2hhRo%h<)uT#(wkDFHJ1Pk5To@oCa9YT=6ZVpesYjne#IkFdIT8T}gL% zx#1lK(~G%p`?1T8cM4Kw~D=s;UvO(A9d+`0;9LnqjHp z4s3@X7V~=Hq_LyC)AMx&{95!xD^qPDo*0xy6P0NV&hp@Offiw~hvd6+LaVpERSlPh z7ym-DzoW=Y=><En#3 z`aGBb=45z~hG~akhM`Hi10x#*zj2lKmfKiGrPpf%{Mfscbs`gbABpo$H`8x^M;}++ zwAUOXk3Lv_MXD{ehaESv4bfYE6V4<$=2FY}N;yCVK_27fQrn67DJ4E09T zFC(B%eNpzr@Xgx>&=IO#_$t&2Fe1C{TSrpxkcITe64ZNnz-7Wm!I(Bzbo&y_ghTU% zl%13}@L?t~knrI4&bEfNE}rD&#=4H!GtY{1t^@J1;Qi1~`yK9g>Bl6F{m+Yj96gi= zx1x|>&`ySr5~5Ngd?x(^!xPF@bQ~aR)Vy$GKk=SlM&bRrY(s4DZOZ&RtycdNAOl(j zbaPFiHbq2kfK4>N@uS82D#fz2EF8rdb6nM0_WJD2;xp?Yc)&Dq>j8cmbqfT{9}lOE zFn0(u(_NaHdB~2)`IsIKbkHQu8}sUR@_pI~cLDL91!py4M>1U)f+&1EZC<3DS$^3= zP6TI+`=Qzv<~DBrA7(%hC*^5v7#2T}Zv1a~=3)(yYnSzN{TFms%p(CKz?d))whuW* zna%Xo$_QYd#`U@_vE&Fpyn4HF?+XTK9e?e->k~DFK6^o{BbcJ=ClbhR!zYZXtcTOo z+>1tU3#{kRdt9V;3ELP%rVDCQ@WClS!*_))F8+<%&0)serJ!MSAI}7K$2#^KKgXf{ zVg>{g)}WAD7L=Ji z^n<^|k^XRP8_6hp1Ob9J^lQNrpEp0E=S|*5ox3Q`48S!KzkeWm2DcHs40w)@pwVnL zVa)sHR9#PEufyFLQ+jFNYnL=^JIh|QSp91Kd6JNckUKEyfPcL7n>F=W7sjA%LH0?a ze(B80^gT+8xAqJAR}4WIV7?cP^(fqjN>9EFF>VKc;c$L3 z7hP|DH=W4hajO9}DAm$SYP*Gt4Cv|v(JAtQZ56Z1&Y_R}mO4`GT59b=>?Lf^RVUE% zfKj*?FbZd|$DB#Ebvzc5D(wR(H(O&zuVtdMdduICj=-=Q$%WKl z)Mhu(D}#Sx1t%IPbK&zQhZHUT5epW1r@j<$vr1ah&1YF_lrIck_J)mr0Pm6(9JpaU z-v2mkBmn*l7+eXBp9I@J2%`W}$3FIvj}7LX{TAZ2(I3rGmKt@}@yEH?V{O2$fzxA; z+v?Z$_lrrccZRDLf!3v0B;xli=yTit1JNRyvcP06b~l?8S^hm4>)kzNED<`aKpiZJ zc*Na=mqTm>qD}Zvbm~j`?ErwZuf13H41c8m<@!~~Akz!Tl0%!faU<0_|@elmS!w*2YX9cFie_z(F=1uqEJ3nr8 z!Wr{@4h{T6{4mIih$@Nl-ggL((>bt3OuEo~k?zGpTxl`YOG1zw5vLCdd%g;9Xb=s} zd*)pN;(Kq@iNJ&(m_ex8k62-ZQ49+$Yqwd`yXcy!q7{ko@uU`9*Cj`FuOW^f);}wy zB!Aldw41#9X}!KMKDV%bceil2@LhJUHqc);)uQDS?G48KaVWNuA6&#tJfZqOLV$^7Pf zi}Xn=Ye6EnrkTZ3omub#gM5sS^U+%S5$9%4tVRlC-4w)V_`BEs{5n2CDf-GnBziNTvPp| zd|5l`2rYO)SZM`pXH+x3qH^fuha$~R<4A+R05zn2Gilj|Hve9w#bn7BG9lgDeHrgj z&-mp$n3J{dGI}0T`H6_-frj_SAFzf$tmZA3vLS`d)lBkx5OSja?&Z}CG#DOYd3ry7 z?1vfW`PZw-hae*TE=PxLZ&tT$FXP>gVH}WlJ1)_MYCA5AXNLRi>rdvBU@!IVaEt%GB2%mYY!Md2yZtUfUZ6z}JSJ=|1((EL^4~uK!gVlJ+&7xa* z@MifuxZ)CPvW6b|5|}L+P4#BRe2xyY_YX{o#{7byoXB>tLeuYGhZvknyTm9*T@wTU z_=Z$MaNNDeFE5RWdE!!|pBH@gxSXS?hyknMRrr!0z$svCfH!zk^{$Ug= zd=Vns0&j9lkA|0JT1Q)|dju!hXS+w`7*8hNI2q9x>vjeR7Pz%OWVW|x@{Xofd{m1v zo>lGZa)@S3X%xLDfqsn?=Q@=HAjPn+@WN$ba&ofjA#IPR{@1ToTmAF@SlmB8NsVk1 z8;ZkGUG>=i8xto!I8G?KMS4q#_!M^W6zs!d=@WIN+Fq}o#-Ty%y`nQ>WVSi2%;28*dASbT-aH{y|k`vitjOgBx02KI-CGGj4xm#9?f z6~xHF?GP5l8b zI4h7hkyu7XY)7QdsU2bB%-B(y=Vqqpdx&56+M~G@0%-4)5{gt~eu)7o@uVt)&`Igx z2-muJko%0cD$LM?nMnD5q=YCu52j0HYI*WK=Xyn%%s^gwO?(JzwXouhDba-`Ll)J^ zbtNnBCCtzg|Jxf+)teT;@Q^zuE(@9Ru^l_J)i)>T;(j#0_?-sNF2RT+$gbdnYA;pJkG;) zMyY!>4T1|0d>jhyB_j^0ZC(#?Uu8yX#;aY)FMmlK9oTiiS8ilOt1r9m^J57kcIt4v zU0XIq?bdpAaa2Dwf6gAeDwmbgQ7pwNH_3eCurA%BNUiG{xS($Q=$I-W5nyEUEch=c z@wd15<4uztUYt=x*Nxs0XvwVHUpwS=K-i@QT69xDsIJI_kJvnK`f4J8xlyl&MPG72 zouT|NscmM9oiMdX9AVl%mB|Q2`84=7MVA_ z2JS&c#}$f(A2CviXiXq9s;6wjPgs6@dD{OYTMbfOdJr6?p+UK8syRKH>#lj47i4km zIvTzNQOA)`OA?m3W?ZGX1T-)6P%0ajhTT{mo8{neRVu5~rGx!|b%}J#90;E#snygUSN?sz)owH<9yDe@l}(A0fbzzq4RBpWWkL^; zf)tfE*W%;(to3Kx1^yFv{4kt~BK$;a$^JGJSCE2}Y??P&PKp0aP^4ckM<~6cyX+~C z7Xo}pB$M$amNh}<6OcW>wf};tK!SQws_iRuvRq=uW5sfA)LS6L82pXCz{mtru)&lG zVfidfU|yrQWo=@5sc}-I?NXCdee#*W(8WO0;4?=$^_$Yy<-Ad4Gah9Z3OELQQG1+1 zZ1$Gv8CsP#JEeyVe&)gEAsVO!hR^w&*RB7uiGSn!2|+Q)u--MP_D`cyAVOsfQC$r? zCUbnJ&|2y~xg7X@xufho3gIVJGTB-6!W0OUi-a&v60}wM9lcix4yE@offE}na=lpd z=VxZm@YAT+PkEVU(9L?e$k&~YP<3N!6oN`LgJDpoj)d3ft0GNEnvzD(>GAme^)5TBY=&DFOl8uCMHhgEikV3F!r6vh!oi?7dQKjh)MBXL z^5FK@H5BD&LRw`~oa5|0p82Dl+>KU?j~nbZhhIDOo0*6vWB#X;cOW#p>caBe5s@xT z`uXge!1!IP=7VMVDPfsiLmXjCFWB_$&O5n{}gAB=s?gxW!2ntP6Z-Pmvznfd&@f7zf;{3;jSG!M{{DzK@s6}}1W@iPtY zE(&$1FR4-umd)Gl#KE}MUXgz&W>sy^O_{Ex|V zF)5tR&s+?=V7Ij;_=o?ajL-gN!V&G|ARB2rnX%{h;-nc2$Jv zN>$Ci_Pg{8(@WK!mZwHJNi}(ekw@HKvNSZSUkdY2JA2+RBcRrRA>g4F5*CtU510TZ zR||ALcwXL$9Q%xebzomPdy()z$neKsMPo5I9n`uBZgwVKm9v*m+w*u$VXi1fy@Ho? zspGn*Q;>5ui@U-YIC=^vLU)NrV`RpWnS45Z)?$0d zr-5DLuvPCRg~3T^hL#6QLb1hlJSi`Z>+BQ4*Fh-oi4v3E^oK<7=LxY5X)3d()W zMB`6Q$9RKaqp2mK#z!-|Fw6|^rJRJ1Bg><+90~_7v!T;lS`M};4qqNP@4CCoF|h|F zl#kd{e|^Bn`2c+E+7tsIPivY%+!AYw+2Wc*g}11F3VUW95Pgjb3_woQ5*^Blr^_R3 z4c=t0r%xCBFmQoMMiG?xa`f=2p2}aU+p0j~%L8NM5ex#OV1-mnkgC(xFaV&kLygZA zQsqEG^PIsX53rD7p}OB{!$MN7l^Cky=5XeaGI}AD2;}_g`G3JsJX2HEH*7-Dy8#w= z8{{9M6249r6(CEhNhsGpHm1kRk$giY7mud$FbE&9$8^$byYg(&+hX-G zv;L&bs`)f{i!t>FwP4<(bjLh*BRPOGE-X6YLxqiuib{5xH5RRl%PA1{1aSz48lRnG z*z}jBr)ZwFt_j&|54(SCP-XlPo=)B4#(W`53(`5L`PO95hf0X?)x&sNO=0?y2gXu# zx}H~I2GV^?Hiw>Zuh&J4utm-%4#@G^1X3|%^zQ$Ebo*~N-$xKi{$#?HITKDdpi;IO5WN$xiBE|_>{|zCz+aCg?P%F6%&sbp ztds}ZL|rPMM}|%Vzn)VYyfL29MVVJ=AAO65$WLm`%hL(huZN=4iRRc85cVD96$gNgFBHon53L~z zY7`URV|E%987=cI6AMS#E4z| z#L?GmUP2}jLXbJkcc1hRIwxP9v~ER~c8*%c6^mVDIJ55ZKMtwgj`EM|Rcbw)31SlD z2)iOe2kcJo9*qekDh_JSAahd0+-zi*xD$RLy)rL#b2LdKFk^+OHrNuHq}T+0Lcj1G z@nm^@luP7+bF*iH!B5k`9PLI}zDH%O|uPSvrmi21!r)hGa!o)Q%ja>$_qQah}DMI`{ z60jk1=BXJV**snqcnk0Jiw(sdsdlW+tx#oIrBNF)h-2?5PSM3d(4PI5&+H)#IO4+I z4iN~rz%6n%&Ka)vea)g9_DA}``2?iB8otdnFqm&Jw5*cSXxz4x{uaY#jzKj_9zIz}5s$##c-w@^N89fR{T?_~I{TQ0H{MviG#b^?GPgbI_7LMU?%*GgaL#>+Nyd+w;# z3utrK>GhmrrX~)YHpKxV-{DXb(GIJGDlHc8Xbi48J{j^s4>GcA`^7tdN0a}2V;9^M z!emlo-=eF74D;h&fmFLTOj@(F$PnHwM@wyz2s0d1Me$6Bt*ms)YykT?>b@B0Hyo;w zukhR+vY)q`G`no4`7z#`QnS|H2g_0Hk?yRhl@@ioKTlorN=;{;Atz|;-JOvuyLYz*J@Nt?YX)=%gPSK8O_eov?N>35lt^tt) zSKala1NQz)lAsncB9v-s6^v2A7qga*h~K--K@HG1ZRhLE!ers`X(a3eL1qwX{PxrE zqZxIc0RwmMmO#^-SWiZ~VwdV1vB~|qN6zkMpC)Y>-{0T47ctkWII29JV$l;NXl~jH zSnRm~RRPKRC@(O1PP&zco}oHRMV1SLPXP_4bSK5P%b zD;YIvl&|vD4*ZYuVnmaK#nF$CIf4 zPkieoG`u~4<=M-9@Bw+Q%07zb%7~%aC)V)bC6j7C(r*+R<~%GVVv1lTF_~TYcoLZ| z`{v9RS1u*1x1;<8mE?Urx`xvXs; zJ{oA)TbPR|u$mIX`PW>PKHKbSf5&BL99=L)hR0rXfO4N%d1DrlPta(d7G<&{uur5^ zmkCXWU?=Kl$n#YvA=_LTxF((!TAt`aIR~@ra_k z$}0`w1WH(oEpk)hRAhs6jp{M%dIFWR-;t{7#TRP{+0Es(MU4p9eq5&zV}AYV6kF6u zD>Z`izJ48le&yUdHlN8>;G9r;spE^ojo*;kp9O32=Ih?7Ar#aoi_qaMh|qCRX)HPD zUCX;t6SR*sAdIAqG6p`}7k?CZ8EmVBf!UHn=OSl*5c-Dgz7G0@@%pDn;bxAl=nO=2 zKg#VncGYR$-XT9oZVhtZbUPeFJq3++Vhy(AuDaSfdCxYocVw3HUNxEbhy~bFPTU>F;@pAa6=!E6uU`y=bV;RHxKDNw$xIvvJ-9Ps@CGomspvDjQt}* zGp?x>to)^Yzk#`3sDtBip$@b0WQoa;O*rt7u5;3?4u4P5rWSx-O+7ky-m5?JLyCw( zQ#Sx9d$(t9C*POBbLHdu6l2fL3&<*VDyIE-#&x+5>#h-(tr}W-#9H;np8h-3G6Rz1 z0B5#beAF-4>ETdnJ7?{2IbA%WB-k!C-^*Tm-;YaF@^1c`bOXW7B6na^uRmL16R=7g@q7zoNJ1-I~W@Y5BINUbn7Xd+czg#k4LNm25g>@a5MaAT+ zA&Z*Ltgkzc^KOp2SYB^2cV0h0Wx-PxQ~dU72O!?2Ig}bXGh_806)v5XbZ6&hc(>;y zxDGGt?^@KUqtYlwhH5>!AIjwT-Ql_Emf6n}EDIiP~5Q2w}Wn7&eWv-7$`jqEI*4Z1`xSS_>bL&3P-|%n)%Wee=Gd z_kXhMKmPjq2I^Yx^Kvv!IV z!8^sRx4CwJA6A+OqPZyUnSZ>V$eoQ6Kmh?WH1ahZgf8Y8z~AUb<4Q?wy??maaJfrC zjA~uGQxS*Ah*SzXsXiW-H`W^fe!xDU+bpssHA@!oYI@E%T$tDOv@|7@?TSk#vnGuw z5JVH?DJY!L)b(=nVm*brU+C!DvoUjn%PE_!oNT`b6%%wqKz|D^;IYivh*3(tt*~#) zZP{_NeCRVEB+hBvdLQZmG+*uzb8zhC8@$VR?IP^HyMD3S9vFBF!Cb%nzFRYHyF2S? zxSC@Hj*m%e@L2P~A3;nmQ`1{nznRrtQ_@iByUx(h*&YHCo=07m_R;BQs1__}=@b59 z%|5s!lM1_82O24(;Wbd9Kyer6Y^BP*Raed-%b#{qgE4sarW^v>thfbFC;CvbjAvO? z`@<9zDeCDG0eyzO2R)`{2J?RRP-a}N04V7GjA+yo;fI(Zvp4^u3jfWgbP*Wdw|hl- zgN9YDg4JQpdEIy9DMVOlipp+KY|-<^?Er6{w#)NQ$X$35AoXE0uIGrRi6*q)bfa|| zm@^x91&m*HO~f+>ZFkX!hxB;oZ+Bw3FvGb9+pye`Ba&Ns4G@PmLsFztzK^4JMXnZc z2NV-kNc_H^DhckCH3hw6-uy6tFHOnih%I#Wb+XXO8;2JIDL~w+@3O;FDL|8lmEP3p zIf7K3-XD%_cqFtb`xL9}?d47-uHy_;TK!dq)r!QlvS)@>?RJU0I))GW^>gX3xOb}z zE_$>&DoW*O^HAn&T5%Rxo;u4tbDXs|PHx*|DzhGJV}?f19l+YVZ~a)$=A1MWe0Q;W zk-fS7XVt^25O2W^W#}?sJKg=AJDc9z*On4zTMI6)ER20b2iWqBGA#0=!E@ZXYAz|? zx=Zr8`VpJMK3Ha_4`HKh8ix!qz~&i5+S_OYfzXJeD4Wv|AKxr`48uTesMaN$eBXg$ zsynEz5eYwc5QAx!?-MrEn7toz$;aFe-22Fkex#IRodLtF0{?#7Ui+aO$r)>-D&G(; zG289qo2M+S-L3gD(ZQv2K4&+g2(=fA`Z3W$3Jiz8ONef*{7uILe)9P}6@tR5YH_Af z_QH`#2V>rOJ!x?ra_!6y+xQ&#kZ+^xyQYq>1g`KSD|qk9U9$`6T%UJbU43$H%s;sJ z?e?3|2Na1OsswTd_;A`tIx`W~I#7|ooC4G*B zY?14W0d!4}f^RoBV;f?355ZoXw7XlocQ<7Vk6+5Ji6)isgx%fo&MPBt#2&+54a}Bo74u zExmW`#9D0yZj*@Ecl4v7FlzvipeKzcy)nKQN9+C^u8X%8ku%EkDW>n3{&*s&{O)C@ zgysXG%k9$TipKo=Uocyyts8|4b;)iWhX0uymqmFd>r4Q&OdKP#K9x~VW9Ap#2ZS5g zsB`|Ma_*|fKNRR;>gsUi6lOPX84f9k<#cc;WFB1=@BWg~@>XrO_2O+HwC3~@0dGhv z2N_wshlp3`ZSd7k8B7m+1l&_SrVXw&8t2`bK=MS?&z@xke&X$mqf@YiaDeV^PoaH^bjxD~5wD;v@(+DD>e z@l{+1q4Dl~(choSafa*mT7GH2fQnccW)ErI0veRK{uW1_jj!FI;l63O@M*vei6}0a ziHft5UiM)7^bjOm#3W=Hr`}4llBTj(cLJb;s`9$seQG3`1PFNVoX$GV+YHMx`QBo$ zGGu4z@g3KFt7vgc?gKKWduCoxzLc>D%x|wE+sn->#PX-K?b5v?_-7KEI-ctgvG^%t zT@~V3R0SVCy>+AwxbzJ`RFeU6b!-~DTah0hKXR5fkxdMLInI;w%`qy3=!VI1%6k8i z+THKUHtc{g>d}bKKVTq(c~;zVWAjLUY4K$5lVwpU@Q4`)fySfUnQ^iXmCZy zeI=hmZa%oYnG_Sc2((leLdT{0Ng)EPFVuJ@faeJJn`eQ^9 zL-kGvOB?=uG1VAy*7=2!iD%sTN;hvE_I|2G zz~Q`Ec2Y%4?_)#AXTehVH=`MfO-nRiHoeBaVN z{)&vFHfcDm&`jp`GO)WYlm8=o>LEB^SVZ?X?%%Gb(JQ(FK4f7rY%5QiihoHL2!iX= zpIlp3&hk#(=N-pFYicN`!V6fUj#Tr$7wQPx%$8n-R18TC72XnudmxTdq*uj=P(G9? z^3A)xHFPm1!ushe2f?T;{9RdqP6X)*8wG@8zQj2yG$kewNp%87!qAk#8st{u9J@y5 zUE5V_uQe^AQ186n&k9Qm_$VUn*X^h}B7<*3lu(BJOj{VKO$E{AH8vhY>xZV&A(vBI zg*Vn;0_0T_b?#>4L~h+CA{ZeJ$H(&84X<%RtM31oSNwzU<-7}0WJ5X3-Xmu`j~m^} zB}z(FSf+BAFj@ox1MIU|M*~XHE%E7`>#PK;J4~<2rwpgE0N4^z8e`4BWob*LXgG!O zy4Xo|+f2~IstD;4?0iuoKj*xs4susLZEf7_9x1dL3369B&gx)mj1u&&TMy|8F<1Aq zzN_#qM900}qMbv_rIXjIo3^RmYZ%+Fp>aoie(^hE$~{IG0+8xWz};_3PN8uX&`XUs z9U{nd<_6juY{5QdTo?<&wE&t_^JC!(pyGZ*I;ApuePU<3CboRZdshd|QH88(j{)0L zTfq~xYn$ffDVdyh<2F=3QK->q&^PFYz-f`d(f) zHP3mR*a6P+N7+N0iiur+w46#7brg@sLX++zqo)jtdT2@-&O?xoN{4}sqi=Jb>LL?!}W z-SkZ>uWH_daN2n??yxvFO44}T_wy$*OL(5u0Z1kN8G&okl%*-SI>~`13FBsAED=xr zk0aF~=6Uc@jtB*E`Rzf6ibt&3zZlRY#zAvpNYquFZrL$YMBy3}8f2R_ssR?eJ!nSP zwvtRLf`*4u`6A=jLrxma0^nVPDe# zq8G|nK#^j$x?fj~aDxsN76)`6Nwqp`JkKo3Y4~Ux^FW6LXTY5Zb@t5uK+YD)A^)%8 zrY3;v#^v}163qX}^A8e$6i(A3ykw&C32rUDsi#d3Qp-Z{W~bchgTHf2Sjs(g!m zNV8XY?5lY*u{ii!A2MC}pO;q*K2Y@URSL*6{>T0eihvZBPE-%+zo{mQI`BYPV2jqk zfBur(^aaajG{BvGAXVnW$y0c)b)Vg8*I2_;e+f%wv#VK`=6VJafafqzaZp1c7?*|11NbQk6g(hyqP5VCDO* zyYH8z68fp;Nu7h-^D$Pf!+Dcf#@oR`_+*-ac&nnY0d-kU;oVh6L>ZQ)^d@fW+ngQo z6s7t89!=)*pR6@Nf@~CRpW*5k&?i zGldCGwaNMg{*&uD9;0BnC`9~Bx-HR@L>-o4?6>aHEYoI4J26xT9UEDwg)M$c`>vVb z>YnPv5A>bj>MF<|1W0{M6{qsIy|q& ztqh5hWGsiv7#hn^Js~FkB9b`$X?HdFu z*W+hG7XQ_+5`dPM={!JK2^qN4f`rU}p2K;3pb@OZ;FbIfY%DLJ%Q5Ao$}LT8L+0-- z`~NqbC*X9@!l)W2%0G78YmCqkq}1a&q0P|9$h7es*Srf$t~*>-no*pK8D?~}X!tX& zX2*{J(YifT5kplD^48`X`1I($a7@Db2n{W+=B2~|u1PpM26p*WO0-IiO^2>D;1Z31 zDG9Z&htgkfe9pv{nVuc!Ay$hKj%$5?f=XuO&w zQvam*kHb9Z2FkHD*osLvP>ZyhWqMqNrn(<#o1%3@@$DNIU|Lz;DD!l2Dp4~|`dkwq z?*N^TIyH7?k5q{=t~Kf^@cnM&Jxx3B%=P%`)Z%ynCe&CH;2$UaN5V^An5)e*MA4fx zq2G0)Iqg2~1&(wD+t3crF9(Se*#SF{WK`@N)x6W(9zwE=xZK0dE3DbV`t%ZAr=*kZ z-j}XrKME_11d9^ct89_$_og&ozHmN!i)05wd8b}i-^0EOz3oS?etj6*NSS;PV!DzH z5XbxeMh8pnM)fCSI)y^iaZUFK24Ct`Se~0^$~8VJ8ddzX3FSDeTFN}x)bbZI@U5icTE_}#PfSiaH_9bRCVjrcaoMMAuigK%O}0iHrCku)t%?X$rQiIJ z!q4_+qHf&93{qt}`>t0Vr29KR^Ch-PAtCyjY;Xcd5L(9@PRrl1M0LUI_uQnvOmY44T-nf#DhD8-5~VH`gwo;jh)&ptzJ(o|LeMXybfJGf1LnMhIXk$$}6 z)^4;&D1GJjBgJIuMWB)D9_YAPbtN*16($Qc`Zy)DTOm^|8RV8zVo>q)uA`n_1JVa5 zP@ob$QiidWsuJymv@)B?Yb)~(=(1pBxvP@3aj->&(scu^kDm{ncxO8Quy$NRWe?dD zSr0i#xKZsw9isfU6<|rCTdmYR1=!Ef&DQ@7-GBV2S=uHiF$NtJt?w(;@D1t34hn1R z1j`ETNAZK|2*~B+0BG$m`lc0Gap3RzdeA}P@3LFXG~+OB$rWO0U)TWKBCA*q6ZA8< zS|WvVz8X>F3)M~wI2-qXj~n&zjH{0gApmVNLie-Ud0(VN5>%=l8HVdRuSP@ht^lPo z+mNHv%0^*}<3TKU*g8{kX=&*qkm>~Lh^G>Di9+)BEF0mcTsS8$Xaa=pUY~bLLl6vd zcPupV??LV*kP@QA>N#jn0F=t|OB6ObfcS>sOk0#HA|<}?Zv%{3b+g-650>|+%TD?D z?XuV^Bzr6rh6WV*0poo`qujYg&!b*yubm|)fIUO6EjaKLde{;KJLxr^C>ApzwAy4u zWg(QQep(TAHG{71pLTcs9>QZ8)oB39;OGNJ5*5 zKir$roUXr{MWowP)(QRT|0%&69_24s-2m9Wc=9H&`{m3RrrVJ{Ohd0CPh8I z{roeE6XIgspL;-vd%Je8VX?f(&659QRBb(KwaKCX@w!hidkyn;#k3XI?H=nZK{#=d zlawLFjAPw06_Q27uqdqsSYv-6e-@r`J?dKoE^UWhq*XaC`jXzfX{3+j>Y~p}vTv-m zAn7McB>8yD;9MO0mCI}|N25^PmPQel%l21k{_Wz{7H!ico>(GZp{=W#brC*R8r1=y zzyM!F7GVJO87XxvSAcpX)jutX^#w>VHY72P?GNZLI?evjC%p?8m^4EhlI5fX7(`l* zqDUnH7G0ZWG^nVVKb1qkj#xJ?TGJk2CyRheqE>z{QmBi6;RqOParTux%KI;ZIFA^G z9_Jd#SBTbPaLRIqwnzp>#lCD6vzl~vhqoLqY3*bm22VxJJ1h!BiBmhw9hB+k*{28= z(nnnlf4@9NC$sp2`Vg(fOm%7|X(S`!tf9dMKX_J56K|BvQ}*fX{lhF4sW|q1M=D_3 z7$Z^K*Z>Xsh8!ox*FfO>+yeMRrk3(|dq$1gLH~-3a`4_Me_p`*nPA{d>;nusYayuX zjpy?R;qTHm{8TA!j9%3qzD>E*b()@@BLUj>vRbVH4n3sD8A#eSz6Cdv5-lhf8ZWG# zPFv$PUafL0zj+7v_BS&kvBZZF4=QIKuJt+80BrV4U8%iDfVI`FWyH)gslwDizvy&X23xzWayTk6>`}F;V*a%pTw3c6+I z=ee$%|BOhLg;6yR$4MH6jHlU!1}5nU+KD=pp5PxYU1+S4cU(S!vA@@G_ugjTr}E7n z0b(G9yQr{Eb#hKU^5UGa;mUWwj=cW;j-Y!1vX0v&xpOSpi3G$O$^piiBA8fK7NIUJ zm`rVAFBlb7fZOh9nRTDTMFbKZ=bTk?Zu>sgB)EsuBIlW5@*SdyRks?WlWu!=5#X6L zXWqv;Re2z5UjSwM;0kfp*vXc9FM#5SLd-g5k2{5M8AI`~MM*a!RPco+tiCJRbBDGZiVqu##SxIc2c)Jw3kgGZ3o%ZN!&eg}ENC;Aw@@#mTS9rJ`@sMWTToDtV9*vrizVqAgSqK~#o@k_8 zhC>*cZ{Y}Vh5pEjPTeAobzH);n>g3X^Y4BXv;sB3G0*q2K(7?}b-fZ<9!j#)QXmwq z0a@+y`hE>4@RIt5yKNh*02&S4{Q=Ya>C9cc_1I;_1?;@js~6vVh7PoIPa3h zT@~BXu%V^D?#gwyxCA^@ryr1~wMU~s0EJs0q^>Je`FZ!VPkTs;ahhZxfsccgu_PK= zR7XZMW!(q|O<$D|a|L-MmUfDqeueffOABwQEOMt@D5wj)6g8c;tkh6}o=G2byN>v>d6Y)t}dZflS z74x(XKwj+vAUyejB5U*rdRC985XZ}_9o0^4$K>BdxRhR>c$u$TA37vusFe#@&+oXc~r$145^BvWRzLIFHcTAAMVT;jyj01O6cT+GQSYQwNEuq?8}*~J!M<+ zU|A>}k1FLBiHWlet@56^KuppJwZRYF5&%xLFf)2Qk6A)gj@+7$vRM`r0&TfmIXJUI zmiTo#eq5&I)KvY>>vMVyVQ@C=lj6EdAZ&E?0p8BwnVtuROQ!IJ7OM7r2{3p?XL$HvLyk|>vecU%|cpyP6McqS&vW&)T@vK8a zG{~#&vO837#Yu?AAqUuaONORO`XJAcgRmp*wo>19tmTz=2G(rcsjbivtmarZJQgJt% zlXIZXxi~!?&2`gW6{%+quZ-DwkYrw^IL@`s%@ZLH^j>}LFx#7i_5t2~>z?OFG~=5S zmRsnT<>Z;zKv&yfDx@qV7G;mDwZU|wz9d9=Zmh-*2sZ^V_i;W$(nmP*Lox)T=QREdxY7Kw-93*7?vs{%1>UoN9?HQdpt|dQC9l3xg39E~O|C_M(kvte5ADE5iDx1`5t}8he#G%9k`1s8L~le?WZqWxOp{-FV55 z5Fl>8S&9t(etDGvdG}^CGfKnye1lr9)ZXPd&)a3K$4(#qkFBc=i)vlllpu8a(@)y}xs=YktfRLDqWf zdG31qvV$s=Pyok5UgqTD(7g%+7sC%3Bo;h1HeZxSt z)*x|NZ)Nd^K#gmAPsu;aQ3vD8V`J{wYVf^t@(LxYXq=rn*#yMT#l_p>0KFH=E&0h3 zl+O~|#K10E2`1?R0I@+rteK;b{dYAfT|Wq3H@6&-lmphJoRG^L>E~>0IB$6xX~sWW z{klF zQQ+r1e8L2X0^LZWb5kNwp0m{_1nDjGt4(imm8KCIhhkH6WA*QuxMML;<26Wb*85}D zL|F?IOv*%2O`h{kIdMB9$*B zKDWNZvVJH&|NM8ot_IIn<76SXOZO{Vxdc5Tl8sxKdwt+yf}8p9TDhc(r}X&~|Jzxx z#oTgo8bao@g||zLUVf=haE8S|PSu$Zy#ndHM2#^8kKQbq);6$$p(?&zb0r|x_Rx?$ z6&+VK6$h;|A+{Xq|Gq!``QEAE1I8={ZcQy;fc+_5gcay%S`=rKY~FO=f)4;fH)5QL>qA|EA-dY@im73)t5 zE3M4}Y#{(3H~0)oG$o!Gso02ZNMxL2$)5y^@>bj3d@H0kvlQbH{2dqY(qfnc_@FB= z{bsrMlU0dVp6l)DJ5zPdpr2(!sI5G}@{-8=RARyHtP6Ar-aD`TT&2d}b4Iae+lgl) z_@AQf?;FJ%2_IX#VhpW2a_fJ=&q?&DpG)3YC_&u#xo)cm<04Hplf0*4ByIXtThr{+ zfSWTz(>0#lGw#%gm`q3Nix>4aH7Tov+w>o`idHhlzyOoKn8-j5XyA75(KcE0XZ|

LsT8b@ytOgCd725y(D8PKMg#g1O3d%zXcB#eSI#_qbvlz)neltsiiRYt7i4^UDn z{4zrU-koyi_^dC;;bTsl!(Lv>OprXyz&hK6FcC?t1{PeWZ7v2b)7aDoyR;g8MJHCc zKD5iJ=wnHnDMB#5A{d3KBHpf?9Tyi@^OPPzb)3wh_><>f-OJyOA~fFaibmUu|An@T zcin`?i}xgmzDw&Sc}*ko^G1Gf8BTxjgy*=2V?Bqzr$cF1;((JBM7B02qnU zA3!r?2)k*QvhCGqz73~g?szdrvbskuCEBcZKXb=b5`RNGiVZvMZjc4cJ-q-3_*5GD z-QUldO%U|%f0JZ}+?Y+p>}Ovf0B|B$Q}MVHPwc>V#oUegYD0P=UL4kfP2>-VEY$-U zCgbThqVQLFeY(a({T}z-zHz6D{%wW1ihSoG>SUg7Ov^a*#bo;dU-3(P1rUYQRtE>I1E-V1IQ9=>EvZbRGIiE|J^Ccca~N#BvR`Xi1A|B$w350-xsp zma~rK6A4#QY1R+v%}C)(wi#HtyMURXxIzjR5NZ(<%BY7DBl#n#xZ_zOlaQ+j-&u=Q zS4%eMvM!wjGP&{R$x{k-x%XBPYz`e%)7MB@9%IY--PURt{n=5<6}2XS&MR%=W~vAd z`057;T4)^$-f8RO73NZe|2{JkktjTtcE6rv88{=^2QEbs{=)r8`>7ph9aEk)L zQY3IOIS<=BaMS%hTa=ReH_FG8fDlD$sk51D{xHnNAJCf6+tHS_Hgf~LkxSrTaIbkH z2F=^K#|bD{ly?5g_qEa9z8gdA90=QsR#q)T=QPHBLA~7KO-$IwKD&zen|612KgY1a zR#{I*-!zo(-o!*Wl}*2~Msg@n^d@P7J{d6x|2foB);>3*C#U<(1IVXVI`1CiC2uW@ zJh~sCt^+-eOi=O_i^9Q^Lc7TZ=3mc$(N^s$0@@SqTAU35c&JKM)a1S$A#F$8YgaM-z+?6HRr zh=OzoBrx~Buonek?*KwWHEL>lTii97<{q|uf9tAAruV(#oq+h51aIq1;0Lat@>&fr zgWJ$LV{zI7+1Iq~XMnJg8m)({CA&I+g6^q&Y4d`0GY6t2z**i)Bq|@N%*`3KQ9*n2s6eWPSKG zsGEe8CFQsxyTg5mNke@G&>TMsM{Ap_ZuA$teGMr&1H_J`XgZ|b{_G66%ARz(eDZz5 zjo)bb#PXci$*d6oBPR!&1>%SJiK~%#Et^kQM8|oF@!Wi}xymHU6|#*zl!0rbV6`zN z(^h7)lzsL-92k>5h}%7;%_y!#RBPniu^kRvbd@r$4!XQkDXc4_Xo|JWAWrwuV!E>6 zQk&m@WzRI6%FNU3Cu2ejoLD>;BNNorTU%XE507J^h&+1%Ss z_XaNzO-SCyL`YH0ZN`rBr3kU zXkhykO;u^;i!AtHFG(WN!u}lq%K)#|DS0EIi|KSI^O}|Ka;=#$`c#@HA}2ErZQBM{ z$dYk=k5A)`drphMa5~QP9|KSbQ5TJiF2{+2t-D&o#5f>}-#F>ug7B=p5aV(m3SI)) z#^f(Z;1$rj0(#d#C9f?t-{6=CG>yTVit$Ul!zhsEei2}{@EN9F1oUvltpNK#4hY}( z+Z$9mr_EBlTG1k1S6%=#C*-+fRdZ-qcD`31m|)u^r0aRuffi2>nh)OOAGo-JstbHaR796VYe1%Cbd`WwQJdy zo1IKmHSsR^0AM|y&j28s2F7pwn_@t9LJ^1IrNkC*{%fketvCf_1m&1q_jFIJGm4~L zuMI|KHbhA|zg5(3=4vFTXK>IM38Ctv&=Xw=JUJa1+d>66Q zq{+G{xwiFYh_^h3#LPim1s0|$c|)Emd^^e)%YHp$w3K73U_F(#JQ8ZqYz%Y(MA<+M zXAZ*L1IttXAXV;T3A5>(3+*?CEK2vs;S)VMQ>&)>)THpmiKi zljVv2$$!WPd^w55DfdBFN1$|CGQ<^tyV3!;UHB&=&jswofAD$-8O+ZuYCpBQSCY*# zE|s@^(9kjoomCx5kX^L}tRiiIKJi}T>)^Tjb+0AKFtoX?0Y;Rwfn*D9gy`b05Y&Ae z^TNhiRSKH&V#}>y@~2*76GP z!e#A%7iYt+wQcaB@=_a7CWz^t79-I)ux-Rce?6YP)=@8!Hw)@lTpSOnH-FqQ9^(!5$I~5Y?%O-Y8zVnE%+4-)C?W&R+Xa~rI(4Kk^4cw z4oHeY>FL%jr%z1A)O`{Nx-e>p1N5D??d}nH?Dx_fyE}rh=x+h{2_FRJzDwYR(@0Zbh$-ZK&UqW9@v?I`alO<& z{1SQ|T_55-RMTD_D?W>J>b&+K$#w!RQnzO>-IpTfTZ*f(J)U=!z9!TUw=hsOeGZ?? zgpfXHhICnIak)e}IqCxw7eTV1zm{-Y?ac_61%Z!oDEY6XN;1vEFVU;6qxTF&<3Hae z7;sA7F9>UK%$-#m+Gno^fLq*U=z`BRDbq!iJv zY|lyL#o+CCK*^4$y+GiV3p65!Ltg`IIlWv=T}ULuHvp}q8Apt*i+|}$e;&VUeeg2f zm21t#51Kb;MaJqNXKPc*pMWXG-A32fekjU)M?xg0Y$JZnYsmE7rx(s)qc)k7vv0J& z-szAUE)k>>N$dwIQBA`7sx;|Jprq-lE45_z*3VAjxC{^j+(y&tRn{U|ssm$In7_X` zC)NOXAooYpU_C`D_R)vpYbJ?yt@wvn3ol<9h5r2X{CI+a+6z9U@{KT4^tYS=a@ZA| zn$g)z?8F29X_uz05MI%IKD)ov-6mC&L&;$%vY2vpvF`f-<6$_O`KY5jWezVW0Gwxs zSf+>uBG=e6Hh)zoH%S492V<*Uh+u2k+ ztS6Ee$klHODK3fYF`CDzu6oA%$oLj|EZw8JJ-m)WVO`X_S?5uNGh~;!I1t5{^D7~h#tZBKfR9QH%hk_F2`DK7^iW5$n2OmORKsqCU9P%t0- zHWTCJRPK}wV4*B?eQ@?;JaSec3R&*KKJT|7@WUkb$vSG^+nWPh669EIAgRv*!-n8t>L7TdSc6D|C^Mhi3wN}N)W6Geb!MW|0r1M{&M zb4;PV3X_SPfkvgo=Wy;j6TCNl?H(=%rqNFCH7(QZ2VrAbG5(HjH4hg9RQflWQ#|c2 zm>I}-aUWQ5$M6e31#xJ0%R<^$mrQyMqw7tyWjJqfisJ7|GLStm75{9*ZB>(65`@(;{mDM!O(i?jZqRh~0v@Fr@8U1s}i^34HsQL!b9{4 z+y`8Y^&o3L%ln&==bJsa-i>Csmd%;K^zdVRPpxJ z0nyc6^83@^E}EF>zGt=qDvlxUUM{69GtlJYiqp`5Ip``;Ayy4gQAv7n4VO^S1*iY` zdis+H!mUi}uE0UhY=jVR;+gC`jcL9x9ck>@%lh*Dp|D9sa1_?$Lw=oppl2X+QJ16y zuP3>L)wh2?IcYj+^dV;W;q}p13M{(5RW;y?Afj@m9#Vb28yx34#4cXU*{ z?O#{=n~6ZV-6XT6a_lWTVXWvLI6Qbr+2+;sH~={TEcYEIX!S23Rzu`L5%&j4fSTQU z)Z&2G=5V7;E;0<+`+Ed38bnZsIEqxD$<}?#GDZJPc*utW3yhE$6>o@ngIr4{`-px*}C>o!pexYV<Ef`&wn4xKzO^_B5)kwz68QHS9h~|PUz-;RQU32H`q^5Mn_=}?Y{)X4KeL}kL6DrF+uh__h*rCM z58KlT!YJia zOy0^D98ut4@nhuE&IGL7pGxnHV0TCkrz3h4K94=I=pmRP4b^$9uMQTZQ(sY+q^;x6 z>&cfFlU&M!RB1kLX}sAsk^!-gP_+S|R*#OvU1O}id#P9JZYm>un!GV401BXp1;6uj zM8akVUK3Pr9?z$*-&AS{OqE#tjD4@%gb$frlCv35ksrf$Gg6BpUu}EfB4we7-}xl- zZJxYG&kLKFxRO!m>=M4Lwr2@-LVK&@0|uK@BT|6&HtQ3{z_Dmsxxqf>tpTJ07DHz# zW&YapkIrFg6m5X?*cka=`s`RftPqAD9D&5t%P*{od$CNoTz}rinqJr|?V<%sD?2lo zzIYnBhxUNhe4R$OFl2`lg^`X|$l@z1&W3~5gT!@F<}4!*-Kzdkn=}rlPAn5Za%NTo zm`&G}^9VPY-Zcn1>sIPi$5m0z&@XtIW3NpS8g*#{xP)771z0rg(DxD+ zXBgKnISQz6wd8qkc-#x@7=ZN{foqbSr&z-(y#RM{XyU0pq(r5a>)L z9BoPB>^w&I5=BY&o(BhhZ9HcBIc=@{>p@9oi=Q$?`qoUbr2U=yxqVvjX(fF$K5=8D zkey+weuUAbanD%Ph>iF@QW5;B>>R;85q|K4>t_;e`b7YX4L?z4TR+<|tHRrHE9&9g z{>c2A+yueAYFIl_$E(K=G)-9&1>gDTxMeP$veVe$>>6;IYvt*mM$i&U(|N&fS=_9a zC&EO2Eg(fO=nd7VcQDX};Ho)=JdxCx$(v4fl`VGVF-oCZRUlQUT0#7@5H%Ix1oaHC z>mbTzZLo9=;N!WJE8~0jYn_23V);=^d1~kYHd5f#fO@T|8A?d8Z6$h^y&=e(pD64} zgc5yY=ax<3Ufal+hxyj{z$_Q!8!|#50zq*}(FNA@zx-&cFOTe-k3!ZbGqs>}V(h$g zKy713&$d%KRT>Hmj%$j9MGQzbo?`AjnKd(={#mX$aP-JoKe5bIgC?caj)3GzgVKTi zPn_V(SbZlq6f9kyNtPRK5b{G!V8>`cr5Q&3{+~w~@Hanhd1t^M4l#``z+#8Oy4`q)LSslzPIkvE1%|Aegx98tUJMX0*7ed&VNtBUs#;h* zC&(1;w4H}VSGU_wDmXhk^RqFqg%`|P3XCSvs?JE%egmYVgd!XzNW&^>_>YOs)1C4G zLKZIj-_>lgUpW*{Hl#tqE}Gg~gIP3c%dRsL*toNlg#*cwbZ97s8nZ4VO$s~KZz=nb ziBRh{Risx)Mpxs^9SkpC-VfAEy&;GqNsu8QZ5 z71bv)j;3hy8G(z>gz2EUQPr=!fOvRA#>dPENgy|iY7G)#X)yai2=vw)a`av{+h<8; z>u0Ri^H>1{CTTJ~J~dse@D~@rgcc9ER0j16*INCv4+na-0E}gJbT3!~KpN|jC+IV{ zOK$>~{Dx>%o#bIC*|fHbEui}hSr#gosP3e)YzJBYhz)#4J#Q`BR>OWY;>k|jPw`YF z>U(mb8{?$#-I!0mSYjk6PcKbWp28r26>Uj?V%7e4>-!Px8$bCBB9B;Z-vCWX(40my za4o4-t7~X8j?tKk0R`a?$PR~`uEuG!hC^_yfEi*Rxx%o42BejjV#?0bo`!LOSBvt@*9;5iE|TA9m)l{-w!^*ny2$6-0< z)=6wePy&FbM}3YUuO@8fQ>vV-x<*XJ!C3AA8MnyoBZ6lpCYRYR-ep~GKE}koLvdh) zVLIZ1fa_pTh|crmHebMZ_FWPKSRuFKFe!1rQPd1MB?e3bI&wfnzInv%v|Z1JXw&O# z1`HusBsoCTlEL#XAcTYH3$!>|e#bn)FcK4hvQfjDG*Wb}9-lOB76^O)cbu~|Q;Wd8 zd5=aDW!GXyzl+`#9j3ih`?KTvr7!GD(ZAflwoPYiB|mW=wA;2f9AuGUP_}uSaLqJO zO`>U5Cmf2M??3GAd;5{5;h$p=a^c2?DRfV|rvNt}C5)FGB6grhN&||ME9@>b;DEvtzN}@h z6mRad3(eKawr9%K2Xh_m7l3?uS%h+62oiu~(e1eO-2B#?&L?vT|tvP~Ubk+@mEM;Prqn5ew;X768zY%eqSXAaih6IHJfUEu% z;Qz5-UkwFf!{k#@IDAHFpYK^wDx|?hY`kylwOKH?IC03#!nVtj#JVh`o0owWXqe42 zRune;E8G)J_wF*a;CVpPT!;~q6Q%@+C&UJaJr&{I_N=JVLj4#VCYJNk;d#NT(@~+^ zV9J4LeJR=7jjiREZim}>)knbI#Uhm&w*C#739nWAWKeovC>ozn&c`xg9rS)wT{ATQ z>1)dHlZ3J~v+TV-_#~?fzx-uJM_)au=|F#fCzxd`iwvuGtn2W9aFD4Sm`zHJH|(@O zkhSAA{8{Ux_p-l~sdMcV=yWX^UFRf~MUbjaKwovH8j}D->jo3%p4Tg9a-AP1@jy6u zMeaPfxx{Mf5jePA=BNX$Im?OmVG7Rgt8}q(58ndp@d{rpCG+*cte;LeUwJw3+YneC z1dci>BhT}jk@>3ZH=b#A$bKPSmwnpX?Z1xAl;1#qWu^6PIq}_&DKI?rn6u65AQw1I z5rox?b0OxAW<#2@Uy$wy4vCgUW)#-@p)^U~lLw94C9OOso%BP9yloHV3;o5(^RMUk zGQ~%UK8Zi}Dmr$*f5O)GzmlQ8O8eMO0rtL>lPrPN)p_Xt6^+}9uGY8kgGb&TyE;bD3=Ms0alJhyU97&CQf0-AyPfwR$ecT(>I(|#Es0a_^%rcHd4 zwZmEJpr-&7l(4;rOwrHoiT%zK^qZ1SVV2OXJ=$qF5CskQq&&e?LALQ#cGz-we)M*< zam6cs={VcHm-##OpdhJpAV6IOv<|3UxJ5Dd2I?f2akdhAru+cihxhtCCsu<12f$2F zC9}HYy{liSD>l=scN@^0`>Dlv*Hj!J&Kq%Ek~g$-*3+#nR`*P^!FJbHmA{kF*Ipg( zpoB$=Fih(M!sh5(o~Ps6G9@|@jjEi$X$#rBk&h!$vGit-ywSluUxA^1;j9kM{{Vfw zp6;L`{lLjf-AGRRrCY61@p6ce5($e^>Sqt|*YBxFo}H)Qs3On&q%?f2L^_Q-o%97a zb50l{599AAgGMm@4#L{%!>Vp;csc5UpM*VfpEJrHXfc{iAL+PunEmtlwwSrw@G}3< zJBCkhnt#13LQ}!HthlU*OIF=_-NmH_Qqjv|N>nqrt)edTs?e3#3Kr1nVXH5+b9J)2!R7f*c z?67UGFWax3e8dnDPwDx`TWTZ+H-qN*Z4yX!(Q zmB4pr!BW`u?bvLQQ01v0li&Pe8f2u+Yw^7jVu>Xpw zUS?`J+1{}14zDNUyta+m--J^o{wDlz{Dt{S|Ipz}Ma+MQYk5Y@xrDou%RVh8G-=(< zkGkBz!*IXWI6*7bSe(n0tFx3EVR@*j=8$4hIH1dBWkXe(GA6K1jaG^dZ`W?Te^b1KQqzL613K^z*uN%>yUS<+SWqqhKnz57PWRE<&JBToF z*=%gO2i^arGV4_RWAnmDqXU)ekZ6|ObFa-mq7$t>w8nyCEBwmw^5k!vGtL3!bEk&{ zJ}-iNR;RgkJUgut-B-nJx28<;De`4Y4jqviXoPG3HITOO`@(qtd?a5PhL)370+8q6 zwJ%p3C>>f$aYapfmB1@0W|m(ca~1WJ{oNl8#3wMNo-E`|wPX}UUjB-(w&x*kS~NXq zd!loxEpkU`7PXXl)`=y1ad27O4U?J7DtdK9d{)yOSDSNl{{X zpT&4v+-T`fe4^>NgwK7~$6Q?8d5p^=F~%lRFr)JS)unvWJ~G0rC?4NwrQv;l$YCA< zbS2daHnu|V6l2~Bc;hsa6O`Y?k1#b2XI{e8AXN&feK_%T`i17F2kOdZ^D(=Dz`_-p z2X-#d6~94qQ39~Xn&WKXoksXwZukI}a69#4!dQX5>0YJF3VAQ4eUl4TY(Mn7RESvP zrDDu30y3(Dx*LTVy~ge=?;$+pkMhbd`>(5>fA2zB-&*9=uY}{JTx}dq@Hdd-sd!I@ z>7~btbAaFieK9w1of7oP|J#IFWPQ4e-)x32SIHt3BF64{GeuO&{%CR-7Hq45etuEg zExq@`8Q}OVWj_a(0cA0PZ&uMz9_bwmW9I@%uh;5c?cCJonJTCtwak~2`UX+icfYMm zPw8J?>xic$#eq0hAUSxST^0A$L%T?b?n(WQ%3B44E{_M^6D@n=^SK6*3bpp@lYXr- zyex_#3n^oF2i=OwU+Ru4#0pO@H2k9$1mu~g#_`n{HocRvE$$|u9gBC!MZr9ekqf4~ zwFv;W_ID_K@Ew-C9cq!tJLrq|gdQ1FZQO#?Ve$ORVacvsGg)9?x>a36w>0b*kl~Hn zu}Q87z`Ii~4M}fAm07c-S-no)x@~eI7ZVUwsHAtOzMprbbWUG;6nO&(-8UZ2f~b!} z&*EAth!^Ihy9xcw^MURU?64my!4e~Ecz15`%>de1vZr+DpKJSbw!$Rp7pnn^)!AKl zY?jOGZNO4T>xJ3_OrGLGpQ~Q2@gw?#4ZkImwxi(-gVL&hk0<&|stT5;TwevkSPJod zHeCL%M2a-G)sEh^6x#9K{;}*ZSedDv)qhb`7K|;A?~xQ190!J$Ns@oU#Hk8Lr`{Pe zDV&NaHh)VnH*m({@knG_Giif`l^-UU`}G2j?VBUBL|c$fMKbK*(-S!ftp~4mbLu#^ zG|RkXj;_Dexx*qFe`)9B7Ad8bFj}AS?+aLaY^q{Zk z*)U))yKmb~`@26Z3(PgpuZLX(bl1zrXZ2duiE8oby@#&eneKdquzIREwZXZ=~rIX9{B{sW#X71^r7pivHaPH!+)xGvR?t2AZFujFgdW=41&)RhglPHLv=9 z7n`ECG+bbf0H>?K9ZRT*S3#_IR=iJsiGgNf{Wq14|v<5tr2YcgFWElD4lGx?n zfdXyW_{ z)3XumAH~c!nICHy(F`qzz$af%xtPtS2V|QP^ssYrT5u4NydF?clONuGV}sCc49nHg zGg7FWUj5IWE1rOVQ1q-VTt9VMS9`qgPwhL41IL`-dt2wMPQ3A~CGtd{=QG42;Y*E? z!DIbmK}o2lTUgScVnf^;Fh9IrBuZH3yfr|pk^W+n2#EY?BVKqbJ8v4$m;GAU%llW6 z#ANpC=hR&Qr$e7vUI*Kc;pzzt+S?rIN1mv?k%z9<8b2hv%poWspCHr_(}k?#J;4PGD`LKAPK$7jN6#?o|q<8+jGA(5dhzprs> z22SuuGa0kYqZaQ|_9kp;?F=C&aUnxGRp4Cx!^@phjl9+gbu7F`E^vo^nRdPY0ixnR zZ=&T@F~$b;kL?ne!pd}*cTz_*KRVDx`7rJMViw_!0g9CD&0)b@@2EYtBiZeQy5Q&+ zqSuA<%DsiNz%n|W^D^0Bru(p^s>$%CZ!$Uxd}il8{NuAuyqGub`o_Fwx4f!3rz8BE zqNin3yDimv9@J&`PFZ+=RYD4Lz>uwvMnFV_o zvf`9kbFX{rzPw5WwJ76}VyJ8GI8wjUb#knQN~CtLk8;0(uH)Kd1Q}2XaP_4glvxDL zo65vhWzx7Ss-+msZfyrDq(m1sOtp3){(Um#07C8axUF)UC(@UF&VcyK-)T&n86rrS zFwNzC)?0#>CjzpDLfiZ3?5E}C@wI^zx@FXsX7eqy40i&sPHVn)0lo&J-9%QktI>ag zb^of?r=(JMv42=EM`@<)(5K;Qf7S`CFY*i?c~*=9_8D5(wrTd}F264)$-=T5apa{S zUhZ%Tvw6LzdflGF-G#F(Gu@vTQt=So#${r-n{Ies$kg8hNKphz0+o5 ztG23BS&G?oVab;F?65{shqtB|T1+~Vw^8HjRz9NhmZDtW@;ONJyrl4ZkzZStX!zvY9bqz3@ED%>A2l_P7a1z|FL6RUl8HNfLH&F44Ynt)vEC8H>En zNjN2KD%B}VVp&#@S(1~+b#@+*4nS=KJi6U)o-Thh-N&Ij1?crioF`3VazszjiK6I` z)~~_QFt?hVjqya5lK^nU)jW?nNe!bgP%N2F;xSSCV$Ish-Jar zG#r2RtYEP-WB!3DUS#fvVm?%BnFX?Ampsc8+41oAQ#YKND`v?}+iT0i=D>VegiB$a zKOQ#qx9QyUZg3g$~U%*0-UEN#G^{DUUn?u85eWs?gl+ zKcynK8_s(;7T;dhkDx%T1vKq`pF`{Jbx0XHi2}`>7S&Y)d1Zlse_Lb*2(AEs5|2Db z%M4qj_kc)jaiM)cQfcYht20Ne4`eL_0-|wHLETo~A|Us0x+dYr8xZ3L-nFBsQ-OFf zQEj)cWS9JacvEml;_X%?kR}Jn5k=a)i_v)si^FnUG&x#=nyEB@KP8#CS8?JK&1nQa zco}43=K*$}t#||!tgofg%m!n5d;gG0h9c4wK}>p8*U-qdjw1=GIx)?pVA8*~cC;Kl z^pYBR;SZ_E>1MmPmYE)wd&SqNHUfqunw8_!I45W&DaZy?D4w`+c_>r2LQk|;BNV#8 z+A>FM303kJ6$3eKvsI8F7qPjMinec6w>qWn*(PFnS8;T5=+*J*r~K~VX7+oTIg&0>L9xqRP>UNps%vHUF060ps@ZKyXz+n?J%&s&E>ymOm15r=?<{FU7 zAoLg!J=z-WwT<8Biv24u=)EgN22lf89ln)c9Cpi7zT$xA6z#YI{YQR#seBeSGpK&qZ-S^{@;h?ppH*fj zA<}1Lg>qC%)B8)(O>b#R?s(h$(0iNvRgy+f6ELT^>pbLs9^s76_w+Y_AN-XC&~kv& zd1k}ySvdXe={fDs(#KBAF|;vG3gn0*)-C@vPhaGr1M&&yLE zP(zFyTxh7C>$f##<>zS>R&LE9N{&ZxllVanPszKw!wzBFkCZ7WQ5H>n9J9JYxtx~9 zl&C|j(d%n`j(x?cHC;vxlF5W!*ShDKt+?FKVvK*B}BhKh!cn?;ubr zW##Kf6_&u*3*ZyLEx$E6^UTkmNFcQ1Tbk|~dI=c53J-Mp zb1R`~GhwU5W}+t7hiA?V2%=%|a#Tza)4L*%d!#M;PS?VoNK(M1)v4z{;6c zS6KqQUyuEC2j?L^D6TDSXzFSQ_Y{^t^S}HNUrijP7Nb= zw$gbW>2#CvcfR-(9hGiFy^1|R0SQY1U$SjW3x1s6l2{A>9WDgde}xDteSCPzNR|>& zIXn(e7>2sSxDD2OXQ%}`-%{ehd~ZI$8PmGBw%Gvh{Jy04z=h;k4b^@oZgHN>e28HHaeZ>%*R!E7ZQh zmFRv4xq2Y_p$lKq&n}U)xNN~T#TZ;Ep%SJSvMS?KvI1CSb5c2RWXslA|$A+ zU8!+n)e)h@h?Z5WKfr=Ti8m3lNYhT;Jz}>u$G6p8DH2?(tMEPJwjSe>T7^R|*cix* zYu=3AIsB{MLC+4~(R+i5v0)a;6C>TF&_HtuwP}}oblNtm^Hpa8xeicKz46wpRl+1& zeq5jG`q|yQd1T>y9_?{S;HcM%iLq|GE1 z=yV_4Lnjr3WSIv*7j2+7XeGPB72f15kax2+IRJ4+0GgL<&hU;MbSV1UtLbLcbiUd1 zZSB#BWOzoMB1H8x{OUA(JB6ePkd0f?gy8KvSJ%@-+5VLz{I>$%^JYQcMtDpD1q#ez zp_Z{vR_cf?h3q~1EG?ru@);5p(aAa^oo0D`FR&06xS%_eh=%-*6~&3M)}_KypiOo! zudmR(*mq&=F(=$OBTBg026!erEfxeart#4M)3Eg5h_DbMvaWM#!r?e(V}hJwoH0r~ zWI7N2=io&JRB7Udrd#_YI#2HJAK)JOEvL!R+rO@c-hGI8`Rgbqr7=x&Y#%vr)w5+h z;qvna`3T6se{mkj2hlzUZq!VeSz$t$ZCJ41?nl??WFSn>v(*^7C`rhD4^IS9H&#>! zn`!r9iWORzybU|8KrJpBApjdk>E%8@>|%-Q@*N7=jCKU&KbTp-9YsgzZ%GN%spzev zi`Nm{fMOuXMaNrK%Srn`v3h^~$stNUR;|E1hjZyI;uYa-jNq=oF7J`HLRTJ@_Vc6qCg3 ziT+!MUHW^*B80*p4^yab_nL2HpYkc5$$;Eo86$b<}PJ1x$ zKHOCNF`e#27tU%Xyjow~a` zxNZL~zZ4&(fKo8+c$(lyx_?-_9U?UQHzLJw9-n(1G>B!S$nE?rZeX-+{PJ{aA&%GV z!()~8HGyU~ltT*z?$rOEuRi!Y(HLA5XI)PSFWetiOluHmCWKxF7;kgTXxpu&f zhOXHRsM-bV?I}LcNst&!oKiHL=jMvsv@r$FR`N_mV|{;2LP|TB08|t%?}XUn3Ed}& zwpqcp)HRtSC0~|1T7mOF2`t+|LwW!H7?R_}iqP$(r7@2Fd|BCj_|f9amAU1I@a6wM zZic%!T7HK_Zz)iHtDlzmf4gn|>g`(Y3Jv!4yo*x6i% z-Sl>x123k2#=oS4q|QDyF@f}h)^;KidfhgA8^TqLgk#KacY!Mv!jC)Q8%j`$_1ELB z(sdWVfbU5E^Nr(NwYo^#>kaB(MKQNG*6+g;OLg8Who$@ff3>#Ekt}?Ai0kcHX`IWI zy(F`e`G`xB=id0oCCU3( zz8y!rL4E!9JNA3i6Hy`&ItImEEf5I z<=(b>X8McMNl5iTwPqNNYxU|BHfxbJwnO76_U6ZKkyVA%^#zr>5_V<(DJy+n`_xWi zM$jnSGEPJNr{MoL%3N|AtKspW()lCAupzlqxQcY+L=%$z5t8c5_>8f#a~$v^=Tbgq zZNte#R99}C3uz&cOa(Gcfqrj!!vLHbW=DgNNzqU)A(_U^!_2yK6Q7R{X}Ug>FmI^^ zy`Vqx3c&=}piz_{ zDX4hWVTl!)#m@2klSA37K+g@QQRYV7=R8Ffim=EL%HFxL=56HKNEa)zY4^+?^4IUU z=EJl^F!{4g;R4ykzE(X+2*V4vdDgM1gsR!IKQq4bVM#?yiM#WmJXFjqa)_vuxbeqP zh(ng{Ruf6kIIKv!#3hNg;$bH%M=r`{b7jc^tZr1X{hdDYX<@5AryP&dl z0SWVkgovu2ILd^37}u91nOgI?4i;EG-kQN>kh?EChb;*Y@%cd-bw9p!(9=iCo@%=C zK(>@|O#2Co@EhclQdF< zL8i2cIfpFRETIi*ypE_N=E3z#j?pApqA5nFn2pHjXPNpIvkIubj%=_+O>Tef=IWHh z^zOgcIP%^gImoqJAe-D1{&Py* zkmc|bg&105^4{RZ5hb9eSH(JvmxpE%l#@r(RxpRkwdo042%qrcI!BOh4x@X?1BJXN?mE1>UdV?((FJn^< z^^MN(m%~k2Lj}*&8Yz_CI+iwABO*c8as@=*YDqm@C<+@`enTwt7TesP7iH*z(5n(l7Bg|)Y_gbf&-5JOK`2ZWip z$$V#O^ocdoBvMox`4XX&!Zu%o9lR#gMeGrSw14IzaQCw7=dN*HSEEk3hD?#Q5`Lr( zQrq$T&t@1cZ`jQ>2K7?!_dD)->;RUHn8^&=dL`v;g$=29x)-$Wq4^xDPoNsu)EaY% z{cYGnr(Y>C@j|FE7PQ5xZ<;|ooA@HN>E<36PF6ehb$@a7>QJ@FH%rK8&38*Wmv%sY zb+lZ=^p$p6_C+m>H2bLlYj}eNjn`5}g5_-sJ;Yvt>uma26DJM6Ebdg7~3ve|?81}0TAyp9_+ zs@&5fax{!*%ewjbdXdat-F)F&GyF;R4yHL53Q*{A)hB`66xc|o?qzHkC_DXvadTrL zPMpNoZ|25W#*QmVECVKPyC93=sMKDj9<>@`0P*J#(3PO?Pf5F?x^pgjvwhtK9w89W zb+Dfn_G3yxXZ#}V>*eH=6pV2XPfGIQtIXrDLz;9D!6o;_k!$k^@8&g_ab583d~Asy z=b#Yyz)4{WZ;Q1H3BFVqj$-cFQ0fi0I2~UTBnk`+M7)^iE`w6}dy7-`(sL(|H_2N> zOnttWdYOAO_h`KOH>X!&$5-KBBq&DHcKb+goyqFo&%)XD)>}6JI>hj>z)flEu4?sF z3fVK(y&FCABhDQztMmL`L1KA@(M(^74I3R>aNb@e#jpvYbI}!jFq6)&U-nnxHa=B1?!9ttAY`2$p_AU+-8gW9CQyoPz6JV(a&eZm!%K z>!~Gl74qk^PO?DxK#wZSLJncsA0dm3=*(Q?`9HG00xHV&`x=m|pp>A}B`q;D3<86q zAmE6Eg4B?bLw9pg5a}GGl@jS@hE%$xkq~Kyl8#{*z8CA~{eNrCVvWlM@B7qw&e><5 zeO$!(Z!hVtUhzLtC(UdWliXx%*vd5tKOCg%i#_@Mo9Dm(IlW1JjPcYZ_P|bEHip(a!hmyik+o?K-tiB^mEB=(`=OdU zQY&}=oVMqxa`Y!L<#t;Mq{CCZ@z-=Ixl-YKP97dcZ3vvCAYO}5@qG5RBD;FaG*lGh z*=*9MZ>pfNgGIVYq32qEJ3@v8x7niTZkIhDfE<4&3d1|TYbZ)I^=w}J{R3-(QGJ?A zT`VOP+&kT^`w`ffl@1W?`~((msgZDfO#sg3S!-{VydbbT^@MS9os=Y6LhGWBR-LJU z03u;XW!_kJ@Gq~@@D@SO85~dYy3d$3ftP~+OA3HoFLk4B+mMBgkM>mGVdXEExk-=6 z!uBTEtkzwKGqVu`X)nTITK0?F>IagtpiZWH?U$V_)g&;T)ceHEPe6Fl;8Ri9hu8+q zdo#a*rr&oMj&Sd#j82y{gpROmX<*7vK*ng}tr#ubSSC;?BSIihu2^3j5?iPBN}y;! z1-rmM5Gs|t@P+-1b3bk=#M5%}!Q>_C_N&JnXurNjs2-oj;)Aw|mHzDIIpeh9^q?WE zP}r~b>%h5hIwEw(F>RT{xQyr7TuFhIm8UZ7p|jd<8s}oWlSS5Ao$WEjfayU=|K7&| zJj5*s74NGiW1vvc>KKOC|LlSrc5+SLJMt(x4T>Wzb@&>KVJ_l}Ysy?RMg0($zvk%t zfki|mn^CNcF2ah>8MBHn>@4m|+I@XeJF)s!2Ag@~QiN>B`)EJ=Pb~H;`1ri`xkchki zRYswyu|_yfI~{ZT{#4Ow&q$`P?oWHkm)zu1+EzaJfq6x!0oD+v}#C^ONC* zY!$Z(9e(=u9dMa(b2HkfgGQ z)yqPzQ#p%vNvfK0F!gPr-y8M76kO2K=`0Agg^s~P2e951Bbc^o*(5h&yYE5KO*H?DcBt%In zaX~a5=(>bQX-8IE;5LE{Gh{CS59gca`aJ;w+r)y^V}P?>eXRIuTX(Yf;?iRptkF?> z=>BgK8!w^Ovp zF40kWXIL3$LLkDDVeJ=7?HP7~L72%V#%A-L$tPOpN$4M_HPPSSKz$ofE+Ko1+$Dt4 z)+sOD##qAWPe~)(#Qw2jK+}K!g4C$a z_LTsq6#8vdn0a4H9yz8dg(grA!x|_z`s5r``1t{Pgc>N9Cup2+E<+uH@gN314=QgR z@1puRys`1&v1i_b%LbltS%}s5-U|%Wr`gNBp1bTBYx(_^t*2wA+~e0E`!csxF@O3$ z@CD+*vOgnm?mU&8T$zxDgZ5C~+f?J12$GUj`bzuJo^QmhHez-WZLN+@9;m_2A@hwF z4XOXTS>q@=XbD%Zc+*_$+YVjpNv$x@WPW4PMH3};0Sq{%d{2t^ zW~<+W%A?_Es*L06XE6a}ca>A53u9fB#gL`Sj2TRCOqkdnUOGE$J{01W-6>hZt>Sld9z!fC%30%g9#?MZg;s2GzJQf=rmHZE zJlc3qzw}$W*^nUgCQG0k?iq1A>LG-hFR%$LyJktXG5=Zhl1OeS?6N6iTN-dG{)_5K z6AKm}QsXUq*iL=8e$}v4dP#-C)-*z#?PJVdO5f{8fwhCNcw^1t3W3;M3(;#mFTKP& zSO2V903ImDbT2sF{7@&wK)iD$(&NRC2awz`T;D&YfNRo*tC)!t_89k-6CRiggce4b4x202;9eq8<`N{%W(|5^}(x^ zzjV@m6E0Ucn;3BZkhCnf=CUB-o_K!)!?0~;_b~G9R|-L3KaCXF#h(3D+A4iLUST>@ zk^Jua7w%xeY#-kbJxLq-wMPwmfgonp+$Y+nVZD}6Kzi2$%$W%(Rzuj2PALGGvJkOF zA>=uuNnG#lWoom%L#0^T8EiIYF!NWO0u)mvWL1B`sK{i(em-`4c7PEFmGlRtU8JL#_8%BC(jD?4X$@$`3mzViZyT7GChIKLzYG zEDhvCuL=9S-g)qm>HlnPH6lR>keOIGJ29F0nx6b zIat+s`zdm=BBaihv&5Q!#*TEBrcM$?@=v?~?Jl?NtAm=bZDRY)J(rI$6Etpgg`OeFtuWy7lM*W{}_LlKWNmObGV3c@VrrP;doNA#NHTCa-WNFsHm~(%<^i z^8lod!j>c#w5g0qxC5Jzk;T$%!v^~S%lRY(`|)xIX1}NK_eun@HPxN?|I=Kh0B*!3 zo=xXJ(Mpj)}4Eq}9;$i5=_p zi5zcQf}1WSYOH=qOhc=E$NhZ7_G!xDoO@3`T3q9vZ{n^kDu4E`edf<$Jy-QdDqt;P zpR66FAs(GCyI!n z!#_vGp~Km(h(yV0yi8I;7Sb&2V!6xtxvKPYLfd%W2?rf9rJ}Y2n6ZA`j+fbLGRrR3 zZ4#x@cQm{)ndQ6VOiSFASW<|ss$dH|Qhv#?2Q@-iy~&70X+ zWUg_WKlcy1$M*(r*B{yck0^C$S!c1+)u9j#Nz8Rb*LjH26=K#|?8CWp?q3xo`yX&V zv!127WU9y2+{8g)I5o#SEBWZ$i+Iy;KT|G@c^QY*Ca|s^_!~0yBVs6#+hX_hdsO2E zu&eFi1QUGquItFhx;n!MR+n_p#6951Usi&5nrR-8&zA0FftPiF>-O%QE8Y#&zzNgi1>PcVSY0Sj00zq>vlo8ohw zfg{gGUuCRE?m>Cbo`#kNQaKb2WFz6MeZaN^+GmI97BtS$k;u!15tvbQr0q~LtS!_$ z&9M`{+#>k6EkI=@bkgEKz_?^1MK*}a#8D1c)OWJ z;6bwFA?@o<3LA_IrgVyTG+MGka!LQkBLP26!NiA@$n082TL?A= z>R@dukt_RA9~jz(u?8<4aTl}MjP=_*f%YF*Fg1Q#A?H}aXq-8GI}Ba-aF*AvLri!U z`Ql@GgWw%)P|T>SZ2LO-lzj};r59XZ2!~thpMK^1E^cP0@m<^lF}?&pnRKqd{q@Ig zGHfK;l16*x@I@0@;}u?bXR|ZZ9wTjhwOj2g>yTzcSB2gs{L^v=Ea1yJV9dp^D&WHk z5&b~ngbn~qCeG^rZKHlVG9`9|eLgux?ezv-#>Va1dOvQ%Fao@P#|4!QAtb z=ois1WU9#hFovkS=!}M&uk_|~Fk2P+K<3n+c#o}#ruPaJ?WvGK@xV68?EUD5gJtus z1M_E2oShpg&cvK0HUwe3`X=4tXp_b19KAkmU0hWKeAm{ovZ6AVzc9a0BNv3+NKR2) z1_;13>3H+XW8=cUyg~VQc1JYT<>?SX!Gmf@jzu7wCet-7(Js~nadqg}j+dbwEp5{u z#3bzIRU$FXvhJdnTN-n`G)^o2bN2>3Fr>U7$2oD*(D9n<;}7IFku9J>Auw!B_+p$< zMi`MsNKyKQ<`fsni3j#H3FmM@6CsLdOZ^_~NOw?+5`~G+F_Dh=$Q57BEj}Z*f7%t45h~#XPDUh??$3{q1NX~i7=1Yu{!M|9(fYLs z;`IBeC%hIl&u>X!e)~Cnl>DhLy%<|reK|YQZ+lu{*XiUP&M@^m(9phsU;K8={>F6h zTjGih^NQ?&2ykvc67FO!;d&w|fj$nIUCHIoOP12#2=OXg=P#3O6MVd8I@+P8c+4Kp z`KFC3!V-#mPwiqXjazw_hJ8@U+_vu42Kt9GL4)ZN6sM;%{N!1}AE)g6i=a6E(%V=X zyohNb#Lp5PAOj;T%xlR*UXT)C7-mBC!Fnh!tH@_0K=_@Ih7A{&eO4VZrX5IMZMEkEEh(MKuf`|DSH0!Y)A_3D3Egn5 z`@N=3b;N>hl3Sr^9_j4WKmCGlQx_@d`LmPx2Lve^!!BGewx5y~T(%|HvJ#;z;JTq@ z7v+# zLKPPxp)K1ee&AZMbEmq@o{EdgzH)ciV(TvLJWptasKDseT}o!gXF}w-_nBCs zM;QIZ_Bte1S<54DN?e8?L~5sj6Zf!epW4kW7~C4(P@SAFIe*88_x-m^P9*4Zty0&3 zs@orw2vTChZ;y??_1#rf2oS#3A{pgQOMI)&NaNrZ{nQ?;5gF0V{rKX)HYxZ01S03p zy%9qTg0LsTMRO-uaOB|Ja0mc$*81*-0zxpQ^O@>%a@ zOCFF)zH=}<{a4JJ=ke*L@V#fs>4?vL9a0#?XV+?nXN1b_`FVTF76}!F_kvLX`p+;q zlptPdpfQ;=XX3Rjf#xxor&M<+C23VEF6>DJjcEABQ@ey{pyonCQI@E^q`CNr^}#RV zGNU64-#=vLr%ohIY=@@C$jQdpmXXkw3VK6mOCiJo3}d zw-P2=7}9(kSR7x7E;ZcC4GPp>Tz5hpwE_huef|H*3p4qi2`MmeOcnnMLldlf zE7b?$evZ?XTbaD|SrJf|_b2{JE}dF2;)N~0vS<$tNePw1KMRtXzI)L_dwFMy!)G%# zPd{BmC_t6@!bc0fnl3>@dLU$M`H!rh*T8 z;lcqzL=R>BWJG!9ImD|t!^ppSonV{-_dV92n!Ersh zi-w{TLJD#yjuS7}`pDXjN|WuTp~&-rqP73{q36d|jt74!1t;@fU1#){Cl`QEyeD)m zF2L}XnP^Q*crm`(=>3m)Obp;$s2DseJBIW;F&bcY)o>REJJwxQ@t19(&5dA&`0m|p zIeYdcIL+n%M0da&CMd|AxOt-uBI@yIFO*UH`-OOALcQK#f+4s8)wTd!E+p0GZ`c4} zrPawNwctGl<)b(kvs5HD?kRm0B{%Bg?VQZcEOT_Rp&T3T7mKe{f~EXmmmQ?~i=)Q3 zEFEd;J0%5|x!^O5{rl=b6giT}{?Wjfo@n#r`U0*$052JXiS#h-^!6d!_*y|)Ku&YA zKh*GbYbGvh_y1qiA&3oU$OB>U))IW%fW&)m~La%t1A}V!x0-T=+aBoR}zwBK2LWva7T6 zpi<;9tBM*1dr5VHo52M64xdMFe5liA_=R+DD`%7#vd;ym#SX;cY9es{0_+Sd9Lp$IngPDZBkQgmQD*k$4*1vbpM0_M0YALR0B+O)n+SHM9u~2yfuz=4r0s*b^D6W}REPoos_vpf_`Z z0cc|1gnRRhb3e(C4&~NtO^Iy4KqnAe6O1N=LB3LTYK;dXo4Egfxx=9TmEv9*4$N`& zG7V%M4S``75Dnic4%J%X8p^qN0_BvxtG$l;%V8>rl{@WceMoV-nBq zcCUVD1mf{z|iyU-S!OJNY*LYMIxM!KOtb{GXGiPdTUiG#p-%GvLSF-*k;o zaIR!cl|?qt3@=7m89INhFFU@iGJYUVL7M*lfZ=`AgQ7qSl8)z8^p;ZIshWymhOxf< zpOfwM>U)cVQIL$_$f_&B- zMF={3R__@SJ4mnm`p%TQM!ijyWivnBMat^pN6Y1TXZmR9F#|*ZpZ3HdwcN7LC*sc3 z?CfmKbyYp7 znH)OG5nO*~lz|h`5?%`RT~*U|(Mf8eAsidGR$z8AbF|xWlvD6~ao8Cj z0m}9~Y3cJ33xE?jhnK?9aLWqIu`+3yqjE)J(FZ`3!7{n{S>MRh_*^H_gz4_*G( zJt0S^-CcIrc9ro2F~IWpBvH*g#CJ&()4o`7Ixz-4GxC@@xT7vWb{^&a{4@;{CN?u} zwS$avWUm+fD5pOkqks5K_{~L&4(`YFkJVPM{EPIKZ_Nf>8`^rjZ=*xD3#PsFIumzW zqMA5YviE|DDL@*XcqrK>c6_pG+tqyRD(~OBVX@U-Uqc?TmK^Q}Bm)moW7EO5wUNMd zim+vv!!^oLgULJKuYkuQH&*u*eKzE&WYcV(TqkPPcf34RCE%(Lx2tDK`GH}0-OJdp z4#1$lReJM3&W-nB#?8jENsThC%de)#RdNxt{l|oE6AyoX_3X2khP(H8(Z}C_j^S;B z2~Cx=$*0*07w&|`j7QxhgY2eOHnwE=?ZvaX08nno%m}?hj7^IXA-8;!Xn)ElE4I_m zb(OuUlaqrcv_8%zvd7CUmelaswAUAb)oOKM+=*=v#tgY2L8oqz--vG^A!^A*A8oxj z=X*|1D5~5~8CsqacmM8mlJNPkNL+kcBw_|c&_h37`|uF-rPzf0{>6X}u~RCJ!%IGM zugI9z@8<*stgHMdFG!;DtqD{V^v~+xpADG9`q-GILm`<={7c1g8|O%pgdZdKoZ zH5OFZGsKiNCTo8#(CC8<>tf3vMrN#wZbFgy-5nO+1N~QaYcSmjhVd5h&!bam?ERNd zW6Kt%4%pDae}!#)FY%lN&Iq^E`w}MqkiJW<#YQ6HQ*m{1SZo|?W6U^L~+@~K4v6bF+64)zL(%MD}qb&st9EPG+ zisBva!j77^>^3(xHlxM<%+H@MIX6F7=Bz8Hf0!@LbqffSCzPE%>Ri+o`6(JL;1YUQn`dXEcA9q(G1oG(k4e8{q;G0K><9`9>J3& z+Q=ZGy`j?lwP)58rpYk0+NB90+2t`>rJ-veB`Mf(>_B?s4wvwML-aqv14Y!pBw#|- z_SeTs+?r({iIPo?$o8qpyM7yBvO#fu0g{P+-qUx6^j;FLivvIL8|4gfyoI^h)-k;)K(P7QnOYfCM=_afBLADDDK13)D&90s*B+BQL29w9h2Ey}s_IO`HMB;$^s) zujlIFtLObEU4RDFm|lHa{Q6-dpGD=zIM;9we0mL=CI<=yO2ng!Lv<=7LygW<5xz&E0YfR!BB3G)oP5d|nPSm4 zYVBQq`u?JdUjhD{Uy$`~O@0{bV4fDeVMl2M>%^+?pSAh>%aW>`SIo9`zo ziN+fpwC@@*W})tFb7_TP1g*TQU+f5+*rFyI`9pXI-WHE`i&z-P*2+4PIC+}HA(f_m zc0~7^&ta%tuBLk5Cg%EANvKXc_h$MJ={R(H5$yGO#fZy(Nw7?TZwMBs`1|0wBG0Tt z|4rr)*v{81kBe%UjTB_t$LIu?NJsZ^bJ_no{6m}&_Bjeud*2ZwyFKpjEJ}o zfki68S+UHjfa)u3f#wn3%M%t`=Or{Qf%H=uzF*uid+5w`x~B$zys?0N%BF9MZFZI* zGxry9YAi9({7U`~8=AzlQ-@CzTT{Wk)8hY5Ibk!&+I-I$-jD_`q}EmGmkamXc%1*UM3i8HdO-2vFwTO4`@)?Ajv-D@YOi+q(-Qzh9U_j{<%&eed2 zVaF$nFx;Zl#$?0Bq*Z77dGC&W6$Q?iia_FxuVGOIGC`n8I#g9qi#5OcF(>_gNR~t90c`uZ2=*W26E`1&N773m7ajiv> zY4d*c?J-I3&pfgHPd4V~EXz|yMeAZJV`@K-=G)Z5&#Ps7+pqq`R-(p}oP6h}N_JMkK zlqX~>usaH}d*4X-;FE;kt^{*-I@oE+LKwTi1wmpy2g_I#6)wbn@4|oYh?4)!Yx~y+=l>y}PS50i|i`)%1(L zt-J2;Py5ffYkXw~^x-heVV=L8v`|Eau&P&%R%xHB{IH8af zwj6R&zFMAsYIFMXQz)oGyLGrNOqfpkb;a*`Kww{bQhxSZ%Yn>)tW(&f6`xN8S;h+3 zM(h{ZK3&~I16t=&#n2uzA)+D96s$voyFtzyMSFoS7V!hjvz00B4J+VCy+fP3Ybt&| z-6c4HLDKtu6$MlbGlqeXM>JaR%6mkm}G$4L1+PO z-2MDaqYmdGxcL!sY!mq~?E9M44A8AdM`rIWd7%Mj5HSi;ItKdg0#iH!I~{?IKzOid zRX-Fab>=vA_w_87KTzxI_0+sStuXxP2sy*)1{{fMPoTZLR&_|f$qf3D+lMC}SABpM z!}XYNmmu1?GC(`%f=%z`+hZWE{eP1{l)}jQmmDC6zhBdWphwJ!8GZsJAqd=CAxJnT?@FxST=g}LvU=uz@KcyesFeAHLuMg*8C zfrdC?#U7O9X)VLG`Ikc356oYZumU$g$4#B{JX^7Gj{PKOf5a|7xa`2=>;+DBw&E-i z=j4bwd=DaObCSI8<)+5vqj~1EUm7tc%i$~3(r@$^0aTvi2&SbPab`YAZFM#SlPT5G zmGwbz!Bo;ad--i0LRZgQ!VL6dgdOTSp|6h)Xa<`|R00u_*T-#@#4VYozjuif&vl>LVp$6V3KNbNNQ z6BP3tAVU8(BfyuFhyMGh(RHw&B^^oT=4Ye$`u^oEpLDK(sjqBaMlF-6xu>vbZ4$3>`f#Dgpa0MF~(B)Q$nD=P{K(7>gc3^;UQJo#O&8~oHCn|V& zBeoAbUfXn3r5DFMn8humClKJ2J3tk}TflCUal=Vf4T8g&@CE}gVqPxAd8TMc` z6ZR^|JyMNylGCGh%1BQ(qkD-JO4IB88 zulf~*(6`MznQBo>rSu=&?!P@*k}Gw@SUMbRE~VqW@xb6uiI;Lw+Y)TJA{ zrkbwelHAqN>cw9-oZks3u}o7KLUZfeD-KGuF-NcJ+S>iyw7jd1>N%fWw}Ab!G%Hx;0u z0Ge%wj?xuVCsGq}b}DZ&agHxhDU&Z?8(*_uMEmw3)?Qe*{ebk;3BnM0vo0UXEfW!p zVfW$fIa0}0oxc-1Kq5g=ZmpI(8M_xzk(d@re{1@pYjG&*B&j1&SiM8XQQvqj z8!4}BJh+~wn&Fde-Y-~dJqT~D&Hb@8iM2lX>6EP}`d2GnWugxM@YCEC?K2~W_dd6= z>yz*@0aH?7G=NmQ5W^|lGTXD1)Y8LJGJ*CKsMApv8k7}yr~$FoL~0xC<(joutY`tQ ztl&TRz2tefQ$=zdoUY{8S9JfFa%G||tc)Sh2%v@#@tbk>BeWD}&46YBb~>u}lY0e! z-51S=K&%qRgI1x zwq*Q;7Ia)S+{oOI#B$8CI=6ZDEn(1-=l?VAMljJI{UR6}18{re1>UHvz3$Go(=g~! z`9f2d-4@XDB~|2A%pJppx2F$j^_0tsL(-jVPdPF?SP2I`W~Sy24*8JF=f?AaztES9 z{wMBB)8@bN2_O-Nk5Aj|LVIx);#&Du@ad38=e6+DafYd?M_rmoD+&5&n*_@tl1}Oi zL!`>+{`-T^`{I2k#*%%Uz#_K6z-lECk-)c_%CmdzqxPU=kPD;EBS;YNG$)oOl>DWQ zXqa$Y_(txdAI5lGf?y&xI_qplWod3=&=HgE;M2X4&pa96*Bjweof12%dn?!=c5yY$ zOV1V}QuSsHst^ExI7SC_Ct5mLXZ6_%j>%ew&*@MWKvDXIUB z3V4#u4TlAi3za<7KsU`;QlcHh_1vAPIOG+@=?@a4?R9!Sh!n@>u-1Qpk|0cS8Gm%_ zFSmP<(0wWF*gLfCrL+VpBE^^PhXFn(+cnkDl22T19+*i8GSEd0+L%5zIWKt%puK7+ z8yp*ktEzzxIWF4iJk@O|@_bM|2_l&AQt|37e|GXBkHHBZUuDu~v;;=hLo45SMZ4(7 zIu}Z zBA!ygg8(*P_;qy0id_V~tBqg|gjG*VIT2Kn`(CC)(^J+VB1y7xqcP{=zXRHjaZ_NHh@JUp zpkPfOmxFDItX#^r`F$hP!~iW|>G8;vhRy!YFs-Nflgw)hCEL*q>bs0Q896W0<^kA3 zO*R<5)`6#|e5DgTLDI>ltJUld*%87=UyR7a#SmLmE!jThX+IM#JH#vhkjhwkSq|ss zN}o7y^~Ax1fl|kn#W_sHM-*qtH3HNnF}sM_oSGbWOv;ko;1jI6Br+A4fM%-CPH$qi z6#JC%HT^f^P?>yCe^+=)GstlZU7m_o0X^_kR6pen$H%4U0$Q;r)9 zAj6t5JdpPNJwCwzC&E>a%T#1cF~6acpJow=wrLJJ<;XfY>67m}+7PBnI^6xJW5xDJ zA>akVl;**ig~%m^2*)0DB{5H4xZt4a%5w2ZNa8Q7QugZSZgxzMh%SqFtLR|zO^{@Q#zSAm}9c!NaaIE+e#p9O&oAkcNaD*;HM#V9M4HRk2zZV7yBcy zxbaY~%@PSB67Dw!#Z*4H6+&;#lmmXp?p{g$p+@uFd8O6+K)=hm=7}}sWUA$qQw8epp&i&3M4Onh?hDm#bJ}>WLTVQWB$ElkDe6c&2=I)@P z&+PZ3?fL6It=v(Pf0Y^qu?n{7-2s)cf2oaym5Gx1pvi{jT@df6llRnpdnHV2M2l3x zCVDDsvbvk8#66MzUn>O<=0u-&gOHYq2uHa=Oylk`)av;@&o4f zHIYTZU2p_khl=znb$0?xeV?N0XTn(<6d-xYY?G!ruhGtvN-WL68ydAEIi6|i7lWxV z&qF13*<+EQrsn+KZRubMMLqKSjf0b24oSZ*On}|997l?m`|i8v-=qU80^x$#XvUcE z0ExXe)fNS~C5)3NR z64{;zC!@xtW8N~yY|%ux;BQ1E>kVJ`lT8d)_}|I3B-$z%y>n$tB?qTj%v0d)a;KAd z$=c-hlGOIAwtGho%8{0@<>!ob7-u1z03M1yl{<814%-;>$-!JRpfz^;8MqtXB}mFw z7N=clb$2!aSoKRaRIW}HAHFnNPk>%r2__*hTTsxhQ?wqxd?n}&O`FVcMShZmgF#|D zYse7t<|DN>or>x*rRO7 zmMYO;j6v2#JH&o{ECo;klU83#9Zl{%K!kdhepgnVcidPRDDj`~`8p#a5_>YYEOL4+G^2S?F%RUO}hq`Hq?AkQkplPJ(C?{25@a)@u z*h5RPnG?TQ`Prb4KbQfY>o*{1Kel$Ivh`QZdn55y%Ijc@mKsh7%976M4GkA#yfs*r zAVq}1vE%stfne0O;a95WVW?!dR?6Uzbjzd*)vKumE(BIhyft$uZuADs{7-?kHVb~Q zwPf<*;&fsYC?GLR;ux@BZxazNRwBXa8{a9+vdkiAWw$OEKCBa{4UzL^iJKCQ(|A{{k~%29xaYXR7czOzJXX2_*%BWI z1%vBzjx;3CPNVcFGzjPaX}2;9>mP*TnFNG2jVFHV_O($TucR_0Nr+}hw(`CM>T5OX#MkZQ z2&=8w1`x0hNA0$XBVE<=LO6+0ti<>K_^HD%K2Qjm_D* zP|1n3eDh@44Pm>K@ldn4fnrylM@C9>tv508X|Za>o#{)wBcaDhJ1s}=BOy8WA(pGT zKs9fU<;~Y=sl#knvWc$bK=RNNUsJ#{-pZCEib1!S?&kad_LLNyCo?lc|oD|=YMV~~| z@8tlEz@|b4)p18-V!kBE8 zscVmxpY{jQwDrr3#66`{O@C(~uzuqDTI9PFGy@o51u0%F_}Pyp3QwU;RF$|~Eh;Q$ z=Y<>8Lz$4#2hy=kr@^R0SWFv=Xh65#uV5{pUh{TUp z+|L!SkN*(+yv>qPN}U&Iwb-AXYnkW!uzzvpPBoBzO_U45bD{=gFL#4NBCvy>ub%Hx@<^4~&BZxq$lWi=Av?Db+)LG;PI_O05n|l3C0{m3HC79#2#H!0PBO z-aIKH&;S}07USDuzegTlb?>QdIS|6x5(zFbY?-4ktlQcne>e1#RP;CJdbFdrE_{}A zMZ)u~#~p$2e{?@+P!@??#|=znMP^x~MwXfxNjxvS$VMB%>hfy3fF2Ut=HS<10afrr zLE3C@x|r}GaOQ=5nIw|YPO`ZuGRtbcg%7f2N~S-q3NX+qkZ$=M^R%A#FnTBVI2%N} z)=}B(aPEy$8bBw$r3XcNnK(*Z7pbZ-49Al1eCXtA{d z&_BrwOX+D@AeI7sY44fw}svuJf~M8H{+|+9awY8uvnw-Uvk&HSvw6 z@&2jxH%v}AgJX1k7(w!;$l|$q0~CV25AH!KTkMk|Ie9KEtw*xH@kT(tMUWIt;Tea z823FhNf&o_ccz=@2Oqq7DEk9cPN5FSBWPDf3J*iFvhZG@{$#)A!-0lDsYt=7rwQv6>$rl!YTI0&zK$3!)x8=VTN=9hc*7%BSeN zSPe{CA_Tv8HcHB(5}RkE5{d%L0G>J^H|@TCJ0bK?Ij^xedR80mhCKIBf%?Pji(bOx zA(&d*{58M_r%9JXPx0g`t~qW)9ac%bn1xAtlv=~Veww;Vx8`NlP+D<0`=mqW@nF|8 z&@wp9sQz z6)>s!mlO!_{>fJj18&K4Wi@QI&$%ET?JI4TZR+Xwz{G;%8+i*H_ZDja6FrhFLle3k zUsr^+M0HV7nTDi$XQMb4UC{4SIVi=^7ZZzaFQkMM%w`l5rb)Sc*xb}l) zxoI%8r?Nf3>@w(#R=QO$N`8DE%8e?q_#{v*eO9Rdv}b@5e>0@Z1qL{*4-KsddQ7zG zBNhi8q6CZJBnmHz| zH}{{C0Z_hkrE-wJu((MkAJWWyZ>fbtJn7hN|dLdkSV{VOp@$!z$owbys`p|jdK!qG8*w}G!}|_esgS^2>ZC2-U*BCv8}&6IWBlLa8o)R=-wQwe=1$j;GTOw(LG(p z>mZ3l#+&bTg(@hKEnW5n)Pq^BQAL*Qr_)fF1xUMN0tb>7J= z@@l)nB!Jkg2JRT;t;xZz4DSmnoH}ufIF231X8FL;mPtEGAN=wRbdg=j%JF9B%15>Y zo;d^EM=2WKbh$@w_r{=O_2(h)d9LE(_)C*b#&*xuuvt#OB=UN_DFMtksfUIUgxW9% z=C!B+C4q-YKtu}2csTaw?SaR#hvH;?*YxG-@xzfF;*CqZajnREh`lmK2ia;DJ&2^fqS{=fA_MONS21>_Z(u-H&n6wK66>_ECceRi-^S{P zZx89O_xjVRM(`>=|K_u;ZX)=r`?L3^-#{z*E}IQ=WI}K%PJ9x%y)+mmatuTB(NK+O z7+I)o1K#^qw^Sai+8aQ5>|P|8OO%7GCB zm8U}>mJbmrJ@Fu!LHoQ%0s^a}ZyFMc?^Wk{$D5qbSRZ2zQ0~$$XgI&x))Vg8jMZhR zs?KZQm;F@7HBgC&RgUx+&@jmQ6w{K26@R0WFM>HYCEo~XL@pi^eA9sKogG+BX`C&9 zm(kWNw?RhDuUNaCo|$pICC^o#21MJOR~DVWLHb$Kj>Pu*RvXW5 zb(#QvOcYm>JKbSApxs&5H?}a2y!zS6z{FCj0pOx#s=8Z7SSM$$F~=vnbIU4A@zgl3YdKw3pZgth$1*KCg`)us%wK%p}w<020{cptDqMNq>i z5T5M37;~*_9vJ|buBXV^&ON294tXR%&e+%5FJqhRnr|0VT98vi6L~H}(IiEb74(Qk zy0ockG7ayFEbWPsE8N!xw&tkv*@-iL?}5XrKHXFAN5(9{D!be%UGMWnX$g$&x#qRA zM_$5CbN3o%QB9We8N4a0$`ESRNn@m*9ZizsDI*h;t8IhY^S*&G_Xk`6CwL&-(*Cs( zb}&x{PzB+%=O;krhOF-(a#YB9sOOFA{t_`rxG(3lnWvN5$B(HKg5z9#0OM9*I)Y%n$&bd?j`HBITMOTC|lX_M&v>*nN)X|!@YtshwnE; zdr4Dcx>!&uOMOQ-b4wq@xF2WQ3@UOV6dhu2H&r_nHj)2VkYkru-o&L8qh~JhgClae`QH|zA43!E88FwtxNgv zRC^||U~eqoR^z(MSmdcr?>w{VWY>Fu3WHcJ{8Eh=Rmt~Y8arUyhNpXTd)r13oT zh0|k=N{>9K@@Dr5!yIRLTmpA+8rMWFu_T$@~DvDmf;@JESq(D-4LDkvGP<_9+-_b z|H4TE;_E14wh__&i`QA^SE3k6;>*X{8FfKn+-Uib{UNuD6HJR{y8Ny%39+LmsoKoy zK}#IwXR&tSmQ9a?KCCowQrGW_v6Y15$7tlbJ(;STJouf86q_WVRG#m**66^RwO`_Z zx05bgs9Qp_Un1Nrkzn(m)g+cGF2GK0Hh;{h1~i)4aF9~a;Bd*1`;L-@N7z6^Mukj| z2zP;EmS9?^A}r#3L@mTkQ*nHaLhgR76<731k;CsZLcWUbOF!6yXl`nErpVsSQ^Uto z>ci6MoVXS0>1#%_#pgi(F_r6@d7^zm|e4QeP?Z&y#9W*P2o6XsT-rNs&l$u zSPJ4?KLVkq5rA#_9D;y;F34Ib`5@cKz=pW?l#@}Gii)AWI9sG!3V;Dt)bw4$^W`DwlB%H{zakRGnl+}u!Gx~m-`}g5-dgiZ;LoyZ<1HvqFr<_e0}57cQN=0$ zzGSWmia!4HYfCLz`$4I1FWH1+a(~}0euiW1gJwcc3 zfTw*=Z|sD2iLo=mjpdKJMQLK?71>zos#ZpYtCF;Qq8p}%LYMxelVJQHfZyIU3lfV zM<3sw`>Vwqxu#3|H#n0YhY`~-7uxY!Cc;V?whMzX!vz+Q{?lB#olh|$H^FYBPhcK^ z#Ao0)&rsdpk&wrNr@0UWloGmM8u0I)=bt+d%rDDgQy90cWdtY+PY*u^2#M0y41oIK z<%Vo3m}bwItvI#FAOp35+&!)c1{{ZSr}Gz^uzd;DC?mKQVVqE)pS<}Yz z06c0^Fj5)9c?dP_y0D-m52->{m5EP1)rl5p|{H(`Npnihn&YR; z7gPwv000_D&x-|Rqix#bpB)Sx72{KTD%tU_2yTZ65sNr(bH1iPz8ls-FjvqfkAy=dOwf3h&ws;_XkMhmJx6M<{$JTe=%Ptau z)_-thHf-3I7svZE1u=j0w~HUF$eyMDsSikSP+e}x1MpC%!0+7yit{XY*eR27TF59X zS8zfg(lVoUZ>(`X8?D2`4HMqH`~NX)TUk1#guMA8tkqcx?%I0CgrH8e_syikqP96s zEd#bsQzXQK%3MrVUJ(&WwrjGhM!FeH*AQ42JUShv6jtTXwt&m|m~#82y;w{G#%bQOW?g0h3qMW5!`*F;H1w?u9MmG(+=~!lwnqM5=MCbd`T6}2c#=Oz@~`z|!?ncY zwx#8FKV!R#0}(hxtRjilZH!;Yb@uHy_YA)kTvyYQehj%A2e~+?uh&sOlUL4cmSEFG zEVGRr80k5|^Ve6wp2fudzGTCmk&vJY!yX3lp8_|>PvVYL&$m@cbn!f~eX5J~*$L&5VxuZ%e6Jizu&J1=JdFO59=`SDsk^S%zu8e{CggE4Hi z?}MM>l25R2HrVs4*LU8B+pUkqRR*`BYeR2r`D0ncYARd!gUTko@QapR{qy7&Jz3w` zZvdsuH$vIv^|kdmVW1hBRT0lKW0a~5Y#baSvfZ+}>3d^}XLtOPL61zl=MJ8^S30eU z97g>R;CjJ$hV`m1&R2d%-QZO&yttLd98HrQ4S$ zu^YwdyDYQo*SX znk9~3(e!4*NXF0-%lhrl5uisM#NR`6mw+djj2BUcu)ZxXcPx%}dC_}1l?~dCb4_!G z9`&@Gt!9^?OXAcs3s6e}h%r)4!pden_l(hta(S*n=69urN4i-2Bu~j11tvvj5?* zA8#W{k7q>hCiVR^vVU`PHdWna{vH_{YZk^g&6|(c^;zGSxmR3;BHZ1=>T1l=O33@pz!h5Q z7;ew@^#c}*tb6zl@4ccmPyfb$`^z5ZW%^Oj<-`;>DJki8Oc$PfXJyb~>0wG(IuZTs zS6MB$bbt~d6FlEbONsw@XHq3(ztZ6o1&zEI49BSe$xDd#NYJfOpA+fRHt0xEdcB+N zn&4zV8YPkMRP|TC?$2^=!X2}Yue79n>GI)RA9T}E%*@sXz4#3bof1zJjjq{UW(kGm z*Mc&JB)M>5(fQ&Ptj$eVACQ)i*uXXFgm_u_hGMKMkDB!hWkui7laz|Oz)u!MZgNOG z44~P+L&SPKh7Wtx8Q2hw%iuKHriCoYTXnBdVU6jxJeb2$^Xtf#JAgDH$@o95ru%o^ z!vY=Q-wf+4+WRIUpWGA`TM_WWMuO4%Q?tb}~j6qg|*raZG*OGK*-h^tUD1@uWd1hB(+k zw)JT)jDl(H^)OBVuq;6EP5{+r0fsjw7bMZ`K&*QBK=yf>alyYM2xr2L&xn2F2#DTC ze&#c0@&m=6Qr|jsM2`nZo*hSiI&Y!Dqu!^aK~fg^uTy~WN%_p{21dG*v)8MarTGskEWK2vp2qB>s5eYayfAK=X$n5ximk_>omAwr7Z|G+m~@nT1^QVpB4N}D(R6F$I4nG!-C_r6FXW#E z9?xTQMaRATQ0*(M?&S`Y1oQ0Toz?PpX z9`(6iS~o@=LrqcdMODk~&XINnSNO6Di!&%JuzX5wES@Nxihd5`1ew}E56XotH^ijIZ!dD0lTpq^0KSLUBlNY_7C{QN(F;=)VXrIp_6k7 zg`mpfs|Km+%#`4%AaZ3&hWzj#cl)IZ-=>bkJImCIhn}>exN8w?K`_alMCHAWLf=CL znLRb^1H@AYoQB+Cc6zW7VBgm1o9cr1)E5~%Bg6nG9Bgz>3z3bZjBh(JyD<+ zGkXsieo$y>40ToW{@&2c^=~Y}*(XNPNTKWK-L!DbrR~5oRW?U#~Ymekl3~cY3=tNZ_RFMT8iz z5&_6IyFz=ay;8I)jc)<1%3IG~*BV6yE0J66X}`HJ-kYsY3}cJ0&b;jp_nO zetnX<9B*y-1|c#Mg4=i)D=kv|rOx|BAwOq~eYev+mVc8a@e?`3<4kUW<*_!Y+oYP% z{%KRZffXUr0jk~gRWIS6b4#_WnT7b2M4o&%M4}@g8D3rYeo^vsA()d2g!k{wb+|Z9 zUX}zEBDMi_eL$7JPxjs7w6#Wu(T|AL96(*Xrhz+`GuZLxouCO;@n@d{`-f{=l(bB= zXEiXE<0Hv$4-u@Lp1z&$Ts8)m_DjKlEOuuqAz#WGorilYGJT*unt6wFWaLn{#IU3g z%#BtEv27(j)9ORTEJ)qt`>p(CBVxpK_bN>>YT*{}ptSJL;ERU&l@KEJMM>J z>>`SklCSyC!@k+R`i{k64hK4+n|5vJF6)k=^1;U_#+%=J-nA=PR~Q(7ABFTLG^ zLrJ@y(L`}h$ZHaR_aL_)WuXh#B>dJ*en{e0pp}B~#M(cN{_22I9tUUO*JXWl++Nr6 z0QMfDbARWzxR3VN`Gb z-k67_r1hbdzW{MUk+VXNx=*&zv(F4Ek#?d0TO0&&XVXUTUtl(9==)RS61PQwR@8C_ z^*2mSlv}n^Lq0pHgmw$R$#jofk&{$^vRm# z_$S%|C_4?uJwu5Y$6aqIQ`%4wP_L0nZgZQ*5=OCBN+Yn-FBj{)x}t3>w?&5#QBpC0 zZy0Uj*R?A~`qLMl5t8#OY&_`GP68Q?KNM8wp#3yx@&(`&TF`Q_*I(K?zs`fs7q`g7 zuQQ8Jx8+6YxRF^-7LcitB z4JPAEX}sQ06f-g0cWO7p3KuHkK^SO(PWzJusZ1h)j@GS`(k`^p2kPDduWQFIy~{<{gIdzcc_-z^O>eY)?EwX$0%5 zwgpfHCM@aCr=K8tcU*$cx3r;W<1lN0TzO@6Sd-jd)&lU)+cc>F=BQ+8W5;3nMCH|1 zCEfb(d9;kKG}g&!!#%#s?icX9ZT^^&ElMdJI(yZoy-YR&D;*XFpB3h|$*F9+1MF)+ ziu>T3?G?ANE|G6I3|(}w3G<%c&`f>7M!M1wl*lk68a%#K5R}MB)&IfmH9F2}7wsSU zN5L8^bNgdN_8x}z>!UlCa!fbgXiLs^hH5^F5$WZ{-`%_Z5RqD)DVl1@aQKY{rVy;_ zX8;>V4dlSQe!*7+ZUW-~{aRZ*Mifw$G`q0mf*>DRGMFok z?!a$$yFIm(9}|1h_TLIkjwV!+W#leK&uo)nmGcro;Y# ztDd2;$B4pV@=4J?H{enD=@##72jHvnB`8uM<0s&zQ8G7=5>eb~`BBrqbs^f^2P*NO zN75bI2*QHu4ljQO2OmCM!aS^(xm@iuKRHJHzCcX0?|N2l6{`onZpUF!H3kkuZ^<7^ zv*Q57@p#K0PV+fHK(2?%0_5a~QY?+LxIf>~q$2R=agfU)mZ>>GdO1w`aC7F*ehNz! z4W1tZtd`UH$d2)Qn#>@YVMrQ$mAyLtreQB3e4OIa%H!?0?qv=gI%z*93e)UbzV=5Z znhyteQs|ErhMQD`74df>gR4-x*S*u8P|6>M$>mMjYB$!pTtW`1e~OdJOJuTR zmfKc8cB?2CGD%sT8zm8Pg@){=G0ei-hx8s;A0+2xBltx{ty)+0?q6;;Js3;9Si^FJ z0A0l6te7U3yT^h;OO#)PE6kPf4=$FeC~te-M6aSwh>h;KGb9NG*G8~(P}@cuPisE`$L80 zD{DO%RoSL6^6uErC?$~j6*@7uc+@a!w~L7l4miY;!$knaon3K+ubQncUxEKOvP=!> zixtw!Ab{2QW$1SZfmGJnF+g#Y08q=-(b;wC8zD_gC(%nqyTHMEsqPEcE|ERUl`S!~ zkY2qTTST{?(mQUt+{O3PHu%VPoA+p8mAtX2m+tD#2Aexg#}|}C;oB)1XyLI z|D*Q&^cVFm~&`5u%u|gPnDSuH5X7rgOS_#P9fMFikeCHs(i8@czAi z_sMu@>FsmAUCSz6IC%M|me?DK7SszqI)wX_kC+ z#<8r-54`7?nk{+yv)?oRWXt*`dqH2c4<}qjj}&kjgYVy8eM5xzAty)4HI!2H{wHuY zQKM$cRSbusm(hER^2e$O zRd4m<0klqrFQP+gArMG;-fF?6a_pF#C?4h3yUwjU4LxW;(IF4#X8%;jQ9V01Z)*n- zImjh<>a;obhviARhs2POJlgjHSxsXE33p>9wu)8(bqQVNw|L7~C(x|>@5NCmmRc8B zA&WYqN4FqsIJ-^Am>%_@z4*bMx5i0n}J~s5=cRleM4Cn8Qu!53q3+f z;=hKZ=+aQ+FN(ba%Q}AB2r3YdyNG+a(jEJOaS?`3o`{2%@p{!IJV>pTFjC$`Nxr}z zG}%!13EE)64~o`L5O%&OrcNvQCcNl7|E2CsrltS@^j#M-w zF!X8Xgth#k3XeKZD0=0iqvASjm-g5k7GH|+*tHGotyZrlUC>(m%}0$=!&(Qk4+p;< z{l)wWvS_Gk(6Gq<>*a*~U;!viB>1|T^{+(z4*b^MXj4eJz(#qsFKvs-$S^BP8IYIZ zg<-wF`Td-oQC~UTOWSiSmSuPB)l~o&@~_fBPEzIlFday`E}3wYGvkZSh$^k@ka$={ zDFYuwdukR)9?!dvg|29~be>F92EDARrI-9sZrm!;&X-FoqubNhad|Yy+GiUXE4kZb z0oZ-3n78T0uP`T#a$LNnzcBlNS{p}KCD8R12KUw445l8OTK2HZwvu24ZB*eO*hu$C znW?zr;D39$x{lfI??h3}{y`o_`#E=)(^pAV^@ykV5fmds`(ZB}NhR~fOfvK9oc(tBa2WN$j|Wr$Yu7PMrSQc*TCr`hY`IaS2@AvJaUY7iG0W$`V=| zx**FFgCD?a(5)(-t3!=ztAo<2tDDn_X*x8ImM#~T*f+@cuo@wwH2sWfz-`K7_3FeW zxQ|s27-MnsP4zwbynU#Kd*==|q(?=_UaTll?b83o(2udYYv#y5A?5DBMk?hh$jfp@ zNr$sVtyTeGXA*l`X)o%vWeXF`2i^Mtw^H_e37O-Q2hkC+vR5aoSXlR!u)omXdYtY; zwTVAqG8u7Q`>0pusQS&r+qs%_PJFmF-6yCkelR)X-74P2YN*V-U-{ zp9c^QL}BL7dJB6MQQIYC=Fi-{kk@W1wQ$P z3*9udZ$DqBaB3mWFV$k1Dj;g_6s{6tN&NtCGcp0!7q*H}-fw%}lw2xZ7k!k2UnG)C zYgg-oFE&K&lDhE_efQP=*9{(`D9ng7!XcZGgr^I?1Q-ig+VUIK4!rcZL|swp_DZW5 zqCV+fI=47KOk+bf<^NLZN(IbmOVvYUUESO2oIkbNwv%r+cC(*9mqWh4fA*1D3;^$=t;_>W><0fnFRaUBxCHD2b9Hz< zO?}rBdt88W=*_}qRh~8aiIapJ-RM5uD)4%|M|*Od53SOVDL?0MqrD}Wtb)ZZ&92v|xAoSGagO*(StTu%8iDno7 z5M+(|)kVr1BewefpxNa9-G({m)*7p{t|mQa@7B2otk3{324XA4$iD9sLmcBnJ8k@t zY-9N?sT%xk0C`G$jFA6svn{+w?HwSe6Zn+S3%mH(=}OogM$dhnqEfiG; zqN*^gnDM!c0dqR>^a)z!%7`uyL>p5_PGWr7y^BNF05C%EaI4>kVfq-CEx{tUi>^0F`a}r^VYB2ZgfS(}_QUI>q4$&En;* zH$|-PfaU0v;VGQ+6bdLZP+$RP9Qe_Ps08k}K@TLa0s-A8Hf(gizwx#jmu7PWSH2qa zveNJ6Ep&gyMmaydrObIUP&f1d!AmEL_I)PR`kv>UJc)ST34m}$iLT3`KDKM(JmZgv z6um*Jng~u4ej=7J!*70CQj1i`m(|s^K&dI(>_O z$_9N~z9;50o>-{xHZeX#Qnd+s_=v?!DKh(WG9B zhaKE?8V7X7xH6g&K>BQAHDmOW4{Hn`vGVY)h|Or=hoJ!@g#*youkG9KXoLiDxr0=ic{wh8`5kQ9cmR(*)=sg7-s)6aJYwW~SRC8_ zFR2Qy%Drb)V?C~kiWkGT5&C+cW!Uiryq8aJN3EE@f``1wAb8!HPdK0~HzmIA+A;W= zQdT@xI0+l8)>%j$h(%bcUf`!nR5sf*Q>b!SW-6?fbO=c&70(9{mm=wtpIalew*nHZ z2CP+|!=smMd~dqmR#2dY5j*Nn?`gQHgO&?;!S(@b*URUrIjIzrCpH#K-JaGs{?L2% ze7EtRkR|tpFiYn+i$I!<9VZAlNgSD12ZgN0EM7mb&5pR3aV);W174R{W#Ep(*mH+A z^%EchNWQ}TXne)C{i7D4w*Os)r*`YaGu92%V z{#jx#yWi*yIKC5LBSytsHm@Gvi{8U8?G zoZzj6i4a?p!nJ3!)UtyrZ5L8zYw%!ebB_2u_SkZIYmKu0+sb|db`bcl)yNBenHJ07 z@#jv$N>s{G;B4Z<(%Wp;!Py@bc~^?m*THWv()R5*+Em!?Bg-%|F82qO*PmE<&h9i0 zM9CP_#mbE5D^wKC7b@K+G72f>RyPSCkYB%7uJ+(S%%hxI<36w`-{@v@3Kc zm$$yBJm1xy<9i2;qWk!xWgG&Y!e5ugec=K@`WM_FjA{<#FPfDYW$hiwp*ZNu3f&>g z3h%OcYJyamUJ7WLe-j)29;Qb1tBdBDguSfC?=hcGu@9-$S-7V*|Ht6WQpcbE9E&}AA&Kp>I+{mm*$=(v-L9JoECikJjNN$sTF_yCgMWSH1A>_*r#g+k^}>AG-eu0I z5v1(@8xY{r4>;{NRg=V1PL4us*Yr`l^NFZh4jlq%sL=-~UG#nl>v{C0)LKkyeykg< zRt^=tY6Ar5)YR^u@7YRspg7vO?-T1YE-#2k7*eht%{o*L+8;H!i9SPxQO-X0zLkO` zCHAYmX7-T8ZKOc$%+210+hy`1cUEyGKrl64h4ocgYq`8jx zc9);SpMRm~Eo7Fu2r&2+2d3BB?NIdySD%pv?l$E;+}+6i7G8l}6uFmg3Jl${;!7np zuou$qcethF)giy!cBU?APn1O9bJlyU!|UcV{$Zd>hz4A$I`o8{=!Jv$mwKhk(;I?- zhO*4vWnb@CU5CZjmpQ!k1N5iMoT#*}^m~P2JTXu%qNNc~TMR*&Z_tEKp&Os_w8C$; zvyc9~08 z>r9XGWwSw||Cdj-`NKTLkQS^TwA01Tx(k<6&pwI(u%ChoA zvg$|r&|8=Jhadev17NIk5Z}Ytt$e_O@>sh zC(Cs!IM5M=$lZM2K>OLjSOlq{t-{Q&qsA|nT`RSYk3RZUzt;yIoHjyx5fx6ZFFm%qAy z*ujk1V)jG4Pf}*0(i9{8%bIZw+rGM+O;h&Xa#!u|P2GHFH=yBf_kr%Vs-hf%_xp*I zU{r8Rl2|e!7YlCYVHM^oLXwnbJ=U#OYgHmSM>#RQK867uYx&q&l`bhZHa<`3`p}9Z znid_G0Cr72MGs5q+T*n|cZ&5jvFFP~r#VTvACf<{OM%4|4YM{{y>Kr4C&2+IQH~Hd zvrD796_A@7-51|y>rjT<`piVfdLEQFE=pXkWFig)2-^4HoKe^&-R4-+1r^ja6Qr3M z&j8`Q)%P)0f-S1@b~QV-!(o7pcke@QRfj|6RyQO}2>nuOrFFN(0gkZ!V9$71OpEGL zF1<4H+syAFk^E8RTZ^-0c2(rJNpGHP?4@6lq<*uu7zbku4fJ|KLkF~7$>$tv2T_Py z`!b>GS{}aMS27i^GdcDAtU*?!I=9dr8 zcSL&BFT2QEpe;BZhck5#q0+n_R)AtRMCm!w9}~bxCq<}aUHe~4={w>9atwbD<^kU! zpf}Hp$^(7Y?@tr{T@>_dXt`~Fh?7KYIb_s|!Zg6`Oowk1t~zxqgpIS}7vIYDq?)c3 zJ}gwT3&#=tD$swD7SY=?YV+Egy1$N$Uy7E{yC!@5?jfgy>cWt?rkx4o}`YEFtijYdf z!m$M@C}l6+q#kV42*wv9|9K=uynKzee}9XXs}y0*GHi=SEl9vpm|c>ANNv)^L=m?|2)AD?Q=~SE>du?5T;&0CMY(Vqd2K+45qKEb`!| z7)o18W54R6b?K6){Unuu9_8NV)1}OChgohmQfs-*RBkihJK0g}RdfE8R9VaywGN(h zREw$^HJ-$jX3*z}Zd(=OrQB}^1628YgTPj!XlQ!Kq=d%vPt2-t_U{GJL$+I=CpHYC ze`1^ykLW7Y3FeElhPbv?<$c_*bP>sB-tVF4$=p`@ab-#`KGpsP%r9dw@H{ z8J(>4V@&S@QxFyLRQvt2)ux<75R8K6f8BX3U}wb5q$*ALZTItV%5!wl3x#FRl;UND?41k23R++UPGszFy0ACya~?YGx&b3UO~bohiv3t6zrR z{E~3F+B|}oL@aU`yBoO}AO?D=qXTQFQ1|{Lv)%S@@_ji{Yq48p*ri4lbgF+@a}Y*huhlI- zpoTh>MJ1FqLEd4AWI?Tu^VXjTgoH}%;nxMo)VFGx}CRuO}Ej!ns8%&Eg*fO z8*LAhGLDaLQgRS9#iMOB`WlOT+;g8kcQYoof&kp*{VLZ%S71@wLzPlM{%PM*A=IXUn#g1!Q{Fk(v42Q7Mh8K(Sa{Ctkq)7~y-UG^J?r z<-S9nYc3}9mt9{Wawz>p!BC);4)rRZwY^S+wJ5&FkGT>MdRa{A5|nRW$~%_SFHOjK z-QHMR>nN>Zo?&fFT9$*PfAFxf+U#ba8_oYf{GS*7!6J-6l;x~8h4bGL&S&7Y5WqIB-u2Q=Cm4bVmOi5w)$Y1aoh%uz|UGN)2SZNtQ zORW@gNHJB9;Tte_O2=L2ZO7+yNBGe@)nDV8|C{|s{;{*%*|QWc%Df|m6MrbTzwTQ8 zGp_LFYf#}y^c!i@lwQqE#f)?szgMqw6;dk!a?OB?;<`ibAdfqSa3lS7NuiPAbW>(l zeRxaUg*@L@?xc23_X5gC0AEJlnQpR`PAgI&q~g)4vK z<|;~jADHtq++U)m0aF%t+;~;ZB5&Hcz3aI%ALsckQN+|^GAERrtmT`2gNC*!L?kJn z+rdAR=Qx(56#MkH^4J}MX#n}EVles;=HYAfvUQFaZByhf(n8o3TmNso-r&VWWogyq zCJwwjY&J{4{6ci`=`J?8S=Q8jqHtluw{V+Fk<-RcF-0S6K#Aqx6f$=@-5EE`%hUT6 zeq5le4*uC-Wm-&v`Oww}Rvqr5PRXlcz+m;W17`9jwpbqaox#ciYQoozR|gJ)D{ppi1s z{^*64D4bjn{*AmC5w%Fcolpz~Lg6a|rP1n*p61RsRpg!E#!<#nI30xvxh#icYEuep z!(WXR>|G7zCUmV_dR+nxs??7WrKV0l7uKm5lSCg+m+ zJU7j&AKE(>bpJwV@>!&$IyCtKa;^miZ<72E^%Wapv;6TEy|VmK6>gC?H@-@n-pLdR zh#YP%RZ$D+DYt`PSl`iva&KFfhA^ z*T&aXVNlOujSOS>5W;4g^-(nToY>PO`0}uClx?<`CtT}RdS0-a174I|h(pefgXS$9 zr+x)7y4@~2Jgq(Xm1jx*DxdK_>^`w(1`j#3zdzZ~6%&XZJIoPsGLtYn=mu6n`SyS7{|4U^4vf~S{7(ZMNSRzTxSTjVb9cRiMP;i{q^W)%owTTd zhEwMne%*9MPo;pYdB1ZV4qLOT)aprjP~*d->+vWSRh+VwmiGLdWNDN;zsbMHlp3e&TLs%@4;{K z@g{3Qt#qX9!HYxH=;lWDu2aytN9K0AZpwvq4cF}*EqzdZ^)|R$%+z#wv81VDZtaZx zZAe9~h`+X1@bKlkCIHmEbaQI(9<8CW6egGy;}k!rX0`o_`TpQMM^=3)%oMFl;R!N` zhzV{RD*0COtymBt>olvE1W&0`SNp&I7*1qnF<-vDdk_&#I1nE~2g}KmHqeV+sck{e zs82cJ^{dK7eKPP#9)Fy2(LeA#uu@TOpS;hnIXwQuIc>r18L!VA!D4j_TEH!_288FI z!M&=MPp^f^1@fdeGVwC>-lrbH?KATCC)FY>d^&h71*y(o zDsD#~pR7Me1#QJ*nk8dncdUuowD%ax%H0?yml6%Bzh6_32E>MR*WRCvB=1{}dd5niWy%A-*{ea43F@71G7{v%SZ>biVbtf+J}3 zS6kb-23DbvI1YzM-@r@}&SB*C_D4F66W&8$gR9W0cS7`)aXN*@2HWqVCSQ~jaX1m6 z&_PLzi{jC0Fl=--p1rQzdM)RE5vHV{1heXa#U@~vWrdvC_j6bg3UNX{x_W6AK>s_* zO@K5$Sclf$B7jEmHeIuY>8?wfKFwvS?PrMwP~8(W*Re1p;hKA~*aUaqDH-=R(1$g{ zizi!79rG0P-8DoFPjf2BCL&H$Xs7s|Pl0(}K3>7dd@Ag}gij^r1+gC__EpTT_?-a( z$12!BXYt-baAXg+{yI;Ti=gVxi`)DEQ87*d{B%YIiAZ1eJHlC#ef4%Dri9GlZz0Z! z6kRNyx{2A~_l zzqAs{m<1jZEiR09P!^V-!IH@z)of8`ZT=xyODQq0Tsf+4lLGf4%|dCOTvz+6ZDMf0 z>EyMHWhUFw+I@ zl@A?S%o7EirkhO@uMG2(ljYGnM?gP>OC(gxSs+=i_#Ge92WBv505$ahGAEyy-L2bo z@OZCEK-PL~<=xCYASivEeaJ5QiNjq3|0eN6E1qg7*VM*%dT_c+>VC|Atg=daG$Cw7I-yK1j1HOy<^cl$ z&5ii;6I#yoq&!pWUHxwspE(o-U<%&)7kTFV?P!_de_%)GDcN%7{P*KGCMRvS>fk*G*OZ2D*l0%8{e3~B)a8)vR!e>ZPqWF13O#!-4qS}Hb4UHFYg}x< z^UZci5h+M=g;V688S(Lwcn*0^&}@wn^>0_S@(i9I6r24dnV!&el?HCgOMvQTp*j4q z2QeM_rJO$=JjWq<^wOPYO-!YuMCb#OjxoOQNGZ~0@V|=hi|kY zX_INhfl(dUTYCpA&VqwGqySqm>Z=^AZc^E^wj8fRl1*7m6*vyWn%5P=p}n7+Q>zmC zu$yGscZWR{Q5%^hRiQSkGK`FkOe z#aK5x>8Q;#zEAag<60Y#grWvaoS>mDoQmbj~YkU%-?ZBj@YHmN0z55^hQ7_33e8Ktu9Jg zA9lF4Cy7?tN-RCgqm>Pl3QRcDU)$k}O*Y3tVQblk`DN|YP$3=#{iGl^^Ohj=j7hY4 zUG{)6Ir-Q<*(g~Q==v#Elt2St=*IBI2AvKowKJDxt7>A}Nl8=5uY&!x^_AZ4GnZS% z^shvJb9t)9HFA}oS^Z%%N6DEno381JV<@~Y{%i(KO0_H)TY3C$JEMW zL6V-roX#!%)&lk7x*+-ECEb^ODL68Ga{i!Y-nx8 zq`+q@NfKW3i!yx922e(m)UP-z_PnjK)zr@`t|Vsf)WXnt7&&329)}e1=xXNe=@3&} zVHqlTh}VLeV-MWZ6276V_4KnL`rWF{Ve*8a?{>zY?V<*qcjMDmJeSNGA`vAt&nveI zcKjtCX4iOesDX`A6=EagwF7B&V^R!L<*0CnpwDG+jb1hVXHk>(}b;L_Zu|ER#Wpm+1f2kg?siJgg z1P0!6WC$W2%YMdr{8(-kM_BKE`~#dc^Gz)P``%CvQKCu0y-Py@dyu4}esIIDYQgcJ z0vzBlq>?5+_qtO=U}Ke*R8dqx(jr`QOXh1S;@2pP3>hTnHkgjLppELO`Y-t_2BaM*Pyd16o%kpfx#QpZ1|!d?YR6^A+bE1ZGz&qtgDzo&rKEa=^Tq&K zR4xkNXE|fMVVaiWUl|wRnel~(WGXWEGC|o5cpaJ8Rd<0IFw4bDILU#3X?cTklAli> zr?s5P#b^c3?ZfQ%?|Xh}C1v2YOPVDse)as^X@%e9tiNWN{^^Zr0`(FL=R|QUQ)l}l zj@~ur3z0Fhq80*m)73$7aj$rT7&McwbHyOfT{~pzupYa9fksi2l64SC9MsJKJ{?!0 zeMy@Nv7~T+!G6ks{mo_>KF0Fyj0+eSw@iJaX|(c44omEr0_i3-s_DH>#|rTM{V`j{ ze%_3$DmQ!hDxskWqYdaNZU1d`s;uV= zI7pfe#($$qux=rVkh{Sf#5 zdDCLvGR49>x1E`T6qw<$R@{>V53v*PIBxTs;)!zW0Q zV;fat{iFuxlr1(DStlcXeSYG`f9f+X$@rodYF`>)7Lm@h-6IR*8m~@AFPE z|2~|*{;+drxo3N}-XVUy=N&dWRsFbLi#ph;LltP*N0`Lf0Wk}&STeY;tS*-+9~csp ziJY(2fU>yp*xtC3B(t9=o#vFc81MeT&zokomN~6wc~b{0LNaFgyXjtzp5>r;G)KcX zdKYx`IPBig!`y()FzYYA2<>j$Hx-U1$BAIm1ch&wmFD(SC0#t zF^LoM3`nY`9g<`}QUYH+I~D*_DMP6$^M)g=q_0OlCU(ZoMdZ%$QNBwPo)zM^wUA=U zLDYy(kkf8Ui?jJku`r-^CUlw8U`0|xsbZ9ankF)9M7m`XdK&L0A=br6d?~L=T!EgU zOK$t$Ld03Y_)P>{r6cL#SKD;F;yhe1WK!hmgK#b)7lE?An(0-7#?hNd_6=FQ%CC%c>i0gcz$J4#imp z%2XI&><-#_dRIdde+b-tKr5_5<+;DbyeFoA6eo_*PAYJgOn2_U7xU6%4_zKge1e*L zIwkV~$MU((Mi+PgtZIOgNC|B(&_DlwWSwkuGVF7KV_J zp*yYuO83wsol+uQf($Z4r<8QVNH_R9gZFyx=lffWF*u|ASrg$3nVpbE>-UofXx7~B$(qk~PtV$Z5GO9=v7hNu*~b>n+%62yT+v}o zsa>EZ(-ufYzRFbjLt}n`jdtWeql@f;@-nJhXNuRpk2q&>C0Rj^WpH)qG__=;@THZT zvzuz1t3p@Xt9d^m1o~t?5B1|Z;D~q@Z9YOj>v}Zy8Ye{B&-%mlI?IU(0NVc}6l3z4 zbK+^d&DEj8m~7{BaTT!TZ}sd%^KtU?XC#(@7s}{|;k4*>ewF;nb|^1WG%WsmKm-7MtmL zeHePlc|0AeVhTry(gliHp#_Q+5lUUIyoS=g=$v|yqE4eYJAKcw(A?CQgXW@{$l)?O zh5Z>X>LFjuW5sNP=JH#;4ZgZ3U`(8(-w}s&dBHGr%bO*$fJhSGceSjCRqQzZz-f45 z*XEm_`Vy_{FpIVsP7`CZ`{>(jL1f=wfv08bQ`%N0|93%Lm%)+}ULV$bof?}mi}owR zcd~k3FoqE;MPl-8m3&#MQDRDZur#^cE^!VxdBU13vOr#&vABdYWhI962uXT6xCCXR z5*No&TQwynsUkI;PcuU4112aqMJ?-PMu`NXPe@JABMEkJ5h5;I3Y-o;CF>Jex5h=# zQ=wCoU2hz2&zVEj(bcESPlCi(t;aQ2^+!Zt3dbC{Y_Nz}wxW^#0lAAb1%k z;(x6f=W*28Z!B5%w;p{`{c8_@`N?Nn-6kpJ|B`urulnafSe#)aMY=I;*Cog9U(S{= zjCQrkYg$+T!gOi7eB7KL$HN@{F}B0OQE~3C%+s3K2Z1eXho~1%#n^;}py#vF$O3kV zI~Bh#PD*Bx0uP6Yqd+j{MIZr<7~Eo0eGw;LdJ#SdF+`muAA{DO_qUme!V@!Iqv$MK zP%anDzC6qWYB5@cjCYd+W&5EzF}PPh;_3V_-{X(IW98 zxglk(05{11SS$9HK|HeGC!-<##)UxENE&dd?e>%!I(#dPV1eJf!{hW$|X zlbSVfZ>kV)!A**tU8M6luorToKeCSNhFy?)Twtcjo~WW-q`T-BM{lJTeq#~XNTwHy7cMR;uNc^Tzmdw|JK#QB#^Fd3m`@@gH>P9W31j(R zsmK_g4o`x?ElI+4C`QmETlv>oHtTkAt^y8(C>ApEf;#@h%8{DC+bBMljg8YHiMK3t zxf@k*cf{%On8`nV8h^)lE&yd;DAf8WAD<%9o_X^d7M?|qF?iH2ZF6GDNw8T z5e$_N1A9;(=cph~101K`_{kKEBi8>OOy2a(BDPW_gN^+C8$AB$x84y+N^Z1fd=~c2 z>uzsaj<6Ga=1b)|DY7F^al%zVI5+!SH2ddh*YU#Xq{)0rD||`*b}4ES+r7KL_-)hK z#_NIN$7+}B_ddREIJ}andqC~jTtNd==Ug8z**lkbhY$yUL*FG3=-uGTopdAdQv zrSetmTDD*ddu2*Grx=9|8{4~FU~1^~Uk?`JW4E}_vlNq#*0HPrm-1cFIB)0HqCRC8 z;dtagyzWqOeSN|WS6HABJD^T!O|Y4;Tzb#UY8}0qo>rK@bN*-fQ{rMTva+69&ho$0 zs*A$BFD`m)gs^CylJ472csRALvbX4~%qC0a24Hc*R<~nvBAjP2CDY%L%pbzwS5AeA z=vD#}E=vP9{0+KmR9U{4nn;${%Szq`8b$a{14K+~U{{o7|BH@KO)m3*JE)x9OB{`Y zy~on!g%9rItJhk+Kqx4Rn!-ACXLG^z35YmA9yz$id~!c%_s9?-iD;{ifg?JPVw|so z+JQW*IfLa*o4aL-1(LmE3sk#>rTRw>NgjpUHw{OY6KkyHP8dEqYEaLT?|J3C{#$Ya z?qZxD*!Gq(of{Y|wSU^pZ#1Ny{YA=2=>hn1EOC0}cHi@M2N0h%1s2BsO~@sb{a{QN zx~9+r;a$1=Hz9t-1f10Tb`uF~J4~dcs#ndHo;ED66tDv={hD{Q+Iq~k^wjvz{mEw% zNM^O=5TA)H_Y5m@TSN7^*eka!5W2^OuqgKz0RmlL7ZRJxZ%!vg|3z~^NP4zO^24>Y zI?v!V!^Tl@O_-#r2U92|fl&n7>I;OQQ`}Vgy!4~RI@pCcZ1lZ!g`CH^iq|ROb&gGA zU)*m_W`#K70=>4rg)TXim9>z~Mq|?bI3Zr}pQAc~5gV<*x?|(F_BUV0|NgDsH>vYC zPq>tl_ol-`?U1i{ir$y!uD~%b;P+RjvfqiI@TN)CsCjZ0=?8EZuIIts*yOD5+6>wHpT7~1r@z)$P)#S2id$}P$YW?MwfPhgN@A-!rZlR> zZ?ki?cAewu2TZa;I8$#<0AcG3_mK_DG_ZH1`h8xY6pXJPbhuNc3igZ$V|}JSW^`7U zP8CraHqv6;`NblcGM12wnv;d)rcY2ZRGEcpCz$E#6QCv!xw(HS-TxF67nfho{`-d7 z{9*H%X~28g`k5}REXO}#j*SIdG(~Q|0j}s|X2ojPf$5x_EVV)^5^nz^m$PwU8&;bx zo#h+5Ft!vMHHk4}$=;4MzzH&c{*_66{b8tJvdK+jb`I_3R!%&!lA+JO_vSu59sj&V zFjN6ig7KV1*KA}~-CI~|Y-xzzxrTR~KA}#@=_n4Lhx*G)>pNAgP}>pLO*xWs6ZzO)KU!$xI>L|9vyBu-%pjP;D;(N4V=bT6 zihM*;YP%{jx*BR5y6E2Bsb^^g3Au4Ku2%gA!HG9Z@qa$E2FdhH_D>=%k4Qm|(LvN``YoM7`3@FAvKR=i&9*Z7ny-v(* zG9T@75+J8s)O60X(3m$Z@YC^Q(>(bn2vlrJi^zT7B(&Et(k!*KoEJHzGzS#&`OyGk zX@hD3p5ud5zY8X4yG=l>Vw;pjthmZj5`N`;*E>h>a+jzf+yWMOauRbw6O16RxXkL> za!$yg?d#+fEx!reNdCJk<^MJ;7!%`Lma~qIi1BF0W`{Xg-_Tuv(`{6jlTK+%+x?z= z3B(b~k!&{#ak5YzT6uhf+p&fh+uiml{_~Aj$3!uM^x2KeT9opX{H^ytq#h$l&+2yu z?Zf9De2(P~-6@l6b(F7ABh3n@Q^myWQ@t)y_J^D`+@Rs$% zgMIE;>lO63{d@0lvGD*P$&TX@TIl7l+iK$HQ0UPRj75m!SJ9TzuBt7~2zfSZ%)nyU zlB90UCj=~RRP%d<@x0gwQNK-iLVN8|%3$NBcGK6r&Zq zwy?V6bU`Xx`1nTU-?hX)XA9NzxW@WuA$JvPNr?KlF zCkT>=Q-ik%Auusu=i$vIlx^i-Wp&6`HYLF{WX1Ki6{#NC^SP5})ilIEEMi!Xp(>Eh z1)cFf=ih(7=~Om`>_5=|>whvmHX`hX^<D z8VgMQ{fzGKW$vpW{g}i$UuE$=(XCOVfxcY`EQ{vfs+rcCIv#efI1{Cv?%CO&rme^9~IKAyo)K4&$Z8%?B_rj1F7bg`F|GZpEu6X%Z{Bk zpzY?@{(tjFV;l(XtU^=yAw#Phf4+5omEBmmlTzh4CJS^@!II)4eenFTgcVM9J>gw` zQj8)POcH8fdv`VR)`K|$dA|yLR{G`2@lx4^)>3)U2NwDg-|C{CGyaBK0s2>kPrF|7 z_M~Qqs=D9yJTl{)E2Xm&eON(nrA~3Mt?KQX@;`6<_mUv+$Li_P!n)TI_3Chxk+YuOSSwgeORr{XkXgJ+LY2zKla^8GU zIZeFhi^TkEoC?AU328)h%nEiL{H^9zbvys(K)Kjz6uA8!yhQu|q(gr;Td581?Df{l z<8DdVI`6f&bK8|ec7R-Jdn@uK4G+t4+ota7Qf(iHmlq3V_H~yPN$!ZpwOpZ`YWo<^ zlUm*fB_`Yq{jztL(Q31W(OG4f?9g9|&hI50;Vp-Tdhy<40XS!!$WI(H`rtyJiOqC4hqLa zIg&!rnEvqQBHXO$AhV5xErVc#Pw|sZJ_vK>6RWN}9TR;yVE=qt%eLc>>t+>IpN}59 z@bVP2bWqGiw_l(u{=BV=Z{E;0zwmlZiq!*LGqVOiJ+!wlX51C{@l6q6XAh21ew=@} z!0D^&wW46el6Eq&L>}QWm>JyTv;WBy+jr)=5mUGLuhZGLB+5)mCtrwsO2309@nA5? zTYUVU%3Yn%+#lA(H93bNO@|?~0i0eYTctswHjYWqaCH^Y&($LhXiX6?^O_3A+Xi<) zGPfE1+A|uFze`fqyo=OsEI-@~5>GKK{VF&i{IlK>db-t4f$VGOl8-+&`#;HBm9o{d6%~b6j;%S2-}iHP6MLBTor2!ZW{Ob$rp~#ItY;c&RB(Ro}mf(>3Nn@Ds!?$lmS*Njd~2J8SfT5R-n)lni>c-$Ff zUG^>qL>%!tc?Kw1+U2OUWub@is}bis2W(cg>+rST6IlLL==`M~G&{~+yZ(c1yL-H} zw=3Kk#`Wy6=qoKue9UAT=E{@^PLc2Y2}GK z0Y2=~>iKAeXa~5TKgd!|O7ajj=&P@;BY02NN%WirfaQlTCw$*=Ktt zz6QOQXB*G=N+>WmmEgJ`d&O;%V5ak6V?E8GEAZcB^~yYf1Fv0(B#|gDv%EBYD+asM zhVk=e4ZeH%aNUw*B{SOeRp@{2XMSJQd|<#fT;CEIsF?BBs8j?mDRg(PCE0F?Czf$R zH2e{H(}Zjql%mg`W6LIdK?KG_d&`&N^03fwKR$?sVA4Ln^Z@8>W-D3_FApooEEY6;NU4XGtzB>evm7LpH2 z`*ZOOUB*u^XeN>6CqyhaL^hkhMg>*A2BX2P1jI?Vx!4*-PDXn;_NRP*Wvw3+S%|VS zpKJi>rL7RSrDp-g;<RiNF#!hEJm zE+5Ktqg}Oi=o0%Gz)zX*R{NTaJ@LR;A7@LCd|2hKbq-5P4Lf;)h>| zf~)Em*z>8XRq5uR#2TFcYmG$*y!IGK4z8Qru#>yJ4d6<_WLW+-|0jy}vf^yDQTMnh z-EJrfEx&kEB7iJF*wsu~shVEW`4N^axn>lzv2a*ULPP#$`%1815wp`CPQsjox6W9r z#~YU`_gv4RKjNA9+DaMRf1>~?d9V@A%qMc&xtn5l5C?h7jBQO~l|!~X{<`>_pfmD*Kf}2U#9zsYi^?l>@$wrmR%4porzpisI@!O7R zV3;P#bxXvmpLMnYMT%{~l%fSwi`IZ7s)`(Ag&!!!T9c2SNY*G}&CtYbhCG0HN)8YR4m*9?q{Wf-?{bpPRR zfZLyYPB75g-_Nj^Z&KUvnRGJiHECZbRuzI6&v^dDVDB3El139#N*Wi^^*e|tw&!YC z!S43x7JF)InWx?NU6}P7Je-f6xPxg`%ZLPi=Pk{+c5;56_pK|(hKO9XO^?|19zA}$ zCBkxIRZo+ODg!B%ElICe%iZ$3ZPjG(lE{)!+tjbKDcF8=&&124Q+D`|g_v5`+JV$)y4H&z3~eJzCc~0z zz2T+zy1`j3*mNr`Tsu)lY&)t=jxkk3N(!VH=))ju31c~7^ew#X*wP% z32r#};py1md)n@sLLSWotCdB~ii9X+0p^$Qi~3oz64{~tK=6m~doKq0ft)Uj7jkAi zA`yGk2+W)07OBo!V}x8}4s%2h(;t8cL5|t=1B|{i#J|c`VOaRU4e_66kNMu10cUGr zL=i8b(<2e|c#1Y{o#r3Y;SXHrZQ(-FK$w1X2VUL0^7LH)-GaMbuzR&Ln|Xfl90l32 zAk92XZvniC4FcK6zq1wpWT0N;Qk~JA6s-qk2pv$;A$>B3i@ZZIiLeAN?*Yu0No~hm zbgCDvimse9_0*u~2R(uR`HKQ&{q`cr7-~egcH2r5x;R`hNUY#r$0fgN10K-IyRIYJ zmAzc+=cqzJRC5F$&n4Qcxrg(`H!O|I6M|d^4;GJ9?lp zJHl&{;Xj9wzYBiRF0{M>T(!-n{`9gaoANoC%s606_`rM67vaOZk*sR+^dA-g&LN;^ zi7>ZQxqLTSfn46kCUTST&!Aoe2r3KcLYy}F6)LCaFDy+C8dL)V9>n4wz2&^-b8%h+ z)8s}QKvNv2h{e>*m9%9^iB0N>tb%-?gx=#zzD%4Dk9wMHc>?U6ozglEIWYEpaiE~ zbZoh&LQ2VPbTT#){g>JH|9=Q6ejvjUk;x#4M^WZb2z=z7J_snd(UC)XKa1eojyXE# zRSN_)i{HwTWYW$c))jJ#=1h_~Li{lO-2*M#16_H}qRdjm2M~#|-vK$~sozN=LYQuj z{os4!ZS<9zavGn%HwX~MZbU@glP*N*3+@owaAnC1kUj+Dl%BDlob6sWiDK6UY1sivjwC{%gmdib(I;$~y4*GY$-;P`YJ$zNlsY*HNscM(aQomdr z7=cgyJgqhMTa|RC2X*1 z%!eq8Guy2`q6UlEKm(tng*F#po_7zM0nwCF7vX<}I$>5f>_;9ZCgQn|BpWDDA z6^+YpBQ7@Xu7C5**fYvUju3IuUrCMHyEAUpgn#~@R>qqn+{g*%R$A3~t+xfU41_eA zDyii;T{I{y_{ID_aO!-J!!cmRDo>o~_j+TjwIF`ONe`l^&f$&*32&}%^ES`@*EGVH zx1YYDxahV1^UE>cl!ESD>FAGYs}FyK)Q31S*t4Nr0-_(#+`=Q2V}?p#N!90t$~trk{Am3PC98i4|lpF{JaW?N>(CcmtDBQo|ccoyl|yMqTWjUrxt+ISh2ulR*1fRJ3M zbbzuupCN4QsVDSdX0jBASQqLq;t*%Gzf_Z~PPmJS^cjojvs9i=d}kmf`Mf-=l{z;z z?*?+iHZU>^`O;J4a6oVb_Jilgi_sEJnbwUzTMMY=H;u!YNG7j+vVlvC9vhCEKo$lD z$~PN>KKG$d^I+WXtZIX4v{h^ddM8DUem4C&Z^Q5dXEBtP)f>T9OK@KwXbV20Q-8x; zgm|OiN>)rThEhc`bbvXC+b|^e4YV4Y`29`(Z-HBwxExQFfL zzb_X9kBu?Ay*@oR)#b%13j4~v=k=iS;xq&&@1oq|>{61>OuIm7&8BPi|Ew*{@`}aH zY7UCM1#ijlO(b`S-+p#A)!g8zyyI9|iDDpIgUhN{@qjGzkF63lwrh`G6u#G&4H6HO z5^!!iggUDStC4@|`CG#QjRkY5=enE$O=2u33w6gc7tQDmX-l+7)4k`pUbbTVNrBpz1F=nT{<*~a z;~}Q1#1vQZbvPNau2prDAHSO?+eU9w^)z23Cm_~WRByIi55Z)f_#^oIP0=-qYieCPP|Gb(T|p)XM9tl~asK_S+6XX*`Re zT)p}U&Nl5+fU$IiVqzu6wGGl#ZBin)=1XcG!}NLZb9Hk0K$Z|b<(4ME4a})uH7BFhZnQ1o8MtgkXhs{BX`g7LywWpq%EM3apjqUp;SgwuVk3r4wp&FR? z)l;*QbJI(-uu_(Y?JwxXzt6zm|CZ9?>f{-hJ&C^Z`e;G=mQiJ187<{R*ex8k!Hjyu z16)ki%zPeaW{^Zn`N1n_;W+XoHIGk$n;PxQh}f{;x3P@pUtfPOy`ODL93myk7Gigy z+F@ZM7@3dcR$j+*7Z4VNLf8;Xm0`8|e})D8R@R$mZ@ycL>r)~EJxeleZBBinEIoPE z49JAaQxT)YCbm%ir45GEE;HQov#DPVf-8v017(q|uQ(9C*ntXk_t=bM+D^*Lt|UIW z_dnxGydW5u6P=W}OX-R4ub;ojRJ>Q!Lx_Af-?U+9qqA}<9%^b2M9y{xV$>!uxejSy z{|`VTcwP_C$|}OwGfvw%yd%Iu3nubjmXw=`c)Qep%@@eUv0BONHmLd$880Tqc3vTteta5)5&iM4J3FPa)lIJy_<(#xHMI>iMHaaun5nnk$z)xy+u7Jr|j~-6IxzKwq!8@lyDQ@x+n;f@YNK+T8~{X@f-$5JE76G5<7)ozaz8;L3>j0s z3HgGlRg|35o!;Fisw3_vAHJ&UXWDg3KzK2|x4qaWlfX779CJ8ZO-9!ez7ZXp6O zY;Yf=zgZoKoqsr0`L(Me2MAFFGhbe;06BCd3+r%Eph;-YKNTAzxzR0%t@favu54j& z#33+%|ITk%Cuf349=D<2m!6LJo{h{VPo0_{PjIi%0>a!c^-{-aOfCEM9TgtrA|H2i zQBJ8!|L8c1{rpYAeEpN^^L>2#KIzu$Sc&O%D?!J!y2DkfCc&tZ_aO5TVLNb5WEiHJ zbieyj-$@{|9@1q-)0n8O@6^9?@41sUlQjtR&NNRt80@)# zXwYY|r7um-GkIu7UQq${*FHOTh-7q8kkjfDHC z9S=-=sCi+|`hHCey=nnz8g+a82fc;4K>$#k%UUwSa*``M@>P(YbquMP$N!+|*BRb5 zM=!Kiuw{=o+EKvuJ=MO?@c^7bN)TuC^Nj^D(&2EC?QgTXJsw(H@m2NoADJ@-5q7TY zAA-uM)g?c$2fsY z`%t8BmZU#g*|kLxgoWq*>-ZjdfVP^O2}?%sjOzWuTAUmrA#)OW6Xnhe{xQ z*v#k5MOFq*{nkd1J|Ql{HUCQflJ{>e1vo54H_n@E!`g-j?`k3{pqkEWqp@xmf7vpt zKSq{X*{NF(44P4I043p!`D7G$0|MT+4*%vY>7}?YCd#D~(N|$)ua!r&oF084*JgV# zcL0ZT5h`^!Y0ksCc*uYn6Px+Ol*zq!GGB-w5-yj&mYz(Jzz@G(6v;1C1UD)M`g0xM z7uF3|^@yEhRhA;Yp?xlN2}T#)ObL@M@FLkkX&Xd_ku;)PXdtG%c7$c>u!@T%oNP68_aj1&K8Gf521XQ5X{RpYO` znw@JXS#hbZ2N&Mwosx~QCz~kt1*TK}5J^AZjB*Kjh=f?xm!}lhjSMIeaQ)knlh~@y z*IO*TQnERU2@QlDP!XYaZ3>8L-LdI<4EqN3G3hE+l^czJHE8^|{+&>iu;y2l7GMT( z8gw5Q@cZJr>nPaqw7K6?V-TQRX$A~4KLcY|TFDsbbaAwAlIxe1sBvDxjIjl&4125i^( z=q-0cxsvf%DU;qOilQszx4FVo>>eXY4B5VB(^|t==vf zPOm&*9zk>LEV&n+YR=^I{Mt>Es1HDM$bL7O>H{J%AYcm<%`~7qK*kB(hvS zR>YFWj#cqoFkZzl#*yt7b{7RI_9&t1<@<-JVPMz+^clE>KXtMdq|o&;eL_%25V#iU z1Cukz4emK>bb-wftQ1Eu9mw^m1p9g?soOVtS~Q`%uZ)=e9j{SVYzcbz~%lGdoRc}hcvWWfyzu~L?D9mMr`;s*_ZTEs43$Ge3= z8!W5u_FczGs@tOFTo7xSXV9;tAar?nS!$$C^=O5({}{U9pOu>N2ZZndNvis z&T@hab}giai*{x19JRT)38tl`98%(RBVI-XOI8~DF?F09ju$YZ3n?<`VY6dk8s9wT zPfJ>ePvI-7`B6>xcYO<78^h77Mb`(Ft^^cRXwK|B7?JkW8JK_cW!*)`oTKq@wnFdz z3bIp#6iZtt^g?{Z)K~t8y}~{5&`*OX`my`2v6?g7N{MxBG$#CK49#U&qm# ziE0M4ZI+A7%`$eWTi_@sVwUS9$@Fn2JLmV?S4cUU5P@jDl{+V(*N!gOt23K!0a0?; zc+k3WR}8zmZm3Y35k-lx;3{Pzbmk6Jy>jiHPPZrdSF<24`1&eA?-?Zic=l8BM09Dm z-D%$~#Yh z5{rJk{EtIH=OvEwPn7e3e>saSFwbsQ^--T&9!*~nciSlraGydu!0PD4iu!iHa82p0 zNJ>szzIclMKDR9A%!LYax9BzEl20|Enlqa2(x2O)t&Gh_eo^bz5!-lzW4dHoWW^#% zh$YBu6*u7JOp66Bj70)Ul#QjglV=?Ozu?g)>gRJ3t(`RF>b)xd$cZbcazuJLp8Gjq zn0DPCvtf0929Y2Qcz)T;Rl?}erJ7&7$oVRvOm_4S7@m&j|A8^*h8)E1wtAO9;_8~5Rt4(F(l z!}j8wyTuYK3?(ylzhbCj7(v1xj9~mrR(oSwTE{%!=m1fESirTd*+y(<3OZGs>}2D! z6J5XO;UU~6k)l=}#}qq@N2aIsy4B}L#FV+!xo&c)0z1}a*3{R4?NQC`z3?tK%{!lL zD*34iWsAw5C+qZH=T6pnKwcV@d_7RYz6iec%4m)C7zX_P%kT z^{wZDOx%F>jlY}T{;nMzh;_DFRvoD94~8!!{AqKX7rQk=Sq2oGa%OutMMxF5RNgEA znXklO7KyQ`YgAomU(>Rq%TW@CWa^j?5MVQ}ye~ZcR?hW`9)wng`yV`+57e0th#6cp z)|0dJbNPTPtJyidXWOn}Z&;7%k_+^wsh7+azJlI!uKsR6!%ue>M~4U9GnejXP23qT z%<7lh52Ho;sO`3fl^ekq1=aQkihcAWO8@CX2KSKmP00wgpIJxdwuOL*b`fBk%Son% zxO~a)CBaP-s)WRgo#*BpJ)PRONjc>1*$t)JcNDk&s%6*&O{(uyY`oM%@P&OIN1^< zWP{~;$Wf-S%H1=}YrmgD5*ng~wOjL#x9bu^^V`fYxhq6?9$RsBTT3RIR%BwvU+rq~ zC;q}(5|`d4$A59Hcdyu5?ATF2>04`tIAZ66;yun0cU3Bylp!)htea_i+oKpHQMJXO z>biA-7!30`()>u@Jik0oU1l|qv%B&`m$V!);-P*G&3(*UAVyM@iv5v52bM8lW%vjs zggTS!;fp?cM7(F;_)08b{+?DX(-Ksp+Ppe#uG?p~?RSdz;TLW-JhXh`&CB{zNHo{) z#NOY8xVPTGy3bZFFK>M!^~AJR70PYkG}0B5+bYhR>B+l+V81!BhOZ7_)QMCSJV!uV?(fCIFCV8oQaDlozo zb7r|ydN>`Sb|nzj3XvzP=VI`#$O67HffX*zE%!sE{IfbHTtN$=T*`x}J{6sckbaAU zR?LKmx`6bnOqi7X*%SSP$jTFTh{Of~_`w*&6m)9!ddWLaj~snF>f!M1v{#5N)hfe_u|1p!OI1IW!>zi;WHH z8ZgpyZJ-v8yZZGcXX0x|DoRB8TtE7CG#C2kMXtrgiDO6)CKUYfO(;C`3(CqNz2;6E zuEp7LH3U6DGnau$%z9;TUc?&r(Y|PDKl;qoXV&C2$){_ssk>T7w*uNmO&`~MUXET; z7wHEc`^OXoq$e2}Q)cwSpFy$u{OSj%D$w#`mN>Wa=!*gKB{iabjCTgv=EXp14QWnK zdD3ddxD%v5s^S=8$PBRA}FAsgR@(Q zA1Czvk)sJ3&fisD=Hv^KmOv>9$0sjR>k3;)sUc44;h_^XYzig3F^IVA2$O2tMHSkn zlKu=40$EPRpZ|DyB}k zSIWIXXa~4-jS74bK^13c#X4eWkhw%T`IDZHCemnyH2mv@u|Yx)ciVzBtnC`j5~D#{ zPKc14o``-iXvS^2`n4u<@foN|*k7}pRV?g>r0EUdVNY^8{|q@_&SCH4bk!8xfRrVeIUIBriE5n}* zE69C);?>`YCGeTtGX9HiLxHz8D}afPb6}g|m8J=|w@P2}j-_dVtIoEIF3Y z%!})!q!Lrb7JV_UMMnP<$37F6p`koRU|!kufKY&~N0OUI;_=Cibp zc6gXBFhvSDp`J&Nz6kp2rcqWx2={?R$o9bB?RAcC=4mi$fLP2{2MLnGWk0NgDky)` z>u?2lyK66@_HP?!X%-k1{jpT`%DBR;Ge%9P9J5cuU?ViG++Da$Um81}#8dSelHeB0RAAocu zs&FaC57{{5wKH5ptPZN-VbuW!=DNfUkG2nsytIKIw zLXtkeG8QXa#@6B#`2AP#zn&QSG$jwJ=>9(MoN%DwTm_u+0&;f{L#pQbBJOKhw>4#` zwTB{fdMd1vI39wF0-xi$eytgfk^l5Q7>nU1q;zlNgvmY73JHPB3T&_jB0xp1fv!GX%gG9#p^5}oh8;%GF+joNvx zOOx^Re*4Yjo|Ac*MRYk2Rb>M1wvctoi1I3Wwy`=v=1T()P!KH@WiYL|A|eMp(c$Gb z7*Nsd8E2EqEa&yRFpFiy2_NvRw!qI~1`jif<`{bNgq%Yo&>E`yk2u5_Xkh#Z8CNz* zw20Wxo-7|EJ1Qn9|GW|Tc*FLA5(&hqEz&cW4680G&l4eT^RuG)wWcDd?_Xt1AS=I<~X3^}@J zdJ%A<#N|Y6RnO+K5Qz!OcE~brf5?O8&s8l_>DVt5AOCFpw^_S6gQWN6bs~{+ZnoMT zx3({Ng;y9v{dfe1P#>2W41L;I72PS$5^8-vk@O0Y^knn6YOE$VXu&`=6aFdOImX%A z#!Ua-5;et?Ssi9n9S(M}uFGueOVH8()R@ANc=daQm3n!F6dYg6liYu+_t>vRQ)6H9 zs3gQO{RU$Ja4KWt@L;g{5RNYP*2ZBm6iMrd+in-%?Qt z%Fs`5q3i|sYSf#5`d-cSYU^MZSBqy;sxiI04)Q2UYJW27{r@pF96zDuKrE*d=G)7? zV6DpPCl2zol(t=j{LB)zSvEN4eO!bsF(?ap4@ocql} z!?Aq{VWDU*#S)S!jb#0B{N49hb->Q{!($C}y`Y$4v1_Eq`1M`2F54t;$1gbQ@8KKC zg-g;9I9@rYq=kqxpYEmG+O;~6^tuVId{V#7C;HGztjis)m#umTFl1EMP)q!We7|lp z#%XzLC&q)qeh4tLR*xC7i%mPqx!U47%6H#l03nLf#?hV`X#b1E6Esy$S}(tus3{6A zysP{;@igfRr$Un4WKwIvM8RoV>*`e&UOZ~+N<(a zBo_}zCw=J(XHBd7T@=22P6T`lwafY#q8;%Ksb+WPOXUw~>n`;(Oa-k{HHna!s*L7( z?KtghD~v`^oe6f!(yX~t5sr+5F|gPPs>rwcbxFdJT8>vbD1-Dy!nS9$t7j9Y=HkPA zR==Hr&kZz&cT4rf*UdED6iEh{MKQVN2X~y>suk)4h1nghBh!1pO1^v@ds5vpVnS8< zj`EMk?y{NI^3kPkT^a%HyqViyPeIv%skx=-nkS~N`GvTdO)~xou4*_Qy)ksEUa}h1 zE$?~&QZ(9$zm@)jX$Q2#y5jK5+s7|jGHI$>Dwb8N!qan6LW5gG4Lx>E_?;nGa|(LM zx1muycR1;to7i?(ZwPf97Svth0^|{Kr=6yRf$Q(&Cfp z<3N=U>-Fk796OHO5ip~bK$Uh;z8|0pe*QZ&reW39gHF>fd=MU~MO4F8fEhtfZ@&Y-@Ljq#~Eq zX(r=Ar@MQ*yVi(hRAAegC4Pht`?qjh)3EhVi9<}%~ z-au;`DtbA9u6=Mb4qKIH?o)#oi`(E0Eu?VayvS#sX?EroL-?kY|5bsaV`IuG;Lc4I zwheXlUM?Ij)xmWtnSX)&nGP~UnQo1@3lG%LH8fnVJfcUxb3b<1LIsSzgv!v1c)Rs*6|3eIV3ciuUu z;fWfs#{O|j-JHJtD&X$d3{k=3%|9V!LXlQXcGaJYCvn=)zd27m6>E& zhysi{&q)ldCpCzNw(jOjA#Dn|+s}nhbR^u2J9~ZzXLq!c88N7TArqH}!(bsamCRUtxXMP)I7s+MssT(M3d|k_Z7CbS$|g7>V!V^cTJgT zt3hfRyi(whQ~2#PBuK@YGo0M2P&VRo4dKL@@20mh)o~ihM_H{BGpS#GCPYMESxML~ zsD0T?Tf*gv&33(YA z>h62LZn=)~N-IUP`mgMF&*npX`lox~*Z0oJ8~a_LI=4& z?hn25W^v%|$QV6#*w&WV&B?DazD}?!E?WF5 z!Yz!`T_+J<9!rg3IyG;C>1UkmucUMrhyT(x@(}N0ytP81#Tg)dwDOzD%l6pcO*|pH zq*IvhAXYEh-3&dPqV8^qgc4?Pu(GM`(K_{9)zGPIv;S=ZnFx$6&vGsGJlBPufy*9Zm(4TPk`uimY zyCc+mb!pDnyu;s_sBs={HB2A_>pL$N!_7IxRNgtHeLc7-lFL-2a@C|>hzngI-T(FJ z;74DK2ISn=z5AXMj-(o{Kb|w$C(0-Gu3!9axA|dz^OI(WOY|ruJ_D2Hf=NO^ANR1+y|3G6^*y9E+0pS znCaTu$h~^R21C|a4p!vr%RnFSgkF2oN2q;75te;&0k??q8&0R>57ry0>l^@B7}DID zdN!Nb$0zaed*%zsxDHTVRqdr?3D{n;ZU99EL}Jx^G6$}zT-W1_TzcU@_drUV`FyBj zjd${4v(7;M(>1Ugph*#=6lY%nPh1vhK}wd#SWY-kHTnM%rpyu$bTyWNiKyo4f| z*~QB=i`4WD{SQEI2|mt?OU_}2GRh|@uQH4{OucZ9j=$W?0XMa!B6g?bHfic3HmcH9 z2&ittY~)4tb#WSX5gSh$Ac!a~11@PE?(AU%T59^sQAAXzU4Yyo^jZW@ZCoRp&2xQS zkAR~VFV{0!ZCv<3<(BCeSIk;4j zNo?FL^94swrsiq8!ZrUlr}rfP76|pOT85dTQ@(P!I?|BS^gpXAUW(&5-YJzEcU(kvH2V{NCgZF;uR@69&>Qpygi9haj_Z~;x#<2_N% zEk<;=fq#k1f8|!Mx1>HJ^wAUx8NAJ8K28!0(Ix&$3_-vUp334n>BQ}; zOWk5qt0J-X^8bWkl^j1y59eO4#4lCc0?axn+_)h!;2~#$_LiTPGWsju&ZRLn@>eYtIL@ z7&qll<|~70Wh^yQP<~~pf@);4$t`NFL%Yl=pj9AuX2~qm`7#r?vkupaGn~rAnK&$w zj@2VF-JQzFzZJXg)_UWO2H4!Cwn&@iFVem+vyP^FZ+wcB zU7JDaeD}fUI@7e!J%^U{A`4Tga6Wo3y$Ai=3a71RP44+pzM-i?XNyHKg4?%7L=u#G z-wg?X(du^n7`SqYfH|)G+znZU$`+>;0pSVkUgl2tyIAUuf>GxUYLhL9%VU29MQ~!P zcbV(MBVOzssBc;%SX$qS8W-Oe%DOKKaaKqZe|kYK#7P@jFFc=TFy*TLDuZpVZg$1_ zZ~v|n`-Q3r1sUVzq5}b>aCuV(^V88VleB&p<9l2DG`* zq7`g(vdu--ad%eD>k??rM~z5zNL4@(Imu{e%w#VMB8{?pIE^X1mh&mn4w<-_*1OAy^<4=}bYp(8IH)pcC3ODxl!nUBc!&Yd#2pVBZU> zJ(UcJRoOA*<9+T@&6+k=F3h#*>R4kZZfdBNHBe;y4;rU}ECQi8eOZ55t#S0#IgK{nQ0 zO03O%bEVf6tP{@=1%hxYM&-=9n#wb#Wa0E<#++lL+U+bp$$XG#J9RYJeJzCDjzh^F zP2{_M#V%Cgl?=X=eI!V|Fq^*TJ!=@&qs~I}bLeiiLjlgbjLQl$?@-Pl2>pkf^fp^llAE9gtcbB~18%*0t5;;miB+ z$n2_R?>*s4dk0amFMx{{@q%-glKZgH2-C?Q;rrDq;tLB$D0|Fufs1*oggZLd5+yG^ zrKzv9Ys2|OR%+AUK-R@Np;_#3yn&o40KeO4MaR28CAzwjvO=g&eQpC$cUokF5^lKG zObP*2;h)PaFxV1RX-)j7JrE_N`dU>R>orfV!~Cg~zk61_F{|x-{HM#{*M1?4$!3$P z_-j^fV_y|C8_P}Cp&QU{=K!r#w=sdlqVk&`xeV;sY;c^=FYQOh&eysx)oni{4qP48 zs5~33sAef!f_0IGf-p#R+_kp%kyIw4#w4LqEs1iA`IFoG;4h6c=D3C2Tr>?udA`rf zSxUkBd$Vd$3OuI1+Qiui=hK_SY^QjydKE!u@iw^@eQu{}|rI*C8a;6Y1v{;GAX*SD#DlKwzWHl#=5M$g@R zgOx@`0sjrfXw&3qbAt@S%T_$PahqLGPejlSkMD(urg5+*MNkpL7RYP8bhBQh7_aC? ztA1C?mU229FkfDtEs7Pw@b?)Umi=&H|GV*Aem!&gZwVcG>-!_d4qb|*mvNqUx2sD7 zW2F?AmSe2VnwDY}=A!IWh{7eaq&S@v?OfPtNYQlBP&!5_fWXccUf zzQOy{Ww?5e`j69)#|U%a8@qP7d{VQd&b5SEy4xXAw4R%i(wtO>QQispr}zvuSc*Pv z6@>YoOJ<^9RlgN3Gb>OS33?eX$DC`XoGh$dG})nK0$rH(hpIe{g9tajgJdSlG;M_M)VhLd=5mB=>pOWBXD~Y+`k{8uxok}Nw#svG-)F!y zBy`)m85rS81lRea1@ON;B6}sBL7DQ-sue3$0DSrFZZI60&zENVatazQbVF27-p-6Q z(mgqZ{nJzTiYj~XaRG_8Npw}~=ON#)3Js!X!YGi?j{avm_i-ILhuR_^(b>+MbN7UZM^Ydj&X~Z34!nfYdro8AEbDy-}EvQ8Lbda zkaZe25q{O0>FTMIl8Bugnh+;6MQf z!BPfKp9D+v^HrC=oc-XXXWGBzTYA7NWrO*ETzA` zd-l!q2ufMwyaF`?smN?#VphLD!dFBGsqnd~AhGrYsyw$R*Eq>NTYA=n#!+yA!Oq)s z8=7*^awZQ$4@o+)kzjdgSJXF8P*xmx(4N4PC94Tc>m(I=4_HH44Ub=ZRgKE6t+>rGdby{>(F^cZZ3bcC1=YUkkBUSEeB^YqMnA<7BTS9``0@u;AA|*_h+HNk`W+8}tRTnzZ zUo2XBgFPT2f%u%4#yTX~>eE!p(sfH=gp+nWf%O=NZV2XT#FC2INzbRY;S{?3y%)n?!}9!pS3w zw#@epIXrJWZWO&cSDYiG&llSxCz-lR;Ae-0xDIoQOuBe1wUbg=b?(-RE-_6sFw1Ak z733#N*K|_ARCm+)Vh}f>!obqh$f(gZGvB67IF#fLWE6=_ZH75V->IAQ&OT)Dqs35c ztU;V3T0C;Cr*tc&3aTuU_TyR6nSA-)ZSJxrkXTGy#P>{v?I)6MDX;&Wcjyu!%@d@I z%MlU3S)`7h>;2i(Chq1?rC7s6?bdxLc?;;o+cGAs%oCz2nc(pakjo*a$^3Pm#ysi= zYuh>l+i(v84I+l=UpAaGM?_}@BvO}o#5cDw>F8BA0O66Y$o;Mn#(|a-mLhC2*?dHf*)q!wX`8l zo{yRVzlZgZ%!YS{Z#bml=R<3AYOfFYs3DH1S&DB&r}`^QesjW!G~0yMT8oa$?Zz+=BIC#wp=jHF7lx_B6JJ(zL#z;j)9ilR5v+m{r6-l z)_)dMCku8;`u+>4epA%5%stk|jajv+i11|NxSv)jhYho@$_g7XV!4DYO}*08&wWqG zVD7bJFZWi-`mtRJWMNXp3$iLrS$5|YCGmaed_#$!NwT) z&eK3w{mm$KNV$zbV`Ac{1=Z=ZJE;8CK1rIIoqB;R)nNT%S1oI%sA9MREQLRS%C1Bc z#~L|!77!-hD2!ApuAk(byupsq5c-W2y~@DB_TB7$R{UA`eM9ra>NVNs&Oa(36BkVVZ;XYJiKnSLc1-s>TS83z*fqav zlCR4ei80&EST!Rk(__eD(nobg>v3I!q%+c8{WwcEon}jU3c3Ie$4nS4MF`p<8mZ5) z@X$vlrF0%)H~rdY4uq_(W?b&I6qqeFIS-xhqYYT^VB^S(T4&83v|X5mk!QV#QHa6G zCH6*(>4kWd!z)F?C@io!cuNN!M})C;laTRG?TyG2MlVNS(7MhjgL8m97z663u`gpv z&`8_4(Zc$iF0mO>(%^n)r!_@J~h-F2Lw@>G=jz1B``-5b+Cp}L#c{Kj`(7%SFZ64X&-OB*h@E|7ND5hz9* zkGs0OYO_+FApgwm@$|vmWRolSnN6`>2M6G<4N%bx$1Bmd8_V|y%3!aHdL+`%-+X!Ig{G0vYr=&^N=1*=wjEy!eNa+iigYS5h5dZwPuX7XFiAOdb|UQVV4;ZAl_8-g+`jJS9>Q6l^i&RWtdxj8}%-J|?-6J;nsIrQFeS?^AdAQf_4(B+hvX>=bkyhGFJX{eT z0^%=ef4dwT%zjVKTpop`ywBV{5878^>WugXC{Er~E!VAH-!QB(ds532$46gYWcR(w zd`!zy`sgui_r0;bLf49{ncc>%K$_5cC&O;fv17WqU?grK^UXGB+A{}S&360i$OqkN%-?%N-bE!DUpGdF>Z6^B2VJR zR)#ETh4hp%E@4RMeUSb(x=VMJn*jZaT-+XEVwV^>He^g!B!HkTz4|b9 z!kbfp5-#qU`9&?11Xi#mH)F(SNSM1!wy-5MRIeWIXC@gA4MfwX9OS9_!vwSf#9!D< zRVv#aJ^l$|Fg9x{DNydCjaG|v=a%-{%FC8B3(A<(+%!{827SU`UU7}Jk#;8(u|3^2 zN_XkCT=Hcad4ieoQe*59bkK_#!0tDQO=y-yAIBZn4H$QPnJJD$JT{j>we;d&&NU^k z7}@aZ0DcBugvMgrfsKpezhlaV>yF#DS34W5TC-e7`+7|6_p0HlC!(gxg{)-`3AKsZ zd`ZRCY$7andenb}U+DKq2-B9wRMt;=sE1uUy}b{p4T^mI5X)$?=zdrrvXJd6Rpsvk z`silv#_&zmn7J`YFs0@+kRc_rG$7PMU!SO~%ysZrdBgp^%joQFlW)4fpo#49A;t6f z*Aar#ib0yd3W2#dW7ae(s5cn9r-WQIda>Q$27JY`=Ed){tt%5I&UBGFfQGvXt=oU(s`p8A>!q4m)1LEtpZ~Ko-CcYAyHVqgBsEpz=w+E6g5mPO zJ!4a6671_X#9r^Eic-n^cf!DW+PcnzvmW$Pi=Dq?1sXIeHFTzYqVZSI;iHinwE7ThssTyk+c@)9mN7TMPKYm3$^`w8YFI3K}}D zHam~Z)G?L^16i>6AV9sEjLVb6OjA7IQ0qi6MQy~aV%-mhs2>?}1PrDcp z_3w3>U>b@IELTCb$PTp@gIgDM(aUR%>B?N60)fp+`PgVB)8@`Az44uwdL6|yg7(J< zB$Z$&Ncu5i>TG&ROe6MTRQ7Ofm1UVCqxw4s5JXijhv`9ePr3=aE`rwWEu<-kdCTUrrUGQkGV+)5_x;?*d zPHx|`vKZ0X=lAao$qEnDEZ^lX?3tmj_j*AJ?WhHU?r=^N#EFgcLjK zai8ndYJ0Ymhg6TJb(-Ba$@(5;J}Y$uLY>0od?nIwwnh0vQ+W5us$jBOCp)@`~J z)mv|c2e3cjKFfrB6}bknGD5ULRF;^lln<{v8dhr{Wn?>I3=VkBjlRx)a7%VkvuD`t zlcbEgS#c`vpqbM9Zs}xNs&NMU!}TvN=`Q-K$2pFL_oM#w6)x+4*LN6G%&?~S4&L24 z+bXmHXS+R@9(*SsVB;l22iCk2T1+vIW}QFk6-p|4DRTsZ#uS~BTj7OG2$~pw?N$TU7QO~`cU z(n(a-Pp#z-{|G*}HK{9uRNV>KUeY3czp@{@5kir=PO!pC56>g(kE~aw<-L@k{y#V4 zOb2B~RUHnxqqgHlWgZ)`FZ-XUkxscvmAphxteiR2F{JPQ?4@$BZTT>^$DaihVA*&( zv%KtfZCJ`VtJuQ7(*+0a>&w;uk`2(Ct`=8rbjpI7bB1P%;XJsv306YyaM`9Y-itGCGT| z2W?B=0wGQ7n)h^`ds*%ogSdj8h4Z-uQrml#dLN6X-KLDji;$MdM%rgEj>v>Pcx=n}rwpIVr{$f5{B(MiYsvbG(0jr`Coeb(o?|Zgk(i z%P85W>Yx>PEZ_I?dHg6u=Y1Z&z3Ok*gnR+I`j{t9DWJO*m?A zjX0@ZNASey8~hP{Wp_*TSIW#&hyJg0Wai0E?_;in+}hzC%|@&z*((O3$nssZ`Ld=g zP1BRivj5(0L5T%A0XxK^sK!<9fMZxsX>pc5+KkRF3EY%-(D7ws>BrY0lAC(NY?BoR zI@I24xtIA&ON0SOGEQR(%%mW1$Z$PYZ0YU&F{vc*37#&^0p`grIuN62EiWRRtnK^N z{+)(z_C4sd2+Su^g1p?o9k%pdhJf|?Q5RW}`}bl_5T+pqq6FLW>j-fERbrqEz)lWa zt8teE!8OxBTHp2DpsrYN@?I=#c*Lu^UFuUQVD<0~PcBetW-Ob{Ju=izj4h6rN-`F5 z$bwuP-|aLE%u`J@9r*c^8Ja(CMO9HA41^{R-?D+pyp z|C59tIW;)u@#53;I$ zrXMv*DLR-ff>U0sdhQZ5-N?kSQ_H`8Nw8y#Gj!H#zXaiOjz2+OU%4m9T1cdI-JO|3 z=Uv)u(YQ^h_dLX*Ntpj~Sime%+Np{_M{n$3>!2QEOg4<0zwiK2e#JOAEGOqc53OJ+ zvBIjVfaeEf1l%B*BEt;Cuqe)`?<1murVL$_Yk;eWY~gswBM$RBh}ex=GI<_${&(N{ zdr^_YoOPtB-}WBq?BR{ z+GKN?qA#mU=yOZl5z#HSN?&5cnSw9qp0M>u8a;p#-UvBp45NzShNL*%r8-CXN1pm_ z-D%&>$WHM*FBf^GHl63<^84R$=2I|{gd1}pLLZUY+i4kU4M&UZ=o)&=xvgXF5qLDo zIDxaexWE>9YiQNmqxE>u4{e^u18Jd`_I>Yw0Az8$6$WP7?eSl`Qd^B%@w z34X~icH_Yd>0WN*8=v`P%khQRtVD)D0jiIk5>e2n#Z)$_XM|NV6!prrok_ue&VaSN z@|h%*ez*?bCiEoIVlXKoo!?LJ=lFS5FQ#w4Pg>;Zz>u9oa^R4_P&-;t2j?xH({2&` zr_e`VyD2ih*A|DK~&>ob?k+yyQ5pb~@fJcJK+rqlU>BpSqX?B?-c|Lwv z{&9$H9<0|p%MnQ0!zMYWsn+>=;l($^c=O*O#UUOumdQ(Js9|Hai2c^N@~21O%$Bm0 zINkbq8Dz1}`NsG45!$1@*mlbiL=Y?6qrt}i{laI?xoNIJ;B%jF>co4kAdK+{#$oN| zz|%$wPPLEsi6GXls$=(}0t$vGLK2Tl*$Ev!aOUsd-x2A{R03q+!k>jWJ$v_YBbbo) z#SfE1R;-ulp4sb(@cK=`2TIjP2|cLHF_Nawf^xe^<-{RzZjZwKSsvmW2(|RiY|qFC zBA|3uHIX0n0Bx}n_!N4RE=!U~r9c~2R);67g;Gl|P3wwO^6ia`6*a=(mT{gH+i@+| z-d?06i~I4n7=P96gW&jq0iGUkYUSA2W`-G(Ee>s$&#rs_Z=rW$7s&)}wp+przOTJr z9Q2FvUA`XyP++-7;M`1m9zpl*07Mv<3!?@eM1XiF?m5r=Go4@VlJiH0p+arBo@+96 zX-gg;G3J>)#|-omQnK5}u}I%T!+XINdWF-TuSAa#c|DBf>kfr%w=xe%5ax6=CFl4$g7;Bt5vt_{ZIPKULQVM`O zK}damw0mewn41$YN(*j+1G9BsWWL^Js!05(dDkpHC4d;EXIz-eeg;IEm{rf)MePK^ z@n^~!IUL85)#XSB!l2g2I}Y1cB{XR_79V_uPaAoyB=(?#c-ue?KVw2{mXv^!3u?;2K-4^wDE*Mr=Ab)#CV zLU}{GFctkZj_%vOAMJi9!835~)wEOwWoVCt$Gt&BL;ai7KixB2XGoL;Bpp$ZCsAAd zVIQqGVuigB#ZDodfyLYT_*-z94Bvuf<$%TAc}DzxFNx%wfmHH0_6MK7S>~(vjM$&_ z*JE6KqO9PR=YWWJ_}_avrMXM_+)sP{VO$+rL1l!Q;Un7dzImlhlhP=ElJna-`lkHP zSBuQ|rex~p)TqL7S^JTdWaF&;@g|ioXTXz9t{eK?D$IZGr;aT?$A@Nll9Q&HJav|X z{+t0a=%Kt~cTcz!E`f3~=LLD{~=8J;jaB^!p!-89lrV@&Xd+ z7wGy0Mw`CpJ#MVo8D>Yn*B~`d-xqqHsc2O;W%%Yt5v%{YpumFUS-{?~??YjP;wZ*> zPd`IuJD3d~a7j*dRh;rQZ+;o4T(PONHDFmKH(cr50XZv5;hUlsKN=_a{&WXcPSRkX>cn*|l|p{Q4AqGTR+U$#T+ zz_FMA>`1}l(cZx=4up=wwa4g*m}^Afq>RF(L-VRzC9OM{RHU}1$Q6zhZ;~nu=66Fk zl*A#Tv_3`pZ6UkLd(F8#%lm4fYt*+HNfICApYRmfmPv$HSU^c={DG-N98qYDWO>(8e8wp~`uT=5r zlCtW?1$oAgLlRcRBX+E7Gu`R_oA;6eng`$08Xf=-gV%Mq&3|f!%T>nIsZ1t|7p&#C z5~Ncnz6Ee>r)rN0pO+ZhLz97p0XM=wWR=hIR}Aj{n?TyQjn7%M7YettT6d>-GpNMN zdp9;Qm&dVupiEIiB4ei4H{)_U2DE@{Ucz=~L*sNV3EUsPan+zfR^fr_6g*i%qjMHN z4qloJ;m!shQlwj4N7L|S{8|BVAlXCw`?I-Gh?wE zmq#Ms>ebciN@4~*mzC&M6`fPwSFKG1(8QCxfYIL1C1X(7&Kw6EGtKQ^yL++k69el?x z4J(TCmXUob=f$RE2_V^YxXXk)dpO?BQj%Mm^MWUX$GsvHoU2TWqa#cb%f%Jq`*Lg? zMVX-n2N2a`{TTA%6S*hCFNKZw-KEby@|Pd@2@QDh zEwTRbas@9z$C5*CvgP*+Z)Cm9-yC6C$qL&Qh1NuDHcuWHMr%Ab9&^qRL5l7;!xnnm z=E$_|S4}Vn+S39=vRzdaoB^?`bU#>$mKG-bRhQFc)u}m~GNI~S(;}xk-v3zy)AE7m zRvlMH@<0D?0xlUq0*Gbas%&b)*flk#~ylFA9$uUvM#NnANC1^ zf+}yDY?*mI_|B1V#f$iEl1sSz+ICt;BP71_COF63#uX~9N*&T$T9yKu=P>-2CA?E*4|1>7$z_uBy|#mJ6-^HA;6PG4Nrb8|vcQfy-mh)As?#gjWR- zS?A^1pZwyc*Y@cNr9`f=)(RB=^+IwV)Dm1QM{rh7Z07vUG%3sPYWmW6sB66uHAG= z?wPt7;ipNfu(X_1j%U6-AzCJ&ZF*!5h*8Zv31W;pql6HBs9*Ee-t_jZrDlf@J2&%g z8@m$6bX2X%f5RbCF|sJEzXx3lCB7iPGZ&s5=`>p5*-uV3XSs_9T*Ci8h0c3K@~2-6 z8QgX;oo^8Yo&sybzjYA!O?52BA)~H@Zd|~72Ilp!!n^ZDX&8sW-hFzRugoK9G47jY zpSp>MCL)fIxND0j>I#jF< z#7M&&Q=tR3-<-|oteM0_2s=+uzmt?~7a(+f?do(I1 zd@IC&bKMfbeM2*2gD&X>Oxp1bec%xlNn&?Kv=+#NK7{?%lKAdcCC$x#|ATlY8p&uo##!#h18Ep zcXPG;7$m|l2aZv0Vrbw!k1O{=*}HP~h!oe~q?l^c;~DW&qjKByPDfm;ljHJr24nse!5d)SSbS8!jna(b)m3PGEPv;8)SWJWI8cJTf#1( z%*|Ir!9WhASE%ZxKsh7WZ`}7Akoixp-R|}3A}+*BS}xySneT@GXA$v8Vb?h^m1gbc zFo*U>O~~YkNA2D7Dg#~R3C2#D`&&HSG77ykaW&-U!M%uT=Wmg#&-IfeVNoPlA*&ZD z4TFN*&PI_B+6Uz{5LGsYJ5PWPZ-zS4jXTdzc?=JA0iL3%O z$$QB;nAbumXdJe^TzhBa_fl>KZ@1g-DI@48=m=SGk6b0&NQ}vM&KJUK4Zsj3j2Gdd1 zBbLy?J*&bLIdXfuBAC&gsrBQw<@N>mjuB6oT2e5~>Y{H+FpzNp^E33qu_u}wC`_3l{|nrH(>K$V|e*3&E4ju zeXqRiVNUf!CpXyJb!UJuJh&Miv7XmzPdnyB^A}N~7K7nZ;gPygJigD{k$TNGV0i+rwI{UOha8{d6sLW4JneZkW_Iv?hCM zhS~^|jl>w;E9TYgQ2+aaeyk$4HI{@?ZuWTv52Xsr?>4_BbV}-+8KuCC90U+MMp#A= z-}fk?`4nV~Amuy0;mb<|{QHB?uw-u4)u&Vkt<#$A4n_D?JinPR?HM+6fk#CN2Bu<)zdL72?FsYd(Sdy1h z+(&tmyo9MZ3GG*BzPNnYdeEH#Te>T`Yx+z4qtikQNM0ic7Ep9&__*+9_nBMwO(6!D zNUji;;%+v??>dJ;q68irsp2cTW&NmN8&L-PeeW; zem#(qTROhyL3}8MXc#^QB@GnUYCefWbw!O>*AVk~Hu031aop9bu6Ug27y4s^GZzGlPQ*G3u-iR-$=P&ELJ`!ex13YWbAXSi89U8nJGJybwHCUHDH{)t}39 z;Eg@2Ck!Aodspj1ypkhxj2_*QY0E;A}{db>sSY1(W(gg_e4=lA{03XQMOs zQqogD3aGB`3<>(;-9)tTMq`~(5|;Z2Mm&)|RB0(1ZN81RP5;?bI@@mbRrxsoZabJd zM{DMRQe3xp8K&INfDHa2Zd$NK=ZW&UR^;IG(5h8ZbiqB-ZfdGfawzWy7EaS;7$0~Z zIj;jJISR_-pulnnkHd>+5N|=bj#jy{2LCZ*NFO?jKOXC5ve+L_U8B3RkUpfEo9o!M z$I}X3yITb&bl=&9G+Z2jGBTMskcJSJA7V!q)%!LfP zi8`Rff2DUaV3V7(U@B7Ze4?n1^2#~})WgVtd3;b8OcohoQr{>?7O&ZxbJTtNX7I`! z?nG;qH8}T$J8rRQ(>tanXY-EnDd!l=^t4VURm+`4Q3elgTGGRR#OZ$Kon8D56B-ccVI);*;hzqp@EW;oD zXIDjEr*=%waZO^dz><;0jjYAxQ=FBhTJHkSj`FTr!xh@@W#nra`+leHzA$x>zBO$_ zy_Scjr6V2ZC2dY!X0(G&apCC&XW|Y!#nB2~ANI<&lbEXIv0ZoW)rkL$HSqUjDOWkW zw`;KciI2(iy<}fClc`r92i5t&iY4kehp&}fj;`S8U4?+wSs{t##Jp^hdo6vFcC!i6 z)Hdn8w)^Y35|C#Q5lJ{V48cRpa=-KF9e8CZh$5U{QMpjLxHaSYS@`|f6Q5DYhf1D-%Tq=2j zKHdIFqih_L@y+hC;H$32-gm^Ln8WDro{jzi4HY32u?5?uIa|_m=5h{Ybb5)=4)a`- zsoUO)wR3=;n0cCYQ`t#*14qk5uoA=SC)$mPL?{+Sd^qo;+wWqzK&LR8{r9mrW94Oi z@F=Mvp>Z4Q^Em4m64v)!iwo{F%tLwn25mAW{brV{GB%>PQ&e>f8ih9+zHm z3HTv_b+uH!8$$6vnw_Uu{$<~>A0^HXSGo!zaQkRSn#&0DMOa3iA_CJ~ji4*r); za3>q(?)H*UL?wa_f{ietg>>c7uC^^$uD+S{$fWb4{eQdCPfea**S+HSJi6fYPuGSA zc#DpZagX+@70xWZ!a{mHn`fFDc!RizSbVNOzcU%qFyuqUFEug@(vZC}nx2KQOD6OB zxe}JCT2nBO#;N*C6eJlt;UydIs(R=dp2Q7lOKt5?%G7)w9K^J2uGVPAeWLSQR}wVi~M0q3h2OqliP2`)Rh zH-gzw{)a{!#a-#R8s-)3n0BYqLoqst1RGXf0)*M;WiEb`{aj0(_6RKrCPlFyPr0&w zYn~aIX+}KCqAts3N7^}qKW0GF7_zS9PxIQ?+-zE4K=?pp*cR7O(n=y~HE4)%=*ItT zu3UC82XP%^fCO!@S@;Y15r?jSx%t_2$85f2d({o+b@?%PI2q%*v}xKD zteW)d8V?^nxh=G4#Getc&)#TcTlS5=0}R}*vGzD$diyao0CPZ#W!{u` zpbX?_)F&y&W4h(Eg@cLoHj~3O3yz_&5y46Fa*~$i9YmVgTSkQRW(^X15GU(~;B%af zX&Vmt2B$21QTJ$@B#hP9lb;QpxsH6Tk{Xr%b)EeA_;slzM=Q(i%#genjQU^ z*A$cprQG5P%mfhz+NKF&%H5x?ZohM%3}+wTs)A+FVT#HLcQ|)vq_9iz^a{+Q_J0lI zTmd@X=k4AKGnAIny^2(_E*k zS&f@HZNJ^lC~v4QMNhH1L7mceZkuG7qghZ2F@!Syu;3QM+@IEu|0n{cx5=UpL1@;V<-5y>|&e0%Y*1&1x3#Yv56 z1tBZ@Bt|?2Pe<6l*_sURtwzoWmU$02jLYJJ$9jd~AUsg;N-yUsUc-R=Ej7i*qGUfk z{N`CVW4Z;Ib3{C0oQ+RW{NaIiY*??(DgdE?mqV$ z7j7vbgve|Mt-!*Pr*h8bmF2dm5M=*B@#7(^C?ADMmQ1Si^>+sU1X})k0S0EX(;eB1 z`Qsw84o3zy9V0g_OK-szp0m?B*6!YMpxz1xcZ_@G3m3n5g`2AYhZYUmKyz(t^fIviRyXLn z4TkMWPp>4ll2lG5d*U=!^z@Q86?FA}=S~00H3-&W5efHFhB%u(Yw&WM^Zs}8C*%_1 zwDo62V-Z?htj4Avlr9W=>+*Y*Kibwmanp6r##rUgMsPjIh*7_*wi;`pM;iu)@IDc` z30G2EXB=!%54Hw|h2V}dwJf{U+!2rHT|VNQ$GO(2;u*Y zOj+u?9|LFjfrhFst>wpXNdm%R;my*^vFoIPgHhX#TNOJR>$!w}SWN43yBUsMxII|3 zKLeLy!FZQx=HkHkcRfgppH~M=U#|2wuYr=KBXs?TjZ&VThN8J~en8Bel;6f6SBmL_jV%8I9#gFi$Y3N zbwY?$RAJF!Hskb}BqFe0FRmFUAS2*(g09i+eo(E8G=ffqd$lFzUs32k-3N}_L6+Xa z@DLSamqCG^5VGef!0p@qYj!_WQn2|GVBK&DIP`>*_DEeIq>93@Kt8+u!z}RU?Cno} zFiRaPTta%$RFnug@ugeV!QqkKO@XIq`psT`6NKtlQ!b#O1Q?uc=yb5r?&shXI#;&s zL0>nL%p+(I>6JbN0MQ8N5TFt7t4@=6q6kgTsFI>$9t?{=@MAIuz%+%?OL{Jv^&|ZI z40U7Hck~GvDVRoZ{Q_V@5_D76kGSwNIv>8Lhs@Xz4$!u$NjFOXD6&-;5&0loPU>Xz z!x*p~{#jc&2<$rb=!C+4XSvic;3-N9NeqFwvc~@<>h^N@&Zy$L@hcMTg1R^*Z%xwdVos&vv*rO5 za`%Wgc+Bpp`j2F1VdiQ}2ciiPGExa)oI-xxsyoH6A}}DK+-W8vMF82Robtsg@^OC# zRNMe~C{Jv8Pjij3#jNNMl%bSAMvcA>Kf#}B%XK3SFiQ-R)Uig&J?)3kldYFh258xjl4ZbmEIJkdz6)C_$&Y3O z5Fa`QO9`mVmG@EH^CW=Cb5Z^MW7~2ozz>HH`ywA)@`0cA)<9?tE_pZ(sQke&T5l_8 zHx&-R7;-Z+ehMZKEfGFmj*mcd4c9AzSw!=s<&WD|egJO7 zs6)yC1(OVD+&|M)2sEz--cXCwZ)zitfC7$ri1$Ic z%T=kfH_3%DUNHD_uF4kZLdI#_a`1SOn-Bf|MbCjtpL-K zX#W8Y#65$T7i3DFea5aUN{sRwiqOkU4mQ}n)M@{>66Fc9Ci+h6C@O|BZ@5#~0L}&b9`Ih507X8rnI5`2IW0nXVeUQ4YTM6E7|fJESBEeh)hMYi795TKCXh{0uYu zeSXM&H^#k*FM)6Fl`TA{O;s1oe>){1>K*+EkyMxv_N;u+bY z8=33_dK2k_w}1Z<6d1gsA%55XS-wkE^fSs07MJ9Zo;u52OH;&hsiw+P8Jc)Xr12-= zTHoWnx}tL9j!&J6ci&5lHgStu`+51FbT^Gm=SYBtsaRC~L-UwjCZPk}er_o~s&oCr zyf1V6!W>m|xyrZuR-Z@tWa21KuGIeTDE{X#J5dVsYqNwO*`OSgj2_1RfJj|9p*dp( zFMh5YCM=`rf@FYFl&vZ3(2IyC^X+=mxOcup)*+w0t<~z#v>3X)Y+Q28G&LmlBSUiK zGzB7F)C5Qz+Tv&?xb>yXYon}6We3pIc9F*qqIa0sj2u zyM)%qGB&RwE>!RWT5z`bZqjlnrK;dbzKP0*k0<))(eueGygA2W6@9|m^UgOQm;KyX z>}ifN=76=R6m-Kf`n{QADhDin0hUmhI#u1Yz|zRzmt>$}g9`~1c%tF+w(T7u%(qYo z>t25q-<~sR7R+@0ZA|D{9ahBcdnisk1Pc~HnvXd}69r*f6@`owwRQQj*PWTFdZjH2 z;wIcFULy-hzm3-3PrdEEpO9ZA4|?qWpIaPwk5XuYERANQj{$041IUAZuzDKMh7NCJ z`Jb%!EQZlNr}o<)?J)-c8ndeQUy6zzwl93a#-kJ-Wc%V4*GJRt(7Ksesf;G@H&f@Q zA*=3Q!3$HLe$^LL7yHT!%2flUk{udOCuLW(t6)Mgdju$0AEbL5uWy(J7J%YDqqgU? z(xctxq~NC!AR#P)7tD7T?8u_+2Qe4ygO_4$G*~VC(!(7T!k8msghKEy^Fx;dOW`3B zo;ywdd>9y4z^$-9_=t8!gCw*1_uWe|8Q?h#_?&P$*nnb2Pu!(>0Z$|um9Ov^UcjF zBagXy&++v61d2lhyA%QGl5P+Y$)O|%5CLgvkQk)9L~tj?nkQ>~kW*jZ=Yr ziKMdmX$8#B(R7!XNy!SS-XG?VU_~NVfEL8o1&5|d7xqbcw~XrN4m0f(xW6RfUcazU zt%W4l3Y%vvk%0`?Q7R>FNwy*Lc7xLr~h$ zvvJ?VquI2jn)98=A46JWjK;2_04OY0&TIlz7(*F#h|D+8K zR#?*;D`bekn>@%BpAuCr(dLd*y~3b_8mtLNh`{4o1Nzd0_Fe6E1xB`!#Kj;D} z-FIU|lv*^mVnFAX@f((mO+P6q=CBU#k<73ZRXut9eMCby)5GKZ1Zh803fRmLK#=ax z%jtSQpEvag@of#ukfnc9?956*a{H95m3fc$1dKap?Yo}EEy#2+ga961&}EmO_$glq zs4Tb=QHP~TXG}gnS!%?fN}BnYqeRwLv(4+{O5flb5EE$gGfk`pqeX!$xidCoJN_;4 z(OxsSz47YnSi2dW~2#E?2n0 zwCgJd6VdIs{=!|GRk07QmG!PG&-qN-BHm$9p}T4WnOTT?($BqVn`br2V%DJd9E>-c zwPCIh-03|mfP1f~Y@ak;$<&lGtcl+f7}zfQkw%^^e6Uu9^wuI9d!ub;o$8T5tfaM5 zOeO^{gWc+d{C$INUz{tZTHh5%bj9+ZZd+3^-$dJ}PayG!yd1y#tIvGn%MyQWSI&wu z#aVn8V}7mvJTJ;0_NN3;-mxHn(*66_O;R_AXiiAD5KwgPv`xNK`z4q${>S4hyQywm z)M))v?zag8<7Ne%{^?syr_VN{nvf>JTZzUhC7llegU_V_JyJ7$Nv=Yw>fsc91KZil zb8e8VBJnwv(h9WZ!Z!+$;FjV2%Ci6rF!v|9WzL4W^92N{$jlbOU$^Zh((#Q7BwOHX zDXmqS36bfQE?|2*17mw_q&l?Ma5;yFjXbNRY5NK=-=B45w#}GNi>v1T>_6?i+e-nY zqfYL2ohBe!599kh!Qh-9(`vl*|0dJlzR@3l?pgzPesRkAJe9G zP{n=cM<fdZz7qlAujXHD7Wwtco1@sln}?R!X?gXId#2WkMK4G6(o$@F zu$~3kcDMD2Fh>sVl6JMdSy@TsZ9{l`^0E=E0}y-4%B2B9SXAQ>JJ@55^iD!}xW(`` zXB$K6As5xjDFAA#U^SaV2I_Kz-~QV5pa=vkJFJTUi)K+p2smiD&s0nMW?1PTSGUIH zW2Sy{u;q|F?fkE6z6E2i@r_JHH4%%?g>ly|j9kWJS#i$R7K=9uwN)Vt z9zhjKo#j_zvOA63nD2xwS!qiPn4G9inV;FFv2Z0BGG?exSo9vMOaz{%4-S=YWcDIe z_8tI+0PT8%{__AMvtCqt()W*40yOFE%9Z+@r8Z)Ve313t*GfVDR?^M0s*}S|SE0$B zaI9`BACKhDtHZIy>ywjog=Xb#@0*9QZ+}=)50b*oRH=v@gw6^N#iduq zZ@)A)0g%C*E#V;cB;TZP)tn%RSXIu;)z5^O?AP3sJW_O1#ECkUIp4eOAi(Ty0oHv` z7k-m6c6&@xd2hGQL}10fwgIa@9~#moMOJaQ z3vlIi_3Rq&w(QjI8cY-=-MydyZ1xoQ_X5N`qTVw>-G?85U?t-=wcA0-KrI+`s_iXHe6v`qKT59ZSPSPjfZC2OyaP#geVPXs8d>V}K znX#v;v+C43gRT7V6smxtHSr25@9bEiZrlphsqDm{r30C?5ke4PG=WVD zRC^(G5_<-I952;5T-ZNJKkMm(Nzt-vzYpg-OblswSDh^~^ObFOIn>$Xa9CmAGLQU6 z-*RBH3+HOwMH|aVwY-rqTCv+meq`D#(*89QZL~mJ4Ov04DvDhpOu|KuweE+9Ws7WE zv}B82t!G3NH94G^17?Z~NdM7&U>HO&JP%>$I3U?FAWfz3v7rW)b&o24boXZ^{HU-6 z2{kTI?7B#ocCpnGw2?c=g8>(CjaZ3hq_J?ut9Yn6;BefgHgy7^Rtre8OTfhP1e-Ox zu-&m(!PLq0mqts`tvSrU&L<-XmL0Ui*NcEEV=LgLtEp<#tesN6@J^H~V=~Km43k;v z63vjt#p~Ievi0NKYm1otgSHrbYoe?5_fxGFM5~ez8+Hs}C|1>YKPlJUs?7puhBC;E zWjZ_MK4UMOq8yZ|22rs%brG);^XfaRyXq)qY&BQEAdYoF>&mt^8!w6ZNNv%uQzoBx9pcEoJZtj|MmN6|({>E`*UW?X`AhcY%jGXRKz-+vvTl zuCC1#=QCGNbx4@lqWB^u6Q8RxVc~hRvw8v!-#;2`zafq_9MqlfO2D|xEy7eW@iuX| z1ZgTOvyo4GYXl3_1&nn}wn$LW{cere(|M%*tO+o=^lZm2K!^Zoe0hHm$l7t)Y}}`& z2$AllLy2g`0Nl6$)W>W^H5+-3&}v{(?J*L?EH!Bf%CL%OzMKj_KG=tETieRHqcSAf zy=c_MGQ2q-BLWg$-F+mm#eaRe7>o>>7Fxgsy4)5#s%@TGu??W}a<<&aK?*IrUQzm! z;??9229d*!22FseZ6$)R0Z4<>ssQwpDv<-9?*bKaXHLGqRuxHHQLuek1M6s2RlEGk zEL{h7eZrAq6=bNk95NQ%%;`-nhJgNO1GsR8g@@vu9^0Kruw^uLdBwRzM zZg`kx`L2lfD$Nalr^wNJD^Yb1@xmUAr?`ddP%34O(?nhco2Xmw3A8SL; zEWzPFx7;Yh9bO}7VY^b6Soh5b-ivfxWdGp8rCQ0RWL=IOX__3NYw{P1UF=CjRk@e3 z>(mSG!Btr(iqgb>1)}2`?K7gtzgqSH+`JXesLtA@31?Bn>O}e|8yDgA(wi?RBUfi@frbT%j!};3 zR(n>F;La4biu_>f1@ly9KGLOOk}x^jV} zSqopz<0`F#>JWnqy&vEXli$T70K0DxWIxzK$)E30CjboN$PDb!;JSYP_?DT>_0C*y z%1KCsAWdjz1gAjh|JJy#fnib+=F^T}Knci;u^{oD_dc@i1ULyf5OujGk*(zByr${bMXE6-X4AtE-{rUNn3Fy5RtjVQh~Vx$1B|=I-#x z*dv*EVnG4JUTfb12$4;cTVUO7lj@}D{E!Yx1@F?|iHs)bsG915J{rBoIvKFHRqenQ z-g0u>!NMesjh~uc;pGx9E0u+_^q?C2ct6rT+5fV|Dka&WS3piD&o#k3#r16?3sV>G z@L~D!eA{I$|3ZxG!4~#^yEOdwiGkhRl&gk8lBK} zKI$A#P+>&!On9~N)w)8b=V8%S^M12EWvZ3tTmJgybotR=!*4^3GPn3%Rfku4Zz!ay zcA6@EFFbrqUyfS~xjG;An)H1!C%lueL2L=L6kT>16M>tVwi*BTV=zJsmiBh{65a1w zb*j#WwTbMMqWkb6zKx#%Q|GLnK_|!|SunCeI5c&0YiABvUxRJ|aHP(~etEV?q01uK ztd{WkKsXx^qlT<8zP|1sIygTBT~76h06GFDiPjY|PzuQ9B9)=XwxuBpa;!1_Y2AK! z#SPG&k%1mBM>XA!OPhv^@E(~WU4CtkY4K%0t!Z3!)nCd@*3T9CAx}d!R~W^YzDFM# z8@xyonXFu#TK2Zz@{*s@J52eHDZzo(3JCGF6`8`cSQ=hTRsd>bp4fxWWkX=^qjte! zK(ExgK#}M#Q{q2wMgK#z=}j3_|3vL-tT6?9J{`fT$J13h1Gen6$SrF6g}AJoS&&e- zr%J@L4<{Bju*|lo+UDn$egwq`rPU70czvt_8dCgMOd|92y>HdflJ11uu}X*-%;4Wh zdd_K+v6j78#(B^0n`3~PSQKw6=4Z2z3bUNxN>rouVnE{|X)v2ZLN{}oA=lyCv)KPH z4T59Y?RJnIpAPS+NK?e@WR5J}!QoGJS-^*C0fxg&>(>OC!r1QjV6M9W{_3@ld$Wgo z7kVwpv?rtFrdkSM`yVUsALPzD^oeLnT0e2JEfNV*cygm53QCy*lrn<(>edx*5nvC% zyME>=36ymPcxy0MFMoQ~K_|-t(fKYo>f+j+n&DogYkdpZ%%1L#4uPD4-9q00 zI0@IO?Hc-k-lxJn++w{YvTh=MN0l;Mw|?KCTPz>%8_zk=2XK)g+4=Z@jqw@~C#@B4 zRPsk{MU}X~FMPcrsfO<-es2K~J4nlr5kde>6+?^KSIza3XNufpFL6EBB;t4jOcEFK z*-3^6^jzlsaux3h6qmlVj2=@kaiLg2p0J1@ne!0ixx=1ypa>+ub!oU01RN<@d!Dus zGkT{oROv)J2Tr_RGQx@e1+r1(S*zyu--F_aaFi1^y(#IyS|8my4P(1v6Ulmwh+0kn zI}Jg(VCc^Ns`gK2p_RsSQgJ=qjQCy}>nGP(mdBBF~^QvDj?i+HUf<@J` zVR~3tV@8`#4WKQ-(2>CNy}Ut$&&EtHfBMW=wiL%!3j24<*WD z)N%JYgCusxZ=dTo^711g1cItnW}nj)5qaz$Z1E*spn8v7gU+Q;@A7b7HQf6l8OsJx zSot<$NJ(iQmhgrDY9ju^X0n?hOze6;oCv4@x(8Q3j^HeTK>{$O-eF6sj?H9H*woa?akHi^j-ED$dImwSw#?Th3Hz~WzOI^ixZ$DUxlc7bq^R` zZ>lUFxPGI2c;1e$$p1}z^9YcjiTxV!bX|o#g9;HSS`RcqWUQ%gxS(g<3!8>FZjr-B zNJntb2Z_#2dqT5kih@`vM+N3|eo}D#I^dYkuHmm{L!RffRA;YLm6z}vV0XXZ|H-KP z8urN0iy-%(4rv!zpLv^nPDJx?QO^{EiR|%J>wLq0W7%vc2TRmsFolNbVq%XQ0h;ULT!uB7 zhM~H!J6shr>6iz*JLiGau;1ghykmj?0SAE$p+?->U3+JFGhStIl*v~4tCprC3$APc z?9D@wADm6AzGtu5*E{8>?t8BVKQoQWU`c!f+!6Dj+t%#jy-d;QDU?vODS)~Y%gYDU z+}Rlx9ef#}+O4t-^$v5l35392!<`3k+8%mLog@#2b4ie!iq8sDv7<1eUm{AC@=6km z{VOt}T7arOHu?tB9HlJcqY1ff$D+^>V^7@*Pq7TF|H(5;qRBkQT?&uA#@ubjan!GT zJ1U^r(v^o^wvfX_HQ^m4j|F_0?=j`?3I-v(PVAGzjS5rU=c zQ4aK6GpdJJtAS3QPG<)j;tJ-gYLJ1$@UJgaKOTq36!b>qU6Pwpcx8l%skTvdh_b|T zs-OX&LH5XX*|91}1A2cn5%ugzH^0Atl;@Zpt$?Cy?|*&3+rmVjn!U5`oY8_|TepCv zMxE{C64!maX;5yHfN;4EC=kl~d}h>%usyZ1dZM)XFt=YBM7SV3v{^}B*JrnaJqt+L zlhVUpA%M7tJ90lscqrmZ|cmXwl-=tDa!wBU_etGxHbmZ7`$&v z^lizo_M#Ai%2#Y+*M3s<+f{?3YXuY6FZPfs3>Y|INEa+@vIz@I@UG1 z7+;p021RQ)q`DkMG_gjdvD~FR^R}y3bpJ8;6 z-JCuJ*^W;k#b$t0nvbc2=@&o`nf|D>01&;&?ueaK_c1ZclykYt%^IMXjt+21l-65S6TJ{$;#a5G(dUalxTN_S~v@Y)4fVKxi1@kAT6>ks`LJU!KwSRd;WYGO7{=7=YUj8DYu$Y^I&@mk3-Q>ach5pk!y*kf*Pv^1v46?=Xp@bizVMkB)gQd4?> zz~S${?Crnu!Qzm*dyW!@md=|(!mEE44{$DY+u(XDro1()LHBkTjvs* zL#ms7s=N0C7%^4HbY~7?7l5koRi-K>l(2WIMWO_d@zGru>W_ZzK5pecIh3M-Wc%fI z^2f7P&3#0481cHQ7)-lu7OIgF(43txlS#!HxLKUP;c%jG?0?<@P}n+#F+>yOCy1l( z%vWt!Oc4f9IBRuwC8D{?Q3plXWrE6;Zi48-DoV~?w?2KmMfGn?-bWVAt(}r?$C%GM z=B_jKM|bp{a2N4sEM6F2SLtslNVZrd0lv?4xfW^~HK|HG=6&pzyXe3ivjq*Q-ED4- zIuTRaKP+Z8&9IjjFbBKG8UvNp@0Ft=! zsP^YWFT)L{I8ke4{tn_?{d&Q$9nh5+V)Rcu>8s&PkVPDHjIVnZ08O`@euJjA+y<^2 z7KfHK9&vG0jRThohh&Av9cs9>mD~0naZ{}kB*XCM;HHC(oDbhZ@^~5bRIPiLOHKt@ zzp|z)YP(Wx@u80T=>CGO?|VhDVH%2T--Nz&2(QDPekwxt%wzH-FTY4Ok>h3AqcWth zlhUH-JeCq8vLt#cDI9(As_pGV3na3)fNUTy?-Mp4%T=bEZ%fWSJYyMHp-NkTdL6x| zD3Kqa$p7j?%VSNeL7eM|j;Zr;yq`T;*s}JVuwS^6g1nKRh335P$WBSt3NX#f3Y~3% zIzL(=QsZ;nPPsw0D@|WgoF7R+&gl<#Z?1Q9;Yc7?rZHT!Uz6an{rh5=5Sv9#S%f4B zVjh{lLD3+sTqzNvzVbqAe6V#P*|@zaubV6NnBR<;zZQvON<+y-obi6Tv+{i+xA9&c zIAu+^1J{h>h17QVpj@t!DsIu@9m~w^$gTVJ%=E$2@T`j&fDoPkzS)Lyh9Ezy^P8V1 zi`=H)OhXor?@@YZ$dq(UkB(oWV99BlcKW>^pdx%7p)ZRSp3@%c)=6!9E)PHIn#x{; z9geSBoM1U`C7R6V9!BoByR^zODev$fr|a%7mL~{~-dj|~P`(s{}_o?0yUJ)nQG}TNCRPRQ7$}^x7y-@-Ehl z7qL|Ynl3np@^RL7D%625UYZD7K+?gML7HmIIY(?tPg|pI8v#c@k&S>d3&`S+w5jS0 zej3Rj#`(4;1FKo=6QXIdoGTSKJ;e6A8e|JZ|A}&i%_{*GTLuRXXVSO*y3HBFSruhU znMZ{i->|obb-o?l3H*^ji5iB{7Rb@e@pFMqNO6c%ZfH~gM0O>7L@NaFm^NP_Zb)+Q z-h8S>>(uj-eDx7zkl7>Y(6{W=lrPr)^!CM(wJ}r9@V|d}`=TAwZi@CU zCX?8!4P)EbOg;mft&9wntnz`xu&Qs!N8bM&O|{*m@bk`Tb{`I}`@5*YENU`eukUXf z6zbRVdk}yroWvrQ*mRwwkjRP|9oI~?&5Ql8O~$A9fGOcNGXe>c8cGSUGNf9XDcE_v z$9tcJBR!gd+BSR z4%A|vH_{YA?Soa+;c$OkwKZZs19I1G~43$;R6!0AWOrO?bZDxA>Z2x$_TtK2FkQh9o}EG)!W##|5YH z3^kjrxN=0dj!%ClRe!$#N|y1d5;P;e@KD`yw(<)Uvk*NMi6E@$ImEAMk&Mx% zjmt*6*^3P%G6op?#t*ipo<0YczmGh9?uya!F#kg&EV_r3hOWSaf+S;&FK6aCcvzqt zyHkuY%-&RZQ=(|@CGlp!VA?dn<0RFWJyaufl1~d;BBNR0W|pM~n(BNOuuck%ut5ch z4V{;!*xIcoq)xTnq3Dhe=_G~fCUCvDNh>)DI(y>B5>JU&8PU@4_G$Vc!`ljU}Z?)BC2 z>p7T*L|NHOK4dyu#?4Ixv@Xm>MCcE{*ifD~@aGJL8(Z9K!41k+yGgzDUGR?)Vcbov zGF}E!47KAD?qOrKaPMgMavSQ>MaHe+{Lk6lju6dc%C_d5XpY*KOh7~`tflFu$zfV(k% z*edIhgVM8TJw4vB@!1nBd#h}QTC+KfF%>J4cG6Se8Xt)ZGM~=Vhrth|;_~`BTX@4% zZM!v>HC!Xr*MEyi=mS0qbeUyU{&=G&t_qgYdSV%C!!4!}l}r`TzL*t%)2DQT(`Dps zY!tP598y!(leE;CW0xLANj-ym&d;|xV_4#fR+TYda@grCYMDklNcmTy8Mtr%=y<@= zD?oszS)dg$$zaobM&h0Gr{T*#vp#?^{>W&y0g94Rzqmp9Tb2Of-B1#MrXRLzKgEv# zPBBr0rxuyJ-FZzQ57wTyu;_63jw>mKu=i+_1p#+P2F(W&wrKJ{hdBK4KU-qhtSDK{ z#*`fVxI>qmlf#uH@1^$6^Q9{@pB`1SX^|xs#i511m@+Ofvy};!8ZY1Iy0G>t?f4Ph zdv~ee@zn-7S<#&%LebU`02*|8?@*Lh1X#B{bq6wpr&)~X2eIP4e3Mv8$-aO+QJEq% z`F~;%8L_Fl=vDQVXhxM`fx4z>b@c334zR9$eH$`glZdIY064CtX|a+O->shYJ-mJi zHxbtGYu7|FS0mEL+zUcg96{#$Izb8>uIz~L()@j)P^G(L%LI?nEQn;i&qFva^Sf{QW)e_- z*{hq?7S_Z}yGP-1ioalgN-a@B{0E)!Jrc?Ljnvd=oQ#;9ncm(XQ{VY`8>N1h=fu1p z@q&gkMhjc=E{%_}IC_MuKRRdB^fcLJ{^g;ghuG+^4<2?^v~BM(t^h&_lc*v7<>fU=pML{f&_brd?c3XcveLwRP#M-kx)&L~TH~ zWqaAzio#q2O{ZOEDBI>QGXwn}#WJG$qq?p<&xkyQ{`23t@a7oayUL`HT67m4xixk? zuy7%{Y+T&_{ilpv(cf4-X1W~iOP$WO==q`&g`EgU)%K2b2)fiA>w{a zuw{%GD&R$QeVYsq0e@S~yox^7yo%=U)mzVZfiTBIai?yh& z9ciU6+)`|1)2)(}ntyQq*OlE~Cc7beMa!M*vYyMOuIqGlKy7qJL1$fe;-9wwc%u&l z@3TKKEdI62KyW?e{g7=D=iuCOhQW)jS9i3D%+n#dbJL;7o3|+C{X@bsa1@{MK!EtX zme)MR2sxmGG_{3PCaBSO3VKibN3#-qZyj#aoDDuub z)7ow%OU8Ko2&KDfEaxx?TRixQgd`lVol9?tFqG_|dFU+}7Tb%V{eM5GKcAY0bSexw z-+*Ik{$EcM@I9DYHW_(fyVSh6y5usELo@($Y(9LVRwnRc#F_(na|-UD7Cz%FHxcm} zt0@Z;bW(p?em2?3@8IApr_Y#O(#z76KU1VJtr){zNp|Mo(`tpL&N0?IP-a~W z<6>(O1)33|7qgaOZZq{4IA2D(*XE`-FmcIz(tVz5)Y`@AK;^nXBBqXmy+@iD?Hg|F z@}EfgSJb7FpiRXJH1l;$>iqXrA_7rBBZEXEZpIIjHy>Emv4;42E?lLR^Z`@Kz#|xc zie8@5UZvEk>2!}mtuesqDd|b$cZm4*XgG?e{MAEd-+K5lPpR0F4s_?oYq1>cWc&3j z=HQD0eqN7&=~?`kM*YiNqjM2>F~o3$^qfZw8meiid}V*HT>XqS(sE0ga=TlgQz=;eRTBj)nl<4e*Clk3`VM>^#U(lEi0{~JcX8vndI+KwW4yAL54--@M_g+{vXHf-1uBm%fxLU>GZ~73dQ|&vE&Hn1<^2Sle#1+h z$)a)+C$ner@Db&E?AW-5u@J1#N1^z;m3b-C{7b^nQ?6M}!q&(&e;(8-_I+_H3|GIu`)KQ{Qib5{sj<3t?28E6yf%m)$n4A}1 zh4mmjw`1eJv3 zuaxW(x2W_@)Hf=}sAF;cxf~HSHqhQ(rBL_%pZq(Gmxpx% z=&R36;nQ$W_jL=VMWuT_Azj}nu3`k|&Z>=iRy|0*6cdO2vI{$36c`$rZ>t&_H@zHW z11@DD_0Kr?uam_8rP0(l?ySVe+xY(HqQLv|+|Qsw>W-L;0iJcx;$02ytzcLme&XPR zX+5C`p^MbDjyn{j9Pwy| zyX^sUo!%%BD`N0{9%zkso1~@=h$?vupQ@d9Ba9$u6Y0|?43e|bqL&asrT2&L^Wa#^ z*&=3pd;t&vzicL2U_-hh78A$@`x|T^$bXC}OPx!?-aQL+#Mw@CCmW{X zI7kmK+LSYPXcHI{JvQdt8xk{Oht$}EtyT^~T4laMkoVq-|XEZ{2H8p&ag;gXvUc1Kq2`5MNvrtu&xnAD13(#wIlyn z_wfX1ZZb@rT^++E;ry3bf^}=5t)A<0X$mc^+q}pNMg)3?n7jB;xsK)v6vz#%c{|aO zURp;+NI^Ywc4212bhP+KBLav%c&RRr*LEMfjmf1riNq%Fuu5HC2+lg$q-}ovxVYWO z40{-&d&y!)9HeDX^(7$t)@r|4J zGgw)ZR2=D=;`q_{dpD3L@p*>{7?zM+^q7U3SO>Ox?iFyk)LKxB8)kT!P`|kORMeGf zPWstNQ;uXN!ylqXAyzJZAqUWdqz#A289PM{4j1)STFQSpU|Y`2Ry&wmylI8`a8OG5 z2oE{&pGN~v=V0A8GG_hW8RKsIduqFoyp(44m9wZKOklB%ul6ZRlHp#H=rjLdf zhon#4a3)=4l+`NM^V(b6Q zbn7rfRvvh}A8#>5jKe)I2%l)`fhEBWs2BuD{|s;kT^vChrMngQY&nhygUVws7N;9I zmbyS#StXf&BfEMK$cdZ~Duh)HR{`zeAb^(kPEMoE18AhO6gebsM!VWQYV?w=Z~__; zt3gO^08((eRA~^&S!yv3pwJ)(+zKw5wg0hw`ZzsFjl1WTfbqC5MRwk46fAFc_~PX~ zPo@#MXpWJw-?7k5f=VJQd0&jk*RfE5n7*`C+-ymD+(rkpAG8iV;5eq_& zfreZQhY^bLlnhO$L%)XRIvVr~%M*)bSMyhV$P-?Tr4VwCPT^(0vBQA(x_)+eds%IC zo&cnt#F@JjPQECc*p%HQiB34vPDT1}SheE97yrNeD1gpiZ>`I={yvu*OBJ3 z+{pJ{!F(6sop4PLHoJL0B0?ll3E9^9z;X@{XOzG*uB(qhZUey;Z=AZ3ee8wUN|>3? zzygEle&B6WA^BVstQQMAZ2__M3LkbKP$x4R>A3)%*2l#zEi8kK0k^sP@0jf;L*two z3a--Q|J8M*k%0B4dxj87dc59(5&c)TN{r+4PVdPb{DJxR1GX|cAKpwe1Wns~#cW3j zx_IKkc)`9~d+wDoprzpt31O;*;PqeL9nDRz-z5JDd$vg1vq_1|WYXm4YT))(7{fCG zx6?rb=C_o| zz-w?Dg(lF#GGqiu45zA>z*tc|_rjehBu!*l6Jr-(*;)a)=P#|x9aQi%c0Yax(o)^-}!a6XhLD6Bjnigi(@0p@`nS2pjpRH7KW! zj|nSkvRpmIq3;{RHV6(aO>5L8@s3nm$kZ1n7Er;>?!+3j+4L;raqlzbm8*7lY{`uC zWmiREDiPWgKWwz4kn_Suc=h3>I>wbA`z|@cPdxn4lY>m>C)ZEQ>Cu>ChpOWj%02Uu zOnU!bK>hc8rW4X~L+mrUMDJq66c9Z1$_S`Z1kne($x@jON@krh;^P+P8|o(wzHl(3 zZR6=kXS@1QZPhy~$U10FqLLdN?B;ZtCQR58^&uGt?6mLXl&GWk%VDeb*;K3J9>Akj z3-&sT^I{5JW`{MQ_9ENO2M^r)Tw1wgNr>(%0i7|~J3^2rr$|kJ{o~5|x~kKK;I7kO zwIAZdb}z{4_sNzuuTPnb4?KYx%Ga%j0nqy{h;%A_BSZiywUjeJZ?6o#9o+_|^P4#- zuTgPb#I=u+Op}G=T@UkpfW2AeGTua~0r|0uPv<=ShseA2Eu_bb#>_jG0@8DDqEnJ~R{+zCsKIBT0uX zfSblLJ6YAf22165!&h zkrK@SKIv=^Q2>tJO$j$%hQay7o#itkuYID-MryF08?b>>F(TRC(Nf;TCMs(O0lC-{^&JtG5f7o?A?f*VFIZ81u7vXhk9$nvO5g5AW5mxNp?2=U=agmSXm-Hy?)N>*<) z(pVERpYJC??d4AAM0qn%Gyl+53#rlpx;_Bia8BVso)GmZt(sgCQM9@*ftO=62|bR& zL6`cgf@4e%*Kq{bQV)!XhAyA*rx~O9X9*I|h6Xb-fufm-rIxrbii$KVRkmD>BsIChl8E|daFk%6 z&5Nd4f!h>t@=3#FR@_eiJ4cReC#B6O>R*<5?UVveF^z_$rR^QwE0+k%i@~bg z>-eKqy-LYGme%a;&!%DB`O=MNd1K>?e#OY0qX(~l zGAUhvq?(P^kX}JwdnTX99X|RSL-cWd(jds62SvH}7Cv`!+9n3S%$-K6qg)W9&0MbV zfqH@2^9AS1T759EQI=N$vKci;j{M%n-T{l7o`3_-Q7343e+UkEOHGR26gCA1Aa!wC zkWw2a7suNz0Do0I%j=|eRv*d71om8nTC+#TE^-QeAJ8Nyt^#r#O;Svo$CqB%hL;yH zhG#Jg6MYb8aIo%l+9_aTrG1S-L-g{}irtX)(pT)%_t~P@wwbqAHIn%(PxWQwcEGNW?&$oP`=FfF zpSHLvtE>po(e|HuIGz%1AVD^tS)|pW*mAAIYVQHdr`uT(_~q^CcFU`L_(zQYu_wSL zW~;|L{lp;r?KS}hh?3{a^SktX)iSxxNbf|zrBp4tdHC+s^BmuqOKj4IbcX&b~w7QsgN57EZ_ z9xor@poN=?r9vP0mob!Zp*=#~33dEHu&gd@shsSSx{!(xLxRANFci~=L>5br$9@zj zz&Ca3bN&&QCDoXhKY7SfEkDSz80~e$T}kH;^SL5G*4Ew8do^QqHm*!L(4%GQ&~FsR zIQ*AwzmQ`os?OJEt8s=~LI6uLSdI;7>Mwnf2nq(lt6({F52f6RAN`D4L8{Z>s(Dz0 z>R`(#PQW-GArh`;tG_%b6Ug2phQrj+)M3Jx3U4=u>+*loW-2G4e z#8>!)X!{w^YAhdR;sym|&5Ld>z*YmjSH@n^D&=E^OY-(*&uRRpGI4k@7vqIMEG+L{ z#f>V|%zbVd%im-TIm4XLL;c z=$CMXSoQV1C5svIs;a77hozQ2d@7eYpG;Kh8V)Z=>nT#n4TQgO08aXI!}3uO_jZbr zqbJ!8lp+zFytDS=J$}jh_s}i;iLvIcCGPI)7Mp9#;bfAH1@X7t`=&)xdE-6R*7+t} zFWowc6HP;-!9F?IIG+uE>2KshmoF;5n3fQ#Hx~NAQTLgjdh+ zeUKTF@;-{M&gQ-Px5fqx+xOP z2fW3+g2B(zF#?GRXZ?g%P=QVC?-&5ib_z5JG86-+5%m^c-r!&ZY^SkiWBb8iFWF#^ z#6dvmsACA0sK*#JhOD{_R_rW%g3o3*G%A^!g5`;zb#*Zn}_nP3jjsAJv%nm&QGo^KDWQzW+@ ztN>*ZF#}pbLXxB_O-g!m-v9^IcmGU(NMuiGkIm2wW=XjG!z*G8hwVf6N=O||E0w8| z6eLaIL}s(FPL2<4@um1;P${q#pX6bqKjy0xz|(v427|3U%H2U&%IDMAvX;Xh%6(#EU=3(?3s zApV|oYDLkvgNeSzb&>y#E%sMy3jPlfP~n;&qT6!a7(t^{Rau2IS!s>tRq5H0r+ zM@nmY059{FG7JecnNP$2K9O&+pm=Jut@`|%P`i&0?PD~hy%l|m9gus#k~WbcQKjr8 z{6Ifb(a9oPW45LOFF-d_K_UahvMvVPm}pEp68a_){SYitw!v!b*B%*z9NUzr2IU)n zJQnqjXtO)X6q5c82ODiUFzIU z{iSZWhTOe!nEH#`fu0p!RZ;eWJrRv~eT=lGIbMhTJh$TdQ_Xl!Pegk{asLpEE{GD= zhnD*0UyH>EJ@3&Rc|GqPaqDS^5#90xrA1Tz1B;pFLegU4A&)_Jx5z+d>kQ<#mxCUI zb}=bS5gM7Hf_n-_gN``2d1i=sJ?6_rVHdW6v;yXk=^`)mDp-iW`+}sY$ITBg)^7xe z{V7_(54R4=hZoJ?4f;mFkT;=258OzIA4%C%CSbDDuifN+n@wu_8K1&^p%mKks_nJ( zSPO6muJMd;8pZ^+b)T4pX%IuKK45JLAn;-3p@INjT+@55S}`ho^@iP@W+bk9{KJlu z$RLyTvNqi?#oO%)q}lh8t%`0iSAg98AAho#~i%g{rgS!Vv8IxbUSy5*Rtt)D_D-ykr4kljNKkt2>Gj%#<9$MLO`<3LlBGQyGinAGuMYh1-G z<*sEoHkx)^m8EuEx>aE)@@(DhT79^KI(%Kz?-xtxG~2$MT=*M^O=Z_Z$GW8 zJyzzstZQ|7@P;@n`O&+NC=N*m2#Ivfq&FPc;gz}|v~FB=DUX_`uRW6FMs+RY{>0rq zX&XqN`NESTX6Xy&3w(+HD8=b)SAy-5obUK7_CYzQP5s}kwZX@Lcp(nvr1W_oa%@;8 z2Rs8KwMRxXjbqiPxPL5Y6Fm_d29~tRW%V3#^5$-elVy(E(q5X?2puVUD+qmMP+KD$ zrAjK?8C!GT`e?&)e*D{`x$5^}7CIF>Cyu+)TEecMizE4-wYmNKs#E3A$1jSu8N6$) zJ|6-lh!H<}Jpa-Y&_tH9K67u)nMXLl&l#9n;18Mu?C`79EmzKLP0EJd$@g= z#liJv7EKEUym)8`nsZCCVhEq|WBosY!4UHQqw2c@sowwpkxJR2V^gl3bBsgCDyx#c z*TF?r5{EL5l@Q0sitMaoi)_azd+(8v9FBGDz5Bh7?!Dj7@6Qw+@7L@7oR8seJA#UT z5N+2exWiSbBR!^A|2|$bEtmXE@9pc#5C(4GH+6k!lTj)o*Onc2MqeO&#^m5_3c&}( zCSej=C#`tBQ!yT|O8Sp4;wnOQu1N7%7e@PQrXl+`6p}!xStyc7JX7-vHl)p!!u<8{V5M z7Po7cX#Dq|3Za|*OHXTyFczvB1G~$2@2Sl8TaA5?E_zJDc0o6i2Pw5}mk5PPX4JT2 z*7B*T+G<-uW4nDZQJwUJFme5*Rr)_+eZq=(g6-9>xQA?r)%%2O^$L+e-chr}WYcX2 zB!WJBt(lMKS?j!u)dpz^4{C%rV!pABoX+Nu81(}Yt%f80T-EeJ#ir%@EjHdA_cWb& z&@RMmvU&b7h@9w$Q}M7S8w2roZ8L^YOIi|G$LRB;<)N;ovEf7dMH*diGR|n3iq`X_fM6PIDen(m)z+TW-0o}p=c5o49*}$T zMvLrl_|P!p3Sn=Rn`O4W3-^=!a9kBaKAq}qJ~%O}iNDSm8ijXaO0VHF9)1F)CogxJ zS!;svDT`3wHTR_>CwkJ&7^=x0a#Q8J$a?9X3Xa?o>+xaoIc{|V(^ zNWN%(!~M-3)VhSBQr}9F zS{R@4iFf*Q7Xj=;n|DTD)NI1|2sp~BC=6E&BnIz~o43|ImReI;=7|}OLtb_n7#$8< z-|KEtSE9ejpRG*`?FQDK=EKwj(${yL<1ACG$m70Ad1eJR3otJEgb3*+3t|lISbi8^BV~#!M%x=kcS!of+2K4li zJYyj|fv*kgO05@$7DT`K#HHUxRmLkjEBFV3fd^Jt0>#_CC%o6DGk8m2oRFGPK zZNylsXm93Ib&vL=SoLOIJ1yz&ueqk-ebca{Zxbk9mvEng%W<2)v~8ByGi<&&H5`j`vpVYop50bGdd9V=Kx93 zYg#g#RCQ&D`l(#qq=DBe$*bHJxZs$zQbe2kihA}sF45uLdOmG32rCq4Gv|3Z+tqvM zcTOIx%COhj1R2uZNi#AU3re;~vuc@Q@VBP=@YyVD>vc1G(?sjzN7raFScjW`ZE69H zRl;?m8n04!v)dl7N$`Y05bF(6BCQ{s&~Z?+yz#=bC?^lXLj) zq}T$DvmtO;etwIO7F%_b7fYt4C7pLdxtToz_4DM->~!Dks2^9BNqV;mWQckdS25f>}UF?~~XhmIP@M$XqM z%O#1q+=eh(NAWYk);WXw@g`DdTTf<)G`cX z;)|r1kmux>4(9MZM+hB%9S_H`<31l&d9!ay0ZDrK3Sn&SPwM&}pCg zancxvoa|gWIA8ujUQ4_CN}liN(#C`YN7Kak(PmxBUez7M?E0$<;7rLP2|)9MC?y~H zUd!exbgjae+d(k>nH+}`N4otq^DqTr+sCleyI5wc`(#9GDS|IPL;)*ubR^p~-*_Mm z%N>$@scbnd=R$zP$1^t4!dNm zoeD)=0Bv~w!?uFhxpLq$^w9WM2j8X)ml9C7wnGmXX2rMhZzjOX@`bbNZjPFuc_>Fg z`EPDS3*6MtHq^gA_TzyB@!4G8^rQm&FxGhO}jMrsoe zslEjF+~MXn`-@{HHhEEfkatwNh`;}uzG8M7(uoi00hABJ+A`e(?&4z)jbf`!5PW7i zCMY2ydEjvk9<6+R5gy5`+Wyfsf1#zlS_Zd{4sXi2OGSP<@uu|^LbKZLNbpc0Q|4$n zq)G%}zEF{$?Pa71+1_U+qCZxYmkkUq@K?Ea3 zLm{5=%M5ia4c&6m4<~?^F7AQH*WIfaJyJDI4Gvj7KWbLh(w(=GFwK?U%MMZom7y7fOUv@*|yJ z_7I8>y6+tRcd>cI`)l6l*f0y%wJ*HM9^PsJ|CrqJ! z6u~BY-ykX|61I6mOUZVWGxEE6I^yGC%H5#i-RB(X7eBAxbe4I#lxN-SJu>XUNO*^e z{I~_4$K+8&{`4zvtPoT9@!fu;^7q$D=}thebP{crl_Uv1ZF>;kxcGKtpdT(rAGvdGP=7ZBExMh=Q zSGY<<%DwNcN5Y?XL1_B2A9ATj-rn?ZPoDQ0c2NA?VvwnVI@YxG9o;fmj06k0zyRcX zOT5;iy4CiMXj(HmJM1cqsw;s~W`9?ysfdY&sE`fO3-K9@1Wo_kBs+~Yu(Vd7Y7~UN z5HLvuk~>b0JyisbwsMP=Fh1q)pT)U{g1@Q|yIk`>drRsS$7MzWI7zSfCed%%6^00d zt?5bBS&fG6f@k)q-yhO%4G!1R-!9SPXSM^Y#x*Peb>kXImKW_R|EB})oA*CYcBenE zMx7uJ25&EdOp3auMW`v6sIgin1%J=|(JW zO(R&AR>ihWM|+Q<%Iu=6NQ)D)In_6NOSS{?!14*mH9^2)_&rQRc~$7aT8w#kjDH$W zkZ&p|$eRM(OX3OfXLVVbb2!iUa}-^dq;rdgsBDj(O>y4LJ{mzIIcSWVl@!# z7TndOVZ)WX82))x7VF}gCdbr*)a%spVAS)>HHlTM!Kj8e#ZsHY>@oAxwELWq!dvVw zs#uBN)h|<33{2B6sdRo6Hxx4AEQoI3gMVvZPmm>A-IcG3RgILgMn;xD zjZo7yNl8{=eZcYMCedoz!eI8d8#rYXd5k27M;1oZhYptQ2!GtYJ69t(_E}=n(WX78JD2 zP8}J6-xW4JwRU7_G1wVBO?5K}l(8H{`O5%+*boLapwg6{NJQhf>`?jsoHZ}z1gBe7khl&cw!EazB z18J`^R*d@=Zd=TZ$EfI`oq|x83UU({=$2bvg?XW8J!zTV|0;CKvS0kCDGj?~A&D zlim`XlQo78CxkjAkF~~*tx(-pYOjuY$Yu}{>8eZCKCxlFnzhv3Ey1w&ts8&;Eg#AS zc@O?lhrDmsYFEv@zN=)vtn6Os@SRJFrpj4_YM|z=^fTjB=uVRVM`JtUMWi1Y(deTC z#S=+$Hd%&3mye&LP5Mxq_NuaPy6GT>+Mc`aqrPWzG$qR9!F!GuXmD_n^_pC4#K2JW9@>7Jy zsP%BXQGSMAz}%Qm7oso(n{^JkFXR={{uwKq2T&~okBaGCe3Csd;$Wm2c&--9_r4@U z)%CG)w!YGzf0cfP7^?2}LD%|r?bd3*$*rPW2jLMKW><(|>GyC@84Nb!NHMm2XfUxM zmvtmbMm1#{qsXbBO3%`y2lQ5e>CfxB6Dk)Uo4fWk#Fo(jcJ z081kPXTNUh0`2-WEO`2f$4M`v0ywt`D6v+6+=1#^rWJihgZ+%!k~GhLrW&I4!)`hm zp`x1-EoblDl7kf!lDJ5l7W)1(dmCagASo^pTxnB^4dGUkf@0|?D@2Hr!sJYrv}B4c z%(K3@Yj!GXOIRdh6iebLPW5*QvneY22O9i@M_=2LA^U+ma!eJBJa%IMiyX4yrzUb& zh!f1tC4#QW8R7@K_0+kOJa%7Ww6suVCe=@!HiuaL0Rqqx_U7fGyT4E@O5`IOOqk0Q zx0fdzxrkOp_M&uCPNH@~b)+OG+uz%tQT>D}PYJbrUKid_S>eKz*`1qrtz$C`AbA|4 z2tjP6YSBgyUEXK5w(mV6ukk&zIoo$Mv16CYo8eOUDSkP_N;u{uC0mEIl!+EfzxZ}{ z#=)*W(uRLKD%y3;f%#dvMYHH+We9o6da=0=@h*aQ+=T&`V1uYtlX?K zv|sCQz6RDMVSXp(Pb(k~!KwN0BX<%imsTacI0VIsoQs}}bGBx?=n8YITXzLOh14vt z){|^RJr3HhSMQfUJkN7T?DmSf*9Pc`Aj>)+p~~aCRaOekiAs-@L%#C@$7pbk zPyjJdy4_vgZU_7kKPf#SX%WrCjkTz>#74$P*fG!HAC)SIP}(x}E39z&>P?-p;?%Dv z5;~Afj7qxB$6+?o$&ubRZ%Yrx;`V$J!RyO`UwI%NQB&kd0yrsTOxsd+mY@8T*<|ty ziEZPJBFua$CpNgK2p_bXNY;>YQdRHAe<;mKCa#vTA${(72``$BIsmX>PAXyI7J90F z)*RC7u#=@cu46m$L5523^yqQrllnpta^mtTKPA}tW0%%DAazopA!~{S%yQ!jjMHd? zi;o_|cP_(@pdK-_KeEEddsJoK@R+78L~IVU0uGI|zBgn_JIhtVXSx6sK@|Ru4OvA2 z?apv{zYzTxr>@~K`uHFmCX2~UZ?3cdpD|KcQx1DmVxHSqoEYv^a%L@6Z|fan+Q^6* z7ZMUP!7)-Q9+zAzdCO-Wd@+*cZmaE{53kXc_^?_FsUc#7J5?Y_kGW;Efd9_i9?&(m z`UT>J2hMDf2%?s42r~y~w?v&olmMn(f_^y)3EjzyzcZsDXx&J zpBAHtbFFO?^SSc@ecT>~A7LA(ZW%Dck2hxORMbW%JQ;i9hvMn;1n56zC$QOSQNe^q` zZvf^Mn!)sJ$51disuq|7{09L`0`Pr=>o;m&)=;EtnD*o#7>kGH->j;wXQy~-5-L)} zy`XUHmFBNNz6Q^am}PzrCwryfVR*F!gz$_~n5+&q%R%59qB~6nQT}X@_uKE{Ssuub zdKF^@-?VrrEBSgCrNLVV*H>BnryFj3THv*Q`DDJXBo2Q0{7jI32OAlaY5IXt_Xj`S zZCK;jdhllBUO~SJGcY1uS}82wIMAdVRdt)I0SH3}k2=P%%lyg5uGdU-D2u!`QfhEk z$Q}6EPm%$f^7h9bbI)$NuPFL#PBRM=o!?CteGD~y^I_h_B|AzDn7cczr_|x!CV;E= zmf(~sd6~~cVjW6t?rnx7G#^#^{FHSrF2Xhpd6wb6A~qNkkvo)Au8J5b!VJqlkAt!8 zlskF{TF#vEaj3b-$ByeiNG|#F)9`~TF!6-Gt<$&Gv-}8YBpQr_1EUQO*Ws;vH!V-B z>V@ZL0^mu8eK8d!+5;UNd47^NkczF|BTJutx0@>M^;ehHPyzE`av>i@4N#}uBs?i1 zJG-m+^1c+{XIP)U)LI#KQESi~Bteo(vMmf}v|{2mvdFbA|HjnQG`_z0HB{hD&${DeklTuX z$|YVZou4?N8=B^35k?Iua4QWbOYQdd2y^+s{|w#Gay4@4YPhL1k<MEd&Xijw*@2iIU8;F?$6w=S;Re(^CHw- z=cEl*p{6J`YdamNGL_ai#oe)t9ko#`wGC91n`%QeZ zDV2|rxNJNzV_2fXZl+>!Y0RrNn|7u+r7Jiz@|3DmfsXsU7&m_gARR1OQKHuv$3dOh zF-Z9w_Z9hnrjD1B%K;G4)JfOGF))t%a-WD9>YU-#E(vthK(RYODB~!_jICcR!`cT4 zC-TPk3T4TNj4Ty~-GYs5!#h_Leh7-b<^^88e}^&4w+WdDUL#TPKg243as@w99I{B)5gG;quo_M?88wwhH%dp&X^ZCD>4>Jsf^DRaV$Y zPnens>v?!Ase<@XBYxds&z}I2kQv1W>03IAuAyAZNkok`RVxAEP>(V9&z|Xb+;4YI6SzaHY%=@(Q z8|n!nO(a!-O5Y}&QcltX%gu;|e}5A*nIXzys)AvZPe#N#n6#EBy-S$_%F2I<`` zNpWi8UUS%}sn0RC6)W4@sPu(_ zC9*))Jw?;u;EqAmNWay6b5wR%S{P>_j2R`jo zU1+@eNN+1j*mMN^6g1CKvJK)DzZKM#m;88cFd%1NOLBOQdH|mJyUJNg9-Ydq1KX*6 zl~lHv%M^Zgw`57ET`aO*b$bODVY)n|ToWEb? z?b4?YiH~piK7HqQ^aQ9pT z2)w2>nFux}D+>PO%cxWwg^Bd>YoBU0|x)k5NEdx|wtI59&@x4G$ETsJ~v8?Et8Wv7S>?IoO7DW%bNLKL2H$@9e!mS~CEBq;TBE);o+=Coh`iV;7mppF_{>7YT zIpq>Oc0CEt=aNu4_g7Y~*N#GJYozit4*?x4nUde;7Tz1QJJ+0QV$_45qP zUwr-0`pDdy{ca^0A|^~^EBurFim=mqo)}i==qg?ZK#Cf7!@0Uv^zlmlMd9g_TEw6A z=w&k1;m}AGX)Cvw@Akx(JxC@1x!DT&y#tA7QUfbC;6Kw}+Bde$tP%ZZ0s258Z@|TWm(!s7HV(brGif`)em>d0M83cF$#N zKNxYYQ?5+33ZyND(uLWhI|LNI?N(MRj%C>mMGu88VlwXS2$8!aTV-f|+<6!L+7m1l zSs5XgmnN1?9bAbO9UMot#)5|%xUhyO38Fpf`h{@X8dF&k+XhFK;p3XLNeO-YqU}8a zymHLfwof@FR^L6;@l8s2lL7UwUQa3cOXcaTyn<3&+x9Kj_2c|Z9dg_LxI*C+{qe@ zC*DgBW_cTb244U__csR5hu8+h0?T`nB+XyB1AkWoy_o0sbpnhALW89i?zG0F?qy*B zF)_$Xc+wQ@8}vL267Pd4M!>9c-9`Gpm`TX-;h|i7VMZ-N74n=@xujR|Mb2a7En;z3 z1$h~8`gX@wsCu+2U$s5ci0tU^p^KWwGQ3~x`A8#NfVHr^naiM}a2)mi-L!6-A4djy zYo|!}*7n*DAa4%QEsikfvZdnp2cGV!AczA#SGat_ttE1#O9=G^Mq_iWG#fqa#(?l( zzCrROM_5)FVwmN2HZA0G8rEb&WuA80*ik3iwC?8rPb2qhrVF`inOpA>?dWY0?-P;?sEXJ4M!N^I;QDg*RrGVc6pzP z0?>F8pY@CoQEa9rUhlG-oymmkM7(E?WZqU}13xar+nLY;qz9vIJZIu)SN8YrPC)?k zO}FJ{!R^r!AGhO}M!dYW9n^rUVKI_Y@6q@FgS`t9ty2vsX4!f^VQt1t#hI_`QvE{s zIU~c4wem^Vb-P|oZcrKQOM%)G+;iJ(p3((Rzw{t(iiD#YeqDz&-mFXpFz%$`qmQ*y zZ?ogNTj5`ia$+TQSK3bWR20V>h!LF=vRnZ$9aGtAKXpAJYB5 zC$fiZ5)&(ve;|eu*F2(8mnwxW&vX0X#Y^e%_9MvaQojfl;;z&4B&iN*IAi`DdgXk0AxA3%e3YNguYfon-twPGk{mpdP!c!6Z4MY4ZdgXz9tuc zZ2=^%c?{l=VK@!YQf+Lfq&+%9{jT-Bq&a{b!zwG%Z^nwp` zYq&h3Dnno+D;Yicl1mG2SnOWi2RFm#!|252B6-x4Pp0M@Qh$Caa~R1hi9W^*r1M~M zm^d&IG&gHcO_MGKS89hw>|tbHRU6f20i@@PFV9$pN?F^b5M-GJTf)|=qTyjD3f5cW z<0kaZP5}((?0r{OK^lvcz2rl8oG`Z~RyjZ(oo$coNS98R-Kx^l!spq@3X;E2sV#@7 z+nFZ2!X+gbPCOR}T$6d3=QCuaJaVTyv{UhAaZ)hBE&)7o@rt`gD{t~puY9)vpd4WS zf<&RYzT+$R)8@4REmS?b*K2gct%TKZ&Sy+I<9JTw=-btwXW>_YPXM zw_9H=7-rJG)E=95`=+pztUH+xZ@c@HO;OqNru@G$DgU4uXV{y5+6(-?nWdV-(<^~z zX9i2JS!9>a0J5lt$pDi88}vb(;~ajJ#iCdTCL8l;&+v<&TaNbywdemV3LfYNj`6~Kzay*pey-i81p z0UCdIA#PZ#3@>ctyu2S(>hyC>V9Vq=+>QxSK^FnBJna2;3y*17b#R1R{Tx6Dpp5tM!F~fe( z7g-%?OU=nE%3Wu&u`1(9(~_K@#yE=%HR_!2q}!B*3R!qISdg{vsxcOaGJ%AP!~g4R zfXzpMT+N$?=gs9ij@ze=yD@qKN2BK#voa^lk`&T=13O6XGblv%cQ@fGo>J5&`}<5Q zt>kDOsQb@o>s;~h+AcxH1@5H(M9JrLv+?)X(iG)`wJ*Zz6qF0S>L=0MC(H{+ zP8axf(zD$?lx+Rk&@X)H(qM3vs|(YZi?M5=mMWc)r7a z#KKo?IrPg!)+e>@+~ywtI5y`{|X zx|e#uBtFx**x0&(gRQ7??biu`%x(=GPVy&-q8$L4m>rxFEIxsK}7iwKc{f{z_ zTxgS%)Zvp>4<|1(J{bMtxGkl`!iSxW>)18ayV`cf0(dZqAmfO>Pw zlVVq)cAZL|XOlYI+zuJTP!>}ZATE2VtWu*fQ$eCqcqmb#33*09knS}Ls-fo2?1-ZUfVh@%>TUktZs=z=>?Jp4k z1hu7mn+v^|x?RyG`jq$an{U^Wt|6{&BA zC0D1X7X$seI*Vvp?ttiKD`Bd*MhfW)RmV_Tnf=vqXSFdzFRAjLN04|!?oeE&R2&BY zqti#|TR&jq__9LM;$gvX&t*QwnU8J0RXf6cPYYE;k}ttMYR8zvoZJ-x-5Ijui;+pZ7A0A?H1&l&4-J3c`(L^<7{W$xw2aqUxkTkfB7lNz+4mn zFW^s4f15Jo-msy9d`sG)d0AZ0KmhC;a}|9amOW8!_jrotA*X>P#*DUN_fBI4x{}-% z?M>p9e5`|5E17VRfgKgD>`9z$hh>bR^&R(PDSy!p^~*)%u>M660<=vz|A z!vzmb5;dH=#p4IB&}2@lxeka1&%TUu6|`&8qN6m&mm4G_;cSy7&M{2NLNC`0?0gu! zq^HKz`zb?_z#tKJc@Owiv!e7*{w&70iLH9#Go$VL#Za%+vjuvd6dEI?PHj^`HcSlH z?=sb8_8lEL7?v|+$}tdk$qDxLTRO$#0IaVcfn`3V`O#vUWC6$<}J*X1Y0> zh)t-H`YVFwYmV$H1}a2>5S>-_?*vkl$VX116M3VXGxI}7IV)`W{exZKs=&3)E>{cC(I zZ=D)6$B|!M>0vyJ*HJtvI(f0ytSj94`Sq}poI|^g@k2Hp*=M|t_IG2{)Eb|dq^JRe z+7(*vr!udH75s#5AG%F;W0#@E;xk4L^$zm!jsmBzee!NbYNj6?b}!!7kvu6`W$9iZ zW4DxJAX#Mw?c-@WUtq)&%6s+k12EyC{0E`A!$D>M8fxaBm$_++;jgXgI)#6S zUE$ONpDUf;veLO7MaE+O7RQF5$TLc%c_A#Cn0SvG*zHBx%!ht5gA0F~w6QPa0i%HM z*487^{%*Bx4O61~4+ABxb5H)&E9qL^_R!hhGPYScZMDVIfg(@|bymRR0p9Us?W*E~ z2%-=$%tLG(g;AVXo8o!R_V?N+bq17yywIRl<;lO)arFN0{Z8o4AA8?mVC9BwYbqmp z`q9)0w}=Xrjbd_iHmz~_+s{tK+v3dlw5#)cMv#bM#K$MYTuwV|NJ|LSZ;4Dk^pkUV zN`H_K?RX7es&^44s1R~$e}>18{k;OBC+cFSdg_}Pnpu1_Jx*i|kT$-Fx1ok)#sMyj z|00QC+bQc(&!@t#1K|3d7e9?0E>Xw8TS^aX66ze!R=lEEawNdO`v0H62-L3yB@Ldq zfx>`divF7@*SM3G^APaW)!eT9VNH{`;)R@1k`|O|FM*BRSO9e zP{3Bf>Cv1^tO~MWoHL?Ifanm%0NYjeV$S|D5Bdqw|bGSx?t{)Y~;<)i&bw?zgZ@U{707?~?{pn)0|NoC$= z1zs5z3RYA%6uQ??lxB|Gubz+Lvxzx#2U;-e`QR||@b9Zz00dYFqvFXKC!^;$>E zPW5%VY#2cDGv!pVd&ceUy7gUh)sH{bhBvH2R)d+=(6vhh00H(^$;;7`!lkdw0TYH{ z==#(0N_@R;E<*htr>{+1exEx0G_KnB&Zf|VF>;A*UQU@fUNp~3&7Z2foh!Uf>#~NV z*dM6SV7@|V>Cfm^;lGk3v+*4Rq<->|*{D7d%9i%=1KTSyJ^~v3hC%*P@FNKseQ6bK z;N1j>S-btUi|p%;<&+^7+v(ggL&;Ly+KJGJFPDp(CXz53f5#Vz|F@q5$lJfZi?e3@ zExt2E9qzm(j#<-FmJ<&xPMvR%Sk{sQ5%XF*RXz@O-`g^Ihg{W?YEyx99TdfJ0{4>M z|22sY!?9hsePYsBC13s-t$IV=y`YdyI0g6b8i}zRu+%w2Wrevqm%@ienTZRXB1OAW zmaXj#gMV$7uK1Cwxjx&!Vhlrf*{MfmdbTa!ibPs!BA#ZZVqD5hm_1!W)p@t`!d|J} zc8haj$?l~l4g4IXtSqc)p%dTuk78Il8?nI^gIR^SiV{El zEF#_BqgSC^PuV)W1wK@+^7{)@tnJ(r;UY5uT}D6P&cepgMd!pish%O~^Eiy&psta{ z-j9x zJKcLk#o%lSHv&zK@UNS_8K4>LYA;?d`UMzh)X56c|6+ znOzWl=TGo@Mo8HEDFavdbxgpv3vyF?6PjXWsy!Nr_;Uxn?`UlJw+l)fj>;`L{7KMB zfqr7R+G_3Uh__VRS$RV-ea}Cyj8y_e4q6$QEvk$j!a5nOZMMPTgneQ>sx$};6fEoD zcCl4>y_If&S)G=-V$@Em0o#%D*v^cX*$&Lx`K+|{8qiesxDTlK?qb|#l=S4&m8Pa7 z+Z6Wp&gml%Q|SAqX~&P?H+G+;AiaoEGtmd+H|4WcX7vyeIb{iAJ8t`lmdT5Iv11jw zwaZQVgEAN%q_gei-oQK}1Xq6acphQh0=zm7c2JD`?PjCQs3}!S!W5dL8x|6M@51;G zoFv)WDhzP-dnhHluYD>f_UvUsl5M&#Ux)&o@aNt!rfUIW29c$ZS~Vr}9uiwt8Wy;z z?){={Q|+C|QwuD_gDG)1ybPdrT&gC1ks%gObKAMvbpL!`#i#-^Znj@0a4vrQ;s#Al z2*s1cWzoSE;gZkYxUvn?g*XRa8bcbEQ3y&U86OAcANiFsN>?D~V#l#seEorO#~p}%WhufgODje&7_LX+013%Y@&Ten9=G!6RilD{#P1t?XukNkvU zjfX#swu*-6vkYRffEsyJrf`;nBnvTgwzHbDtD7HYLTxk7OjsnLs?}23eiV`_-pUa# z^e6^#UsoAdo$7xg>fR#W`FW=$K!kQWYCfw#>T?)rph`CRZw@jP*_D1~ZXqN9n^QgdRp#Dh3hlm{lta1)vURE$#BCGyP3Tg&w?My^((q2Xaz$Q9TIp%4O;M@ zFCTpy57{~f1CT9sn9OFt+nEo*n4!u|s0MvlQNmpI6WmP#Rbp3bcsimmrVxClRj)W_ zl9+t27Zsuw16AFz%g8KK7=}h8Z*3}4(6(XSWd;jgYwERTswCUS!A5xF&f7^`!~70} zOw^iDw4WA)BJ6nzr6h|JzZ%~n97BS~$@U5N1H|U?p(x!=Q;q(x6eSQMEdAG;zPCPzWIen$Z=nOB&% zk+7dQ7G8OO)HEKJBw0DCC|jH37-px{;K(|pZkOnk^kv6BhHF(XI}+|05m}aC#;4w_ zseg7iMR%UpW>iW~bykrjabZxwM$@<>(@B{{@|ehcmd@=$$=s0GCB{Lm^2LQ(6(dn1 ziDIWJ+E%};c;^W&`&AZZ3g-R+>^W+9+*%zQS&Q7uCIuj(0FOh0p(MUJO6z`oZn&gf zqFdEBY$Ow5Z~RC!as`}xV!jr?T8Fd}f*reRqEvh2`zy)AAq+`bOjn9W3?WZAu-3>) z!646Citan<5)UD9>6lOYYdPTr{Z+lBkJ`lN6P-&+OFa0=GsoNPQtZpBhE8HL9lMy5 zieFh@m@K^0hUUtB42t@};9pE--ZQ6A*Y#%RxaB-_v6!nYSB1&N?ypGbP8SEa18spm z)xTN*jP27FRS=>=LubdQ%`J*yz|=HalDPa@yW4d`@TH(2K+{`Z6CN+QDjZzu^z1Mq zO!2@^vdV&<$LFPkd)j%s2hcd+@m46B6t!Mce98h!tUCjpfVM8BSQVft;t!@8@HZ$& zO4aoQ^uC}`|AgFg-R2E#aJTJ|NAuKT6txZWUiGFif9%ka&F&xM2>=oXk}BP)6TW>F z!ibFyX{%PW76-^G)c)eAoeT#$P`f+}TYIFt=Fqpu-Ww>se77%r4Qj;|@E4A-9GUVM zA02E+6@Y~b)2?TqZ!LmR;Z>}GADE>EN7O1kR)^<(ckUaIl0->pudgM&WsvcnOX81T zx=k)@5R<7tO+UVB;&l&TCjqpM0vuvI&P=jji?UMzUZy)A^jMvNZwYVhGpoVau4DON)a_q|m4Ehu zt|0`cI*pFYw>0rEGC2ogS344x#QwPZ3*ADdZzIJzKYF8By#nZfMrID zx{UV)0i^>|@pc&=ne09F&Jq307d4FHaX>(SB)4m#kE5K|w$#zFG_bxKubu3wm`CYG zqbseQh}_H<$e(ApOG@c>6Jh9i}zV&+Y20NuXcsK z)K*I=UUXfwmD-Iu;xmd*oA-(ETRccp$*Z$us8jZ+Nt>^Bt;o>xFjRZf^&g zC7&X~#70`sVfN{ImD#5!u^}MEJ3D-$T`kBqsDxq6QF*e73~-_djz%7s_2^*r>voA` z4=Hl>y#A4jAI?hY0z|?=1t&)74dwUuMTB>GuZLZ&e9u$hN=4Kv*n(6yVFNuj|F)QI~k^~S@ZBy5v&R7oinXAxwnG`${AMmWR zmc<=B9E+BHODXT_8lb;NDK$J*cNt9J!roA3u&Z^E3Le2(|GHZ<_GT|Or@&JJj{yIu zEC*MTbOMGjym{clY8}cNE6!m|a`&hv67f6ZYFT{wW$(OmGn9pjYC zq;XfyY@GBSY+@NpKJwCsM6lg1l_uPaztcshOep4N35jxRnPim);X?V^-KIEOV@57DA{dJ+3}xJ)>^ z8$0M`(&NG=e3e~4Arj;9$0yC-Fm)yVT^lgUW8`EGmGt;4nFT`Q|Ke$@7w_v z9e^JXlY4nDiZ5>IdzUnb+ptFkJrw>a|Dt?ZmCuNb8bJb9#os73^fFEqUmG6On|;bF@YveU-H_#T z=2`xCq2JbSIzhHQ+QNg@{cZV4c5q_XgT&KuqS0BvOY#EMq}(pmipbjK35vd{EWL~G zxc;54@)N0huWBP@PcmisSOJOLc$-MP4C|#2HzrnFjaP7^4v-5p5<3T6bDufmjAv_) zHr_NEU%)%@W}~g-iY>NsLmq1ISPOTI%idG4~fs(0xY+Ec(>Kv%20SUTKanC2+TP=(ByakJ z9syf>%k6RCf?VBSK>E)D;4&o(0)q{jACHS`g%E&rHfyOtMPKFD;n zqPHOLLa({^&()L@AoA?|tIJ3io3#AlcD?GW?{>?TWw-66X=mSQ^>cX+g`CJ&IlsjE z+l8X6qLDX!2X%_SH&lUUggSSSp-tHs%(*{*SR-4E(C#=RSQ57sUo4!ce$ zt;}@ncKQtSlYd8dJ~sqC#VA=-$|2YI?Y_`$9b>Ibx!(B>L+&B>@2^KmNJ{L(>n-{Q{rX$=J9n{^Oirwo8 zcLDVKnhUnpocDC4$(CDUY+?`vtK1%0Lb9E~A3DW9fPT631WO)v@o4PkxN4F8b^m|X zdw^E1`0uk_B;G#MSoXKd%*3I|VePnpV2|Z22k1}^!MSBY>1A-Kcnk31)TFjsFqUkm z2AyoLnh0C-P^m^|TfwsT`tVZ5_-Gr~D?^)Nc9LzzM@}Ndm^F=meFY9<0DQ~f$BQUs zRzpTQ09^5AI=TpH!8YBgPJq{)d^s9a&FdI9(Dl#7x=CC;oxgjp(rvqCbR{e!s#w8+ zc(2Ep+vJWvxY#B31$-yDFz;njem*mnCC}Zp1*v4XKxf8Py{N8xA@u*K`tEot`~QFA z6fGl_O$e3D>^Mk9ge2LcBu>ZP>li62d+(K%bwtNHX5q-@9FCpB!8x)Wd;G4W`@X-Q z-@iQ`r@G$jwVp$3&J~}K1k9OaDSW)oM>K@9ieDSA*4s2jIzTR!e9{=HmRk}I4P3)^ z(=>uYszrs2;am7TjRUBDGdxy69zZKtvQrOGt=p$#L1k3#zpY0;N)l5l;UadJvyE;* zRrywx8BkeDW}z7OeDRiCxXY01k4*j;!*G+bD}{JRO~y%Df`cFs)IwCRXYz6@Jdl4W z&L6CJbCbv?-IV!xv(k3zSfD1vsJbSmc)#~{ZM0e{_>!ltwvf(CnYymK|LzSyPZVhY z%pW(mi@#eZnLRtz(~sDA0rHit1KMkhi@$d}&A0FQ3MUU|yqHbiU08ZS@b=fe)~df}=l~S&fB-70`QqVA zMYO(*YhUPkiF3)G^rhUtc27XjmBH&RdZ*)9m<1%(!JrCe?C?RU8KG&vikzHh(7kvj zrO=vP`r(H{{S_5|A#Rr>9f%4$;w0%B78d`y631=nW z<{{~Ahm8ON>^Pm($}I1(Zu4h=dA(@6J%D7~2=ir{6-70#y%V;R;EVL$mIY+K#y>J_ zlC)B50hfA<$=@`dH{lH%UFZ{h)-lAHB)m_8{dJ2C$nP)a*7ytU*3L!>fHz{It}HG$ zpB&?oC^>g(I}(gG+*?Qj@Lm4PZh(&$mZn62RITuFXDf>!x`H+J)938nCF!9xb%5Os z^K0c?hquOUFJ=Q{b^7uWK=fF1k*mtqS!3pI_wCgIOl@<4fIa9$r?Sio#2rI{OU`{n3 z0`w*}b<8G`r>j(Pw((l-mr9@k7S6vy9B0G}yQ1IMNo+uv01q@e`T_sDQvs+i9gYv| z95Y4pMHzRXEWRH%M<-aNLP?_rfLNzT%DeGa%Z9Ws*CLd%cME(Il1B`^$G;ocZ&|Yr z>$%DuyqI@gj%wHJce_h(TUjkNIQD33l2F7wFmF2W6riWb1s7OL4VuM|eA?yeO7>yD zHXNWU$0Y0m=p>abkdVwX=XAX8JCg)#PqNpuspLOpx5o{JTdXJ zx^^(a;pxxzIP(LhUEj%Bj&@3pwZM%FfzA6R%^yGS&Pz2LH2|<7AqODNF`n#3ofvkV z>Kvbys#PlLAEB@SUdx2NQ|;+yz8^X#93 zwl#|Yi?FWtX}gYevn>_)ty*lEez1H!r|o+i`!+o&dq+2I;KC%Tjwial&`jm5xfWV# z5Ry6~OT-#^uwHaU4ov}kf^+2Ctf6c!>PEcVOn+KQ^6ar3K8rafp7$2lB@~a+h0pp* zUnGjm%nJPYGra3Et#4pfm0(lk)|+)fs5U!3M&X3%y8-2%*!7iQ&O^(Td6Iav^V&Bm zMn6I34luRVHxfX-9ni|h&H&=Hd*G9&K|lY6w{Y4C6w2cNzyNggP{9H;M0&g?f_z2G zNYFeIfCgVz3Y6jT`Bd`V-8t1a-hou;H`_F)Q6!(U6$@=lJ=V&$%1;lBPaS2~h3fe9DZBx=OK4qwzWIsk7t}{AdL1 zT+SYO*Y79dxK7rTxkZ&iKcx7EUS??YgPaR zrZsG3|FRi9gion>)uOCtURmI6Tn;=8BQLXpbs`Sm*GvFn;zmSAqW!z!jvXz?hrbCN zHTitkBqL?jh;nj(h<2(^2h}H*BaS66oqrc2gZE6^-p0hiygHU z3d(@_!HXs>B2sX0bqUR~dV44xs%Heqb6q0bbnshV<}#kx5PVvv2?C zqxaUo(fT{(MExd_VIY+oN1a(mV_DV_C)>IliY@YUl8(r;isLFwK_K$8obX5?)@{MYy^gu2oknF8J3hDdw-{5IJU~m)Q;{A<)K5127*3%=#3^DM>SW5M@crj8+E|{cGY`(|q?>tC zw>2fq)<+NHbQ#qIt^zm)iHOa{m!3x`8DKSmN|-%pGWC1Ya^_Bh&7MAhJgGqfy%f)v zb&WdO&5q2z=-&ZWMXS$bPIS4W9e&N`V}`uR3|O`(vMnZZ z3-j=i%doXB7~+=-24K%fxfiLjji#ol*8mBYQ#dA|R&Q)kZ$}P21I@u+T-T-HY;=2& z@C?ztPt0JnZ*c|qD?IvaLx7y{VZioJ&|+dq@qt50=Hw&3=no=SE?-x_Ngu8B{TJ)? zPZSgs-#%yGJbzW`3g{M$`SA1Q6z=?sZ$cj4pu7Bl`pHLb%jECZ8-#CfK7DkKo|;+R z#r24AuuH@RtRnB#*L&{ZUA1>)PV}p?s%>rwRnT*U% zXJ+!OO60l>1jGWwt4F1kyUKlBOs!FfM}yB!SFf6dyf+4>IFNpEqVZ$RaDX~<@JF6CD@43e7+IWSw}#&2X0^!%v7T@%lqak7qou{9!ucInD>=%q0w-O23c zbm#Se*=g8W9VPxdItJ`9c>;CBx42%6-#rAq;i!EW>C+iw#c6*lX_xzfKL@v-2TPs- z>$Rc|Z-X$hS&6f+qt^M1JbXkaUlvj=jpB3a=d8D%?p(C`N;_CBK2YXAF@)$A|JBwGLDOJ1ytW}TMceY2QX8Z2B=#pId2;k387qFj3c{> z1(?=C4-q`z8%aHsTPoe019|Vhiu9d>sT&5X=+QkBg+=aeeQU2KrN2Q!pe+|Ex5 zP23=?&}U{>puhq2_IZtU)#@4(G{roAiRCp6<74o|9FfCrDn)Tz__7@)zw_x#6TR>6 zOv3wr*K`w@OEP6);uj5d5S|Os*O$Izp25pRH?brqvUgMc7B`d45%z6g`2I~@I_N4C zE@G9z+Y9hY|KnWP-?yX+{qaA4a@srrzJEZ|%si9O{(2~<+KqJp|WCgluMc2`Mm^aT5Dy|2OC@Bi?We$0QVXN+xAu+7BVJOY@ZgY0?}hrOF1ZSFswSdS zLz{~pnbHa*4bSf?e%-S6(=*v3W96{kk4EL|owTKOVKr^oUxhjEUzgwG4O?*2U~nw< z6!O`p=QLI`qCH$dC&^UD`6fKR&*Bj? zq9>pFt9O5x&gM2X42!kuQOlriTmHL4j}_i(P;Nw;jlCH!RHSH-XHE{s*90;-wj~7 zvj5Z_NX@qc5d}YRNfzb{5RIhcR4BX%-n37U_1XO$YEVRZr`~;y#3dVs0p>}6!kRpr zHZz6+caJAqS%XWq^-KRl_9Q_D!t@$%UX2odJhCV9)c;k?vZ0owQ`8KoM?o{}#UK<)d37nKu@^AY;|1I48Ty z6h07>4fk(D3z*4tOgSY_6=h~u8vx8f{LUdpNG4uhO2=dQP~d<=-+jB%r*6KYnLjUb+LdtRiK zokj44bFak(ya@<$q=@SGa`D%I-&y+a@c{BLK8kyK&iDO+PHdCxLb_{l+Rv>5_mieW zoUlQZR~t_;i9%752{0=37cxA}i9ka3#UpRUXQW>8h0{L1(7+}O=Hb3Is3tWYt>{g% zl`AGOZ}v&HD)MVVw57*$I>&hu{kJcZcsTm2zz>T_y}YX66zm?rrYZCRnzqJ(vkL?g zJNy5gV`k76g_KLPd-BLQQBSmcm;QbiX)sCr zz=uxe0Xv=nHTwV`_egen*}=1(AYielLF7@`;2CpK0Cwo9R9&Z5F{wiREM1yh!kU&g zeKDqEyo{-_*)I06V!>nr!>&+_H zJXK?nRj0E2WO}c4dWSWEkHdcaCHm3keZ>*4AIG5$4R(COO4d2bA>wC$pJxeW3ERBi zQ#J(x&B%?pWpghVqUvf>$-tvyzgorNljB)>a&rW-sY%g`sRr0;}2n&};;Z*IE=pAK= zi^oWZPA|*{{$%vo=Yh*%GuIee{R`m9Of!oPn)?RAUmAWd8I6 zen_&%i)so!76!X6zevL)&Wm~JC&#NN;9q3dL+U+O;X=s8PXrSCS&-9n(m4VLJ{aqV z2YdoN!2F#`dI45$D}mIiAR-Fo%X~J|rcK#_p&{|PJk2GSfmLF80E`HXUKWODId*CZ zZ!QRZrac2*_3kg>AbwUGrXo`~UXL2FkjsOSdk*Eo3N6q|na!TNOMs7CR^bWJ;zZvO74VzUKgZ#o_1bVV$&P1#*y3my**m0Z zRPIkKDS^W43ikI$n$6k$*1iz~;7tHct>|D6;A=O5$vVmJH~D!1zJWtjjKp`}mfU*I z|CcIR^3b;=1#Vr|s;uQW4A00+EZN8}Jmy0ygQfclXJXq2m)alb8DUw2a;Qa~%?9gn zByP-@(Q5tXo%Oao>MHb2-rtX1VP9eBL^>j8U`v&a^h|HaHf3^XoplgrRuYn%J{}-+ z4oKNz%RJyY&___Iny;^>^bh0Ctt&^cRox{re2SvOpeJpqbr4bf3H*47Z=9zP zuP4Z3Z!;cMF$0mm#R2g#b~NyLt)QA-b%OkfW#Xm18QkjVcV>0jF~Ile{)H6btq)GS zyLA`Yk^p!!@-)n40Ng`jP-AWs7^xj6cv$F9C@kie`+aVB0eG>I4QT-TPV(WM9GuH=6M*i9uxe|x|6J4Ow8>FyIE3iIa0QdOFp<@?iD z)Mz38!+runFg^K+Voq5P1UyJUX$i2z!(ZjM$9!;Rp*)Ikp0g$pz5^hy(+QJRmgU8JPPni)r)G}wzsPAn!)s?g?%M>A%!;3> zu{l3D;%y<>2*LqlK!YLS-;2||%>j>NltV~e%JzPyO-wu6TM~`*R%L-OU=Tap>)>%$ zrT5<=9^l$w755lp)KXMwt5{3YJIQMb&f?!$%785ymT?hcD&7^f;rn zdZ`2Z>B4JA>UP>^<)Y)P7i2!|8B1G>f|yLSgr(525(aI1$~*VFFb$C;B7_mOh~uw` z+j;A~1bZV#aoQKX-5$LuTBg2~&WC|~b){#jySTDNQ#GL&=c&27OXJ@MW3IVZ+6y?U zvp^AV4%OP*4+sg4>b~j!Y$ae$!2SetZ*o+iEY0cg9+r8}yOe7}+wyvN>X#O^=o5Y! zoR)WCapBRY$$M`L#J4JSaN^8B@BoAO1Aiz_;yeIYxcoh!z zmC~C+NTz4QH1GZS42j8qBMGU`aj8Vo*1kdTl3Pe#4X!i)VNHsG6}a*HLQA>rdC?55 zJjvKie2xk7A+Q^Z&#fAj9{Q&;W)WGNMz~T>jC^3tjF2>&`+wU4U&sQ)Fc*gwkHlI*&E+3qyjSeY; zID<5thknxbxp$0oze^4gloSA8ILY_;>(V#w8uH`@yzh~%#kpceUY~gLED#pSAI${GH*F-|m7DL#Z#_4tB%LU(Y^x3ce}d-Dx#m6e-y^`L|pI zyw*AQkK1a$6uJWR_btNg2mZ`a+W-JTF%0qBqnY`JuIS(}A@zQV2J58CZ9mM5?Y;7R zi4RFV0lM(Ed~a$|RjPP*%*_AE&h+u}G@&&>*t=7+7=Sem;qbKHGyrQ?_B<$W^KRqN zh`x};!ct|9Ul*?BC&_M8;YyC3KOtH8P-*lrww)2^J?$ws%Orcvr`TYRP??m$bGX8H zBxsZYo5x{F)?NzC`_AcmRLx8-napi}M^Ae{8Hp(6T8GOK;_B#IvIdj|ydH|YR<+ci z&Z9kx@k;L~9CE0!CQ`TUX}MYq;a>iEq-r?~Y$BdlEmCc&cw&AZ!0vLj$n{_7+`^T) zt=aQ!Cloneo!x)k7ccq|+64D>=uL8nE0;Nn3R=208~g3P%ZE$VVbPfuwnq$VI*(0F zWpO5X4h#RzbVaU^V|mq=Vx(nhkQJG#Sx?prARah>ERec8#;`x%3bUqpV2Ks)+i(w>pTE0*2-feNqqpyS=;45#ns zu389$OW*7TdU`><_Z8!iJ_!?(G1rI?xdIjMNA-sAm+OW)X>h@^qKjL6BtG)ij0W4Z zvP&d1POhT~*_#$cw(05jnbF!@nn8*==m}^I_T;4ShNHtuxa|BOliK#A@tacnRJsAL znS5PF(E=h`*-8$T%Lbc)yJR?kB+rn~TOB@MDcq%a=Dbp4&)3V z*ysN=#SPiWdq>i~GpgKJNPB(hE!4~uk*U>1%eT8~=KB)sw;YsKd-z7{JO6SRimpLR z8-td{6^E-}(2}zM7%Oc8>gH~0G*gGE%{A>>v=k(-_h%#&(o(S3{@x@QW_MdEQTGMI zvgS*g?F=76(SiSE>9Sd1U|2(Yl0XJYOSqm*^1{|*#q6y>Sb&A|YK6C$c6Yf^Opa3>R=Nl{Qm4Vg z0-Y%9)s0JLk%DG^&i%Ws1zLrEGeg04x78HA&MlzReH;p^blBvToYy}Km$jabL4x$J zXYYMLMz`mtV+2!w*7P8sAAhAKKMWh{7){!RaS^!?eJUKwf~#uTZUH^=EwcUYi!%qB z!f|sG-*9(h?-H!!x;E@XMxW3S|Eo&wTp-8xuwT0+6W3jA%Z}S{ZF1!?3nis(4zi3l zd0~6yy?f!_PhSn?1Gmpx<4?h>uU#Gs`iiPv64FmEGIKasT}+|3hti zug^f|-&IYKR)Qx=IK$PybMoNNHY+)ur5$u~N9hLq9B(DZ`JJNH>WQ#G)YDrOa(CYg zE8n)?Bk*jVKUv@kYAYs*w6pUYfJnBgqy7q;2wxs~F`n`TWVHC!B6I z6VI%QmuwOhA5xhzwHB2+U+P~@sbR$}ugGue`st>0@me{u(uIfae=7C07h;c~i+e25YhZ7Q<7s{D*-0AycEA90n$qbcKOEg?xFJojsLuo^=bO?2QOu|~- z;}PKni8nEKK(aFy_cyq_k%Y%7?pA;VbPsiqkOEzR;*vf?ma(T^bJTXeY&&ihG4|t` z-`LTxX<=15^sNvoW-#W}-w6-!R7!yi_4p6YD6&v}=)YuI<9Pr`KqEc`tLvq1xqcaI z5?JCOrBts$TY+ToK!xiv8=q_XzYqj)``92wznkr?H)ni4GvVA3z3xEwI$BL}#~v(E z9sxBgN*C@l=J~=%-uRU8J-D$5W!F`;lD7{R%K)2!#D%upFVC~OZIl3BEb>#R^3T>R zt98M!jc?C};$g#1uou6O?b0rs?9~PBfXN`<7HiK*9BWVo zeT(0uo~ylwvX20?hcUoa zir(TjqD@45nKBkm3)Bn%ks*hAk;^sf(AeN$4UC)cxJTE*h{^ea=79F1Yps_xtnnJ% zh$$~>5q_r@ozTM@t7Bhn8Z=%RKOqE&YRfEk><$MtgPW4J#kyx&N-J&7*QI4#&#n{G ztx-+GdZW%lWbeEW??d>3+$`RBTfB%IG|gM5bBD0ty5Wvy6Lz$(zDu}>=*o310*|ON zhCuPRURi^I5Ds~!P|vhX8FgO)te>Axt)Jf>#Rk>IT_;d2%`f#u?lNBET-WrakPK`d zu#5jZ>Z;iQ|G?}X?C#i4;M0~>13yq+p1vXO)R41#y-+-}`>UPVl;Mdeg}{SA;lMRI zC0d>vVRwbEC)s^87_~F724N!uIeNaOy)09LI+n7MZmS*r0OdTGeSqaB%4h`k_%m(s zgiXo19@-UEg#CWA5Sq9BgxA`k{?hJ}QK-Q`day-5}kIzd9O{ zG{Zb%EABD}>2qaQm>6@CK=oqj>uIy-7zbl8MOY6{k25IZTxU_{*5|l}!=E^4wtV#X zAWC!jscIib#@{Rg6jb}{byK7qJVkdpA!@5a9jb^)0*!GFXfM-7SJ^y1=y=^NnUuRk zTfm#=F0c|Y*j@`S!}i9y0O8}bc{CA02JNH?i(!mf=Ad3_1e#{P^f%r63g0HpJMy`y zvr=`=nBnt1v~Zt)CTph@5V`22GS+i~CAVcV2`E3N3h)inuk?l-6h3qe7B@Aql&rOX z|6}?f@hBoB%lvvCn*ulCkJ^@!yRRKH|63vS=nzsvj^8uk>pA?dd`1r+BgWY@ukyjd zYBD}T4eD(A(P!z|chVkg^PBz5j^C$#fiEfr`o0l)D{kY;`S4nyT;~kZF?0wgcSKM3 z^Vf8exP_P`qUjT6@|-~`D%$O%)$|ox9icMEd-i`p7r~0@j2+PjL-P72z3W<3psEmP z7X8jv9=%jVlpr#I-n4_ITL-IDsuWl z^Uj!sEg5!i4Pw>r@RPPn*TqI;a582HB>?rwBXW^^p0#muAXvnGNGZ-#_b+o34MBloSX8N`6*kPApWG;v`R+SyokU{Wc4 zEw){Pf2+(U?oZ0OQa|Md~pp4Q(_WG zws~T|T&3zC&9QTS48i?J`$e8E62(meY4ZbM=Um7X*FiGMjYch_~+-$2d9$}_w zm#)ORhZ{(P!!(P%#OzM~bk_zZubXK-;Vzs$cWm2vr-$@~w?C$!5}}WBNz;Zn zkpgWA5HFhNU$>{whWsd54T1tn?RX@-F=AUDVrrG=cvQ_9qS(NB5TEV-Y{YEIHO?Fl zBYY_Xeh8eNJ5=QJFFDCfX{v>W|F&GP;enFUyuAbSdfsih9L%pZlsP|K(lu#GbX%94 z;H#JY3XGBjRRevMX?bQY9PoLE<+zNTwj{nP1=g8S1!}PLYMwbn{BU)7XjDCB2V&yi zz}NfS=2Ql&L~3?;g*^|a^mp!8GPxRx31xDO&CTdqBhpvPHPMfP6J(@LzBeExC4PCd*+hSja0_krSwMcVe{^QO-ekFHD#{B1}Y@>3WG z?<2UHUPh<8GzKo}!R<$a6&);sB76vtj5cbrLp@O zc4QT8a3DdxGnSC}p%Dl=L3>;+^4ig}f=2sHg`Pf->4}3L{?xVYGN9UmXs^Cky6kC{ zwW4}*E`-OXY4_UObG3#9$H5EJQGrgw7wq8gfFyb%3lsbI<^p<_*IRs4Z|3Y)l20dJ zH@lq_Ut=%nyn5Z@?l5lV*A3WYoI2HltkY+m5v36oKYhaz!67x@acYGN3dMq!da~dN z!c3rr+#I7dK?H~lUjWKu$IQ>9NmBGi5qv85Bq07AXgw@XnURc)R@gchQKo+bP=7z< zu4?x+c41g(a?mizyVU1v2C8ksH;$`?mXSUY$CWKle~>Ksai?PYjdO&SyS=rHJqxw1 zKW_XI?_MV!*~M}}qO5Ar-*&36#u(&QW>vPx?cQEt|MejT`6o1QVIw;%9;%k*W>@%F z)qu4C-B;~#INdChHJ3o2yeuM<7&wA0Cdq>djzLtjRTVE({8FD?)O0IBbXFCW9VheE zedwq>82^ze_GscA*$DWo59h3wi$(qN{o%%p%M>iFc1;~O_GH(r--=EkS1Lz`HhzJA z-VlII*}EV8J=Xwk;td&;Cui9vd2FPL zM=k3KjbqV=zlxjOaNlYYrW0MzOi2wFt?a%_XiK~T*r!NfuYBp5Dx>By|HSnhI#N-j zmV|C4x&sz{F>x3K6X%nx&C+Y+@DZlGI{uBDQ1eehoxFbG)nz5&Uc%Qs+h^sg;ga>{ zz)eA^T!%O3A8boA+Vc_wEiLbW{Ezk??q_$(A=&{LVBb($Km>l^{-PW1%1iiZ!o1xeDkbx?v zKpfvusp}o)gv*#n$e$9W#(a(|4ZTRn-EzuTF+;@J0ST+348^2Qw3m0Em}VJdJdZv~ zY@G>-?s~mk{p8+f;j`wuD6jZ>&8mM~;7Nr8H3f{u{CE9g`y2CT-*dGqYY#KrEa+zu zob?5PWp=NhYX#V5?Ux+q`)O&VJ*q;6yneiOGGmD8XPezw{@GYR$ao_|><1Jms_$Q4 zsrghY%I>a=lllF-+JTQ^yq*@Hfqm+{cV_LTfSD0vz5PJ6a-3S$YLrPy6gzeyI z5~>2P%o4WtbmiktQAhRZF4HZ3<$MwGvM&N^BLuJOqSm<($*eL?O!bCZ zWY2D}z|B76V!CAnsw+|)kMnUt#kGP`fWC4@MkY!1=hG~dH3Jctt3wtGALaY*7JzN1wVTi3 zjZ;z9BkgH~ucWeuc)dm5q=}`hFai#w8bQq5bW->qkBgz08ldN>Zy>ow8#lvE`+dAP z0`?L1-urC~x_x5fwpR5m10TJyL2$wcQc4Eo@jWu*ULQR1E@gw7rcDI&gLqcu$pIB3 zVz*?0N{ZXRNT48lMMdEXoQ~A`<@%vlc87*p?$V_tc865^pOVh|tOsgWN+Waf#OBKB zbe7nRw+rUtrt_6OkRiL~u_MC<5%h2)#-hdXRffc2S*#II4#LEkU9X+ajXmedqS_;DKtwmC1y=t7@4Uy^vnqcb8h9!q#3 zr^Wwwavcs>bY04MD<1W+!~Jf8#Lg23*=dwa%odYcq-^Ic$4Wh;L30;97JR`lz$->2 zR}uDw(xS#*C0Uc0TPyCIWRr6Of12#iP)Iww%yH}Bn5dX$bouKpi2L8ZId32{W42c# z!M;FsIZKR&6kDmLgjh})uQbOp+3ezaFRmRuxWDN?8k-OQP%~O`D0Hz zP6`&AgDaX>5U)_aIhqAB_{PD4uzght1p2@$e)>B_APYLmS z(Exo4=8?f6_=o9N(Fx<94-L)Z>l}d}8Bss#Oc|8tfrABvBF$aqWDX=vHg21F_twyo zuaHw5>t3{%osyb(lelts-(*82na`9P`@JSYDqkYOkyO6|FHk85c^SRawxFm`-NW)N zwjB;z;z%sZH#4gJC-9~B>|hLF>tISXGwQ;w2j5T++v~h*5UY^e>SvkLYRC5FhQoMN zAnJ@&?%TN&6;oF+kEI(C!J^Vf*-->N<9XQwJn}Rn2e89?PK&>ls!Wd6wp2b%CUDs6 z&m+C<7?uq<+a5+qmVVPP@tjzAqOhwpV^vcR>D7S++VWfE^zs!%Oub41h&k^S8ysx| z8q}W105HrL;le27xaHXZ#Iro;hZz~|OzYl=yMaAgrd$-^%|&#s=i+~!pjBW+aPg?F zrPc}wds>QnEZGwn>m6~k_ZVsFgvI^QAJ62lTxMMpcH`)$+G$#;*EXX`Z8Ts|^=s`1 z-pD;HnNeJ7ZcfrHRi|1;yZL1^0qo!dV^P?Z6&+Y*>WK83ON;D%x?kw~VzAew!2@6Q zXx6@3?_2xw-*N`fnrNQsX^l%(=geC_V%$%b}>{W{spIw>l*if5)(Ru^8V zZxT`8V=);jsU1%!xCH6`F=!$2rP5_0KjLxiwv(-OvG zf+dIxP%m9j-v~}&&52)dE9x~|H#J-2E`om!b1{&uHj^~N^IFG`Th`dUaA3#r2L`N! zPcIE)H_g}h%eHyr{6Abuh#yTDjZYl5PAU-?&R;JO1O5j-U8HOexYs}!zq8MKA;9~? zRASGxXfzKh?PbDbJO}Z%p7(qU@%trs_EK)G-GHW4Xj5Ip?DZC=1@b-aD;-bS)PMpW zE8dqE`WK#T#YH#b&?#pcYx;fSPsi)R?i4VE>`;f7;;TQ4B7n0<`B%o$ET6gocf)88 zbX2UOg60QFIydQX33FljUu{|?DvBc@p=kj^bdG7NTVA2JxTi+>l7O~(=V@_ET4i9g zX()yK+~)Z0Z1Y76t$xV?$kZnqtk8_{!h4?Fu zOz0K8`<`Dj75|bygAfW&Jz81%N$8E_QKBU`7NAsBLQ;FPo zHgb5LtshUyqr$jbM&GnC3YH5tA3m|u@Iy=1iwj0{Ak|G6C5Q~1RgT7~HfAuF#)%Js z+xp7B(+I%mr)T}C6vAQQ6rK8ro_zbpy+@IH&c+4Qu@nE)%%`l&*1#qhMpH&!+}VTNw3sea1y}ni zuj1>hmbP^$$zhFAp;A6YY}P0^q`dsYbms-vy<(>`dwDXYEp(6C>lV^x>JD8oogH-A zmcekxWRvF5>Y6A=szlK9iM9#U?0|=C*59}GqO#&j(b^<9K`1|rF3HLqCXQS-aKkLL!Hp0UWTs5sAw&!O|;gi{igWJxHomY)aF zzSi_;|5+gL_;?Mp&nmwg^7!}} z(%PvGge#!l8i}tgONh_=7Cfu z`5r~M)th<$MlY;RvomM%Hg}6`yE5zyR$^{9u2H$-lN}7QkR43(S}+9K<~~^U`eT&y zvaNxTtrq{KH)amBtsG<+GWRF6v+sLd+|7J(=3c)Hbs7Y?BWJB={!+{s~Wl<4M_ z99oIJ{h>?kl3Rl+6NcK_s=!~H78tlvm;=qk@~pd+eO$i@G&`q3rc+eFlYqmZ!DQeQbE}EYmg()4p}nbF2UlJW-Y@)FNOB)sA8{=rhySPmg@2cMa`b zWq=3n5ob_Fovbt_6#Ji#kkhY#vz7CgN9neHqnzcbZ8s$g%7;9yu-2=gu}jRXX@&w; z5h8DA{6f+%r0%>Jk4Da?tI;~P;lt*B94rF0p1EF^1n|xe02r=PbHy;%W-bBC(`8ZI z-)z|T;S5bww>Se4d?R3cNl(rOQ(%IaVo{i3QCoXzju2YN5~k?kH?z3*bQQpgb6BH% z!n=Q-Qfz<`76O&OWGWQd_YyQt7fn0z>h6t*s-$)PERLU7KdX7C+EwVL{Fr!vd^R&k zs}}NUnjMp%hM#yl@${hB`3`v40}YpMdJmO*cug%U&J}5!2K3*?H!Oi_>b|1YQ!9i0 z967VN8RM(CAvh2MNXb0re&9mbJ7c4JQ>DfF#7bnN(SnQkO^Lv=M z*@}p%D0Ds-Rf*2Cu*ovr@d3ovt8Dr+u^~=b?b)qC==^o;_jEvHw*|{XHUT3itqKR0@{OJ60Z0I|6z!M8of26d&I_jAi+ef7FZy;z~ zqIN3p37R=l^}U*OC(^vqD+w-OhI_zajf+A)<#R~oqcCmuy42xdjl3o(0QRUi%cLKD z09yOzN9M#(UV3o)E|1;^&9!|eeo;c^tP8#!G|ObBV3`&7Ev1TEC;v3Z*Y^s3%l_44 zWE~Gsj_qC1G|dyoC-A{oqV{gIVnRC?T{fHT9^O687=U^z-(5Y#{gIgIX7u2A6q&1# ze+WhBFg4o>&)od$g84rLGJ_NBDS2<@1iiyaJR8EM<54h%$K)0bQXq! zZR0e%5hU$BjEbV`TWnOdG=$WV-{-n+QELHrfi6QnQ+<@{s%U$vlYVdyTpq)=X+#Q^ zY#_GzV}pTB|_idO^T*Ta*&mG*Cg@ZTGGqC!5Ie_X%D^WY5RG7m-6rye-J`7(FKbc z@(6&;YWC2dj~A+YyHgQW{e!fAfJz#;t(sx8;=Q-g4I|RT-I@>*uf?Hj$AeUvy3V-n9=RSBhjwj<8B(znc^rnjK}rI3bVNHI=EM3G^10Rqztj;kj^q= zAQ&WL6oy&|ePdz=B#oS)WC15hMqWSt#z1v%lTnJUKnu+nOL0to0yo)MdM#81P*$y#{V8o&R zycI`~ct651#1oBGg{iPxSs0k%xaR~>SrSB#v&@7cwiw6Gru|R%0l*~6oa>(7e9%!w zI=NlcKW>F2{n+BQ5$G@jmwCM|N#gKfzVGex@Qt9D+<4JNx7bdbaSo8%6f>N;S#}vIj(?w2y)FFKb`+j(sdgt|AqFBcpiLwCk@d{7ho=M#T7sAB(dS;>?$3 znB42amsJ&)j7M^mu-a2!XOKz1by3)tAVo#1T+k(U(4#ukXK6eDnKA&02Y5L5=ejn2 zci4GKADz`_{ALoBhW=p#j8)9i);0T*KJQ)4JnNE&toWnv(WH+(4c3yhEs2T0?x~LoFzm#cfz*vt4IGs4Uro8f#vmIg%+&+o$A_w zU-z6NpO-u{E@JZ@025G4V9;EsPIsxDwcMtuh%cibTZldtI z&tDZ;?c^v}IP=+uljDW+y^*)fjp56H>u0Zm*`#k>F9+4iR4uhFXUkz2+rsbI-P$r| zh^i%#dXd8r*1NYbXWR;KF(C>bTGcSj`tmaZr%{%=abB18qHMM@;4Tagx!5F`g8($dWcQ9?=yX%LA?cc&5~B%~W;)Hb>q z_1;6zInVFCuKnjZ!0!6iCsGD7-EWm=Q-Kf$28U&3H^(Dlm>8eE#o50BI%|PEkk&&bG|UOq zkfeB_RNlO>Xv&BKgHiFj(=567^_j;(*B*Sji-jp>;03(W)^_Z(%!K$Tl zZ$NDDA}CY6nQ9rKlY8me{UxVgF7h6!VmTpV|4xr{AN%gYAvpRs#|z*@4=&*f{3PwE zqTXuPfIjV7uquskcXUxKq?!kD{0eChl`x>>1!%(DIP*0}wA%X0`fC8N0`$fu07-$w z2G@~*D{uDl2e|@@`TlG9y@$gQQguC<`QHmexXj($X`C(U%Je6k8R;cO;kzXdVm}Wo ztXTN1tsmUg6CswMfB0rx9+BStI%;6hXYGAf6F(33N7E8iojKXMKnq=P4Xmp06e}3T7lZtlPs!YVrN>TK z=yUl6gw_{eVv`F_gEdmm3+lTkzfR!g1CJz}YChY|Ww{Vk6@(5|_=aIKzHTY0eGfg` zx#RoQ!4F3gLZv?V4$pxucKJn_#t%G-VyJn&%)?4y+7sd#P3i-0wy)(@^U*~4|$;UZps9vXZNTla@H0&p}2}IO07Cm2UHD4274o2 zmaK2F6>`{W7|dh0^t>JW;sTxrTOW*bn9H!?I}<#1L5)SH0*WeL1!*WhV*IlgQ*gas zTa@ABw5U7^6%U@FLD8{Kf)wqs1N%e#LyeYD%$5knl5QMu0>?abp@b;={b`!h-)*Xs znq{d|lJ{2L@%)IGq(Ly5j=vTqHCl;2(D;&O&qfPnbK1&`?ztnR{b66_K6{V2DKFl{ zJKwcc>JcycaO}$dX)RIVZQX30uomRukkE*FpyK1WRpV4FZc=GXUYHNk(h2~v^)G>G zF*1IAFI|^^02J_0K^9EA#*%}a-7p~vX$Regoor=m*J#8Ox?MHS;_njUGS#*^%I>jh zD&uP}cbd+@}5g|=kG$#ZInhas7$EbJYD&`M-_SdF{jsT7qH3&>ln*u6+NBICfyXcuaa^G_p-uc-9(a2gefV$t8KO!52WLMSjHJ9I+cwX0|GE* zHEnLGE?!s;h6I+ss_HpDK+MYZc@s;x6FCO{U-B$l2_tn`B0w!G7i;4?bOmzo1NHM^ zZ(VqI%?(j+p$5)nga5qT&sUEd&*W~CY;r6+G&nZOdWFctn3FTvZE!z3umbt$?R+~M zJB8|QiNfxO5IIq&iN`e(Op?ijvMYfD6b?MJ3qfgZ(q5nPV@Lac-CB8SN@OSAr1xlq zIV?8j^*Iy*%LZv?%&Z@%AsP{3GPolLieh1c4YT=~kDQY7i2LD8W5$dSCB&`iENsIg zD@cmEIdkjA(eHl;HRjUv63riY-2o`k#^M)eXQsQBMeMK)wc><$0H^kAgZOjk&Memd z(;3h%29H{?{)Iyffz;|)7qq$5dt)Y%-oM{d4A%g?(v)%<43;ar3-ze1oyzaQe~0a$ z>p=HPQ}eNQtAoEIl{m8-clNO+pfx~FoPErpYNLf*HBVAx16cA_)qKD9<{1<-kXoQ3 zaX60}HHpYCZ0j=IE2%M9fgTEd$C_Lw3>5g2NK^sFauiAokctI!5fPH!*R|~e71|Qf z3i?+BfBf$q$2NnR7@9)v1l33ENrG5D$c&0dex(q-LSZj?O1ai2}?CR zXNO6g9rkKtExUK$Qq+CHG)o>gV|-@;{dWL`b@DtXPK)eMI_6?YXW1&Et7RzQQT@&E zc#c9F>-@d@jzuhuGu8k7Il~@22w^uNIP*Ba*H5O`pEqt&HLAh39UHJbaZB+);s~L+ zAP=riiGxbOPs>Gwixx;wy8#5t^Io6RZ8ZLSSdxR{ZY!6!{dGc|vEBom*@iTe@BjZ+ z$dDODrBT>#2vl=~xi~FwmFtwLnk+2+P9%;&%2<-GZ#y{W?b&z;lUddGEhf;TYe*wV z6atNYR4quzLmp(=TMN8jrZP4&Gn=6}&PY#992`-ue!Uim?Vo~%{^FXxZVsSn?LJ3a zsH84BdqIiX39parBG4V}A(%%YJ5RkSS6P`>Ddug1y67~hIcX-#K(`S6Q!L9Z`WM;$ z?o6K5`ZN}RKuz!!fUXw?lTbH$!5m<`lQLmg)0Hmu?=FEDErAD?TT33S@h8 zRm67(@$b6>yn)mjnE#Amqo(jJ8WdLlJ?Pwc_@sSp z@A5o?ao5?GaRy;V1ie(Y7K+u9G04Q8{49Kp++543A) zI8}1n3Pfi#to|V{hDJ1H#v1C{v#z%@M&tp>fr-^4REi~FtcRcqF{{9SPHU4%YF@ix zD`p5Q=rsgcIRa07D+=hizTB-DsV_MFF#{f+eZZCu-2XzZB9w;Nt86srs;p||u#EQ) zyxVa1+fL%-Jo7GX0{&aC+jMtVwSeB01Xbt2=`&BcnDRFbrE3?RhI(0UW9?j>6#PjbL9e;Ym6 z>hdJsdE~z!8|sfEiSANGp90+n=x!z#G~46%708SAVoedq*-;GSFb2~^M|ftqoiL(F zJ&h=u0bav(*a(?NEP1?v+Klk=+68C4TI6V@6z%`+xU6vRCeR1)L=|471%kRmz>8Q? z8_Sc;DfC)k&zZyN5mA*OzOc0*xttNDl7ppDYGLDY4mJRIz(uFk&aAmDMkhhckx3#@ z#^BI5ZXRQG5r7`Zboq^<13pA45E;ReaIGGsKH6C^ldPq+6Nar}$C=^phsx=02xPw-FIMH&euEm6VT#p1x!&ByzE2GN{UKWSh z2D)!_3xV)J)UfEjklV84tqIHz>TeVrDu5~8P9Za#B$L~0M3$ZUZ zKKCq(AC~)HUW3)P-Yl&p%g+W*kfBeJ)hOtFa$goH6Ls(e{*b z-Cbtl=cRgsC(_|zPqTV#=TnUCIacfo89Z=!4%*#z8aDT(7s@sZN)k{-0Lyeg+&RB0#2~T;h3fw-w}QTm<9Y2{^bE75$JJCE`yf%I`PP}SxRFL-Y|h-~d-g3#f>V7BJVJp9 z)(U^jP)c-V%x}Jbemojj5x%exoj=Mry?i#?G=8ZOlucHgiE*$pTEOLFCfFF6e<(OM zzjF{R+!z`A{9xH`Es&4%Ok6w5S=NY?Xt^P+K2l>k`qr)*vrnj;a8~iHxPK2X_LyN9 znj{kNeZ>41XA^_)p?6>Pr#H4TNU)-Wf3na(*8O)L_(B51DzMr+GQLg+B=Lo0AFzt) zT3E$&ngu0aw&HaUP$$NVAIFXt+*R>(S)RYKtQO&+c~ae02wk<>O%rb@bIsM8vV;sA zWP|z=WcDHsN%T%`rLRYjf6)za;mHBc9KIw;QFAh7~(OGYG`e9JlHw+f=*8N@;?2-5A+GpD6Z|z=MpNAtuHbJMM(|wICSCSfx8D5kcK+= z$XUu(G;9#|7;?H5tBn=&A;F&pn$O?DB{0kQDtd-MBrDek7T+u5eb@!j+B_0UeK&t^ z`F9|lp@CyJ|Itp3k&O+()4$`0r&PF$C?N=#K(Lw#1{CCnrdbr%SMz2;-%pDCSsWnC z;(1S?kbowETy+;JSS>yNkdFkdmC6;-C$}rUT8Oy@4=B3&wfmi0lQNev;!&09pm;fN z^~HBPrVisO1@t5;J2Oio*9g{_>g)urzy^3DeK3(Bh*fpES> zp1(XT0h-u3{^xLYJUwc*}MkL8^LkXKr~(K2-E@NP3)Z^Y&em zY{wY)cM(F)N^8wZ+xaE71zrDtxo1J&WM~;kcu~X6VlxgdmL-wo>Q6QKWqNsf?pEb! z;B^w~9uagNX2~6EnU4CvUhs`%&-h5GaYt#0?rf881hOUy>Q^+O2-X`Now~8W&sM{Cl0}dq$UUrOz&=ug;Z50L3)d=6?SiGGc%G0UD8K3^os{&rq^d2&BTb&ZS{gcN`8#^t^K@^njG$n5+}F`kx}6@8W8#Q$hFu)u!3xQM4KsUz*^=Tw-DU?dryy#pPijP<74fYW`1 zut};*WO-fcS14WQK7H|OvQ_FD7#{4GE9v!=4>?Cr>oAln>{`VCL9G8z7%iG2nnWn{ zerwoi)84!IdoF~lpf{_S(2uij@>39cgEo2AE)|rboH;Ug zl#Co{e9Yo|(nA`jLTMH@2I}q-?-~GM$=yeiTS1zqAepksMwMvNk_hyOZU4k@(-Un`SV7tsuh{-;h5l@MK28GSUq~%3y3hE4JC)BCEIZMPH8IgE^Ot83XXw zpS1(#3>#=?w_Igrj;V>f(y7E3f)e;gT=l+TK(8Us-yA;;4LhP_xuap=MRPJbFH)^~ z$)M$h5n zmeL^#sg71>HC~_(RM_>|iV}Y{Xp>pEJjCR&`zw41;VXTWGuoZY6;}0>Z5Trz6-=t3=q;q%<)+OEREETG5d)z(Xm-& z`J-k#>O!!gu`x5yM0OMv{ZS-Vmb}Dx^;@Vz!-cD76 zj|zkZ;<$Z2cDiR0X+M5LUKLutsf&y}KNtftzWg6ZeIm6*sz&mWhP4eGQFJ3f57EK? zaB73upd&7g<6V_RF0}vqr9_q4n^n0R0>n!!q4KKjF9iZc#ga zskNV8-k`tKBhrx*)+ky8bfNz^)1~;dUo6u_x+}{I8~)HTzPk(*qm%PH9=A)UwQN;a zJ-+8OtF;VBLt6&i-8@NAYuQEH+81hzbpzio#se`NKCMD~I(S1tuG8qB#1fEbXcxO^ zwn}^XVY$_qin*iWi=VIWU7r86PFejbzdQ6-Hu-)BFW$+XxX^Cy%#?&dvkbgsjpkT+ zJ{l%+m}%EBmH$8Fp1%h{L)u7=Y9X8d=juPSe?VD!Bt;SnXS4BC`qX=m_y()`aE3uVC5W0&n=03Z_w#Th!IQWKr5ETb|pRT!(}$Sj3m|QL>vzo=4epJ;N+b= z|1P7Zuj1WpW)nz|9S<%5TYaOG?O}>;55Ge;SZwMb-(ya<`{etQl5|0_Ce36<1_R@< z4*AeIX-kU8Kg zF$^|#wEv`x;70?A5HF^~ht(qT6vnZPV-FHk#-zL2%dMUoXw*756GiaZ44FRjDd2#P z7n|R%#%Sn&z6mFsF(&sl`sYTUFOj*dluDvo>B3@WyReF|xU%zyePydsTlP|pnY$MA z{JV1W(D=`7yN`7+Hg&0(^qM5$})+*Axz*!594q_YXvYZ~r<^L#j+i|Hia zXUC=Lhn{hI?Vw{{ zUQw5jP1Fzg^Sb|}^BXLDRZq&_kx`k+CaI53Te%q_5E&DY|0?EFSL%3n5~s@8eboiQ zh2T|r8^wW_2E|twATA=#i>PNO8~PsBRw`PwF;S(T3c6nZazfn(e%-E5r-uq6YxHF= zDI&&KGNKSwt+oFaSlFB7SItPzV&JuQ!e-GZ;K#;do1;<3k>B$A*ReGE%QWYBz2`nI z1U1mbQ%^*`ejkqP0*i_4^j59WH1xL8Z=NJI3MEx+c+j5krCKVfT8dYHYtk-G-BW2v zdu_X-$r%fj1fJ~XYeSD3+Z865>AW8nZg_aM(lTN`Zn2jywv-io=Do$w+?q~YZCjVy z)9-}QnLBIQdZ{-=SM@l7lM2WX6)6*OFY>RA(LH$Vv6fbSDW$VN)8nvNtx;YtX#|@` zaSH$oP}bP$`&+E+977fb`40mmM`0Gi$(>O)QOcGSIW^&wT|o|(lmlOr#T~6 z_@-)q_&M~^>g{{YhuKGzBZha)YJ(9sAVhf>^$#SYEu;gP6g}q(kU$l@ih_QMC1g*v zmm+TcepZdioK_#^#=2jiSef_M|5zH1!ZQnwTUr#r+Vfh!t9E$;G(ox@?w2^no@ znOYiR6Z1gjFfv{8rfK>2zsm0wRJdMhfEcyLGdukcI{!~`Mubbdw$T^(CHB#rvPTV! zQ05`Z(q6;e?&9}!YG*aN1rY?vmh6$bjtiSM&9dVY5A2G##>BACh+q|BD4PukFYv>V zb|J)N8Jo%}U0U|tLVQom?{+KyRIr%Ejariva$ofm8d-SNI7t1EWmr4bhMOopAs%B!n_ zBO8Z*h3&1`@bW@Lk3X3dWeQbI{1b)#WI{%nKnpI6R*S75mhTgL*hSR$VMQv@&CYkh z2Q>*@DT-V=&hi(h!0cHQ?Hng7Fz)~B4e>l_EN2W-BPXOnx<7^xu-AaY>t>#3Mm`5{ z{i2SF%Cur{w&*4@m#`GQC%?9%^!6t(Lr6GFu(JO^Hp)rB$7QkBxw-ndr9KQ`pW{3i z;B7)Re4KhaTWr5SE*j5m&WcxL0W8L;`V5d2Cx_oOVGJ*(jKDPPo(rLz^uDQD`ch0x z*|vn!e);D~WV&(=i!xfaOglxn&ZRhYM->_)F~q^B5>JYFv}*n1vdZn;~g2U?P&0uuXu zaY$af3#s&_-ExZjUGhtMGT^Hv|A-;i>BNdwylm}|OUGQ9bjbm)Q|?M3IzbSxhEDj! zgzTi*&-$8~d$79vt1y1XzFnqA4!n|WU)=dn308I=up@)K)k*RM+gYBTI+O7S1XYXr zAT9o-pc-MKYpGNh!e{FcM$+r8xY8V#H0mrtSD61}mxP z_nM&0J4jG(Psuoi)S`#fZ0(sxPDt;QR}?lGoN3M-PuuDf{3; zydt!(Z1F9Q3u_B!iIK)&tyOF=S zfhL0p-q0pWB5svS((?N%Yli(N1lk{IoB^z^se+MoKRtW$7R8%U)-PQh#Yu6hdvl}6 zhIewe6Wto&WvnKI**#z?$*~wqBXYs%e`f;LRCr>ATNF5zt2woe|MR^^xe0b=d7ys2 zhFsT|OK4>kVO}1T)y75EUamVv+FUBXmv`O*V2}W9^ljH?`|E?XO;vCCPQNA%WFn!6 z`x;5dptuC4W6$p*9@3sFnC5dCEC`)Q5tPva>7k{jccdmOj>-N0Jjn02u~L9Ucr}fP zvqluPjtxi#dLP~`O%=q>UDS;Tk$I5Sx2AK5w3W0uMkn?=R4i9XJ}IW|P(J)> zc(~NWI9;Y)C15`mBsf+cqjV0f|L5Be0~6jt8sP_M>mW41tNu4#95p1^X;LG(w>wK) z|Elj%xX6Nk;$j5O(e?U!H|1ayS!#hR(4WVJht!^5+${}k%qH3UE_T?|4TkeNRFpe( zotd9n9DSuXVHD@PfoL}@#E{(MCJ(o%zTKEdVe`nOE}t4^LyLDua$}M~awZmG;8+yh z7@h=xsOl$f9CsQsCiTgEzPcXYG5AUb{t~CgYmexYeO`(sJKl&$66r9}7025enazG+ zkOPZRwh)F!iF*{(7C^ywSJzuG77NZ$vN4IT=35Y4kC199^;t6(?zhh_ma~Xt>_s}( zR5%p>u^U+g)0ehUl5S`}#~s`Ia+vmknwNLBz9LOx)YORulm~|*vb@w96-@gIvv*it zPHHdSu4u#=rQo0VKXHcI*O&G+%1_w7!%qC*sdFcc(2?*4sy&a)sPsu++U?D6h>xd0P@;Ux%aYfd#-I;l3NWw|}IiX7k_UwJ9s8F+F-VOxKK;ZXrZEcD&aan|H{ zwmn;v4W1scD_KI2)u4;iN4|k$Qge|}ShM#w`j@5N&w56>tA6SG30)+Cgeo+9g?J3z7us$aFZJEm9~>&~1|P5%J*`Hf;;F&niiLUNnHPuz$u%lakfZ zHc14$3hr#9qT0MU4HE<^Xl#N_1v+e8Y%Ge!R=#LfntR!1pIQ+J(D|Ly=~nM~2Q}hU zska-%|H&uMla{PNU)r~*NBs7$T7N&w(U7RJC?zw}Pz~ae3`Wa*a~re~xVx5o9#Cdf z(?${^YJ_Y!ii5d?9rjH2;)#lO(?TrNm~9eSr+aHgoEynm{**`n*qI{yl1y=F zmQr=eWUfCr_sw|Zl?*xP9hQ7$iUIZNp3#j!?-pO3P&pmwBbEO5X;dGR9vs)SSd5K-tuLP z$qb>^0ql-011#G7E0MXlp6NCpoq+v#e>5``FfryzmOdIyj1Ux$_das&LBni>eK4_r&vx0Wk89 zlf<*94c-#*%o2_fn7wkJlN0i6vt5CaSRov_Y*H8T)KGo_CKfC3M2_~~CNJp*1&%oJ z|2ACuyn?{kZe%B$N z0c|U_;n5`NvxRgkPV)}~vDXuYcia_i<_#I_)|aJH*BGa)CLAP{ku_BuvPPEyYGzyEy^IVAzOUs{h|7k+tnWO5ew`MRwD)YJ3O}0&l{@2$sdsO zb$^s`7+$p_Y>+^^t#m75p~VYC(whGM630C?mZRdTMR?jm>#3}c&04>6W`s^ppTs|e zMxf6(TyRqBYf;{v|6jQs+VqOFFFSi3JzLOg)B1K6c7X$j65Zb z?e~BBp&1tL*QUB=L|4OkRm&b~!MN(nzKmfFb7^Oet@z;b#*ro=mTwPH&)}Z8X+IOz zqz|-WyzV>N!fF^c(?-q3gT#8y)Fd@&$16sLRDQds(>NX|EpMJj3+2t^1N_kR;#-#Y>0G z--P8f*YDZHH(a-QaETj#O!6NsfHHA2n6oZ&ZZyjw|HUu5W3|T}u>{WC_u;W)`UXck zzNvF%KJho8F5{0s5*J`HH9SJON}-&!RJez2cWPTSs%vlP{i*w^0VWXi_}Jv$6gn=*PKPYKafrT0+C5U^-uz3wR+)&C@V_Pc(ti8Cgox5&|)$xAC)i&02RRdHl zcB{8#jLe=ALL0cN7O>X>7u(rrsv!*`L7Gd4qSKO{GSW5b*l0unD;=o58Xv#B(2CWH zsHB@BG@kgw(F9v*G!-Jr7>g#9h$p@@2U7^w_txX|Nql48P{Etnzs!ys`CvK&XmY1k zt+f1tIph=hG)4Bg$Uh6ro;_ejs%Ct(y-PEX5MgWYW$s;@s!c4ZKsk4_Ub$P)Apco8 zH1|zrxz=2Z0)10lh{>~(ojXfW=6O`c1g`^phe_60I+TI3Zmhtq8wrI7YA%C$F?M4~B#CA-2|AP44hw;$#m&C4|!`Op7w9 zBaM{^&+lF?_B1BwBn#TMiK5<5f!cKH^}o5jwBppDGVoF?1|e2hwwI7>NiZDZyC%HW z%p;FT@x)aZ+j2|I5U{v|Tpm;r8oZ&cYcu#V_A~i*d1ki$ze92QFT365KMDP+dMe9x zT|!3lazZNvdKgyEgm+nTDb7;4^766|>LqZhc&iLP{f*Op3wP-Q^^?Z$!SqRz9R!r~ z>%gQtF)~jO?+Q@AeQOxr#1thPR1?SSo_0~F^zZ)!R`EK4kPrc~Pp3frSc~Rji z;F8~gk!+84PFJgi${=$xe@xDQ6&8k=Z_h4dK&RQ?=|LqtWu>+8} zxhOp==|;#+p$+DE{I{1&lJCc=a?wGkp(_!@`!WR*5x)e7hMwOVkv&Z$zTCO)pUzkx zq8To<7ulLg9!tv^$vK{3@k`cX;s>&pcnj&lR)8+&cu|$Q{YYy6f&LL3rfOZau@I9SfJNW+%`3Zz#J@HBd7^`z###_+EHclR!^! zgJ;u{?@rMZ%_oJ$`pVB9s1@=s508FTD{i@Qli^8vbu7Dwf4vbGPupjDS-q7Wtyj)$Vy7*msyi9U=51gMl(pEY) zQ7iqOEMa%^y{so#Yx7KkkF;4=xBKTB3`V0LUiz4Mb>b1#Wc&1fn1}+I`9-{>cK`YL z2d3&NmD`6w>_NUne?L;WWKB>qfyS@&sTWw0PK1F11x10ElnVoG#vjTtykt*i>` zE1u|2X>y|KW(;zCcddb zI(*v$xU0PiU}MZ6x;JWKvtGY1khc@)(zq2Kb_Jka^8z>lT`*bE8fU9ak3P8^+U4OxvhODX247`;2V4YK(6G58|}t_F#G z7V36)TcFMTnM%^a@;|$sko~Oz#59(rX1&BF>m!BUA&VS@?M!y6!vTp@Pi|n(+ZUZn z_Y9gNow`LDJoYCfcX-1kxYtZ0Q%-hWBvQ043RvJCmEJiTto+18yU&t;b=Aw8!$}8i zHeuLiir}*@U9Jm@fGesw{^ZFJtaW+eVeg2xcW+5d%jSpeL^uZ`I&T=|wBUIM_LE#6?{ANVQa z4u9utiz_%H69u(v<0|;Xo}(LM2J_0vBK$v_zo(HB2ixJalr93h4f6=Gtsn#Ug_!RO zWT`l^zS}e41+WG7e&HN1ikkWY4P3@rXO{yTzUyoPx(rFU_kq(!l|5nw9OW|p;TO#{ z&gffhzQPU7#G*ejJ4h2SgsJF=VwtF3N}-Za7JJ!3X&33tI>u%>K>JnFPz=uH^N?|XS;sSU1#n+YHz_9 z9-NiO%jwU@>_PTI&@__RT|)B4ARQSg?%$93UA(%~?@y@l?$JoG)?b?&)DUo}TE?#! zMMj8B=l&*d2^t7rc&TtC8=f6Meo@uh!f$_}3a&E#Q5F8ZOvEzVEVYmD&EVm4Hmfnh z;ck+{KV0HZ4E4EW3Q(`~4F-wiCCP21&20b@%zLPTVZ~ z5f?^y+RNP3y@v%}vEY?6lct)h42!Lwtz_V{r=biK2HclN(DduAj(v zr8j6MdW0#j{&~pCh;j2)_LtKm#$~B&UIG95)G6X;>)hTP(~ z%zn1JIw2^8{%*exkl0tlJ>m$zo4Kp40lU8d0BxEn0r8kW@6`2hA+~bjJ6o?PP9(iV z=@ob#F8)OVxac6Dm^;m#Z@*Ec63f=Q2{h&S*fyTH z*part0xG9m^gNF@;i~|TnJf+VIF-Ctth?HuRpth|5p| zL5}HRy)a_lbzEqRtlYykHlBh!-2Ir=q?%``uGd2c%qyb>O84@guQpw#s=9flY6c$A zVcf=+)_ls&bg)FU;tCN=K2gjr9I~#qy(7-Krp`#J(Vck_8>a zv~Ah?b_s{6htN>zr2^F&-S}{hZH_j<2eP1E3Lgnf9iO;$|JH}{d@t;AY`=nQxQ-g5 zo4(R=9lBAY-mhb0f0p+5vU?E#mr+YkM3~?YGbbdeE3m{2^veUrqkG`$1qwCp;4DcV zo7}#aw7+rm_AmI`AZ>@7wx6s zi)W8g$L~xM@^gV_D1+dD)OT(BtL}&J1+Ov7WiOgMDv=9d|Gcix5)j_Ajl7I}nJCm< za!75nQ2*nAl$E`(SlDuQSM2jL^p?S`#Ecp*VLGl6EpmN&*q<-^%nHxxpvjg3@iA0m zePD2NzhD@er+q@1?z(1@P%@2Z71z&JdArPhbR}3~q;_U^98zu_NG(i8Q~7@Ba^v^n zD@T{l_#{3eMjvChZT+6BAJwabh^LT zw2e|0CX_c4ow`>^ z`ij|HBo$RNY>W4Tij70yr~!9MCBq|}w$Tl-A7SCpr`tLgmhPM)sHm)8)(Rz$@CfEW zQuyOFzN*fy*u=@w?nd`5aBPV9BMAIPUZrLn6B%gSv zxC_f6Xite;(yX@loNLMQJpOMYcpN2_;%#<(LR|wt5^7@$8N1n~ngbi;{U-eg`*5HX zX2JaRO77x!GU6Ujq8Dr=sV!3`5#n0QmnDWotw*{22WICxqMBL!wRKEmW}O0K2y_Uq z%MNJmbvC}szIp1YQx&k--RgKP#ks0g z#bfi81HT+<;$>3gN%m{c@^9?kob!&1OyjS_K1-17&hsCjxe-Ewlz}_;i)tj<&uH+( zRNN-hwAu!?%$sR%lx`nBP)aWJuX)j*HGX~74pANbIiMl#+SW~Ms(AjBZE3}nUZyzjkexEb0CN%ef^Y!;cjcc9KJdOP?}H6z7NI!o||jW=bKL#yJZki}JO zdiYEU&u!&*x~j1KPJWU@Lw6Oi^4lSYh}<`1FH&%NA&T?7DKhmU z8GQ)r@pgJaj!?Jw{FKlXbBp&A={F0hzy19extZ1b^<)of=C| zr3j&$8!V;kV{kiJ;$PV3D!eB2hU~cbXH{hy@!sH)s}o8GK(;5PW`*B61}t&(GG@6q zFm%_wpnyS79!SRv-aBt#bH^!g#3FRKKrK1o7R0K2zq>IZx{*f z`Z3sq{g*GnL>nH-Bb-MR{Yg(dziwAh>gi`idwn*&ede2@{RX9tCf|T(MhtxNk92=3 z&D&#Avum+n`)>MsguTdt@fj;0@Ila-kX|m-IG_ZntdHx;_0QowPS%v#2H)zt5B71GL;78Ik&a#0(OF zUbown#4GLcv>p~g_u@|;Q|Tbe%kmdrJdgcnVWh0cH})Uq3#p4W(?7$wG1XWOMU|77 zlYE~rQ-F=N>^7;S?Otl>mFZqQ4iqo59^;*OQ@eVV{WkOaYXYP6KNF2xzl#xwiFV?0 zlXyOcVpCr~;-|h*G4Jnf-00kQZ?zWXw!CI=ewXOs1P6Mb;hZlc1IYP9Su*1TdAL<> zqx<^}dxc(!8RH~BZn$JOGhp1T)Mnf)#MJfZ{(LG9iM}GdM6*wUTGYy0 zdvK^QVqUmyZwSv$A9NPKf(*CuH=+EVN+r>QpzTNBj_3;L)SV^iNfRS%g!HLqS<%Re zz8@!w$1t83MCZwa^S#-1Rn(~UfI!SnC?#5#BP{tjGA^TY<%Q*SY!>q8L=jWASV!-fOVO3$oi=b$| z{$VQhae^(~a60@T?90IIiF9HWVW(#%v>A;B85Sk|SlYfx;rIHk-S;T2k@{L2chTTw zrs{r0_VVTzAh6_Dj9acEb!Gb`Xs?R8uIZ~kJV@CB*wzyL+fN-_)~BJuG~`3n8vLjl zbV054;5Rk)r>AL+y<*#;yQrbAVO=W@7YN9>zbF#eU0ZY(VxwJZwQ%xZQ0!1D&F?SL z4EyI$gSsZvw=p7Hhw-N7&)6>?iJl(0MRNA+TZqj3V!$VUNc5CNp#ryKYp{6jS@5Zo zIiAqxW#3kz0iicHp3n#56f`@nq;JzL_ERo7i)33Ta5jV=B0j>IL4!SzxILq+(qwWX zb2shwPF?sBS_DzNGvYIU^UA2tYzy>C!(?G}E9#lIBk8RBcgLV5!d=7ufa{Dqd)+)Q zcT3W*w>zv_T@(;w`K8@bs`z=^^aHZaX4vI%;EvosmBil_O(WQtoDK-2f3kHwTm`@grA zYu0$zx)<&*zH|0Id+#GV3_K+M1OQIc9lVrR66;bi`P6&8O{l!dn&uIFi6NKN?8gg@#Y zDue4>%|y^~z|!}35oKj%Pa*}k*T7cEi_WiI%3po~bl3L51}b7;?<769^J?9Gel3_Y z@X^P5^nK8!rso=SE{UOnqbfIL{J-9#qP8h%s< z)-7)X)8dc4t)-zObjkT=nwKH6t^ihLeE}GHI8h(U)*4&`_SCn_0KjV&ur;8r0Jgv8 z5vZH;qrIhwn8XJj=JWh}M*uQBU1!|^o7$Q8q9f(=*eSTJ+(+zFM+kP??NkzaC_Xa_ z-4sDjf5;{8=~(zi;j!!Wh}e1k2qDA+a_;wu+#gC7iHdvW6Gm-nxdJsP|K>djUEBrY zI-QM#ZkGP!SJx_Q!tqc!`opuvn=QzfAM7j=-qYGA(0Y}IIA;g1S$C5lg=AW5owL{O ziw+0|KOmh;obN|U3&3~2Uzmd3b4c%R4_PG~@y*N3Q1yFG&dx|_lV}@f%wS)tr zKY=KagW>#x5hJms*2`x?dXdx%R5SKp_}lXP{j|$=T+ZhIMvA{bd&bEw9uK7C@BvF| zP$8g8FY0t20uxrLq;0F{P;G90Rl95D%o#$j?6Nuky}^Sv)`gtQGk%h}oWN%XY>yyu zZ0I-Mqix<}F0xHjp+lBIwdSz37Vy`vBR?Laxr}zIK}ZR$o`3afT{Rb2PCuBV&$INS z`m1h+;z&2ROFciq(%A>#Cw(SxEb~8s(cFN1aRflY)JHnu>RcpH(&)&G5dO1w0c-Yj zfcWvahX#kNYr3_46Tse26_<(fd?*~Qm2Pxc9`)93W=o6!j@5y{!n%af$`r=PP7D)l zkF4qIfSZ%A-t5(G8m3p;G^oaeQak_f8Zyrkzt}L0W3z>o@1#HKE&_b2`@E<7$iCSpgEC?KPCLrc7CH3zBBE z?QK}h*R8dRny<7^+7H5PQ)0EBUD9pyZqKu5ktZUCSXf&;qYK1q+;Be7Fk1}|hmfH3 z+wZl6OY&RgQ7A_E)SZo~R7kh&ND+GwryfBoKimVo^;Y>MWiUs9rPa$*?$K+k&L@Rb zk6c^2eH4wS9zC<&*p3!ji`V;gLT9d#`2^{X;JGhGK$2OqO_w}F5Eni z8cNluRtr$e2>n+r3L~23dN@Pm15W9akHX=jFy@{!`zu_c4uEVHFF#7 ztNRdm8!A7-T~FI=q6&;t+BR|A2VqRKdg13ID8Hyp)*tWJuYA|wp$!=G?9-*yizbAq zq#29{9Uq2W;!pnx`d+T}T+74uLMqJR4w{GaWYOLP{P@73Y0=-^vl$E#>UQs$oNyn^ zGqpagqNB&{jv%&97fN>zRi)=BYz!A%d_u~XE;6Y?$?Z-Nep4W=!5h%})B$^7$27QvQ2ei`jz!R`H-!7fSd=l?lAGz5a@AkGs`q4MnyH{{Ri z$5if-;b_>DHH_+$XC6!!=||peUhugh#3r#8P0ukZrKcg@>0&^>fX(Vr9(%T>(|1h@ zPBt$PZhP8;tl%9tl9X%DUB^9)d(DUj@^n{$WI=XXQhAWYmiWW8hFO2h2GbVbPr)gw z8D<=;FP-d$LTrF5q`asg!LBGHlM}_2p`X+*q`CNfpx_sc2FixxX_A_p`^jt5PMj45 zh}QTs8i8`Ybvl(^r!LGbp_mFv^VMcv{6#i_BcxD~cpOrQ9m z9PhaREuzj*$g_$FZp(~=?tMF_lg4mv^P*%#@^axQ8oY}4mG?(bG<4za?M){mAh20N zeW!shv`jMq3QN>=DBoOtWOrjO;`S!4%{n{;{-v9o+no#sMy22hu?p8eoJwq_aDS!F zMUf`D9wM5h0bdigA={+14YB3h3q|2=#;hZ_)XYUAYQTxl3oB;zfSi1bN;9dMdM>gI z4w(|dyk(Pq>z!NP1JI0RiKd?@dd>^QM&V{8g5)5p)@AB>G!*2j{a-;Pj6iWWSu=^S zs67VVMua{?5S9Z?VZSy-48L~mxaZ7r0?9r+$(Y)_cC3lD-RY9{OTSS&E{)p#q%zC0 z!rW_bG;0>K)Y1Ro?kl|1#N!>QRg=4!Nm8bJd>oAP=gHAYp)uRSsM$bKF+B|#!AaDo~4mEH5W;V z6wQDbm-y~#b3=IcZ?ftv!>pQn(~GT8vyKMq%gT@)>nG4~(1NiWp_{aVDcq~Z>=Jcp zM8+E`X+=gsC^;C(*!6Ec21xJ_EMiX21hgj=oJKY5kN zjGVsQEpX#?7H9fiJW8Hbqt*nlM5mtKrDN`Uj>T`P;g(h~bA!dy%f=k-NTj{U3cMW*C=Q~C=E!i0 z-L&#N$EjXNHWl=`RV-gL^bfsgR`_(=6h8=4FK8EqxMX1}t@6X2wB{Mokkad)-nnFX z$CE3_@3b&^75%i?>gQKJQd)E3K?d5qN~rSNEMau9Lh~WMi;4xGkh#lqzTt$EL|?mX zH54lA6WhG^Ub@wyyE3>^+gbCtIeM>ft!Be%@w)~fp8u{@AqUKeLur{4j2~r)pua$r-86h@k|S?(Rx4FIlQxkf zh7Rx1n&YwdY3%WYuaM{582YH(rFx}&8;sbL-%EO2@?D?C?OH&}tgqGrd!9UGmjHus zl$e$%ma*MAxSP-$Ps5K68(p&W$8vE4`%{QS3V2=1=f|K>1h@}3%{^u`IJpNv*98bq zeo-rX1F?9%kZ@Pk$Q7VbtKfkr&m@0!~d{Vx?tihCtHSt!{Kdk72U+KW%vJLq>PND`TW!34QP~ z&wFMtW{+?H$a51PwcbmvKz85Pu8!5X&3^{a{fJ#{A+FV)IqdGT@1sqs9jmnf6FDUZ zCC>>}u4gynq_Igoe7tYY_~>pkl|$d0$>9p)=HTu$w~td}n{*HL5{ zfL%qaYI>oLeBKnBPCq%m`JLG02sv~}pJNtWXVXu}O5i-?)9xJka3Y!%J5?Dajg8OH zyu;NCp2pu(vEQH462xE>vr5tb2yh;X3=scnx6us0u=68pihB#py5x=r%KrnJIE3si=ZI{b$0sLG-HnP!Jb#)u{!qm8EL5owN% zadi8D`V|^DVYbP}{B*efBMWg2f*1=102I&SIAq215YMV`ruO|{6Flb{nBm#~MJ+}g zg{%Edjc}zC1zINl%)0+2*XwZjiwB1evUf~MLKCUjF7R5KYm@3M1K>Ppz|}=v&=zL? zT2Kmb!XqZc7BT9ZJ5dPGa9+zQCv#Uu!~nYme=Jd3X=rv zPi^WoG-ZlnyR{VWVWs1&7r&pIa^~0)EnKqe4H9t z(c&Va-j?BGWiv0QkEocX}vJ{{mqg z0y$MopYRV@Hjp5JO~!3A?DQf^Swgtz%sHQ>fsnk$W=ZWUdXn_; zlN|9+j=TDWK?i4+?oY5?{x{_L`@4ZG$(A=!vznVrTC<4#RI#fxm2`Mpu4o9eg{d?3 z$gOu7_xl^5MkV6?k)+ns-(FZJtMzjY)i<2vAmcK2uu~TJTMKa|lbZ}LRmJUc*^Hwi zZf9rR$s;<76VQCfDCy98`c0W=q`fstCQ7sXgz57y5;+y%0YM4|FEPxLoS&SIc);*x zzqp&(^H!)Q>)Q>={HA`sjHC|}9Y&K8Mg9?W!z1*z*AQz@vC&LOwm7B58aqz=ih)G7 zo!hme%|3_M?zMP=(EF_!);SZCrwfYO%j6lBN9h}WKSc_f(xyDcS}9W?DZaEbMeIxW zzk6%sCgznKTw60)iw%iZpBfIn_0OsQ_wf(DgrpJ8@{V5g&EzG)C%NAV4$J&k)Bj9m zh77;Xp1^Tm_xa5^L4S-F^ygA&hEvJ|nFaW5);Ph|Q%h5x3ACm?GHp+NlQdI3O;-J%R7(8%S~>c7eeVg$XZ{rXiq;bB|4cArr#8D5_u+8 znEMxy(cgG{+L!7HC2m;M`F8EypBxtS2?8!X7qChFEmGacwNS*`?xSx!#yyu-uvzZ0 z$C(B0!D46J>2REyRpJgF_eBPut{-0wu9>vEdmiWa^Z}a;{2r&_uOZcM+WVi+Oe_Q9 z!02W^(Xfa2i)$DDG(+H_G0-KcAinm@aL z5)qynbc%uQ(RfH&e4+z|Qg7+iJ6T>7xVsl*&IUPSd~mN(w&U8(Q71U#0>c8bq}Od; z3z!SbWHquM^L~k>* zCOncQ>B@B8Bb10E`dMU_-`A1F6>IW2`{=Eo8VzA)FaLVOd#1>6)g9;w6PtODKz0Aqz1!Qe;F=aN(Vk;N8sd8_K9VuLw0yPS`VszqH=?l_c<>E%jjqP42t7r9$r2bc~FG`e+N=btDW zf6B=>8BNtM$&M*Vl)p}x-VX{fz>mL3+0FDVPhJbbcl`qaC0%W3iai=lZH z4bh`>R@r>=%m(W|j#n((PIps1^N&305pJ1#A<T7Ax!d1@ zq;*ap;nKnML<9@~;hc8_^R%$EEOAEM(ZM@$O~bhrM&s+}3<@5kDz9Q8erJ|^>X3V) zm~htm&lgM10(BpV)( ziiH-nej|4td?~vd_>RfP$gPQtio1u<``p(c5;7}_Z~0)t)y@)fjR|8{zp9=W)mP{u zV{RC9nAX>ND^_=~s(93X!M~^q#s%e0-SCCdDUAC|KfsvqR)SqNZ%#$fH2`LOwxt}} zB^Cb{Z9Bk+SQ5Rzv9{Q6VIc;Co)@5=$+|sTLVn%0>HW#pNc41RSE_?tfvuuzn2Vq$ z$O4XtB}tFV%y79O1$SzHzn8tAPP8K?fE`9f`v>INS_q03#bHi0((cEL?C^C5+|lH? zV5UGau92iT7k^Drcdur2OYibfBGFEO(wLoD&6yi&aS$#t;mL}pOB}t?5@zM1x3C?> zFi~rD^E>Ig4TWJx#IJ4iLRFe<>GTsAQfsC4BRP321~|h6}A_ zJf^O^zG&S@B@<&?p6F<8u9bAfaXyh%K8g1c0b2^a{ubE(ahF z4JdKW^+Zn~1SE6Xj_z+^BTdT&(u>}zjRs_ZYu^ZnSm->KGPfH~6gF|(a zllMR`%c8P6=UW!V#Re$GE)Jx|4%2a2 z7V=E_B+$w9s}uT8m>$s_X};|+)rq^0`_f5YBA29>RDZ-k(71gB`ZSM5eZuC%xo!t} z@eb(?Q*!(CIM2!uuTZz5eKINJDr3ICCzW0LkhTSL$4X-RUnBW{b+XfOh|ZUnR+4V& z6)hFiPgn%*+7qJK9hC76^vif`+*i=D5BP1)7lpchi|3QM*U35tWFRXb;oMY+#p&yV4A7=M9}U4*=K7!bT-X)dBT6Sj9Fjtdw#{#^NaAFFJ@nHG|zp*{uf>4 z1N@b>XN>&I$;K9KPe^$)B4>!w>aoU3OoT3pPCq!Yi)^yb$z%A#jQ#givkWg|Ad6-B zrea8@C5W-q+V%Og5sY!wB*G^+YsSTN=TK6%M= zIy3>Zyy`;A^0%93Oe7#c?`yD#qSEtMROSodge7{aod?Z}C5a+E^%v1^OA7fx6T46` zi(E!bRl|cvUq;WDSrn?`>YwEC51!T*8fcR0Y=R)&B)$ zOur^FT5b}0?9lbfUZkMrVRgxmZ!h>)05qmWz3a=-r>FC=cnqdMqpcin&Olyi=_4Ng zZUPL5IH!QT&WO0VWabe{$pO>}Be3r%gKw<{52TGx2TrWf1JV6amDt^i9@o0v*&c2v zTPN7CJPnB2*;|)&rhHA|M6RrXmqiJO=kdey#MC;@ehb*nj@<4Ykyzw{k0_MZ?AN;l zgn`GQUC52z`fHnEL(9@_(CnFZq z3A-QZ=_eHH0>%p<%~-bWkJO1Yev-88BVi8I8BcmWxOeoo2N&HUp$?Rafe_Ot*k>Ns zooQHQSy6Q_?I zn2S%J|Mp^|iYk)jL`C4f&)|~WQb);`(Ri@C;R5C%v`h;GO(QB~osDTvF}xA2`c0jFfE+yQ zI%E*anV6FK_9D#-VcvUhmt>R_znLt2&i0Q#B?aN-&^$0s@p?5wAwCo zAbma9%ZtRt%s)=3Z6&>$8*`e0#OYplh7s*JMWb`rBm@%EChegXKuHJfvv^o9l$`47 zOM;=W7=|h8?JM$F)|C(Htg*Iu=pvyy*Z6d~oY@(D@@~q!rFUe;=74+{A5GVOU;pMT z2PHd7@Ek-mOFjeOTw&1);El6Adt^SeS{%apEEUGM9qRrYXZ5obNh)Sj{C~8#nec5N ztOQR7Zp$xz!crI%pfh)zS`#-dQcvmkX4RR**=qkOpMPV7S22kGPdCF#CP0F}jj_2` zvJ%*4FnP0vpb4}VtXEjAxdeO80-xKe0xp2gZ7|BNo=(sX#qt<2E(6AcwJr#%l}<5}V7E*N z$YtB$s~N5+9=0!1RWMyH9=7`=qOD}=TwHov_x1h$({UntIc1DvElw%gz5M$B{^c`a zBu9KGWJiHyy`YD3n}55>+aagji9w|4Q~vHd+!EIq&zo`|RH`B&%d9)SV+N?I_GF*6 zC=aUv?z(j3``{p9{gX50-?e7Goz!8NlSDvT+aVj7w&_e&vE1%7eNMdI%VFHzC%@!rQ={>wEI;-8745TBTqqu4j7+U_~~PhZ((jwYx*!cC0bl*V#f@wX!W+t8~l zF(7JwK=+qz3F`V74%u!XDmXs|6lp6UtDhn^gkYz@OS|WBNIS;=0RLD9s1KVKLk>>4 z8%h1B@}l07Fe&7Pa1!|Aa=@5ya1xQzu=ZGVfoNYl4Ni|M{#7qt;5he2!j_Kt3^gnE zA}a+?^L*DoZZToH$o4;&vu8jhjC*1)x>hVTn4L{Ma*yrW0TYwG4VUF9a7KBs)k*Kb zu^aM9Gt}NG**c$De7m?Wii2$(F+`f?qPxAXw>#auPi`$LqmtjJ9>w_VTIPD}d#uZ= zL@GAF{P3RJFw2AN;0Oa%bz_*Tv#tTqqvQTq#fw~fMn_i?e8B9Xj$wM$EVupA4)>7O z(KWw?RmP7Iu|tDK{9*grWI=FKt(t^&XIW1SJOC%3uKgd`12|2$_#tUjdI;0mFPY=u8x99|)}%`GVG_ zTCxIRGQ)Fd4{1vwpURgB#au{gS7kl52|xSCvtc`>*Kb) z#+k6?!?kkW^sC(rnUCVy)d0~KXqu2Y?gC0q;{V= zU-ZpA*rJW82uJBVUJ|g!a>=u3pKXE%$4!#3qt(Cb$=y2mVh?fHT9i!6)T=T6w+ za+0^8jv5%7hixYoe?JuNhiTY~5LI0u0sIE|3Lgj!LZ&|MNbBTb&F{0!B*Z?f$Qh+3`$OY17mN*J3F4IYOj9@ubo%x{srR3#L zTEvzAoNiFEgp4N%p2_3-;%4QY? zcbE~R24ynxz0F$#el%Sc$`*Kxs$!r=dXnq3$#;~GI^Z1j_)$v?8o^Mv5Twh*-pszb zYtoZ-M|IQBVm_GkR%$n%IbXWSD$}5>3O9uZoGxea+r#wNU7xn*f?Eu=MwhP&eh(7G2n(aV4EK|E%0bd3;oZ0rsRIj5c`CD1EAJ?6^n}g_Mi6P z8w>GxuGpQ@^JO>jo$eVVduF5chXYwV;iUxG+SjKur`B|5?vTSaeC9~KsZuX^(=@3ZHj_pm&j+GLl-=)fCB+@=AKO+)} z&ckyvB1HgCh5l;eb)jB{tWZL$G6lyub7=5t`Y^3t;&T_ptp_4Z7>~i{woPdVUU42; zaLen)=OsSTxvg$cw@}aQ?!jsdUaI+iX# z!)qHjzlMR}J>`C4lE?Ol!x+Jnbhy8x-%eKrM9^cxIq3;$amVd7pl%V`w=#*K-Hu2Z z!~g0I*oo_+v!(_7B_sEqkMP31`Q9p50y@C^m?=f#o-O5TL!oK8NFMFMt4h5TI_iEL zaJX6gqDdD8#xHc~)El~lshZhSOJoI)jw1t0Ua-H?`L)#QYL%$b_dk6T=U4P(^UCtH1>&E@1c-a;iQ{RqLpKt z>-tVj(gr7zbhWlvD??U{@)Lq!y{q966nksv|HlK<>t{PByLuj&4a zTN-!I$ozBByEHRnnAv^jkHCZJKyXLO+M2}B=dN0-5hd=@NYvm$t-%bV%em|FP$^jT zGa>)@r!!$N3^YbV7Qh-BACb-K(3!@#4v@XbKDZ0ZfBcxYPAI^ld;z%38$H5BD2GE_ zy}i)^j|nK0rPs%iU?8PEI=TwSHU0x-ugIinxs5tbxkwOAfJJaAhT5?GNI40?fI8I515FUG1KC9qfZ&% z&m@~c(YN}m?=WCGdQGM(BjlTR*Xd9iP8`5Pm8suR&{A>BVTlm|yPEKb$2hhDMqCgH z7=Yq6Y(l|NoeR{L16Qk3uqfUVpc!r<`z^; z3F6!3;i7V&9wc*+OClIY{$>n-9__9Ixb97#1Ax{3s|U7qgqWPO@a~~I@0B!pulw$u zW1yv>6=C28WaoC95PX>690WH&+hl+~)|UcEn<&@87Kt*j=3!uOdxc}P3b>2HGJ+D; z-Ktk)s()53g}&n(<&<_bFd0|{in3CX{ED*>d7GLFP#`C=f%BR1#sZ1$#1qxhFDfdC_}*2IDG7v7pKkA5@m62yw= z>2_iqa7dQ{%WDKRKj)|oIpPvZ@e%|*ImsomJbCG>@@|`c=ntU$IpYbm24?OVV?aPg zn~SlI=f4(Bk$c#1<0t50zk->+Ox_?^Gj%AQmpS*C&!|rU1fVxP4{|?JufR1@H+9jb zQvwcRudtn$a|08+Lr7A$9#8`wPbAml@HSG&yb`c}XSpN=0eza9zX!C{qnS}-SQ5#} zyoeO?*N04o8_-Fp#w*|NKmW^TZ1NokG-ww-mw+uGZrfl{hh+2Qd_;MX{N!S80e34o zIU?V@`XqUy)%!m#Wzglu^OK}Yhl*Z~wYC#8j&zB2|0E)&MZ_#tRKn#s93qFpNr8^2q^qVljWY_ zWJ8Di;Nn+PZ;8A3XqNX(KGyuzOSKF3hz>DqCI&p2zWf~>bd#PRItquW`(GmJPXW!58)59f?fV;3UOrW)0>YF;^c_^M!$|C8a)iTWFLZZ zpP5A2+4z|wfhgTCdpUkrc~NBIX@?JgR1c-3GBf4wHdL;HT1m%nPA{F+sQ;I|>mEVg z)kx(FIvNX;tb#{#-F)=9M2$y4uX!1EjWzdp1;~KC!BQnMAp{s%utN983M`fVfmw6q zFVkhkVf~JpAp8(0(jSD<9h@l|3`EFTCpZF#LR%yR$FAyc;;2A&W(*XB|Vy?@$#Fi zgS;sd9|82XCk;uB_>5EOL6>fVsTXV9*BI<|hk-2d?lnGwD!bc*PklG91B8q)ABDmwK-fN5-Pzq})=3MT(aZyL+mSo=7O)*Gt)$zo9A zZ}Hym|B)U^3JhBoEq%%}X(>#@2B9V@gLQ}7z%U!058EF@)4>-Hk9O*Ldfl6w=0b`d zqA%p;F>!8Umc9FQWlak#YH8p^t)F3d*ecjh_^28<2?ztG$-YXk_i7zLPfX8XrfQgT zTY_K*?LH7sQ#!$nScoN{7^|0x|yxVTOYF zSkcPMEiXmPhvtb@0~B-J}^3bpoRDwg4Qq>63{_JdI>J%Xc)ll3(}~ zNWQsMU1C$Tm-PFz0RWsoxJ_HnYS1v{=2wZZ3jMj5cR;)oBM|I=Aj)CsXI~WKa$gc! z=c8`+Y{UL;`n~!G-^m9?#>w#2!E??>dKbGL+~#Xzy)Amn??@Y1^5sCMjjOxvZ<%iq zCdDeB$lR5#W{Q@<#8!Nj!(><^qWfArCo(*=EX&hk%L{J;HufVA@m=`*#b>^UHhh74 zlOHqZixq%Ts}j?G$qL_KI!k^X@bV|`f+OuI>?x>d&xKQKri(Zuxx zPtnw*)DK}6|3p{YT&$r;daRMcvY)$Q;0NGY0 zk3{X+oJI^7nTbJ;8$QwHjyJF|z>!b*xzf?N%rHp1m7@;=L3$DTF6yG|I3$ep2M}S^ zslj(Et>Kc7e~ueSKrcsce?J2qq1HeEj*>i5+KBZEG@a`E4ncxjCz+8Hr%&rv)N-bQ z*gZShh)OTmfY5h}{(T7EY!cT$OjenJ|IXia0}RTLj6eW`vdEyV~-x8oc_E~b7@{L$}=D^Rgp zJu||YK`=feorj{ni}ibk$|U$VjZe{r5}$HJQ|PPxPL|GDE(5Vlz5~2JcQ|k-0zA`@0ZxB3koo;LJ>ZJ-4LB$@eK=ZBNbD$;)1hRN?Od9bxt}D2N07H@ULJ z1TzY7RT!(?(`NWrGmXTBR&LBvYYd?ggEqt}2h%eOVbv#Zp$~@Ue1T_LRag9$~m9>tkOYIwpALXEv&!}*+1<#~ zn~~)qJ~-3Lkr8Rb)d!f1L=CF>$FJ0RHh_@h_sBn!pAom?^XtkrT!`bb;rrKG}ppS-bfcSN4j&u(`il3 zH#w#4&f~HbSUvTCxhO#1e;urZGtQA9OKXhC6Xqi7I57dnpSaBD$)qI^hLkbA!H0xQ zBm0h&C#PvkQnI*=$eW`w@t0aVExiVAH_gSpOgRO#mgXc!s&M-1B-YE=q_X}O4-L)g zen7Z$N|I1_PI~|PQ3lb$K_Lx3Mv+w|qm8lKe7z0F!Q>VKH`Y(wb z3)&&5gua2q*4zm&Q2L^dP(EI(j7vc26*bRWhv}-4eb(fKesEuw#6vuxE7&LN#iOS` zzca89Th#EU@c5xwB`TvilSw!|e*d{BGDPhD{q5DkrxE$@d9X%gQK;QnVFY8*cp=FB z3B6kM=rkeS2lHyWKpQY&yT84YP>YW`S&EkA*rd9L8 z1s|Q6*H8V7g^8CPxJhUFTONqRb$4M@kwgn#MM}L?XH`h{XwOErYGvejNF;O*-Cz7{ z%CcwS8ZeWR`(76%|0w{ziZylFaGgWG2MRgxRPz%n!oALuBkqlBei?T->bZG_Oc*%HmSS zE&W%qdwX$r#8T9Vj$u=l-2yFT654!sD#cd6@?v8oWNbJJ;yUo2Gqn$Xu+I;ZQ)Xy7 zJ#;gtV*e9%@J!_rGp3fxegfGzM{^xawmGt*_IWj4ENd{t%8e_hG9fskg%3*{U&Zc! zO!BrH8z(D7RiZm~_gD%d;$f~I!ed_#?N*@NToV>+DW)TWl?yidM7G|)V?uawhe%OJ ztz8>TY-d7P#8U!gnpJ^6B-7NZ*E(LaHi5v6O^**0_ zWm}_n&%Ly0ua3fiwsNyglJ4nBk?#dT**3Cgro0jIcf z?7_pmcTyruM8>N<lz1!&%fO)md?_J(BP$%2=q0txxjkD2I_KZu_PuS@u0gt-wCyT=@Y` z9*D;H+~uar5}qame(6dkDFd#anaiqz{Vj#GSO|g70sF5+tAT?sSL*45{L0oU@0I)1 zAe11#PBZbG>0hdt?G5_v^y zC77Zxda1^RQzEJLeKBay)rgQRGIg23gs26SB6n(X=?}(sn}#wR9I{5hEt*u&r)8P4qt z-&wXXn<%lycuW74%Vy=yFTVh`VSBj*H42hU9D#fya2O-K0|HP8;&+$rk=JP25wPNwmDM#7+QN4J8oWvwYs9}aDhb6=ls^Zr>m#|}LajXJ0U>!^1&&#Omeg5rp&wQ95!ZCLe zHjBjD#B0gholcWL=y1J!QIB|>{}k%q<(lv&7?K=~G$KX5$ZGp`(&Bwdy9M!bWCh@i z-g?v>vUMp&3Yt+yH(KN@8c++I&bv0-9$Y#?$s6+J{FP^krQYW?zl<;2fQ3w-Q8;6$ zZz{_3yL$cwf`p03Z!Xs#A9)`i?u>zP%7ZxNql&|ykL#&-L$z%Q3~^4V=- z>tlimZ_;A7ft;rlBvM@h{y#Ye5wbeq*Rj6c@Y=0TZn*QcY(Z_*gS?_^-_Wp)t}(Be z=NJop5h9uAb)&yck|wUbk<1K9h$e`=k@=@>(0t+9U)zWp@yc}vTCdFMy2sUZVdy%BNY3wM=PKcr41jT|L4s6tD;>d zhTg>be(Xp1QcK81OwP%Ee0Rknmt5Jn3Fvc3XH`=LGX})Jh@-u;s;FkN2UfP8v?G15 zr?L;9vHvvFx9n@-!-mq`RRj?(i?Mx9E1G{$$k`r0GIx{$5Fvpb@qLXL49zqB9lba5 z1HgTR4%a7bilpxjpa9JMS3*mTCr}Pe%G#KljpxIH6(Ll;_b`Z%NhP**A5@6XJJO#W zqd0Qe>wk4lTS6F~c($A?Zmc<-iTUYJuW-!|dpuX~-ypj}G{mpxz7O%VKq)TrG9GN@ zSxEaHcSW`fMAjLsV*3R(x0So!Ss0RizDk5-<#wGMHtwt}xa0Df`+r^-h7gpc^ptO2 zI--*4dd;}qC^8Gnsl`95)6T}yfiX>PKz>RT)j_PTmO}31eCw_OL3f4DprRWK(j)!c z2{(|q)Zwe_@D%m|5Q;-Zbc5ex{=AR!I(oTu6US0W$E{_tg`X&z6 z+q$C&*<7PQUggB$T9L&Su=zbQ@L@$g!VQ{EQnb#~*psEclHvcesK^$12WTXgfq9S$ zPJ4~p`C1J76G!n-Bcje0R|BCdpjpF5oZ>+`!sNv)XsZ?bUr5(UzZrVB`qx!7>er+n z(sK?ZdF6r2Z7mznpXIY~#6m^vniPl6>Md-keo{Uge~AQgtyUc#XZ!ZG)2|SfzjYtK zzr2bf%9zm#$iQ+&r$W2sKslT9d=+JgToIcqR6!sW#9z#4#M+LNcW>O78n;o@Oi6%C zqfH~}4!aGZQNOgUw@bPnE?xL~cTlod{1ex-Frixk2`;G!5OGDw`J(8P>^@+noEt(<5M0!RWxy`rRRm#wfWdH-_+v`?p5baoRxb&V zBkK5aG4uWXMSs9mQ(DBMbT2Qp_ga)WulKCJFKVuWyy;&Rl{s&W!T~nBzeKJ~ z@D~dL5)vT|QE0JN+qN13rE|@!Q*Dy-eV$Mh(urp&{+bnZX70}6X@GzbEQ;|HAriu| z8*2aK5dU>deZCs}NXA>er#sd~KX3Q8btm<0#~a3KIw5n{Cv z#Tw>@B2qKR3Gq9n-~4udx&Ts<(jLQE0RkQ43H&V~a?=H+v{MQIbCQ6fS?iuJz~T0O-b$q(W#jg!xO)Z*i{hi_Mc z=UBThxoUR9zWra6!LQ5Cew7GSm<)YwL-+jA?X-6`uc`zVykgj^M^A@5q;wZAA75{2 z->o$3zLZ5XwU+WB$2t-}ZP-y_0Vn75yUS&+HvsYt03I z1N4V_qh3~d+QpjCLlDqqOVBqzI0~@y@#h9H#{^~b7%*%Tl+9FVw>m)nHR^@4+%v2I zBa_CCvhVLSgzUA2{xEdBBrhr6>qci@+`y}#JPo-ynz}?u+&uZHTV6&;rBhr5Z0nWG zA~N?n2anlzy0-qgga00w`~kea6jspa4Xb!)#^nOvYq=TERq803AN@$**k)4NUVvDD z9p2|Z%Kn92<6~5=iV#h;o>6`qLkCE$qV6}!k7+>C0ZS88q)GNFNG7RP0+)plMllAu z2zA*+Pm@=S+B|P#2_D60C(v@8Ea(7`pU=mfjzgw%(8QKj0sIS19c77n@@*&SI300& z>C#RKy02x##y8)IH#gXWY!1o}wZJ(AG-WPnJ8|K6jpw7lZb+zPA`rPE_8-^!5u!e!lq&Vt7x7JexRC_ygT2i@pAS1$H)I5@!w^h0LjzI z@GuL_)dlHqm2wh&Q=0ZxxnQmrNfhVX?iZt!@lK<<$792Zr{JpGApPH-k}}@l9UT{Z z(G0z*7`t$p9!zNW|Etrz3|oQ>0=l-jHJZ&9Oi4>>K@x*EAda4{ilRYUgg3~183V&u zdt=K0FOyxj2rlnJ+gQ2QpS3#{3Tw^GD`2rL1)S^W;1@G@s^-Tp9*;T+fsBbCjbGn< zk0XTj!ARNR=dS_I>QA4P$kkr)7norTfqLJxC06NJ@fUryuAh2{&tz0a0sXcG=BDWMvR6mu7h_ zW&a!;3nGy5TF^BTRN#f{6M&y)56sNbhu4MvBm~zHqAPH;_o5vL`4V83OpdGrNeS}v zYrfX7*&X!pAv%n9ubsu~SAKyx1NwgJPI^%ShEPTdf({=za+HHimNMX+KLa78xFu6$ z{n3ViF6IVG z+n0Cf1zxRv%zu^_n$~3RThO0VNMBjzsR04LB!+JWiEpLE1IjR>vo{j zO_;Nn6LRHp5bxK@%gRoWD^J$~hS?iIOdhU((hbA_wI&mLT*7Umm`gE7LC)0llz8vg1wbt0XW?_GY4fPjaF~(P<}em*uI5?36&M?@vc1lEA>)-tU54$a@1V8x`rRPYCSXHzDE-c+&3j%zEwAWmNs^4G$Ck zH@7wR!I2B{D08#bm45Xf2DBuV4~fzVQ{+K~>3H>J0v)fx6VOkH@g-8xOOP}!!xlK z#H;}8NAvbWyuey8F)8fEFS0fBEQzLU*2f;p0uULn+kfNS%*^eGKw)V>2WITGaq?(b z6v5n3==s@qncRePKZ+<2J^zuVWB5fJFl4_T{Rc(;`;vqhoelP7?fiP{l6&{Cwz38% zN~lk~JaXmphz8+h>QN-Svs=rQ-vC2vM=B+rDk$&%npoy)*@{eTPfegFXtP6lDKX;@ znrIW1DYXke_zB`DCkSaWpcpU`(p|W``frqjdBYf+LpmcTX#oJUI46vWtYZ)FpE1PK z0n%MzD|CsPZS#d76G3;-6_pe6w87!RY{at+G`c3l3|qYPUVuN#4)9X7te-j?PY&8U ztgVvXJ9v?}p3sQELc3T*6~hH#cxv%7c{Y!+qzqcB9+}t*cjG)G;ZptDf98>ujYvT zYVc6mwKDiuZ!DVrD8=AyeReBW%Cy(i8K%5w9Vf#&RsNgUwvkDmb#MwhOcsR`hE1~?{>6`BA zgTTW+;|b0HTKVZoOcTPu8_YWCyZzj$?mA^BH#!6ZK-^^Ye;E7jc&^v?f25O9k&(*Y zGfJWC=$*1R*(up8dsAeEgk-PmEh`xjB}Dcfk?g(q?|P}u>73K~{J!Up^YBRWe%-J8 zzV7RKUeD+A0xr_R=$AFK}AlY=m0{ch5YfP&^26>BZefFCI13jW)7JR)tP_F0WaA08O4AF}&> zgtf3oBGB;Itw%t)xllhbx6I41Ch@!bXG*2j}bNb=hSU||M9`7IN*9#P-OwIM7kB7$1 zk&n2PI+imrh5b+e2YQudY$z_EKrHl0JO+{UPDo z&(U0i#?g9uT;dNiDJnb6XB6K)f24DH#-5p7B8uc&@FXKv7}gDsLSlp34LPI^Px-Gg zYJf$aV;g9#(N`CzCKOp|_o*6-!iKmHF5k|slLU9I*ib^qZl7DY2gr>^lKWG1EDvgwSj*H#nHqHY7Gct>3?Oi7%*5hKod*;gt~aPUST z*)64~laxc}D}goBm8rskP&S@7nk!!zfp(ZUJSDAV%k-2dFF-`{MggnOWG=y;bx&;#e;6H{Xb zZ?t%em2OT8@J%vV6Ja3MaC4<&fwoZP^=!}NQa*M;<~%S&ndkz1Cn|MO@&_=}gB)u_ z(W+@T*opXrg?4;9OwhPZBf-|pkUnhXJ#xF+%=L9! z6fF^?FG5)-NIsQoZaUoIz@+y-(V)_vbIyjo^AtqIzAFViN{tNZcmX+4F5=V2mj6Nr zm#?`u+lMA!Sxk@|npax6rd083!ttB05fAAtw}$&?Pq+|eoWJPWakg8+Ao4#xaR+)z+=ziW?=XRjwE#VEG4=>xumD%f&-)O!ExJ`bRE->H+2 zS=VXd>?|EN8JO(+k30F-cmS?D!%njMtf>xnt$Ay)o{iq(HCLb6mG4_F_MzR}4~q(% z!6Gi{iIWl9>p%p9i(BefHmXnI#qH~Dg54p)Ff0aIU%v!L#XZDG90VuqlX<)p%0pmq zqKv~2#IO@_OLP4txiL=juWct{?D|gd4Cq*34m@Z)FN!$Z*RCmjmZ64`Ue23@Q3N6{xl?A=AyvtX{GJG`7U5`6^>RzcSa8 zMoRk;k76KEsFs)w);LLDxkv%L?8kzu^Mm~iGO=XuAGQz$K}$#rtn)OkppC|ukd_ml zlKXXE8Jx!$vTXS<^@604j&w5b2O9P3a{c@3LI(5$_WBR%hK0AuHvM{Ejw?_~wBR{3 zc@KSXiF9lzJ!w9(az=*b1HU%+$9WiMgPS=H-af2ZSqQyrWb*M{j0Ix&|f5V_JXc37UDd<}6;^*Fl$!!8ss$;msk{1-IM2^A4ca#%}v7aftsA?wqFM zR5F5UlQw|=TpKuDsv*X4bHKwpIlgpT=gE=r{0G$Uzcjw{J+(*1Z*MWV0iJ+&prcd> zj8giS`H57w-^wk5G8nJ_W8iGQZDr z4ln;7-=c~Ues)Pd+ve6Q}}Bq&Rlq#7a5DEYO@GpTD(9C3l95t9{)n*QyG6gOVDSdkm(pX z+5X}!otv1e=;#j4asB{nE3;?}1r)Rj$nX4hieU`Gi1mj69_Rd*ceS3amNB5>&HkyW z`nBdn3P}Zm#giyQ%j0b4Is65Ux3X*vP4Azl?57SGnc34&c+H+J-E|(wP*3qNHk;T5 zyZu}Gz^s)1N_%!um)f19{OkC?GSP&yUU;`G>~D8ho!o{*JLpbwMXvKWD@=sJM_8ec z#wf|KcQzKxEfjZ}?J5b$c25HHi8%V;^GQL`C!JeVVD?KEx!>1ac^?_=wYhph`RlvX z$4iFVg(p&Lf1=zYFT63tC>&k_nfE|{L2@KJx=Uz(pkMBm)h#_H5o)vdB9EC!s9&+R za(zuX-p$6T`KsPRvn*WVW(~apr9#T_WP)y0PQ2u+rxFL;sy?`pFF5qC^y)9b?5ms> z{e_~TqS&IYSCtEbHtR!GEBr?^;?pkIef%czf{FT$b>(hHBufmhO#vbp0KW5tWUS*f zGatwnT_m}Ifxe7P^^UoW@m6>E=(;x=CKVbwmT){1X5vG>qIS(=7qvOn*Ax!_5Df$0 z{m1wJ_GKLvKCvpjk$xZJFk$cGOZ23T#X_Be5m{j~yOVuiBs&g&N(X62I$jq*HHDnX zgES*nS7k4ODLW?z5JV%mthi8vJ8*81___{vwtAePi^f1|N0sQ~X?dU0)$~$#a7y7vu?{E#XWSaCtTBfKFi5YIU}^{iwU2>X`;) zz}DhO^_POQqJ8t7!3J5+gUnfyF6bXxDIdGKF|hZl;KP|83**rzMHMm09>c`ZX+Ou% zFhz|l<=>X-=K^-{V_X{uGd)jmN4hX(q0%`!+eVyvJbpL+@UIJhqWWBDsK5&U$=GEo z6q)q4(e9;d8(I>(9ErRH{>r zQQ`*Rq~@+2Q2D}A2~J`bNkdyouWm{|oFZ$sZ)#QL!& z!r3(VwT=nW^fmrLhSG%`g~r6fCvpF4!Gz=8F(1}vr*s^>{z3WsDtep~6_#5V)!X0Q zT4ExkUY{s%;sGNYCD!FayAPh&7Wml7vkm&YFB zvgZNWXe*EA%ya^Qvn=wcjD3ifVD<95b1Ii>?OW~o`VT7i-gXo7;L)mSpf~wN}|y`ungOsxurKr5D0(cikmmoQEdtxbFK+3~(%7cacD$ zl{#-dph0_YPSE{hHFOD|`beW}Xe>4HdzFgW*RS?c{>OR6cNZ?kZ1WvR9BVCJZ?!4p z=##!2o5s-@JxMYUDe?1BVc%{BxVLS6d&Sggp+vMNO+s^)K~0<4P9puLg6!$*eI2+y(bgI{ zY2FT?apE3+bUz6ymRj6{@VcGE?qzOAa&vuNtpSBseBWc#zkiKvJJ#e*8I=X@Tt=yM zF+=c-Q6H<{$}caa<21%Lc^sB|(YTqFG;%bRZVHR=hLy&%yT&v#^jr4+k3e@%g34$1 zGDqRqXsPp)WHs7ZR#Ke5e-IB1tVtJCMC?Ng+Hlm1BZphLjuwA^FA!($tJEqlVbo^y zrj2vJW@!%duTTE>p!$Fs-_s&PDY1L1ueV3cv@_+G&yReWhK=cirb;+ZqYpoaBTa=h zTWD$U0ren4Lti(&PnKp7-`%k`YS$p3|Be@zQP)2 z728=EBtcS+nA@$MF=r6t~NwS9YLfwjwK(fizgZ^TOMNnEn9w72YqA#16TA)j8D zHxd3^Rr*`P!vT*M#Kv_BY%}sFqf29Qlt#I$)*0qy?nN1E=d7^ISm~`x72Bm4|51E+ z%RHcucQbEr-5Mv^fifp<=3LVwG|c7e!a`5KguCA>Di-HHy{*vu@*M7pXiR|V3&Pwa z4#C=6F zqs(IQM3>P$v-&^R@hG)y3Qu0^K|F50aSunwl0ed4tL*@L|BPf84tIidu~^yC1&&}% zz3s%RD@H+*-8lP-N-_088GWqnHkl2Kdrj9Rc6oW9r)es=()Y1ORc0)+g6>j_+s;O* z-+4vd=9uH}(*q+~k5W~3*YgZZm6EwhR;RvnzE>g9e~gA{gyXR&pud@u*kNQDxI;AO zU;HR$o9WLZ5&r^nl}}j?)1BAa{;swp+U$w%M~NEW?JikWPL@iR%GHNk-@?;*>iS3y zIO%1t^39un8V%3giO99V(Coscdyu(O$DuvrsgS2d6|5d(?)uuHn(gOS_miS>rl8Cw zOpXZ%)bG3TRAu%}<37YhQ=0e<`!5Mr-W$$Kl?HBkjfEUN6R*0|emx@@)bShRs9We+ zcqhMqURfvX+5<`-?QSj3XtKYyIn^nTMU+zF#kFKo8SdPSoh4SMnhTAAx0#n3N;tB@ z2aT!CG&6R^J9FRWzS!5JDGrHCzwUIdn^!JMakShyPGv`5-ml1CIZTbL7Z z7;jZWOpdj_zmpDSc9N+BWl83&eX;R#_G|pG$A7*6CT@=c-7E3hKBt_B9`QSg{A*8V zP7`C6fwaysuVJ*t{UY~muV}yh@k4RxEtg)KgiH6$m zR12j$x&{o~N|m)&Pi>2m$Ax4GWu1Dj^ud2WTN&HNnZMU&k-FEucrUq3D}9@Ib5xru zIF5IipLIqleTz^CZ@

p z^(rrDi{N)P#~Rh>%g8@(_u!Ad4ju2^DsGRYW(}|-s=G?1QZalB=U^HE3aKv~DI6=s zs;tADDwg%foVv@FzCQ6BDph4Wq!4;vKMSH+^nGIj{^1fb^%}TPVSWVFGrO zre*>T=#zj%648z{M-YtiGVTe8p-^dy)%*vSyh3yPJ=1mbxplv!g*2#C;Cln2@p(mt z*I?Nzgy!`xQ>G;;2Hri@q5QsONB8qTt?%zj08DP@>flV+dM*U?iOFV&t{lA|oF)ic zA%k7v^IYz8s!BM5-_dsXyV!EE`n}f0YYM&WeS?)9q+M7fUh`RG+8cB34&b9LP@RxJ zS*ON@z1ji#u8dnL@a0qfg?ve#SlA7Ao?9U#mhU92!|IndfS#1nBo5b){uT_Ix; zIOflP2;20E&K}o|edKiIrV;@vdslsi4Wo$zU43?KD&J6;PJhkc-?0@Bo&Yn^IM%T@ znWdsUw>-31a-^kN4_csq8+mx%p&PP@L-bpv@gI52Y40Iymd zc2m_;E8}`sr^j0ctxJSJJ2`)w$i3t4;FINbHo1|2u=iSr?w8vNyl?+*KQqtZS6SOO zhF|t+wf%NG%XRA7SAit=55{2m&Q1fl=FHh85x`+$>8f?vR& z@RNS@lSiYWP1y<9Mm++IhW0zl-|HuT0#$J_4h4zXr2^5wYlLzjtA*Z0?}1Z*J!qQ( z9mFi5VC7{o5SfcXc5v~P*TcKXrzL7n!J=P!yB@uoOfTS_Y+QV?rz=r|4T&L&6E?8G zwPT8-`9Rs8MR{OVQa5DkQ;pzL1YX6z3_)Dv_^`D4LHVZp?{ck;4O-q3vqF0gzZTf0 zZHkF(gKj+efXLmx#Tg$($rR3)c<^b0d;l~XzdzZ39(vx8n02KXa(9YHc&f^qzDrZW z$xjsQN!N^T+8cc}@#;NHhb|R{tH$KdOs2MWFgTtf@R-!-xivNxQ-^e|E@LsqJ z9?#IZ%%xpc-wm58%)JBF}&!f?SgjruhMwWUm=$b4|Ih8 zqZ}Q~(BiBFZ?hdpDRMhpDMQBWTl~PO6%Ln+!g0HX>({BXNgCFv@ud?;_9izNr<0F% zQMg7DB*L=k?7b9QtOeN#%jt0@g^+RN2IB z#3thmx(;Z>`Qa>(K}rtj@JB7wQ*ePwu2qR090MCuBiDR5$-Ost0?LJLX)4hc53`4j?*k8i4?c+2c@yRtVWSD}*sM)wKNlDHOMm{) zVfcMKG&Jy>ua(44OhVmy*`6~PM^Y~SGU@u=qQSU+zO#b;{9|v|zm_k0-XUwfTs_u> zfpsF>b2ybUu-tx8FnG2xH(n!s-(bYdhp%10_erbu2>^Xgz@>VL)CVO~mNu1m_PI%Y zjb5I~q(l{HYKj43kMyU|e2#MfQN#AQAXD`!)Y4XRo?H5wWiAW7$x0klFbax?Kt-~R z$sx|D1L7}1_}M*>%1B)b(rb0~7pPsQoa?23xJ2PZ4fXgm+o?xDv&mqhEux=ahx158 zv|s~hUbD}znTc0g>>Dg)Jwh!S>s2#b*@y;fuw%Hk?yn28HJ^ruN1r z>YR=14C;j^xIB9PkFoEL$9jMNM>@8bJ z8Kq>e+a`PO&F^}v&Z$1_Am&bbeV+x>dKUa#xAp4anwK25Q$twfE5KE7@c+?>=) zY7fE)1X|qxPAI}byN`-q^!V>;n`Yj0L|X!pn4=ln}F6-Mxf7gt)SC<28#SO>dg1sS92(64xPq4 zzQM20H+F%3@;MHkuLyQqcY#?aQ5_8*DcuSP1YGOjhZa_jKkez`$5SLvmm=oApxR~; zEtoy=7MJozWj1!k)7R?3Yu^RUi1wh-mYKK%sN3r;@w-2G^_KP}6t^K&^44D*Xu8?z+y#da@df7~QXd?1fmvAuUm0S25Wxj1*8foLL!)=%O z1*rQw{2#n9vBY}$A$^rO?=2h4?fb$f@~%{<^I#wT8SJ4kvqSwEGbeW&b&yD*=5XnS zzQlz}ZPDwhU$^o_(k=Kf(4&|L!(kGlNp979L0$OIgZtkjLZ`?nDfV=2J|K2^5<;+R zA(W`!|1YXY4EOb(!NZRt9{ut`rqWqR`MZG;5Q zj|cMm$3k$o@1O4k*7H{L>bFTAMJFn80+!Pv2hMFv?0#>SPVix)zwLC|I22xG#iH`@ ztn{jT-rDjdU^$hO(TyM2xmq+9cL>YBe@2u$9zFkTV&?nt)xhTq^_&u`vj4t^zi;d` znb?pZ-;;O)n`8mxpT`BoIAsFsENFkQe48%d4)Th{V9cG|)=nVxN;|FziRIlO%apDG zPaf6=VO5LRv}@lBG9%n3uizHJ$<2G(?w>uIQ2%;v{{A8>qi2aBcD+Aa#@)lNR#QgW zr{&Hzjx$X@U3!DAFaiH+;Y6~K zRgd|In#aG!-@iYec&rYVo$Tv#MX#L0Mz$@l(Ma~Jw&Z4tD$jRck&H_qx;K@YY>o(p z-Dyqg1`S)9hi-`C4`YI1J&-wLgX|+tWo|m%e@Kcw?+U$z%RJ8dwAfnoMR+5w`&XV|OfyO~z*5H{!dT zVTO-d@xq}!g)KD*MC)5?Lu#_Ld8FJ-?D}R@ltrE-|BMrKPYAK>Y7@@Yco7;cIb^>e zOq=rg&r1S1U~s5;F~9Ds@O(CzNKmx!TTJ*`tT$!GqEzIK>-i2Q7j7fg6Mewhoh9&u zsCU5}7wITzy}9kvzKL)fq3@rxJ3_OcgG{CjKH>o}Hjt6uI2(M~V0pZea5WQzSgKO3 z56Dq?6+muWgK^WYg`p2(=YoVxTUGd>W}ee{h_;ICXztxXf$5A}h~%UE1M5ABXcKZ`Hd^lwf6PU? z>{Fh)lV0nyvhZO)`SpdD_l7MQx<70Fa@t~Tu&paK(^{!#w2~;+qXH?*wNLy!@qZuv zKVNw56Aii$*6)A2Wc66y_v97j%TnVOX3_f6*ZWV{U=LK@ZMC6_8$PkUyjVxmv)ZXo zBkX}cF3dL!(&9%!T(AN$_JE}1AIczz(cw_3*L|pW0mx6<3sB|!p^HHMTPr}wgh2mu z0P4#0(I8JC02(oG0~eHi9OIG0!e$?2<$Ze2lZiz0t;xYvTGn5lUnm4sddkzgNwSf| z>Vty7g%XRhJ|Zpxs(2y}+h7U9=n$VtRR!kpCNz z7!A@63T2c#a;wVN|A#i60mo^KMft*G{r=ybLViY`{1ce-#p5G(UjmGt9W6f#X=QOV{NwQ_xI)p!63-`(oW_6wuqUT*>c|-M1<|kJjCXU87FX!FS|Z< z1bpWW5No;fi#EridH`#k=^5B7+#Tig$b|#KlkYg07=i9{9I#*(gxd@8hOuas<1GL< zi7Twm*y{p8SY5r(NncJvrmHKqAD^6w8_0gUXq}Jct5L2=Ws!@Kvi(ton zEGfv3=cw$j4Gd`RO+2ro{3y!B5?s3T_?pl^jfnr+onCfWQeUqXbTG6BQ!`F3XQfM2 zj+BR2zw2oeh`DvHBWON!7-)2?F1R3ut+`=02lGs~vMaVs`IrVj;`s9Nbq}eO;38fW zACm2P(I#hIfb#r7+VkW6L)7XE+mi|&6f=X>osi!QUcOg-ToPLn)y4}Y!Ut1l;T!E$ zFiOkAH-)f%JW>!#gQ`iwLQw5{RSsSg2POHPe;eY2*bdy9$JB=3lyBIb z{|xljo-4si8?Kz{7ayq5Zh05xY|-|&#-Fyc-uS@eb>?^z4)(W2H;&;z0?kxwC*Jz2 z>(^%gL1R#%#Gb8Gy!{I`AgCkNGW~+otCaoUD$TzyOwy;fpzHyekZhq#EMu>zQFyzH zAwq}vB}T3RwQ%Kjy8bTf#QsXMh)?+1PhOcB4C1?6@7zB-1jMqVL8b#+9Yx;v2fT2o zi@ld=&{f8das_A%$G=94wd(IJMIeks{21z5e^=n13C{);4u#!omLTtj3jg%!WyL{Q zx}y680?d5O!xwArj1E1=l@!&d-W89=74s!1Kth!iQ#9%GUO+n>fT;Z7a@S*OA%SYj zU7Y#XqZnNi<*{pjheZm}cY?iV^v=l{S6sUOGZ*^fp#JlpvB$CB-d9t;oKE1TkT?3g zWl6f7+>}X1cl6o1y{+sII`wTBj3Z>5x><9X?~ptATO9)hzgL(s?GnyiIX4SUn(02I zxqz;50b*T3?Y?nyyOy`&_{;5u$pI31zA7ZTAdu{kZHe`>`-cXpU1n|~osU__$qfrW ze?n|*tN@l6^n9dnI6=M*6ZmGtC+)#~qZEu9C{N=h%SJBkcMB4pPxa3@7NdI=MbCtJ z=;RfN8r-Ym&1x6w$do!f18#g}P9h_#d7G7&p8lN3o{S_IkAzjrE|7K%!<$}7m9#fz zexvi=c#D&Bsmtwt=T|-UR_&>2D)X~mlRX13Z%t>FS(d~q4opjbOfErim=`-zm-L!( zw%2CO5a)@_dwy8t?AqUsY8RM}ZchLg1s{Vf_k0)-PrHC3_`yiYYh&(ZVe)HFtDNfkl z#cVy=^zWZfI92p4aTkrT_=MeGx{r16d${kTU@kN>BO~K-U#3HQo{>!OO0q*@joM22 z33ly`{m897^?h=d*{oVtQ+?o23bDqtOtX$=i#ro{v&0VqJ*;nUG(z8DaR4;bbY4mG z5w%@DMsaCDW|tz`e6Z_%JA^rHfc8zJ8yFZ+v*W7906gHQmD zRos~qA8F^PB*Qx&23`!s#fkbfeek4pur+YH5^Uik)F_YxfAHh(`y2|u(Jqr9$MAJS3;kcar$rqXn zt@Y9|^0bgu{B@!JS-!*cMB7&%D2{Y1>IGe`j*e<*|kU-jXK@14CDRCWvcN znOS2A$naL|KJq+T1*#hl`990=t7^AJIvV25GlF=yBNENK-|CPS6m*nUEhWyk#SyzU zImIuUD&+Trgz2|C58Ja%9xcYt=NBZ2ha@W3W54+RScp7PvuBC0o#O}$%B-W)g3wnR zKRicEhEo0agWiZnsmhWIMfha379>t9TvFh;<-Y#>;@={e3%&+JdsbYJ6e#U+l>&6bF&9Q?m zL&VI3={KRO%0%3GI7HWI+F(9(_00EO`7i5fC_@|lksVDV)G1IJR4CC%vdUyC!_ByL zxTse7$6iK$%AG0Ju440?G`9|!L}jbbwTFIV8ufk3tSzKrheNKg@IU1I{=2=D8HvdH z)xuHPVe7$G|MGqZPZiZb|3tbsslt3~*@AK`J@Q*O+0VUz{K(z2gIK8DyvMlbc|9K= zJYuh5;zvgo-|rhO7ETnr4b-b^yg17_m!(DZA6tYd&8~&j%+R z`iM{P2B{s)8?{Q=j?*FWFf*ex693s@+>n=dqA zadS?5`Jr%o_-td{>G0LKv^sKwVC}PdEwgNvF%vKNS(QKe)Ga9Or|?OU=}+2fX^~}D ztsqaa-UAaC#QD;FSkdG{geONVYDx+A6#x5xK$pVJD}k7-%V*heHnrBUbsT^B@6#ql z+~k-rw5`PdeCD#6zMX8Zd?%|-)POcg5mR160eh!S_-YB)G~;!_R{lfRO_7IDLfllz zYgD`Tr!#<-|Vd|zp5|dtf@x90Q2aXCusRk2OC>-5wo;GjfmMpv#zgF_N(MXoV zthTe6`^N)&@))JHBGO}cHNW#5*4E*(dom9lPFe-%Gv$5CXBXDk5$JSU;^&|d_TeA4 zZ)+5Hn@av=r~dg8HHJ;>H@4J$B1T43=*ahTLwm|=Vtvl9g78bB+_a$}V^=a|l$=MK z`Z%j!hhI_InIESU6PNS>DpL5})&2V%)oC}&73(@57fLpcICSe7b7|z@+2?V$toqvc z&9Jtf*sW1AGZfHRNKTs)cUNuKj{JIEO?c9nQG2+vxk7wgZB!$^An#5(Yvqy<=dzKl zhDKonZ)Lbf>>qpd$EJ$qA$}AvI^1bMZ1s6XK;Xo2JquUITw`2RY!VUffbGM1R_V+W z42ga`4CS>F=g5H*oTDo;>{hru0PR z;yoB&Y!*;Lcbj-5jpHu)yE$snU=gLrVXvIxJdJkT_(E53|+vDvfY}O$0%LYlI zz}nK$3nP8!Y>hhjvKLy0c5ffGQ@Zx_=J+pHpC9f6&NSlHRCZmDJR7yXApLHVhguF_ zr2RB1+-_FP^yAx1E01cVxgU0~S>Ytfjoy)EK&w=a`JrDEkB_=kYKw#_XV_m{%oKnD zO_81S-m;XoO*Ko$L_zJ|L)#`)LJ{jWW%X;ug2t1?)&Im33@-x`X10fiwtPE{EBBpJ)Z>Mc@OiL?Ux*=x6eSuS+zRYaHsE*B2*WA5EPh{a;Flu;xAl2Q9)t>nF`G^I77Mg<=<&FOzN+2-MqUE zwR$FKI^2h1_C44HfV#j-z$~KO=Fcwy2sjm6cAdg!4coTX0(C2n* zYSTdc@i+ooiR_W_W}C%jYpQH+NtS1{WBB$gTaOfe*%s|l@7VYG>L2RtxmNMS4U~Ug z!MBX_z*AU}Sw*H}5Q9JjS;2IL4!Q$4DxiT2ZXTiMoQh0G0qTU^s*eY{U)(IO;Tyd$ zrX);Lgt!>O*i&SiIu%Io`5jKB(}T8*h0P_GIGkbb%U1dKpitoJ z`6`eUwgB!TJW?wKk==Y`u(mA5Y3vjty`5pvQ|JISRU}=HE?D|;V4N1PO99C86KobE z#H$lB<1PZhnL(O+q(Tb54?O*=nVS0mle|OxCiO8OHy|yc#9^XREkxB!T+m$Vk1B={ z;4%ywuz|bV`YDvDCn?I(6e^rFx5XOrim!z|me=><*o5m=CR^Qtj`4K8t7BV=byW%b z0G-GFvqT-~8jbYzt(pz#LaEq@l|(m3+03vTQYOLNZ&dt|sVDmI0oKcXalMIU3#|tF z(FJD4pwf^7egYp4KRknuxhGlk5NTz)EmYw!4!tI5I6DB3u_Rwq+A)m$kJRGtEfkAB zj!SnrF-y%)&v)5`s=%+1Nnjn%3Au|DeXzYfIg~Pu5@u7g65JM-H|(FQ%>J3A)X>K^ zGO=Mz?dCog4$F2M^ulq*e%E9Tw8$uQ#s1#GIPqoC=m!qcQf21dT;yp~*6%PG&-5Zg z=8pYN#wVu=;xVK#n1-Uhs?onM17q9_m}qv7o{><(VUmA2(FapH#gen2rk0AdFh|7z zFZ_#qUaGnix}Dq5p+meMIOa7}%O_3?cmNVK1G(+pD3F8dLp)Mlpbv^z0@#Pd2*5W4 zR8~H)Wfi{%D&pMT9!YAa>Rm8%E1!iKL1+QlR3EQ+NVg(Ie~NuLPHfVxVE@Yfze*Gr zb3PE*UP&;o5j&2vdPBh$8B*iOpBbpEM8>}7T49cR+@ZOI-}j&7=l7$3L;-~*DD=6N z=IHdP)SwuTz3!D6<;J5x){xu=%F0gU6d2kW3#`-uFWGqa%o5x-JHh^|zO~L-X}3V& zO>cTLX>U-|cg8T=h9cW(<)s5Kyo4(|nzXf5%s%7n-h8tYVzs1H0m-%(T5@o|^4=2d z(K9#hd!L^;VsJY6rxt+Asl!(kX%8Jn0YUJ}W-dUePEQ_O4W!V5n^CA(45<+T;7u}@ zJc1RMVD$N!d%kIV{e5BPE-!+^o{CXhM#&c_()sO}H6M(%?+8%vn44%{+@SUN7D2Gd ztZ)BuJ_o10KJ4{;o1IC_pSvUxg^E$Am##X#*_0$lx$nuS^n#j5syK6g@O2FkM=bCUU1_R1aYJTcM!ZeQZ>bpzgCG@)nJNw z&j4}MkQ@wMkjMJ45XbSjwV*>0ZLrFw{3@Pc4ro`FEk#!P3YqE5?k`4w+@35s*ospP z_fkv6))bq|=4galHnG4IW~Bd)>ePs?y7*QZ=>34Bvn`;gmhe*`K3$rfIpG$OLBuds z)dSf7@W?8i2MlpfZMWa&*JV?V7p`V~I)o^@0dQmMK@VNuN^Ok_mLsw8n3f=VyyvwE zkfZ~p-XAr<-&vVgI9~PHflqY(&n5&&%5IQk+8<<*Py9VAFpl`nym!hEf)wk%Ts+UJ z&zhp-Y-a+3Rqw-Dag3b_$6 z>lA05Xg6R=v#&$tS!2Xf?=keNH0!q_0De+szFNpV&W-F!(N>+U&ld|mt%WKG=R)i# z&g^(HsN~_4UTNB@KYUf?N=y4-OrGX4cbg%eQ^JnVY} zCz}*t|1B|i&7Z08e40{!v1edDJk#uqTz^mqW0rAe4@C1#fHO}1CH6Nz7i z=sRP8_?$Fbgr_-zU_>5VCPlqG6wQi5b7Z>;laIoaG*=;VO|rNZ%ys475;_vw@dQ`N zy9$Xy38buQIn_^-Upc3V+<4NEq4fr#oprMOlRW>uNe|LpHEG)D4Gs=;=tT+=tKdeJ zMEQ&S5GpM2CS-`iiw8NBuD7WVs8!cJm=?d)e)X437r7A?&9Pbz>)O6 zs<{B%lHMj{B(GMhLWVa3c+fZBOLBhxc~Pt*>0I@Uti+_OBWahecas|3q!Mv|8||^T zDdmeK6WrQ1_hIsrFF8xM3=N`<#+^F0a=oujI*k(P_DC?xOG@ph$GpWQ3If$KL+?uqYgThvpZJ(Keq7qFi)qyS*yCRy1;3EJxJR|A)&{b~HHwx_Jp zZs~NnFD+NVpGWSFoyhKriiWrJu{2v2fpB6Ct4n>C@j6Ho6_i%7^>=~Y5{^F;*3a`o zU2}Q*7$x=jDQyYfYt4J4Rh;5S+jgMa{Yuxm>hKghOs{wr?0Nx4^DZRlriZqO0Q?A)P}$i$5Yxi`EnJ8~o#j^_r4E zd2>&uzs7v3&Wwxkf#Y?9a;>5fN-e(v@mfO+y9oIh*(37erH?XW4o)V=Sk}L0z1iQc zp>Lrev>GM^&HpT3H0a#TK+^4)8Rq@YClBP`M56g^lltJJ@YoECck-~1qZB#)LMym5cx{EWV> zE8L)IZ%u+JIKwTNiT`3foj4XIttP>vT$yGfazyKt(HzyE=kG_psEX%BTiUuz9d4Z| z!1^A4HczUJ8vXus8Vg{>4NOGh4r)rBuFf-m0CK1XhFDrd#14(UsJG?pfRsoZSfVT) zlL>PKVZ3_}gbr&Wp-0GN3chau4bv>iZI!EPe*L7oR*>WD$x1hGk!H+rdTa)mWzu7~ zLY=5%s42X;A9GDHc_tm2ftB^En$U+{#W2-(-%OOm3?o_u`?6K@j6*mCmlAOiFD2cV z$tfO3Oq0sNMG>>_azAxc15Pl&uO=Zg50e zm~>J)gB>%+g8<2+M|l>t&!?BZ?lCJ}9T~CD%n4i4H9&=@1in|D3Jk_LPo=f>FTiz* z1-3T^r&{Ltxj21SJ8BDjx46v03x<_MDS_DU-)t;Mh;hZe>Jr~3?yj?V-&+vE=JN3O zvxedI4f{2|GFlhHIy5)s3-xl8;fjZw$F-zO#&R zUGg)&-R}QFCFF3pw&wA5(jkS$(hkr%2*=7}$r}XipP52pbZRd|59(-zX{j5@Gr27+ zlO>VfS)b%za31DUiL#R=KJ_sEo#m&k@OnP%Th}2v?7juO;${j2e(lj+M3-t_VHxLG zcrB~0^>eXhpT_ovv`4I z<$g9EdI1u=(xZ{_=dN5I6~Um0}bsh9#P@!sG1U zH1I5wR&VWO(4Bd8NzQ*GImKtG=vG}&(ls~x=?;6!a{B2QLC36e*rgnds_Lqnocm$5 zd`ivBBAJ=xCi|>M(ipFNJ4~)~15_&Bz>^lY zT1+oswj+8!%%RNkMhSsDdB~IpsNbtwvJdq`#bOLmu;=n{ACMEKL-wFx8)b0^yG#%A zbr3^PLHPa%uU*$QS+$t>y%fehU`IG#5=s0LNS?U63#28X8tZ$O$%ViT?}m=E36MP2 z8YleQlIe2$VG{3-$Nu&}A5%21Nz*%(rGBt(9BpOGiF6HGH_K+%&d-6gyyt!l%kCol zrWjR>lZ>k3A0z@!L@Lu4L3y~2hWnZv0{{u(=iLwRNMu?3WfFI89&VG~FIkde>U-Mm z4suAI88a9DJV4(w8DzM@L~1Fg%`Y5bH|rS)f7@48KzV!tL*?g zDXI;m>;Z&MHe}y0fuHOY=VK2j{)FlWAr&*1ZlGi*Ze=0X&%b7pe9Z%pphaW9oAL!R zkCIEm2{hd=-D#=b)!2s0yte{MV-CZ%;(L{@PC%HS1x10SMQx2!H%H6Hbl={~^;w)E zIlI{W3Z^ecCVd%9+TK>p6NOtthm*&k8W$*EnogxR8jjch1n!Ez6hA{D)T&Xh9nz8u zYM>(<<;W&zsm1ysZ3>_80avt0wA)*K!VodPb*TQ5r!@5Kvd%Mh3h3L8no-Z5J;46U zxd{GcfrgJajc-q!QLv*T8Ek*3`)t2Utn2^Y1yo;@t^>$^nny< z?Mo34JmRiBCMP>rPl$K;K4D(K+?jsP>r{~J+ls8ts7M&_Q?cHB`!(S)B798AYx(kp z5OHjbxZcX;Z}9;f(#(4S&&g&~XZ-0f+6dk_S-%Xlh?|(a&B#`=cBjJNF?;QcPiqu& z_N*~a1J%m>Yacqfn{DoLM>u`ZDS3R1%{TXER#TS1ibM|_JmH4$G!Zp@*S=!?$=bj6 zxl%`Ljk%JZ@N8)CqZiV(cG9-o$6VaU*aFvS*5fYCi7n8%iwt*G1yLNVroRGe|EeUR z6KU3n=-Vdh0z8Yfgte-xNNJf;fxTkR7$HXJnt)_e>TZ66Pze1jHvc~6KD0y+?nY*i z6$rsFm^l)(h7n@`#t&>=;^S#e$4eCH->EX7E1{x5?1?9vu`Wgm3S?tHrl29Cyt*yu zI5*R12u07hiEZ=O)yKuMoSUTEq!zq~^)!2*hO4@Ls%;|hni8jNuRN8ym0zbk$7wXc z`Z+6Y?ZntI`@D4b`rRM`T}}q56W%RmL}K%0n8g?$xPk9h^cZ zTTV{~6-Vzp3ij)Hy)0KLmb@(_0P|Bi$;)gsX5YAQ+!K?Z;UyHy70at9>#sTnoUfZ_ zPqbIp+JE18Yoq&dC)(mId%JzFuyf|JGF$klM*DW^++KL;zM9+qbLfWret6SFFcUl3 z6N%$@Mu$URjWUw#-w{q^;7E+W=8KCVGZ{39kDZ3+rAo-4vZ@w3JH&{6JzH7u^ zTid(4OgV%GKWQ{q9v2|>3V;)!D&)0rk=%VQzO#ARMD?vK+DHtF)gxl4ee#IARh9L9 zR2ymv?wb`>UoGd6yNF4|sG)`HbwR+C0&%LJ>M8A{vOh0uwvv=+kw9QEa zP^SoE1AJcDe{U}plHgVj`j!?h-`FQgnZte`@jgzzC}zqs;_LoYZ}9x5fpWD5#?AT_ zRdTGYa>mnsbp5HES}rFeQobhBW|!tvE8w=-VfkU-^FyzA1P94(n+ou*;dr4@w0-k0 zYi=t}?U6+hPXB^>x0z!(J7{#jKV3$%;pb`|BUowj5cVv2AosNXD&SNl{_DrOqi# z$T-sE<+82lL}rn%JxTj&m?dkTf0fZ8vg`FyLFaM8rCi ze5|uga{Z!I&NA#K$8`SN(l~ypgit6OTP@+U3RLVLg(++st}{g9CXrno*iktH=BBfP zT{u_tii0}1y$D^HFMFBCP#GD-_ysz=@Y7}FiTo0t?Zl-e!^0YxwHdWqpRH7-n`fID zcXlN`=vCeJ`_=mN-Hv$3czfmdPq6D(xF*Kgl!YJNtBlih__P!EUsM2qFB`Y&R~Fb? zaZC~iFA4weA9+NImNmB3iFt8NyfZJbur8Zq)n_yH3^aOHgd36!&0-pQ+uPL}`-!Qp zJR*P9kQ!9TKtW>hIm~xXif&OiX(U%hmezTI_`0h3u{E;m8WpTYWy))E^Z`Tq7dp<> z>b1_kY1`J=lS=sCFsOJDccNQgi4LRVsOt>Yp9jrf0!3qz;Jfokk4Z2XEzzjb{W%B; z00dZ9-o*=?j>KJR>24!%F0hK&suW{rD+tCAe27}5CEBx4_-2tN+#PY(gJoNNY9sY-+8%p-yQZpyZDf%NycMoE!83Yhq2{+7MjJ_p~PF3Kco2C_`lEMqKZ z$3=c|maasS$-`x6xn~E&CdevRTnzJ`rg3(kM4wS?#hY-9H)K$YsSP{$1GF5tovgx* zR30sA{~$FX7mB}}C$48r#i0zs-(iv1BY4Mbia}071a)mi?{H$`gs@f#bOJIn~W*L6`4j9>vt1pt{Zi$ zkC&?%4}|WXZu*YHgrAKaX#LE}dpP0G5`&h?i#813K8LU~PThqsxTCvEavXOg1^M4z z)5UTU#!C_bUi04RtwJYj^PVcmi2d;ic!-*8slpXI9}2yZ#8rKWx|j9Dl%V@H>xu?d zjGei?eoMD%VI`5->*ZWfK+1b)&P?=(xXGnx2AIIBm9;?*)_)*Hv5d!Y4FqOcoT;1( zftMbWYjkC*47!!INpDyjJcL~{zMR&GcFR9@N?2QkG$YKX5?0#(9QJP?uZ;q0s7fxS zV9`2Dz@2Iye>W~DSgJG29eEU9qC~ia1@Z3%3Tg;9pZdwg+xT?hnEmg8`w?o}gHapq ziCa!Vb(^$`NxZ0g*P?^*A`ktx7nzS?eX71&Rn|n471|CEIJ<%du`sIEEKsiPnYla~ zc_^eWP&$ecGurOJ=y+c)&g0WhIQ?fm^LI=~w%$sw04LTvaOsI8YhcU1%o^vG#!0KP zdM8hHH+75kVqw}?=3|AM<^ltTzj9(yE@STYlew?^zcL#WpH+%79AC^G5(wQiOeZ$? zC>th2_Tc34Ym9jTMW1a8d<6Pw$_aSlz|Qme|KR=q#M6SXchKJaCYTLQrTktn;P@5n z?zy)?^cNBT?T8ZtR^s}1R zhrL=Di1w&7povS(WExx~Q3u?e5iyT6eyF8wA(pvvk1bIaYpxqkChE|;R- zzu^4zGitAeL^--fs!DYtngc=&};~wm47! zN92(Ib_Rc0J5@6*sFMqz>MXOi0Ia$qA}^1N{E#JdHw&x&qTHZy@@4h_f}rgg1fxLg z>R(CXApLSF)gnM~dSG{1$viuE0S{~SixXnVZDLoo@I@=WBn4vua@xg$0{K$=(HJ@s z`RC`WU}`~Vx-B&{IGWq+nOr1SO~S$~^j85Zjfsq3_HoC&kbE*rVQmqzs!4Umn4WVZO8JEC{hY0jI{l6Af^OwP>I1X(=H;e{451ZY zfsp7e)p$3*j$=H{0@2atYTk{Nx(x0O)r`$LLj(1SOs|BT%(A8#qb|fOkJ@I5&m771 zouG)EV-k313#Gge=I%B*ftK&GxP9rK_^q}-8z8?w0z=Wkj5lBNKyOQ=BSDgI8tOj6 zW=C-Z-zj@(B+O{ z3C|fT2M#@3adWA?8F_CxOn@YrcmjuG#l}=aswm5x)XBQXqNo6$qxhZMlP9^LJk*h` zFX94xfmztxIl%5HyS4~ceB5~C$ap~Txs*j#X$Du2K<)xI$%trW$XUtKCpPaqF*#OL zUJN7r?JXcYTcGoE0hK5@seBqVs4;czUjO%xXgW{s>>J^Wt3{l}hH zJc3QCq`+bi6e-qz#Ww1|9Y=uVwZO&bz!c91pZuBRUt!FOA!};@vt%PQ1Oq<2ZRqnC zo!9lzrW3?lrN>asT=mx8ci%0(JE+Gx@d^MDr zTT5}^ggvmgR`kGN#7q;2#@jad%^EXQ=PG@AaPBPt$ zrNd`^+7B}%x5@QgZAviCccsV|X3^~d3iS~IbXLK&FwfUiQsj@ zTN7u=*I1&hn2ERTLFC6(1k{uAVMMK#037uY?7L6a08nzJcvCV7P;B}}w5e$OV4fmu znD1qo&2jy2l3E{2bi_q8zwa7WDca@;i^#$|ld={g4$adLr8h&n8O$2(T$3D1vPh_s z3x43GUN2-(G$%gbbXrqSEu7-g(J0LNbb7>vBpos$kF?yCE3pE^*H#4`(u8LVxOUaJ z3%Qn)dQ^!T8He7*6!?z_+TL*rPeCVR~h`WG$_-k+B@r~;-- z9n4&fzOO+fRiH1fo6l@B2;&%>3i{_3hqcWzM?>~66@umpkHd_nf{0GGo)Gy-YTGXt zgfrv!rRcSeNDQ!tg?pdf!0>t@PuneWquDLuMWCz9mrUfX8)Ni#WE5ru2dj^C$$7;g^0vE6>%Yn z>Cks_Se1(OxGw=D4%mcMA`sCy(yOMfRa0YE#nI;9&xENz-(NoaRE8rLZon=3iQtaZ zjzBbvTdp!uFa>vg!Ya2}$CokyvdjU;$x5amIKK0~7!iK+ag|jM~qwYE=w`-hw zijQw&r{k37uI1gi*kJd{!7q4*cPY7Yn++qoj0{2>OSrT+^?B5W92qO_=>A&IG%ygo zFTqjrpAz@Qr8jLf(rHNi^!3+EQBerba4Vyjv8(-o(1h-@Pr_WepX{<;+=vL0OkGJ{ z7F+lJrQlnj^h%P8+HxcZQsv5De#_d0=9719$hsA>wy8lCk1pxIuuV-b7o)Lxl27dm1}NBEKb$LdQq z!_}%|;M&DIPaZ0gfw53|{`_bPY4qoaHR1%25%oEYl?@l zUtbJ;F3;escUXoYnyWMmB%)o`R!Bb>zqENMxg;ZS{yu>paX4c!ROBX@qwypQKXT_D0%FRl2qfU1njPrwL$mD7_PrdscIvmpzE=c6A03dL5?LY zs{ZPuUP-^?eW=(%lBV&8#e5vD&mI*f3y^$m2O z*G%c=0J!VR-nkFTGEc2pC%@#oUv9S+9l^RZHISZ%l@VIR0Nl_){ z%F$fc;&Mew-3J*ieA!0 zxZZ`bK6cuyDm@E1Y~|9CCw0Lv zp8>)zqJpT<&lxv}w6R!R3UVRHH$Oaj$^d}zpg!2OYtY)8$KcDBGXReP7~M>`O;X>%HH_# zzir}QJ}B}sWfJ@S`h(Q9NI$u>MYVQb!;Oiw7Xp)SjS{gEtOhC=kH?BT1z%ozv$bPV*ocEnA&@g1S4fDeT@oFRY}%{ z5pf~s8_>pKdpwG7A$b5DK_(b$h&!=AwygG2?QsBeCq*OQFDhaz@`pOQIF+;6N+5otOw?g&Khfwl`{0D%!Z-rPKmDrBM&~W)PJAL9~X(*MzKLu?+gJgSY(pe43a8T2 z5gzL^UPnQP853k87jfB!`c(w+7D6n}U$GX$A65}WcL)*Bce)xA;=@v6dEN_yuRA@& zXr#mYg2%#iz;!ur7X(!;6G$rsG`95kGwj|;4_t@T?K-qCQ$1jU0pKAjv%b=Tn9o^q zQ9RUD*+7?nMOUnpaP<{>9XjL7PTYH8D)NhMl~Lw-e%zM5U2Vy$3|E2h#W@Qqhxstz z*#+6?lO}x%=qt3Gp|w$^^CGn%tGPQAG#V;nmt{Dv&8wxKFSH@~M~e2xCjRX+o8o%` z85d2O&w(xT0xZ$$GaaWp8nzE~$}~f`);+?>^uNhqJpW>5aqjho-0e`AvIlO>mnxg> z!S!(dedXG0kH#mo-_Ei1Gi`q6@fML?qo1!%kC|nOepS)k<)BZf&j>d$PqO$lC&x3w zjq|@)`|ly~_hZ-|iORxy7vsM5sh$+$e8?apB`vwW00X6M*~HUlc(WH*&ytkMS|IR; z3@Nr!$RRaR{iJ1v-jjj<-wzhvuiYQj^!T-HX<>Kba0p94+RFq|*QPzz5lN%kzY4z` zzot5;_dttl)MITI4rhb<(n3nP~0+&yw&liVFr0xUKqoOC4rSL6;(0*S3M1t(!FA zN_;gJT`$=oAv4i`U)|W-L?1`-dqe||4~b@m&kA&-eH~>uMvibNDZVvByWbssHeGDK zX=E0Yu3r+w_RymwBX#%ByQz&H3;)Eq98fddeiK3Dlbw-W9j{@~Dzn(R{3l_%=M8I-DC9eke)D z;}pyB;IZB-yp>l#-Yoj~lS5!yA!l2KtyXf{`m5QUl9@9R7ZQgx7S8Hs;za#dY0bG(+tX_j@;% zHtmv(Q2cqF4wkm(I%_H)FO+1cF+0H5KD#enVARkr;Q2pqHX_{13-1mwD!qIq2s~sf z2FyVa1T;moOITP3f$TD+wDnbJU8g8CN8SR#9g*A(u{i7kGSTFFLBDDGd}LY&nR{A8 z#_Q-Sq4Mj3feYeiML?EsaDb^F@)-+|``{qzg>2Wg95Ma@(&(7Cu*64|QauUGwtEtP zg*D=ww)Xraze^$FvSyC7FoXo+s(rMtd!zh_j6}d|W~qRI`4=cmJ&BwSZgXW^^Bragt5i=BRyjcSyZ{8}F!Xw}zeU%H=Oo|35*9 z)Yp@4RSxu?wpJ!b2(GEF!#>c0=Eshit21Zg}CtzQdk=PH2 zyQJNVj3+oX7kx5!o%La?~!l_aUu`e__8a6MQNDJ5j~0Wt7IgRtJH1wvK=iq&R` zURZ@jCo~t<^D97Asfd6|;JGq+y4_4yb6b*)^m=;yG{WIG)BP-LGDA;sbbN}<;SAP| z+<6pPX3S^uQJ<44re>t?Y5IoEJ3n<)q;-C*jghr5LzYBrr95=f{lA#R{}Sg=v-Asq zLzfoMgA;%~26F8Mp1N3j7BlTi^DbCO${_4MU7%wfaPF>4V1|{)&$9&P01EVF9gOOF zB-}y0HU~IZQ4hSOo3~*=%gI@dv#b0?}*y)mao3YF85LUAw95s znt>L=1;1n5x4{OA$8JJV!8qfiJ8QIneIDq)I-_r7p1Ba;LZ1HdHl*0=%LbJu*+!ez z!=Agr^`Iu@&v0;PRkbX@ZL!{057i+8Bfs8H@PW|xdT&`U4gv)CeRg$6x!ce zJ}{I3Jv1lQ8-BQ48T*gC0J{Vy5C_jMt|y_TOuJar^mibJ}pkV{xbFa`uE(zUf6=jc$ zCRmk`P3EU34SOUve&&AK6x`P?mr#kr;rBS}Wjh5(bGTEr;8p+s1iPqXwXTy z@LdDqm*ng}UMiO3imx7SUY2!KO1ZF*? z9x@K0z*4(H+llK*%=OG?!B1`Ir0svCFTW%;v0XrDYKi;W5u;N}X8L$d`BA}<+^fZ$ zKH6*%_Jg}Uy7fM~9JlTcMSm86%83h-N&&TAmELrTjWgtHH+ zMDX2hq~b+pqY=G5zlk8|z)^CWhC%A!Jjyd9Zg6FD6ZEhjBzde%7Q!iFF)i)s#rgnMEI?jOg8&k$i$LmLdSX!tga^5J+1a)?qZj8(7V(Rbdq1%@&y6hffuMpV3!S?NY-S)_qHyE$5S&Z2*am#E6z<50qS{{~y#$QiY z2F-;)>-V3d@hud!60kVolw?L~`gjMDvF@zUOt%|RpNmBUz1FQ9QoI^ZRv0OXdYtYgGQt`bU)PEitK_q(TE6p+uY!IDI#tx_0=#;=TLr8a@^( zI$cg8)lork_7dVi+&3n&zZtv>a&38lzO~d?i5fAxZ%45M9TEYy(;pM!u)- zK0=5Qh*aKnUjnKb{6jNBq9B~YH_w31-FqD_&gJ=j_}(>E|BQbtBZkjutZMM-Td+`ACR$SnyuuAt z(ZzVf8BHiJHT=Kk@6)xn`2N5{{d;fmkJklm0!ChjO~$%Xi@}<GrXNe9 z4_C#eXe1Xv3;N79jeb#Ov+kXKT4Gk_eHzAn6ck*&in3SG0u_tTt?oa`IbNs-?`u4w z*C=>@ef=XD<3}PscpQ6#Kl5$mjf}*W?7FB8`@3l-9KKL=*WWm)4qSeZxQX~Yb;!XH zBh+-7XCaz*5|+U;z;C97iSHp9Ee+-(4oOcRBB+1U=>#K$JcXKkCyxY_oTO}`xUl@; zUFCB)M!D72IjCtK`+lI*#AYX#Lw|Q%X%}12I=psf2ao38GBp$3F@a5IqkAy zGPmh>q|zgaHV8i94wx$u{Pal1!AB9W6|wIp!2;2AjMniI+s?)sUxbNBrIi9w*HDGK zxnKNjBAwEAZfCq5(y*%U=!LVqKh$?sYJ0=A2ZlD(b%_IBDSzHG{Kv-GhK15PCizYI z_kjJ9fz~M!Xqk)DMyT_}-t^X$4&`Hd~FI2PP%fOzH+EGJ*7#8kus(ngnd|93AwpPjKPzLv_`%E ztZ;e=LeRI)eX@5Wfs#i8j@E1deptrTJfBAJuEO*!N;)hzi^7=%k3MIp7$aaY*a1-R zH6rD5<;qylP!iiAg!UhLwkZ_4(+Ba&-%nX&$&CPV$l{NjC$d(E=nN42I&4Y<<{%&`1JVjg)LDs4JXP!pkZ%*j?AE4P49=-uN+#)A*IpvL z_JKd&>Ho7%_RStJ5So>zD=yn)Eg+Im8=c9n>HjfMmdU&kNd84Z&Q6U zWIiEi>7sqDq|2-!Vlqa=e!O{wy^zjZES1}xDwipPDPb8=WE$&xFlDJ;`@v} zlfkVYFQa@aZ4Nid#dO~s{X${yTw978`zsXhjiRv!atyIp)GhmLg6D^#KB4(53-{e> zF029pehoOPE94l=Aj8}Ytm-1Zn;DWy2_2X=sVUS1E>PBYEkVm*RxInZv2gBWwY;Mg zL!9l6bmgHyUwjz6a1ns)c_a_}1_)F~Q-`@{dLRS+XV{dgv3``*tPf|Hu z_AVFRJ0GC~m!aMBpeR6UzF?NpLbAVYolSgRN=*x>~yn>kl1O#f63JQpd z^BjJ;436RDm(;s;I8KC5E6RzV?Y4t-6Bc&W#P=XAAOgo`kDSGbGYf(iF!J)AkG`0$ z#GNMN_zTE`73KJpI}mOO9tNcv15(sxVY?<_Yxw+>BK91$L70N(RcVHs7VSjZHAOr@ z#<6a?{0nD{hjup{W_7}Ph`0;E>56ps>8ASFR<#&Ul@5mR%P#IcBTy3CekPUv9bgwl z3tzQsxu*2>iUZlUhL_N6MH=j*`{i&g8thxBXM^HDy+)%H)aALMO`320CeSl_f_7hq zYCyS0fX8{m%*ew6^m`Wq4%zINQYcKLoeEt~^4+_~?UrA25l2UBLhO2C}g}FW~xPyM?XHG5bA^;+l2<7&;d3*(;$Z()8J?;k(qY zhbv;zQnk+*QolbZd7Bnji_kv6tEnGQl4-KM~Ww;61vca^J;}4H+YS_f9 zQ2RIM#r|R4=1;rFKU`bFl~YmgZd?)KK0<;P&l=F?4LXmA;?H5AQj^p>SYC>I$gf>RBI^`eqyGA0e}E)e}Z{i;@z^gCDA1 zNA3*xL34umONsrz{ICCZ0q#btpBfDT@Jfs|NL!tVGB=GWV@HVk<@ITjHX1~Jj#4n5UX z>TJ$fheYx2-4Vr;{d%Weh1ZJvl>xw)jrgU`;U(SK@ZpiViPvtL`M_{jf9BXHl})YV z1`Hgpfo}%qFabzdO-ntfeeVOf#bc{KoEo-mOM52IfH>Hxj^TFGoUlMtMb|nL6lBUk zK=`0OvC~i2`6F*O#YvfBh=XLMDbnO3fAzV(q8ubL87?F@i zg+?mmLe=BE5^F|GvijI#x?R{@xumAzKy35*eSbg0wd zWg?8yyi`GeRmLMq_Zaz_g$o_P97>WcVfb4wEq9&HO-W3Q7RBzWq6a%Yo$*N`N;+8& z{oqF8X1Sz=qgsw;-tv_bL$klrKo)N`w+~)k)G$7id`%l`(je?Mj%EVV4u;ut32AeY z1=y~q&w}qiH+>@eH-F&*^)`H#s??@wjVc-MuZ2^7MNjQd7)3eH&}RPDJO9dQXZWY$ zHf1(e4ysF0UnE1FVh1@+ z2D5KI!)3bv&b!}zt}}MFSU=3Tn#I_Wg)38zaLwD)8aCePPhGnIM?+jLVg9l)TNNOG ziEh?`zxUAa(ta*Urf0_$? zrTm>zM!CmA{uI-cI=}vF*@cmd>bG@F8|xbB;=&eiJ7Ebskm5Dl^E%`Eq7FALGbT?) zFdHB5KfeF>egJ)_jQEsBO2FGy$(JFIW+cCcO{}hZ8|tuYq)5ICucO=J*k))|=jNR= zNK1a%R7TsN&h&f7$y15D-Va9;PbAn4*Si^Z+25=U@=WvNdi`sMb1US`O2KH1V2@t@ z@t7y}vuC0%n4bGf6zShFbpVTmujtB+*CJkO9lGW;1~(%xZ&d{l zI1SO=cu9&6Ac}V2Kb?DKIonP~dxfcr@ZitHeuW(k%cgjHIH&ijGJMj$rQ&>~L&;Pt z;jF>>zCgjZ+|?(Jmk0dblWKF|uHBMhiv*I$30q9sci%%SpdGR*0@jo^%b-d#fFh`B z-=Y)&A3b%~}AX@FnLb}reA@%-KEZhCgVk55$T?ljp(DgEn_gF`xy}j zUy{GN7 zDhemPgpQco5In)piw@GYs*~u_j_`33DpwO3F_rwy$K|OsRrf6q3kla@l|vfKu!W=T zbhVrS(QzI<7#*XbE6vE9>ji-rb#5vrqBg_Aw2krg0nKzd~;SJE&NHcqYW@ zg2`(PA749kGDfFQ859~1-hl{|PJ1vRZAbhXXDY!>g(P5ft3d-<$6Apv)Nuph!7W4u zrlW6J{qP7f07UN$1@erfv{_?wIPRt#z_ER1c5yy5 z{JjoX*h0d)Vy6BtUY5D;Pt9)+ceglj?Nw3s7Xg(3GI z;7(owm|4ej8Wx;#lp?;(B9N#wz_OL@mVQqU%0%_i=YmyV6?POq{i8OGykj>$vljs7g2IX5e$mS5@ zONyungIIk_<*Cq%GSt4U#C;u{yO5nQ+uEN&8`>2R2;^ZR+MTAbn&y)QFHTBU9!{eR zpj{W%YO{gr;7au=)wPS%xJMzKL}Di-0vSC@;l3a33arVHCS8r%fn-ONWtWAQO5sd~ zO;|{#1sjFbTs7S<(2o9zy66(9D+&Rb?m~8R)RQtbB0zsok~+mK=92s|a#YRiGN}vX zwyR86XAv1Os=NBaqs5OpuSOv6MSXuP;msAGSmgnQND7(W+XBl2?6&Pfi_rw&!-s- zR4sNprvAT33I5PU|HR7_V_67U5fmCcb{M2}DbG9~dha%E?|rt-R-V*YGu!N<{XqgB%FbTkt-U8=XDi{t0-NSMds5@4{ zVzss=5PX1n@zPYH%CiX=V)Dz$X*V8>qyf)vm!tC5JOLL+S(weP&ne$ zxJkd3zM}cN5v5~1m7cc{K5^ArdTtKX4RO{E)P&ZpWMew{DKflY%SH6euOYT6$9CZX ztq0G`J^osOG+K~UMn3!pca*;>`3fqh zVz~Vw{m=D1J|LdwbF>h;&I$XGMs)QjZ4tF;3{`gecGdP~u6$1zOByuc?q4{fN;#Fw z`67}+#OuO`R{+v*v5%DXs+;89L>KP7{476j8i124o2d9`ZUb-HW>%yJQ- z5D_L_Uoam-2D~=@yK&Y2`o>a03V|*66b(}L_#z-2BA4FO);FcLMXG?2?J`tjqTkj6 zD#rrO8WmmcCn%lJv8c_4&!@P<8Fl5>VctDEd8(!472G4l{ zd&GGS(HlxFvT|3i-8c7!b3tRmwJzj8tvUTkYa0HoZOFNTvGI2iDG!)Lw{GDN_Ou&I#VHrqe2(D1lo{8#EacZ zd$>1W;HrJ}5j)46-KfK*8qmW`r9X`r&>=vArbFg|B8ZquC<%`Fq`d#=Mu%?%t!|ML zFlu#z-*BqvL&Lk|u+La;D!n|Ls+4d~^xn7=%Z65;MY;k*%G@?l;?3cTkc%Pe?xGj$ z%|3F*aC7rwXmzt$Ded|07bm|D%{)vzn-D>-e7a~4ZQfBOxS_UVPZ&iA$EbHiM4o!F z*m&aFSq>hInH^1@+!IBW7S%A3G}V0PZL9SrDTzk08Bfs|3u)gS>y3gpeOlS5DcBHeC%2H7(4S^Ct#Vt>7JljY#FE0w z%JX?yi&G-k-u+87rE6VdL)#!}{UG3pZRG)<3lZDP`Gen08Am;$98aA`P3q z1FHGs1dA%Dei?KIgRzYqRplw971&ge%86H&UqCznG0t%!_fUCw+~J5OTPOU;6IYuB z(W#aaC~3(J{pixvGrpnB657mx-td;Ae-Jm--|T~lzLg=A&!89`(*VC_WgKyRJKW~{ zxh_3rgQz+O)?`^CuUk`G#A&ourJyg&`I$c3EQ-NxyBs5zqi>eOp8X=Ir7*L2kAdJq zm!Z03!K5O(?;!cfOHkVhs$exg)yF~qsMyFLf z3~bNjmuAO?$G_T=78%ZJqpcKd*A*F#tWRpX=nD}Jfq3+}RJYN#HWAfI-p?TK?P+-=Q2FBa$=5xwpNkhFSpT$qRSMMa_4_ zYz?N*)=}4LY$uubsK-m{*(;?oQ@g>`X#Ok$#q=rLxa!70_$}A#`??z;7RX*_HE#WS z$$h(t?F!Z`vUDehawJ;xh85R@ip}L)&oK(AL3JiePPZiFNwS+X2M#0yjZc970KllBSs5vT{4vySK{A5C^t$ z5Mi`_|89a|9v99$vHbmgAfr1Cy0L3RZY(*h;-|zBpV&+usg;-+Hc5AlFU5<921Ip zDHAKukWI|hl}I6;??&%F+L=v5`f1w;doKz_cY&HfwJzOyqMvE=;dbn6o8oW-VP#xv zilg|ruIiF`v~p@qxqW|S1+LlBSwC<-E!|SZmITT);>G;**nM0+q!7$@(eBE)t<{2& z#Egh6;WSJBexY2{>zq4{3JFBA!;XdOdTCEoj?Kxc`JfS|Db?$whi@NdCEM1S8nBGK zE$iDNKw9qjT>vwp%_EetKF$`d4l5UQa_VcB8ET`O~x>JV2hH|!5D7}Jki2XZ3IeY5qTe~0J6T;bv11vwu@Uy$y! z00XaokpSCR7`b4gN=}dwSyJZ+CmQ`!_S3b9O;7vZ;?Ls23JW`bk@@yaLAx_Evi#^W2qXGX+6&Q+9@dnst zP-hg3U>RF$49gr6&6_oE)W&El>Woah=<2MI$9J=%D7tD$A&0OBY?B1YvPN-Po{8BF zr9aQa>BrlmCDytv9{GO$&68)5?bX${x~$=$Zo*|b*5_iPT|edB>=`Uo>r9T~6KQxk z@_4zM?a>w1^Vc0qSgRczeT@v;NyQ@1wEkJyNb!G%bZYU(-I{`EdG8m>r<9gO%12#!EV7IE9UT_jRu&(#HHvdA z?EK;=wU-V!l8}|?*0Yj<@s5-Bq|Y~$4RzwOiezItJG8B~yZat> zLMAt=AHVH#DtqHAPVdXL!{zpb7ipPc!lCxlN&IyjUW$z7E@ch6QVtiY-125DLMwms zag@2U(-1#sRz9655O*i_>bvKp&R4d)U-az@g|jQa^OZLXvGAyuYT zw8M9kYV@U(bC??qRz|#NR8X_qd0II2=1Sb{W_~@sPm!M@Q16@8QeKP(9{69&z~4|+ zm4kP67X}B)v@FRHlWuET2Wz}4#k-w3-nfmoOWuhWBn~mwMw-91&=;h<;0riROY9JS ztC!rlQi-Lsaqaq=zjh__O-65@P)M*&s}3aIb4Wx8=qKg-hKQ( zYX1K5oczlHRVumDEWuF9{L4`{x(H?Gj-iTuq3L z+xq+QSS(hb&&k8%lJ)L)@87>~Tj|`)CG<~q$2e>aSQ$_^9Zh{pkz;+*o})~$^OGg( zO8M?clDtvaxfp}TMZaDIm11v>;n+#rQ>?$yNxqvr<5Y%gGBid-s~11-yK{z^WapK7 zql`JVvQ0@!GxgZWE#xBDTyRGVE_YinEkuhwFP$bXq%H!q#yM5t;XA7=np#n{tva#q z!%P3f=__qT5q4#b<}04DOVdfD^5A8Hp=Kw0eadi|ynGU#!>CZcjH&-v&~M)1@V#lT zGgUJe>X+b6DkfEQn0hAq@W{6be<{lrisTo`6&z0q?1eby^2VnALh>a!L9f7}lBk z#n*BdKLr<9H|RK6XNAWeJkMWqV!^D$J4-C7(WOtms<%U^IGaa`82z(1B~L%r94)wo zT3!|HtK0LN_gj}LPpa01$0|;B?TZb6%0zMKLT61KSBYGlYgg^yb9DoQ#Tc7;7yx5( zU@dsRig@;nLBHRnsopKS=$_6n?CwbZp4G-LbYp;)jV*mlZ6W6Nwea{7dtz7?;UC#( zPi0OTnpSp|kFyZYwY@DQ-srX**5|jJ`kMzTKkMC;wKy%3x}jd7yx5vb@nm8}am8?n zBY(;9`dYr>Jco>xa)DGp3^5dG69y%wC}Qza>R8px0m5B^a%*g;&j#Iydh9W!RQZfO zxh$&+Yon>VT0{L7KNgPH8y~wf_3`6JhS9M-JU0|t0}b=#jJUd&AB_occ65%NJkbgF zi{92KRjRT13*&DNwC9yf*ot%sFAN!L<1BjKjJ=s_k8qz4l~T>OvuP9+Ke3+G8tSQX zc?_Hu#z0G)UR5Q%w7AILIi}dr59$XkV`F1tnU|yfHfeW@^zw}AHj8-d(Q~>ljtc0- zV%Hg093}5e_daxWTx*ixYU!yxm})TBnKZX+GYAn&I`<~}cAaSo2P?V8b8=93L=!K%t|U;_Hcy`LPHuAjIVc%P zu8mI@*PdksL1NR>PjlqZW$~2dALK#ug5W@3(*v(cKkw~)U5t(%03x zgH{A)jHS;^Z`zfPY}T(wt<1!ZPkdWses;~Fsdh`PEQ%yGZSUH&5<0`$FJ0+}WfiJ8 zqqHpUS7l&P66)2{yRh|R^W%EZiu%|lZT}GtFOo!!^IJcU?X~01GgcojS4nSd&8~l( zQ6F33sD!^*eg`+Wy7^_gm zC3%D3?Y5xo*aL886zr?wp|qTh-+jwQIZpR3O5a~s4xn*i6sLd>ML9LuoU0}Q{|A5(Wl8`5 From 1229f8fd09725227711c91799d0f4dbc18842ba9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Nov 2022 13:15:08 +0300 Subject: [PATCH 446/526] Update 02449_check_dependencies_and_table_shutdown.sql --- .../0_stateless/02449_check_dependencies_and_table_shutdown.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql index 49e1e36acc9..c586db447c0 100644 --- a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql +++ b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.sql @@ -8,7 +8,7 @@ INSERT INTO view VALUES (1, 'v'); CREATE DICTIONARY dict (id UInt32, value String) PRIMARY KEY id SOURCE(CLICKHOUSE(host 'localhost' port tcpPort() user 'default' db currentDatabase() table 'view')) -LAYOUT (HASHED()) LIFETIME (600); +LAYOUT (HASHED()) LIFETIME (MIN 600 MAX 600); SHOW CREATE dict; From db6d867bf2b9f3c5eb97c0f9296e4eee666c7359 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 14 Nov 2022 13:16:23 +0300 Subject: [PATCH 447/526] Update 02449_check_dependencies_and_table_shutdown.reference --- .../02449_check_dependencies_and_table_shutdown.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference index 0ecea03c64e..37cd3d93e39 100644 --- a/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference +++ b/tests/queries/0_stateless/02449_check_dependencies_and_table_shutdown.reference @@ -1,4 +1,4 @@ -CREATE DICTIONARY default.dict\n(\n `id` UInt32,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' DB \'default\' TABLE \'view\'))\nLIFETIME(MIN 0 MAX 600)\nLAYOUT(HASHED()) +CREATE DICTIONARY default.dict\n(\n `id` UInt32,\n `value` String\n)\nPRIMARY KEY id\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' DB \'default\' TABLE \'view\'))\nLIFETIME(MIN 600 MAX 600)\nLAYOUT(HASHED()) CREATE TABLE default.table\n(\n `col` String MATERIALIZED dictGet(\'default.dict\', \'value\', toUInt32(1))\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 1 v 1 v From 986e9a60daa24ac6682e92ccf0247969d1dc1029 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 10 Nov 2022 13:03:54 +0100 Subject: [PATCH 448/526] Analyzer aggregation totals crash fix --- src/Planner/Planner.cpp | 22 +++++++++---------- ...regation_totals_rollup_crash_fix.reference | 8 +++++++ ...er_aggregation_totals_rollup_crash_fix.sql | 5 +++++ 3 files changed, 24 insertions(+), 11 deletions(-) create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.reference create mode 100644 tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql diff --git a/src/Planner/Planner.cpp b/src/Planner/Planner.cpp index 56cc73456ce..28be1a83088 100644 --- a/src/Planner/Planner.cpp +++ b/src/Planner/Planner.cpp @@ -498,17 +498,6 @@ void Planner::buildQueryPlanIfNeeded() should_produce_results_in_order_of_bucket_number); query_plan.addStep(std::move(aggregating_step)); - if (query_node.isGroupByWithRollup()) - { - auto rollup_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); - query_plan.addStep(std::move(rollup_step)); - } - else if (query_node.isGroupByWithCube()) - { - auto cube_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); - query_plan.addStep(std::move(cube_step)); - } - if (query_node.isGroupByWithTotals()) { const auto & having_analysis_result = expression_analysis_result.getHaving(); @@ -528,6 +517,17 @@ void Planner::buildQueryPlanIfNeeded() query_plan.addStep(std::move(totals_having_step)); } + + if (query_node.isGroupByWithRollup()) + { + auto rollup_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); + query_plan.addStep(std::move(rollup_step)); + } + else if (query_node.isGroupByWithCube()) + { + auto cube_step = std::make_unique(query_plan.getCurrentDataStream(), std::move(aggregator_params), true /*final*/, settings.group_by_use_nulls); + query_plan.addStep(std::move(cube_step)); + } } if (!having_executed && expression_analysis_result.hasHaving()) diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.reference b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.reference new file mode 100644 index 00000000000..7c5d87e1389 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.reference @@ -0,0 +1,8 @@ +0 +0 + +0 +((0.0001)) 0 +((0.0001)) 0 + +((0.0001)) 0 diff --git a/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql new file mode 100644 index 00000000000..6cd3e6a9385 --- /dev/null +++ b/tests/queries/0_stateless/02479_analyzer_aggregation_totals_rollup_crash_fix.sql @@ -0,0 +1,5 @@ +SET allow_experimental_analyzer = 1; + +SELECT anyLast(number) FROM numbers(1) GROUP BY number WITH ROLLUP WITH TOTALS; + +SELECT tuple(tuple(0.0001)), anyLast(number) FROM numbers(1) GROUP BY number WITH ROLLUP WITH TOTALS; From 769cef94588c8e0e97a807ae1ede657561fdfdfe Mon Sep 17 00:00:00 2001 From: Alfonso Martinez Date: Mon, 14 Nov 2022 11:51:50 +0100 Subject: [PATCH 449/526] Replaced str with chars directly --- src/Functions/FunctionsJSON.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 0973c1f3e39..2234c582ba6 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -1424,9 +1424,8 @@ public: if (chars.size() > col_str.getN()) return false; - std::string str = reinterpret_cast(chars.data()); - - col_str.insertData(str.data(), str.size()); + chars.resize_fill(col_str.getN()); + col_str.insertData(reinterpret_cast(chars.data()), chars.size()); return true; @@ -1443,9 +1442,9 @@ public: traverse(element, buf); buf.finalize(); - for (unsigned long i = 0; i < fixed_length - chars.size(); ++i) - chars.push_back(0); - + if (chars.size() > fixed_length) + return false; + chars.resize_fill(fixed_length); assert_cast(dest).insertData(reinterpret_cast(chars.data()), chars.size()); return true; From 65f00cc246f83b8d28f25578461bcfb4f5b0e240 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Nov 2022 12:17:21 +0100 Subject: [PATCH 450/526] Skip binary hash calculation if there isn't a checksum reference --- programs/server/Server.cpp | 54 ++++++++++++++++++++------------------ src/Daemon/BaseDaemon.cpp | 38 ++++++++++++++++----------- 2 files changed, 50 insertions(+), 42 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index b412b579539..87282483d09 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -805,41 +805,43 @@ int Server::main(const std::vector & /*args*/) /// that are interpreted (not executed) but can alter the behaviour of the program as well. /// Please keep the below log messages in-sync with the ones in daemon/BaseDaemon.cpp - - String calculated_binary_hash = getHashOfLoadedBinaryHex(); - if (stored_binary_hash.empty()) { - LOG_WARNING(log, "Integrity check of the executable skipped because the reference checksum could not be read." - " (calculated checksum: {})", calculated_binary_hash); - } - else if (calculated_binary_hash == stored_binary_hash) - { - LOG_INFO(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); + LOG_WARNING(log, "Integrity check of the executable skipped because the reference checksum could not be read."); } else { - /// If program is run under debugger, ptrace will fail. - if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1) + String calculated_binary_hash = getHashOfLoadedBinaryHex(); + if (calculated_binary_hash == stored_binary_hash) { - /// Program is run under debugger. Modification of it's binary image is ok for breakpoints. - global_context->addWarningMessage( - fmt::format("Server is run under debugger and its binary image is modified (most likely with breakpoints).", - calculated_binary_hash) - ); + LOG_INFO(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); } else { - throw Exception(ErrorCodes::CORRUPTED_DATA, - "Calculated checksum of the executable ({0}) does not correspond" - " to the reference checksum stored in the executable ({1})." - " This may indicate one of the following:" - " - the executable {2} was changed just after startup;" - " - the executable {2} was corrupted on disk due to faulty hardware;" - " - the loaded executable was corrupted in memory due to faulty hardware;" - " - the file {2} was intentionally modified;" - " - a logical error in the code." - , calculated_binary_hash, stored_binary_hash, executable_path); + /// If program is run under debugger, ptrace will fail. + if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1) + { + /// Program is run under debugger. Modification of it's binary image is ok for breakpoints. + global_context->addWarningMessage(fmt::format( + "Server is run under debugger and its binary image is modified (most likely with breakpoints).", + calculated_binary_hash)); + } + else + { + throw Exception( + ErrorCodes::CORRUPTED_DATA, + "Calculated checksum of the executable ({0}) does not correspond" + " to the reference checksum stored in the executable ({1})." + " This may indicate one of the following:" + " - the executable {2} was changed just after startup;" + " - the executable {2} was corrupted on disk due to faulty hardware;" + " - the loaded executable was corrupted in memory due to faulty hardware;" + " - the file {2} was intentionally modified;" + " - a logical error in the code.", + calculated_binary_hash, + stored_binary_hash, + executable_path); + } } } } diff --git a/src/Daemon/BaseDaemon.cpp b/src/Daemon/BaseDaemon.cpp index 758f85e688f..7283973007b 100644 --- a/src/Daemon/BaseDaemon.cpp +++ b/src/Daemon/BaseDaemon.cpp @@ -355,27 +355,33 @@ private: #if defined(OS_LINUX) /// Write information about binary checksum. It can be difficult to calculate, so do it only after printing stack trace. /// Please keep the below log messages in-sync with the ones in programs/server/Server.cpp - String calculated_binary_hash = getHashOfLoadedBinaryHex(); + if (daemon.stored_binary_hash.empty()) { - LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read." - " (calculated checksum: {})", calculated_binary_hash); - } - else if (calculated_binary_hash == daemon.stored_binary_hash) - { - LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); + LOG_FATAL(log, "Integrity check of the executable skipped because the reference checksum could not be read."); } else { - LOG_FATAL(log, "Calculated checksum of the executable ({0}) does not correspond" - " to the reference checksum stored in the executable ({1})." - " This may indicate one of the following:" - " - the executable was changed just after startup;" - " - the executable was corrupted on disk due to faulty hardware;" - " - the loaded executable was corrupted in memory due to faulty hardware;" - " - the file was intentionally modified;" - " - a logical error in the code." - , calculated_binary_hash, daemon.stored_binary_hash); + String calculated_binary_hash = getHashOfLoadedBinaryHex(); + if (calculated_binary_hash == daemon.stored_binary_hash) + { + LOG_FATAL(log, "Integrity check of the executable successfully passed (checksum: {})", calculated_binary_hash); + } + else + { + LOG_FATAL( + log, + "Calculated checksum of the executable ({0}) does not correspond" + " to the reference checksum stored in the executable ({1})." + " This may indicate one of the following:" + " - the executable was changed just after startup;" + " - the executable was corrupted on disk due to faulty hardware;" + " - the loaded executable was corrupted in memory due to faulty hardware;" + " - the file was intentionally modified;" + " - a logical error in the code.", + calculated_binary_hash, + daemon.stored_binary_hash); + } } #endif From c9ff7e8e012b303bcb35ff3561644189b18c82cb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 14 Nov 2022 11:39:56 +0000 Subject: [PATCH 451/526] Add test --- tests/queries/0_stateless/02480_tlp_nan.reference | 10 ++++++++++ tests/queries/0_stateless/02480_tlp_nan.sql | 15 +++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/02480_tlp_nan.reference create mode 100644 tests/queries/0_stateless/02480_tlp_nan.sql diff --git a/tests/queries/0_stateless/02480_tlp_nan.reference b/tests/queries/0_stateless/02480_tlp_nan.reference new file mode 100644 index 00000000000..ea4aa44fa89 --- /dev/null +++ b/tests/queries/0_stateless/02480_tlp_nan.reference @@ -0,0 +1,10 @@ +nan 0 1 0 +nan 0 1 0 +-inf 0 1 0 +-inf 0 1 0 +\N \N \N 1 +\N \N \N 1 +inf 0 1 0 +inf 0 1 0 +nan 0 1 0 +nan 0 1 0 diff --git a/tests/queries/0_stateless/02480_tlp_nan.sql b/tests/queries/0_stateless/02480_tlp_nan.sql new file mode 100644 index 00000000000..e24bc9a9830 --- /dev/null +++ b/tests/queries/0_stateless/02480_tlp_nan.sql @@ -0,0 +1,15 @@ +-- {echo} +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT sqrt(-1) as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; + +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT -inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; + +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT NULL as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; + +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT inf as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; + +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=1; +SELECT nan as x, not(x), not(not(x)), (not(x)) IS NULL SETTINGS allow_experimental_analyzer=0; From 2dda059cbd414d401e333f04cc648f72775f339a Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 14 Nov 2022 14:09:47 +0100 Subject: [PATCH 452/526] Revert "Randomize keeper fault injection settings in stress tests" --- tests/clickhouse-test | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index a07a39600d8..20e63412d91 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -456,8 +456,6 @@ class SettingsRandomizer: "merge_tree_coarse_index_granularity": lambda: random.randint(2, 32), "optimize_distinct_in_order": lambda: random.randint(0, 1), "optimize_sorting_by_input_stream_properties": lambda: random.randint(0, 1), - "insert_keeper_max_retries": lambda: 20, - "insert_keeper_fault_injection_probability": lambda: 0.01, } @staticmethod From 3c21c1cba7aa211b089cd370e3da904277de1595 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 14 Nov 2022 14:00:47 +0000 Subject: [PATCH 453/526] fixed tests query --- .../example-datasets/github.md | 72 +++++++++---------- 1 file changed, 36 insertions(+), 36 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index a571ad82458..6412aa61d3a 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -1868,12 +1868,12 @@ There are a few ways we can address this question. Focusing on the code to test Note we limit to users with more than 20 changes to focus on regular committers and avoid a bias to one-off contributions. -[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NoJywgJ3B5JywgJ2V4cGVjdCcpKSBBTkQgKHBhdGggTElLRSAnJXRlc3RzJScpKSBBUyB0ZXN0LAogICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkgQU5EIChOT1QgKHBhdGggTElLRSAnJXRlc3RzJScpKSkgQVMgY29kZSwKICAgIGNvZGUgLyAoY29kZSArIHRlc3QpIEFTIHJhdGlvX2NvZGUKRlJPTSBnaXRfY2xpY2tob3VzZS5maWxlX2NoYW5nZXMKR1JPVVAgQlkgYXV0aG9yCkhBVklORyBjb2RlID4gMjAKT1JERVIgQlkgY29kZSBERVNDCkxJTUlUIDIw) +[play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBhdXRob3IsCiAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NxbCcsICdzaCcsICdweScsICdleHBlY3QnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCkdST1VQIEJZIGF1dGhvcgpIQVZJTkcgY29kZSA+IDIwCk9SREVSIEJZIGNvZGUgREVTQwpMSU1JVCAyMA==) ```sql SELECT author, - countIf((file_extension IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sql', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, code / (code + test) AS ratio_code FROM git.file_changes @@ -1883,26 +1883,26 @@ ORDER BY code DESC LIMIT 20 ┌─author───────────────┬─test─┬──code─┬─────────ratio_code─┐ -│ Alexey Milovidov │ 4208 │ 41799 │ 0.9085356576173191 │ -│ Nikolai Kochetov │ 446 │ 13361 │ 0.9676975447236909 │ -│ alesapin │ 1893 │ 8796 │ 0.8229020488352512 │ -│ kssenii │ 649 │ 6769 │ 0.9125101105419251 │ -│ Maksim Kita │ 272 │ 5862 │ 0.9556569938050212 │ -│ Alexander Tokmakov │ 949 │ 5727 │ 0.8578490113840623 │ -│ Vitaly Baranov │ 988 │ 5521 │ 0.848210170533108 │ -│ Ivan Lezhankin │ 270 │ 4698 │ 0.9456521739130435 │ -│ Anton Popov │ 264 │ 4346 │ 0.9427331887201735 │ -│ Ivan │ 862 │ 4269 │ 0.8320015591502631 │ -│ Azat Khuzhin │ 957 │ 3697 │ 0.7943704340352385 │ -│ Amos Bird │ 121 │ 2901 │ 0.9599602911978822 │ -│ proller │ 549 │ 2377 │ 0.8123718386876282 │ -│ chertus │ 17 │ 2359 │ 0.9928451178451179 │ -│ alexey-milovidov │ 167 │ 2321 │ 0.9328778135048231 │ -│ Alexey Arno │ 67 │ 2310 │ 0.9718132099284813 │ -│ Vitaliy Lyudvichenko │ 247 │ 2283 │ 0.9023715415019763 │ -│ Robert Schulze │ 111 │ 2196 │ 0.951885565669701 │ -│ CurtizJ │ 144 │ 2158 │ 0.9374456993918332 │ -│ Alexander Kuzmenkov │ 134 │ 2092 │ 0.9398023360287511 │ +│ Alexey Milovidov │ 6617 │ 41799 │ 0.8633303040317251 │ +│ Nikolai Kochetov │ 916 │ 13361 │ 0.9358408629263851 │ +│ alesapin │ 2408 │ 8796 │ 0.785076758300607 │ +│ kssenii │ 869 │ 6769 │ 0.8862267609321812 │ +│ Maksim Kita │ 799 │ 5862 │ 0.8800480408347096 │ +│ Alexander Tokmakov │ 1472 │ 5727 │ 0.7955271565495208 │ +│ Vitaly Baranov │ 1764 │ 5521 │ 0.7578586135895676 │ +│ Ivan Lezhankin │ 843 │ 4698 │ 0.8478613968597726 │ +│ Anton Popov │ 599 │ 4346 │ 0.8788675429726996 │ +│ Ivan │ 2630 │ 4269 │ 0.6187853312074214 │ +│ Azat Khuzhin │ 1664 │ 3697 │ 0.689610147360567 │ +│ Amos Bird │ 400 │ 2901 │ 0.8788245986064829 │ +│ proller │ 1207 │ 2377 │ 0.6632254464285714 │ +│ chertus │ 453 │ 2359 │ 0.8389046941678521 │ +│ alexey-milovidov │ 303 │ 2321 │ 0.8845274390243902 │ +│ Alexey Arno │ 169 │ 2310 │ 0.9318273497377975 │ +│ Vitaliy Lyudvichenko │ 334 │ 2283 │ 0.8723729461215132 │ +│ Robert Schulze │ 182 │ 2196 │ 0.9234650967199327 │ +│ CurtizJ │ 460 │ 2158 │ 0.8242933537051184 │ +│ Alexander Kuzmenkov │ 298 │ 2092 │ 0.8753138075313808 │ └──────────────────────┴──────┴───────┴────────────────────┘ 20 rows in set. Elapsed: 0.034 sec. Processed 266.05 thousand rows, 4.65 MB (7.93 million rows/s., 138.76 MB/s.) @@ -1910,7 +1910,7 @@ LIMIT 20 We can plot this distribution as a histogram. -[play](https://play.clickhouse.com/play?user=play#V0lUSCAoCiAgICAgICAgU0VMRUNUIGhpc3RvZ3JhbSgxMCkocmF0aW9fY29kZSkgQVMgaGlzdAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzaCcsICdweScsICdleHBlY3QnKSkgQU5EIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkgQVMgdGVzdCwKICAgICAgICAgICAgICAgIGNvdW50SWYoKGZpbGVfZXh0ZW5zaW9uIElOICgnaCcsICdjcHAnLCAnc3FsJykpIEFORCAoTk9UIChwYXRoIExJS0UgJyV0ZXN0cyUnKSkpIEFTIGNvZGUsCiAgICAgICAgICAgICAgICBjb2RlIC8gKGNvZGUgKyB0ZXN0KSBBUyByYXRpb19jb2RlCiAgICAgICAgICAgIEZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCiAgICAgICAgICAgIEdST1VQIEJZIGF1dGhvcgogICAgICAgICAgICBIQVZJTkcgY29kZSA+IDIwCiAgICAgICAgICAgIE9SREVSIEJZIGNvZGUgREVTQwogICAgICAgICAgICBMSU1JVCAyMAogICAgICAgICkKICAgICkgQVMgaGlzdApTRUxFQ1QKICAgIGFycmF5Sm9pbihoaXN0KS4xIEFTIGxvd2VyLAogICAgYXJyYXlKb2luKGhpc3QpLjIgQVMgdXBwZXIsCiAgICBiYXIoYXJyYXlKb2luKGhpc3QpLjMsIDAsIDEwMCwgNTAwKSBBUyBiYXI=) +[play](https://play.clickhouse.com/play?user=play#V0lUSCAoCiAgICAgICAgU0VMRUNUIGhpc3RvZ3JhbSgxMCkocmF0aW9fY29kZSkgQVMgaGlzdAogICAgICAgIEZST00KICAgICAgICAoCiAgICAgICAgICAgIFNFTEVDVAogICAgICAgICAgICAgICAgYXV0aG9yLAogICAgICAgICAgICAgICAgY291bnRJZigoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnLCAnc2gnLCAncHknLCAnZXhwZWN0JykpIEFORCAocGF0aCBMSUtFICcldGVzdHMlJykpIEFTIHRlc3QsCiAgICAgICAgICAgICAgICBjb3VudElmKChmaWxlX2V4dGVuc2lvbiBJTiAoJ2gnLCAnY3BwJywgJ3NxbCcpKSBBTkQgKE5PVCAocGF0aCBMSUtFICcldGVzdHMlJykpKSBBUyBjb2RlLAogICAgICAgICAgICAgICAgY29kZSAvIChjb2RlICsgdGVzdCkgQVMgcmF0aW9fY29kZQogICAgICAgICAgICBGUk9NIGdpdF9jbGlja2hvdXNlLmZpbGVfY2hhbmdlcwogICAgICAgICAgICBHUk9VUCBCWSBhdXRob3IKICAgICAgICAgICAgSEFWSU5HIGNvZGUgPiAyMAogICAgICAgICAgICBPUkRFUiBCWSBjb2RlIERFU0MKICAgICAgICAgICAgTElNSVQgMjAKICAgICAgICApCiAgICApIEFTIGhpc3QKU0VMRUNUCiAgICBhcnJheUpvaW4oaGlzdCkuMSBBUyBsb3dlciwKICAgIGFycmF5Sm9pbihoaXN0KS4yIEFTIHVwcGVyLAogICAgYmFyKGFycmF5Sm9pbihoaXN0KS4zLCAwLCAxMDAsIDUwMCkgQVMgYmFy) ```sql WITH ( @@ -1919,7 +1919,7 @@ WITH ( ( SELECT author, - countIf((file_extension IN ('h', 'cpp', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, + countIf((file_extension IN ('h', 'cpp', 'sql', 'sh', 'py', 'expect')) AND (path LIKE '%tests%')) AS test, countIf((file_extension IN ('h', 'cpp', 'sql')) AND (NOT (path LIKE '%tests%'))) AS code, code / (code + test) AS ratio_code FROM git.file_changes @@ -1934,18 +1934,18 @@ SELECT arrayJoin(hist).2 AS upper, bar(arrayJoin(hist).3, 0, 100, 500) AS bar -┌──────────────lower─┬──────────────upper─┬─bar─────────────────────┐ -│ 0.7943704340352385 │ 0.8033711363614333 │ █████ │ -│ 0.8033711363614333 │ 0.8199118213401927 │ █████▋ │ -│ 0.8199118213401927 │ 0.8378309872629326 │ ████████▋ │ -│ 0.8378309872629326 │ 0.8530295909585852 │ █████▋ │ -│ 0.8530295909585852 │ 0.8828273906355679 │ ██████▎ │ -│ 0.8828273906355679 │ 0.9237540060993992 │ ███████████████ │ -│ 0.9237540060993992 │ 0.9477682629346298 │ ██████████████████████▌ │ -│ 0.9477682629346298 │ 0.9627948304418104 │ ███████████████▋ │ -│ 0.9627948304418104 │ 0.981300247585602 │ ██████████ │ -│ 0.981300247585602 │ 0.9928451178451179 │ █████▋ │ -└────────────────────┴────────────────────┴─────────────────────────┘ +┌──────────────lower─┬──────────────upper─┬─bar───────────────────────────┐ +│ 0.6187853312074214 │ 0.6410053888179964 │ █████ │ +│ 0.6410053888179964 │ 0.6764177968945693 │ █████ │ +│ 0.6764177968945693 │ 0.7237343804750673 │ █████ │ +│ 0.7237343804750673 │ 0.7740802855073157 │ █████▋ │ +│ 0.7740802855073157 │ 0.807297655565091 │ ████████▋ │ +│ 0.807297655565091 │ 0.8338381996094653 │ ██████▎ │ +│ 0.8338381996094653 │ 0.8533566747727687 │ ████████▋ │ +│ 0.8533566747727687 │ 0.871392376017531 │ █████████▍ │ +│ 0.871392376017531 │ 0.904916108899021 │ ████████████████████████████▋ │ +│ 0.904916108899021 │ 0.9358408629263851 │ █████████████████▌ │ +└────────────────────┴────────────────────┴───────────────────────────────┘ 10 rows in set. Elapsed: 0.051 sec. Processed 266.05 thousand rows, 4.65 MB (5.24 million rows/s., 91.64 MB/s.) ``` From ee1a1d852d6c92c4a0ff6345b30815537551a07b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Mon, 14 Nov 2022 15:02:55 +0100 Subject: [PATCH 454/526] Address ubsan on readBinary with bool --- src/AggregateFunctions/AggregateFunctionNull.h | 4 +--- src/IO/ReadHelpers.h | 9 +++++++++ tests/queries/0_stateless/02477_invalid_reads.reference | 1 + tests/queries/0_stateless/02477_invalid_reads.sql | 4 ++++ 4 files changed, 15 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index ebc91ea5100..deed06b8bf2 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -153,9 +153,7 @@ public: void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional version, Arena * arena) const override { - /// When deserializing a bool it might trigger UBSAN if the input is not 0 or 1, so it's better to use a Int8 - static_assert(sizeof(bool) == sizeof(Int8)); - Int8 flag = true; + bool flag = true; if constexpr (serialize_flag) readBinary(flag, buf); if (flag) diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 27a24eef804..41c8f1db975 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -1032,6 +1032,15 @@ template requires is_arithmetic_v inline void readBinary(T & x, ReadBuffer & buf) { readPODBinary(x, buf); } +inline void readBinary(bool & x, ReadBuffer & buf) +{ + /// When deserializing a bool it might trigger UBSAN if the input is not 0 or 1, so it's better to treat it as an Int8 + static_assert(sizeof(bool) == sizeof(Int8)); + Int8 flag = 0; + readBinary(flag, buf); + x = (flag != 0); +} + inline void readBinary(String & x, ReadBuffer & buf) { readStringBinary(x, buf); } inline void readBinary(Int128 & x, ReadBuffer & buf) { readPODBinary(x, buf); } inline void readBinary(Int256 & x, ReadBuffer & buf) { readPODBinary(x, buf); } diff --git a/tests/queries/0_stateless/02477_invalid_reads.reference b/tests/queries/0_stateless/02477_invalid_reads.reference index e69de29bb2d..a04038dea65 100644 --- a/tests/queries/0_stateless/02477_invalid_reads.reference +++ b/tests/queries/0_stateless/02477_invalid_reads.reference @@ -0,0 +1 @@ +ubsan 30313233343536373839303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353637383930313233 diff --git a/tests/queries/0_stateless/02477_invalid_reads.sql b/tests/queries/0_stateless/02477_invalid_reads.sql index 37fa8ab098e..08748af3378 100644 --- a/tests/queries/0_stateless/02477_invalid_reads.sql +++ b/tests/queries/0_stateless/02477_invalid_reads.sql @@ -2,6 +2,10 @@ SELECT finalizeAggregation(CAST(unhex('0F00000030'), 'AggregateFunction(min, String)')); -- { serverError 33 } SELECT finalizeAggregation(CAST(unhex('FFFF000030'), 'AggregateFunction(min, String)')); -- { serverError 33 } +-- UBSAN +SELECT 'ubsan', hex(finalizeAggregation(CAST(unhex('4000000030313233343536373839303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353637383930313233010000000000000000'), + 'AggregateFunction(argMax, String, UInt64)'))); + -- aggThrow should check for errors in its input SELECT finalizeAggregation(CAST('', 'AggregateFunction(aggThrow(0.), UInt8)')); -- { serverError 32 } From 4a90f8d3f54360795c5876fc8aff742b325170a9 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 14 Nov 2022 11:34:06 -0500 Subject: [PATCH 455/526] Update permissions-for-queries.md closes https://github.com/ClickHouse/ClickHouse/issues/43068 --- docs/en/operations/settings/permissions-for-queries.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index 3ba62b78cfe..fe803e7b1ee 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -16,14 +16,14 @@ Queries in ClickHouse can be divided into several types: The following settings regulate user permissions by the type of query: -- [readonly](#settings_readonly) — Restricts permissions for all types of queries except DDL queries. +- [readonly](#settings_readonly) — Restricts permissions for Write and DDL queries, and selectively on Change settings queries. - [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. `KILL QUERY` can be performed with any settings. ## readonly {#settings_readonly} -Restricts permissions for reading data, write data and change settings queries. +Restricts permissions for read data, write data, and change settings queries. See how the queries are divided into types [above](#permissions_for_queries). @@ -37,7 +37,7 @@ After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` When using the `GET` method in the [HTTP interface](../../interfaces/http.md), `readonly = 1` is set automatically. To modify data, use the `POST` method. -Setting `readonly = 1` prohibit the user from changing all the settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). +Setting `readonly = 1` prohibits the user from changing settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). Default value: 0 @@ -52,8 +52,6 @@ Possible values: - 0 — DDL queries are not allowed. - 1 — DDL queries are allowed. -You can’t execute `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. +You can’t run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. Default value: 1 - -[Original article](https://clickhouse.com/docs/en/operations/settings/permissions_for_queries/) From 446c029810174155d5604450ba047b901908c9cf Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 14 Nov 2022 11:53:44 -0500 Subject: [PATCH 456/526] make read only clearer --- docs/en/operations/settings/permissions-for-queries.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index fe803e7b1ee..09797146773 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -29,9 +29,9 @@ See how the queries are divided into types [above](#permissions_for_queries). Possible values: -- 0 — All queries are allowed. -- 1 — Only read data queries are allowed. -- 2 — Read data and change settings queries are allowed. +- 0 — Read, Write, and Change settings queries are allowed. +- 1 — Only Read data queries are allowed. +- 2 — Read data and Change settings queries are allowed. After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` settings in the current session. From 4b182e7b58d9de1d0c6070c8d4079f1ff541c936 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 14 Nov 2022 12:17:31 -0500 Subject: [PATCH 457/526] Update set-role.md --- docs/en/sql-reference/statements/set-role.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/set-role.md b/docs/en/sql-reference/statements/set-role.md index bf998d7841e..e017160623e 100644 --- a/docs/en/sql-reference/statements/set-role.md +++ b/docs/en/sql-reference/statements/set-role.md @@ -41,7 +41,7 @@ Purge default roles from a user: SET DEFAULT ROLE NONE TO user ``` -Set all the granted roles as default excepting some of them: +Set all the granted roles as default except for specific roles `role1` and `role2`: ``` sql SET DEFAULT ROLE ALL EXCEPT role1, role2 TO user From e18c97faf7e85e31003cb8d009e1aa49fd52eb48 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 5 Nov 2022 14:09:43 +0100 Subject: [PATCH 458/526] Remove dependencies maskSensitiveInfo() from Context. --- src/Backups/BackupInfo.cpp | 4 +- src/Backups/BackupInfo.h | 2 +- src/Backups/BackupsWorker.cpp | 4 +- src/Backups/registerBackupEngineS3.cpp | 2 +- .../registerBackupEnginesFileAndDisk.cpp | 2 +- src/Interpreters/DDLTask.cpp | 4 +- src/Interpreters/executeQuery.cpp | 4 +- .../maskSensitiveInfoInQueryForLogging.cpp | 161 +++++++----------- .../maskSensitiveInfoInQueryForLogging.h | 6 +- 9 files changed, 73 insertions(+), 116 deletions(-) diff --git a/src/Backups/BackupInfo.cpp b/src/Backups/BackupInfo.cpp index 12bf1f91e86..a10bfb4e82a 100644 --- a/src/Backups/BackupInfo.cpp +++ b/src/Backups/BackupInfo.cpp @@ -93,10 +93,10 @@ BackupInfo BackupInfo::fromAST(const IAST & ast) } -String BackupInfo::toStringForLogging(const ContextPtr & context) const +String BackupInfo::toStringForLogging() const { ASTPtr ast = toAST(); - return maskSensitiveInfoInBackupNameForLogging(serializeAST(*ast), ast, context); + return maskSensitiveInfoInBackupNameForLogging(serializeAST(*ast), ast); } } diff --git a/src/Backups/BackupInfo.h b/src/Backups/BackupInfo.h index 708238e3868..cf5d75bb0d0 100644 --- a/src/Backups/BackupInfo.h +++ b/src/Backups/BackupInfo.h @@ -22,7 +22,7 @@ struct BackupInfo ASTPtr toAST() const; static BackupInfo fromAST(const IAST & ast); - String toStringForLogging(const ContextPtr & context) const; + String toStringForLogging() const; }; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index e235a34a09a..5be2dde9a3d 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -166,7 +166,7 @@ OperationID BackupsWorker::startMakingBackup(const ASTPtr & query, const Context } auto backup_info = BackupInfo::fromAST(*backup_query->backup_name); - String backup_name_for_logging = backup_info.toStringForLogging(context); + String backup_name_for_logging = backup_info.toStringForLogging(); try { addInfo(backup_id, backup_name_for_logging, backup_settings.internal, BackupStatus::CREATING_BACKUP); @@ -388,7 +388,7 @@ OperationID BackupsWorker::startRestoring(const ASTPtr & query, ContextMutablePt try { auto backup_info = BackupInfo::fromAST(*restore_query->backup_name); - String backup_name_for_logging = backup_info.toStringForLogging(context); + String backup_name_for_logging = backup_info.toStringForLogging(); addInfo(restore_id, backup_name_for_logging, restore_settings.internal, BackupStatus::RESTORING); /// Prepare context to use. diff --git a/src/Backups/registerBackupEngineS3.cpp b/src/Backups/registerBackupEngineS3.cpp index 7f5cb7f12a1..afd4bd2b21d 100644 --- a/src/Backups/registerBackupEngineS3.cpp +++ b/src/Backups/registerBackupEngineS3.cpp @@ -47,7 +47,7 @@ void registerBackupEngineS3(BackupFactory & factory) auto creator_fn = []([[maybe_unused]] const BackupFactory::CreateParams & params) -> std::unique_ptr { #if USE_AWS_S3 - String backup_name_for_logging = params.backup_info.toStringForLogging(params.context); + String backup_name_for_logging = params.backup_info.toStringForLogging(); const String & id_arg = params.backup_info.id_arg; const auto & args = params.backup_info.args; diff --git a/src/Backups/registerBackupEnginesFileAndDisk.cpp b/src/Backups/registerBackupEnginesFileAndDisk.cpp index 6a3e1669507..def9c5ba188 100644 --- a/src/Backups/registerBackupEnginesFileAndDisk.cpp +++ b/src/Backups/registerBackupEnginesFileAndDisk.cpp @@ -99,7 +99,7 @@ void registerBackupEnginesFileAndDisk(BackupFactory & factory) { auto creator_fn = [](const BackupFactory::CreateParams & params) -> std::unique_ptr { - String backup_name_for_logging = params.backup_info.toStringForLogging(params.context); + String backup_name_for_logging = params.backup_info.toStringForLogging(); const String & engine_name = params.backup_info.backup_engine_name; if (!params.backup_info.id_arg.empty()) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 980e8f6e7b6..9bcc086d6c7 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -170,11 +170,11 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth); } -void DDLTaskBase::formatRewrittenQuery(ContextPtr context) +void DDLTaskBase::formatRewrittenQuery(ContextPtr) { /// Convert rewritten AST back to string. query_str = queryToString(*query); - query_for_logging = maskSensitiveInfoInQueryForLogging(query_str, query, context); + query_for_logging = maskSensitiveInfoInQueryForLogging(query_str, query); } ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 017485cf76c..a6f3a38303c 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -392,14 +392,14 @@ static std::tuple executeQueryImpl( /// MUST go before any modification (except for prepared statements, /// since it substitute parameters and without them query does not contain /// parameters), to keep query as-is in query_log and server log. - query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast, context); + query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast); } catch (...) { /// Anyway log the query. if (query.empty()) query.assign(begin, std::min(end - begin, static_cast(max_query_size))); - query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast, context); + query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast); logQuery(query_for_logging, context, internal, stage); diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp index fe05283eef5..6b2c865d166 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp @@ -1,12 +1,11 @@ #include #include -#include #include -#include #include #include #include +#include #include #include #include @@ -45,7 +44,6 @@ namespace bool is_create_table_query = false; bool is_create_database_query = false; bool is_create_dictionary_query = false; - ContextPtr context; PasswordWipingMode mode = PasswordWipingMode::Query; }; @@ -272,8 +270,13 @@ namespace /// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']). /// So we will check whether the argument after 'url' is a format. String format; - if (!tryGetEvaluatedConstStringFromArgument(function, data, url_arg_idx + 1, &format)) + if (!tryGetStringFromArgument(function, url_arg_idx + 1, &format, /* allow_literal= */ true, /* allow_identifier= */ false)) + { + /// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`. + /// So it's safer to wipe the next argument just in case. + wipePasswordFromArgument(function, data, url_arg_idx + 2); /// Wipe either `aws_secret_access_key` or `structure`. return; + } if (FormatFactory::instance().getAllFormats().contains(format)) return; /// The argument after 'url' is a format: s3('url', 'format', ...) @@ -309,27 +312,36 @@ namespace else { String database; - if (!tryGetEvaluatedConstDatabaseNameFromArgument(function, data, arg_num, &database)) + if (!tryGetStringFromArgument(function, arg_num, &database, /* allow_literal= */ true, /* allow_identifier= */ true)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + if (tryGetStringFromArgument(function, arg_num + 2, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + wipePasswordFromArgument(function, data, arg_num + 2); /// Wipe either `password` or `user`. + if (tryGetStringFromArgument(function, arg_num + 3, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + wipePasswordFromArgument(function, data, arg_num + 3); /// Wipe either `password` or `sharding_key`. return; - ++arg_num; + } + ++arg_num; auto qualified_name = QualifiedTableName::parseFromString(database); if (qualified_name.database.empty()) ++arg_num; /// skip 'table' argument } - /// Check if username and password are specified - /// (sharding_key can be of any type so while we're getting string literals they're username & password). - String username, password; - bool username_specified = tryGetStringFromArgument(function, arg_num, &username); - bool password_specified = username_specified && tryGetStringFromArgument(function, arg_num + 1, &password); + /// Skip username. + ++arg_num; - if (password_specified) - { - /// Password is specified so we do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - wipePasswordFromArgument(function, data, arg_num + 1); - } + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(function, arg_num, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false); + if (can_be_password) + wipePasswordFromArgument(function, data, arg_num); } static void wipePasswordFromEncryptionFunctionArguments(ASTFunction & function, Data & data) @@ -410,7 +422,7 @@ namespace data.password_was_hidden = true; } - static bool tryGetNumArguments(const ASTFunction & function, size_t * num_arguments) + static bool tryGetNumArguments(const ASTFunction & function, size_t * res) { if (!function.arguments) return false; @@ -420,11 +432,13 @@ namespace return false; /// return false because we don't want to validate query here const auto & arguments = expr_list->children; - *num_arguments = arguments.size(); + if (res) + *res = arguments.size(); return true; } - static bool tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * value) + static bool + tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * res, bool allow_literal, bool allow_identifier) { if (!function.arguments) return false; @@ -437,86 +451,30 @@ namespace if (arg_idx >= arguments.size()) return false; - const auto * literal = arguments[arg_idx]->as(); - if (!literal || literal->value.getType() != Field::Types::String) - return false; - - *value = literal->value.safeGet(); - return true; - } - - static bool tryGetEvaluatedConstStringFromArgument(const ASTFunction & function, Data & data, size_t arg_idx, String * value) - { - if (!function.arguments) - return false; - - const auto * expr_list = function.arguments->as(); - if (!expr_list) - return false; /// return false because we don't want to validate query here - - const auto & arguments = expr_list->children; - if (arg_idx >= arguments.size()) - return false; - - if constexpr (check_only) - { - data.can_contain_password = true; - return false; - } - ASTPtr argument = arguments[arg_idx]; - try + if (allow_literal) { - argument = evaluateConstantExpressionOrIdentifierAsLiteral(argument, data.context); - } - catch (...) - { - return false; + if (const auto * literal = argument->as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } } - const auto & literal = assert_cast(*argument); - if (literal.value.getType() != Field::Types::String) - return false; - - *value = literal.value.safeGet(); - return true; - } - - static bool tryGetEvaluatedConstDatabaseNameFromArgument(const ASTFunction & function, Data & data, size_t arg_idx, String * value) - { - if (!function.arguments) - return false; - - const auto * expr_list = function.arguments->as(); - if (!expr_list) - return false; /// return false because we don't want to validate query here - - const auto & arguments = expr_list->children; - if (arg_idx >= arguments.size()) - return false; - - if constexpr (check_only) + if (allow_identifier) { - data.can_contain_password = true; - return false; + if (const auto * id = argument->as()) + { + if (res) + *res = id->name(); + return true; + } } - ASTPtr argument = arguments[arg_idx]; - try - { - argument = evaluateConstantExpressionForDatabaseName(argument, data.context); - } - catch (...) - { - return false; - } - - const auto & literal = assert_cast(*argument); - if (literal.value.getType() != Field::Types::String) - return false; - - *value = literal.value.safeGet(); - return true; + return false; } static void visitDictionaryDef(ASTDictionary & dictionary, Data & data) @@ -567,11 +525,10 @@ namespace /// Removes a password or its hash from a query if it's specified there or replaces it with some placeholder. /// This function is used to prepare a query for storing in logs (we don't want logs to contain sensitive information). - bool wipePasswordFromQuery(ASTPtr ast, PasswordWipingMode mode, const ContextPtr & context) + bool wipePasswordFromQuery(ASTPtr ast, PasswordWipingMode mode) { using WipingVisitor = PasswordWipingVisitor; WipingVisitor::Data data; - data.context = context; data.mode = mode; WipingVisitor::Visitor visitor{data}; visitor.visit(ast); @@ -579,7 +536,7 @@ namespace } /// Common utility for masking sensitive information. - String maskSensitiveInfoImpl(const String & query, const ASTPtr & parsed_query, PasswordWipingMode mode, const ContextPtr & context) + String maskSensitiveInfoImpl(const String & query, const ASTPtr & parsed_query, PasswordWipingMode mode) { String res = query; @@ -587,7 +544,7 @@ namespace if (parsed_query && canContainPassword(*parsed_query, mode)) { ASTPtr ast_without_password = parsed_query->clone(); - if (wipePasswordFromQuery(ast_without_password, mode, context)) + if (wipePasswordFromQuery(ast_without_password, mode)) res = serializeAST(*ast_without_password); } @@ -602,22 +559,22 @@ namespace } } - res = res.substr(0, context->getSettingsRef().log_queries_cut_to_length); + //res = res.substr(0, context->getSettingsRef().log_queries_cut_to_length); return res; } } -String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context) +String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query) { - return maskSensitiveInfoImpl(query, parsed_query, PasswordWipingMode::Query, context); + return maskSensitiveInfoImpl(query, parsed_query, PasswordWipingMode::Query); } -String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast, const ContextPtr & context) +String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast) { - return maskSensitiveInfoImpl(backup_name, ast, PasswordWipingMode::BackupName, context); + return maskSensitiveInfoImpl(backup_name, ast, PasswordWipingMode::BackupName); } } diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.h b/src/Interpreters/maskSensitiveInfoInQueryForLogging.h index 3892f89bc52..6284b9a9294 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.h +++ b/src/Interpreters/maskSensitiveInfoInQueryForLogging.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -10,10 +10,10 @@ namespace DB /// Makes a version of a query without sensitive information (e.g. passwords) for logging. /// The parameter `parsed query` is allowed to be nullptr if the query cannot be parsed. /// Does not validate AST, works a best-effort way. -String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query, const ContextPtr & context); +String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query); /// Makes a version of backup name without sensitive information (e.g. passwords) for logging. /// Does not validate AST, works a best-effort way. -String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast, const ContextPtr & context); +String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast); } From 8e99f5fea3e91c862437fee7ff616516f7e5b687 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 5 Nov 2022 15:53:08 +0100 Subject: [PATCH 459/526] Move maskSensitiveInfoInQueryForLogging() to src/Parsers/ --- src/Backups/BackupInfo.cpp | 2 +- src/Backups/DDLAdjustingForBackupVisitor.h | 2 +- src/Common/KnownObjectNames.cpp | 48 +++++++++++++++++++ src/Common/KnownObjectNames.h | 37 ++++++++++++++ src/Databases/DDLDependencyVisitor.h | 2 +- src/Databases/DDLRenamingVisitor.cpp | 2 +- src/Databases/DDLRenamingVisitor.h | 2 +- src/Formats/FormatFactory.cpp | 3 ++ .../UserDefinedSQLFunctionVisitor.h | 2 +- src/Interpreters/ActionsVisitor.h | 2 +- .../AggregateFunctionOfGroupByKeysVisitor.h | 2 +- .../ArithmeticOperationsInAgrFuncOptimize.h | 2 +- src/Interpreters/ArrayJoinedColumnsVisitor.h | 2 +- src/Interpreters/CollectJoinOnKeysVisitor.h | 2 +- src/Interpreters/ColumnAliasesVisitor.h | 2 +- .../ConvertFunctionOrLikeVisitor.h | 2 +- .../ConvertStringsToEnumVisitor.h | 2 +- src/Interpreters/CrossToInnerJoinVisitor.h | 2 +- src/Interpreters/DDLTask.cpp | 2 +- src/Interpreters/DuplicateOrderByVisitor.h | 2 +- .../ExecuteScalarSubqueriesVisitor.h | 2 +- .../ExtractExpressionInfoVisitor.h | 2 +- .../FunctionMaskingArgumentCheckVisitor.h | 2 +- .../GatherFunctionQuantileVisitor.h | 2 +- src/Interpreters/GetAggregatesVisitor.h | 2 +- src/Interpreters/GlobalSubqueriesVisitor.h | 2 +- src/Interpreters/GroupByFunctionKeysVisitor.h | 2 +- .../GroupingSetsRewriterVisitor.h | 2 +- src/Interpreters/IdentifierSemantic.h | 2 +- .../InJoinSubqueriesPreprocessor.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 2 +- .../JoinToSubqueryTransformVisitor.h | 2 +- src/Interpreters/JoinedTables.cpp | 2 +- .../MarkTableIdentifiersVisitor.h | 2 +- src/Interpreters/MonotonicityCheckVisitor.h | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- .../NormalizeSelectWithUnionQueryVisitor.h | 2 +- src/Interpreters/OptimizeIfChains.h | 2 +- .../OptimizeShardingKeyRewriteInVisitor.h | 2 +- src/Interpreters/PredicateRewriteVisitor.h | 2 +- src/Interpreters/QueryAliasesVisitor.h | 2 +- .../RedundantFunctionsInOrderByVisitor.h | 2 +- .../RemoveInjectiveFunctionsVisitor.h | 2 +- src/Interpreters/RenameColumnVisitor.h | 2 +- .../RequiredSourceColumnsVisitor.h | 2 +- src/Interpreters/RewriteAnyFunctionVisitor.h | 2 +- .../RewriteCountDistinctVisitor.h | 2 +- .../RewriteFunctionToSubcolumnVisitor.h | 2 +- src/Interpreters/RewriteOrderByVisitor.hpp | 2 +- .../RewriteSumIfFunctionVisitor.h | 2 +- .../SelectIntersectExceptQueryVisitor.h | 2 +- .../SubstituteColumnOptimizer.cpp | 2 +- src/Interpreters/TableOverrideUtils.cpp | 2 +- .../TranslateQualifiedNamesVisitor.h | 2 +- src/Interpreters/executeQuery.cpp | 2 +- .../InDepthNodeVisitor.h | 0 .../InsertQuerySettingsPushDownVisitor.h | 2 +- .../QueryWithOutputSettingsPushDownVisitor.h | 2 +- .../maskSensitiveInfoInQueryForLogging.cpp | 11 ++--- .../maskSensitiveInfoInQueryForLogging.h | 0 src/Storages/TTLDescription.cpp | 2 +- src/Storages/WindowView/StorageWindowView.cpp | 2 +- .../transformQueryForExternalDatabase.cpp | 2 +- src/TableFunctions/TableFunctionFactory.cpp | 3 ++ 65 files changed, 154 insertions(+), 64 deletions(-) create mode 100644 src/Common/KnownObjectNames.cpp create mode 100644 src/Common/KnownObjectNames.h rename src/{Interpreters => Parsers}/InDepthNodeVisitor.h (100%) rename src/{Interpreters => Parsers}/maskSensitiveInfoInQueryForLogging.cpp (98%) rename src/{Interpreters => Parsers}/maskSensitiveInfoInQueryForLogging.h (100%) diff --git a/src/Backups/BackupInfo.cpp b/src/Backups/BackupInfo.cpp index a10bfb4e82a..009c262ad67 100644 --- a/src/Backups/BackupInfo.cpp +++ b/src/Backups/BackupInfo.cpp @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Backups/DDLAdjustingForBackupVisitor.h b/src/Backups/DDLAdjustingForBackupVisitor.h index 63353dcc000..8c8ac1c1ee5 100644 --- a/src/Backups/DDLAdjustingForBackupVisitor.h +++ b/src/Backups/DDLAdjustingForBackupVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/Common/KnownObjectNames.cpp b/src/Common/KnownObjectNames.cpp new file mode 100644 index 00000000000..45066fc93d4 --- /dev/null +++ b/src/Common/KnownObjectNames.cpp @@ -0,0 +1,48 @@ +#include +#include + + +namespace DB +{ + +bool KnownObjectNames::exists(const String & name) const +{ + std::lock_guard lock{mutex}; + if (names.contains(name)) + return true; + + if (case_insensitive_names.empty()) + { + String lower_name = Poco::toLower(name); + if (case_insensitive_names.contains(lower_name)) + return true; + } + + return false; +} + + +void KnownObjectNames::add(const String & name, bool case_insensitive) +{ + std::lock_guard lock{mutex}; + if (case_insensitive) + case_insensitive_names.emplace(Poco::toLower(name)); + else + names.emplace(name); +} + + +KnownTableFunctionNames & KnownTableFunctionNames::instance() +{ + static KnownTableFunctionNames the_instance; + return the_instance; +} + + +KnownFormatNames & KnownFormatNames::instance() +{ + static KnownFormatNames the_instance; + return the_instance; +} + +} diff --git a/src/Common/KnownObjectNames.h b/src/Common/KnownObjectNames.h new file mode 100644 index 00000000000..f003af69904 --- /dev/null +++ b/src/Common/KnownObjectNames.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class KnownObjectNames +{ +public: + bool exists(const String & name) const; + void add(const String & name, bool case_insensitive = false); + +private: + mutable std::mutex mutex; + std::unordered_set names; + std::unordered_set case_insensitive_names; +}; + + +class KnownTableFunctionNames : public KnownObjectNames +{ +public: + static KnownTableFunctionNames & instance(); +}; + + +class KnownFormatNames : public KnownObjectNames +{ +public: + static KnownFormatNames & instance(); +}; + +} diff --git a/src/Databases/DDLDependencyVisitor.h b/src/Databases/DDLDependencyVisitor.h index d23a7a697a9..7f285af42cb 100644 --- a/src/Databases/DDLDependencyVisitor.h +++ b/src/Databases/DDLDependencyVisitor.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { diff --git a/src/Databases/DDLRenamingVisitor.cpp b/src/Databases/DDLRenamingVisitor.cpp index 7ea5dbeda83..443ae37d66c 100644 --- a/src/Databases/DDLRenamingVisitor.cpp +++ b/src/Databases/DDLRenamingVisitor.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Databases/DDLRenamingVisitor.h b/src/Databases/DDLRenamingVisitor.h index 44146a8ee6b..0dad002483f 100644 --- a/src/Databases/DDLRenamingVisitor.h +++ b/src/Databases/DDLRenamingVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a882fcf5009..bf1168dd8a7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include @@ -444,6 +445,7 @@ void FormatFactory::registerInputFormat(const String & name, InputCreator input_ if (target) throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(input_creator); + KnownFormatNames::instance().add(name); registerFileExtension(name, name); } @@ -453,6 +455,7 @@ void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name if (target) throw Exception("FormatFactory: Non trivial prefix and suffix checker " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(non_trivial_prefix_and_suffix_checker); + KnownFormatNames::instance().add(name); } void FormatFactory::registerAppendSupportChecker(const String & name, AppendSupportChecker append_support_checker) diff --git a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h index 686594c088f..5fc98a42634 100644 --- a/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h +++ b/src/Functions/UserDefined/UserDefinedSQLFunctionVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/ActionsVisitor.h b/src/Interpreters/ActionsVisitor.h index fea013fd075..20782baca6b 100644 --- a/src/Interpreters/ActionsVisitor.h +++ b/src/Interpreters/ActionsVisitor.h @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h index ab01cc6a0b6..447cbbd054a 100644 --- a/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h +++ b/src/Interpreters/AggregateFunctionOfGroupByKeysVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.h b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.h index 81d936aeba5..02a5259ad48 100644 --- a/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.h +++ b/src/Interpreters/ArithmeticOperationsInAgrFuncOptimize.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/ArrayJoinedColumnsVisitor.h b/src/Interpreters/ArrayJoinedColumnsVisitor.h index b64a170cb00..c842b729518 100644 --- a/src/Interpreters/ArrayJoinedColumnsVisitor.h +++ b/src/Interpreters/ArrayJoinedColumnsVisitor.h @@ -9,7 +9,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/CollectJoinOnKeysVisitor.h b/src/Interpreters/CollectJoinOnKeysVisitor.h index e52b0c69591..8ebbcbb4685 100644 --- a/src/Interpreters/CollectJoinOnKeysVisitor.h +++ b/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/ColumnAliasesVisitor.h b/src/Interpreters/ColumnAliasesVisitor.h index b593842e133..b7f271a0381 100644 --- a/src/Interpreters/ColumnAliasesVisitor.h +++ b/src/Interpreters/ColumnAliasesVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Interpreters/ConvertFunctionOrLikeVisitor.h b/src/Interpreters/ConvertFunctionOrLikeVisitor.h index f3da56eb60e..c0000c371ec 100644 --- a/src/Interpreters/ConvertFunctionOrLikeVisitor.h +++ b/src/Interpreters/ConvertFunctionOrLikeVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Interpreters/ConvertStringsToEnumVisitor.h b/src/Interpreters/ConvertStringsToEnumVisitor.h index b1389f40654..d7a77eb651c 100644 --- a/src/Interpreters/ConvertStringsToEnumVisitor.h +++ b/src/Interpreters/ConvertStringsToEnumVisitor.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/CrossToInnerJoinVisitor.h b/src/Interpreters/CrossToInnerJoinVisitor.h index 704be42d3c1..21647feb9f5 100644 --- a/src/Interpreters/CrossToInnerJoinVisitor.h +++ b/src/Interpreters/CrossToInnerJoinVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 9bcc086d6c7..331cc5092c5 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB diff --git a/src/Interpreters/DuplicateOrderByVisitor.h b/src/Interpreters/DuplicateOrderByVisitor.h index de8cb4c8f32..0e1bca60545 100644 --- a/src/Interpreters/DuplicateOrderByVisitor.h +++ b/src/Interpreters/DuplicateOrderByVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h index d702404dab6..4c8f3bd58e5 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.h +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/ExtractExpressionInfoVisitor.h b/src/Interpreters/ExtractExpressionInfoVisitor.h index a8cc3a39f89..873ef0ebb4a 100644 --- a/src/Interpreters/ExtractExpressionInfoVisitor.h +++ b/src/Interpreters/ExtractExpressionInfoVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/FunctionMaskingArgumentCheckVisitor.h b/src/Interpreters/FunctionMaskingArgumentCheckVisitor.h index 1f3c44f8087..548cdee67d9 100644 --- a/src/Interpreters/FunctionMaskingArgumentCheckVisitor.h +++ b/src/Interpreters/FunctionMaskingArgumentCheckVisitor.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/src/Interpreters/GatherFunctionQuantileVisitor.h b/src/Interpreters/GatherFunctionQuantileVisitor.h index 74388e7161a..374c6731e52 100644 --- a/src/Interpreters/GatherFunctionQuantileVisitor.h +++ b/src/Interpreters/GatherFunctionQuantileVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Interpreters/GetAggregatesVisitor.h b/src/Interpreters/GetAggregatesVisitor.h index 036d50ba4d6..e7fb010cd27 100644 --- a/src/Interpreters/GetAggregatesVisitor.h +++ b/src/Interpreters/GetAggregatesVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/GlobalSubqueriesVisitor.h b/src/Interpreters/GlobalSubqueriesVisitor.h index dc9294be878..5698e07fcf9 100644 --- a/src/Interpreters/GlobalSubqueriesVisitor.h +++ b/src/Interpreters/GlobalSubqueriesVisitor.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/GroupByFunctionKeysVisitor.h b/src/Interpreters/GroupByFunctionKeysVisitor.h index e21e71d7a72..0a8e3b4e7f4 100644 --- a/src/Interpreters/GroupByFunctionKeysVisitor.h +++ b/src/Interpreters/GroupByFunctionKeysVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/GroupingSetsRewriterVisitor.h b/src/Interpreters/GroupingSetsRewriterVisitor.h index 3d56bf1917d..4388a3b9b0b 100644 --- a/src/Interpreters/GroupingSetsRewriterVisitor.h +++ b/src/Interpreters/GroupingSetsRewriterVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/IdentifierSemantic.h b/src/Interpreters/IdentifierSemantic.h index 178bd291beb..65b7d87b428 100644 --- a/src/Interpreters/IdentifierSemantic.h +++ b/src/Interpreters/IdentifierSemantic.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp index 58faeb41a15..b0bd645cf50 100644 --- a/src/Interpreters/InJoinSubqueriesPreprocessor.cpp +++ b/src/Interpreters/InJoinSubqueriesPreprocessor.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index 2864b433e00..22386a6e486 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index c695dae6d53..7f12bae06f4 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -21,7 +21,7 @@ #include #include -#include +#include #include diff --git a/src/Interpreters/JoinToSubqueryTransformVisitor.h b/src/Interpreters/JoinToSubqueryTransformVisitor.h index 96420512ae6..596e7274c3c 100644 --- a/src/Interpreters/JoinToSubqueryTransformVisitor.h +++ b/src/Interpreters/JoinToSubqueryTransformVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include diff --git a/src/Interpreters/JoinedTables.cpp b/src/Interpreters/JoinedTables.cpp index b88bb5d1caf..95d7be4acf2 100644 --- a/src/Interpreters/JoinedTables.cpp +++ b/src/Interpreters/JoinedTables.cpp @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/MarkTableIdentifiersVisitor.h b/src/Interpreters/MarkTableIdentifiersVisitor.h index d05c067397b..ba52fd656c7 100644 --- a/src/Interpreters/MarkTableIdentifiersVisitor.h +++ b/src/Interpreters/MarkTableIdentifiersVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/MonotonicityCheckVisitor.h b/src/Interpreters/MonotonicityCheckVisitor.h index c95f5209760..cd10ba1e6c7 100644 --- a/src/Interpreters/MonotonicityCheckVisitor.h +++ b/src/Interpreters/MonotonicityCheckVisitor.h @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 26b8bce1f4a..720636d0434 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1,6 +1,6 @@ #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h index e8194f0dfe1..d9896f29454 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include diff --git a/src/Interpreters/OptimizeIfChains.h b/src/Interpreters/OptimizeIfChains.h index 79a4f497ff8..6387c342f16 100644 --- a/src/Interpreters/OptimizeIfChains.h +++ b/src/Interpreters/OptimizeIfChains.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h index d546db40df7..cc1e285fe86 100644 --- a/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h +++ b/src/Interpreters/OptimizeShardingKeyRewriteInVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Interpreters/PredicateRewriteVisitor.h b/src/Interpreters/PredicateRewriteVisitor.h index d2b9ece9306..cbaba04f7af 100644 --- a/src/Interpreters/PredicateRewriteVisitor.h +++ b/src/Interpreters/PredicateRewriteVisitor.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include namespace DB diff --git a/src/Interpreters/QueryAliasesVisitor.h b/src/Interpreters/QueryAliasesVisitor.h index 6e79cfc77be..d1cf68478db 100644 --- a/src/Interpreters/QueryAliasesVisitor.h +++ b/src/Interpreters/QueryAliasesVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/RedundantFunctionsInOrderByVisitor.h b/src/Interpreters/RedundantFunctionsInOrderByVisitor.h index 60c9fcf2a24..21042d1e25c 100644 --- a/src/Interpreters/RedundantFunctionsInOrderByVisitor.h +++ b/src/Interpreters/RedundantFunctionsInOrderByVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include diff --git a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h index a3bbd562407..8cdade85b6d 100644 --- a/src/Interpreters/RemoveInjectiveFunctionsVisitor.h +++ b/src/Interpreters/RemoveInjectiveFunctionsVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include namespace DB diff --git a/src/Interpreters/RenameColumnVisitor.h b/src/Interpreters/RenameColumnVisitor.h index bf0cc21c630..aae84d04ecf 100644 --- a/src/Interpreters/RenameColumnVisitor.h +++ b/src/Interpreters/RenameColumnVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB { diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.h b/src/Interpreters/RequiredSourceColumnsVisitor.h index f197d126223..0adc80728e8 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.h +++ b/src/Interpreters/RequiredSourceColumnsVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/RewriteAnyFunctionVisitor.h b/src/Interpreters/RewriteAnyFunctionVisitor.h index d29af322711..bbca6d51f1c 100644 --- a/src/Interpreters/RewriteAnyFunctionVisitor.h +++ b/src/Interpreters/RewriteAnyFunctionVisitor.h @@ -3,7 +3,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/RewriteCountDistinctVisitor.h b/src/Interpreters/RewriteCountDistinctVisitor.h index b15c03f9c24..dd1d44e2ae7 100644 --- a/src/Interpreters/RewriteCountDistinctVisitor.h +++ b/src/Interpreters/RewriteCountDistinctVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include "Interpreters/TreeRewriter.h" namespace DB diff --git a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h index 4d064bdee10..71281cc9501 100644 --- a/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h +++ b/src/Interpreters/RewriteFunctionToSubcolumnVisitor.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include namespace DB diff --git a/src/Interpreters/RewriteOrderByVisitor.hpp b/src/Interpreters/RewriteOrderByVisitor.hpp index a3857e046ca..b6ea2828066 100644 --- a/src/Interpreters/RewriteOrderByVisitor.hpp +++ b/src/Interpreters/RewriteOrderByVisitor.hpp @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/RewriteSumIfFunctionVisitor.h b/src/Interpreters/RewriteSumIfFunctionVisitor.h index 86aeef65377..a0d7236bf4c 100644 --- a/src/Interpreters/RewriteSumIfFunctionVisitor.h +++ b/src/Interpreters/RewriteSumIfFunctionVisitor.h @@ -3,7 +3,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/SelectIntersectExceptQueryVisitor.h b/src/Interpreters/SelectIntersectExceptQueryVisitor.h index daf6d2ea0df..764d9d233d4 100644 --- a/src/Interpreters/SelectIntersectExceptQueryVisitor.h +++ b/src/Interpreters/SelectIntersectExceptQueryVisitor.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index da738d3db1e..64b2878a0ab 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include diff --git a/src/Interpreters/TableOverrideUtils.cpp b/src/Interpreters/TableOverrideUtils.cpp index 58e885380bf..9703e3e9e77 100644 --- a/src/Interpreters/TableOverrideUtils.cpp +++ b/src/Interpreters/TableOverrideUtils.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Interpreters/TranslateQualifiedNamesVisitor.h b/src/Interpreters/TranslateQualifiedNamesVisitor.h index 73e45fc7ea0..86e8e5df8d2 100644 --- a/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -4,7 +4,7 @@ #include #include -#include +#include namespace DB { diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index a6f3a38303c..e1966597e7d 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -55,7 +55,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Interpreters/InDepthNodeVisitor.h b/src/Parsers/InDepthNodeVisitor.h similarity index 100% rename from src/Interpreters/InDepthNodeVisitor.h rename to src/Parsers/InDepthNodeVisitor.h diff --git a/src/Parsers/InsertQuerySettingsPushDownVisitor.h b/src/Parsers/InsertQuerySettingsPushDownVisitor.h index d1f161fc89b..0bfc5705077 100644 --- a/src/Parsers/InsertQuerySettingsPushDownVisitor.h +++ b/src/Parsers/InsertQuerySettingsPushDownVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h index fde8a07b555..53b8f64b96a 100644 --- a/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h +++ b/src/Parsers/QueryWithOutputSettingsPushDownVisitor.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp b/src/Parsers/maskSensitiveInfoInQueryForLogging.cpp similarity index 98% rename from src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp rename to src/Parsers/maskSensitiveInfoInQueryForLogging.cpp index 6b2c865d166..ebadfa5425b 100644 --- a/src/Interpreters/maskSensitiveInfoInQueryForLogging.cpp +++ b/src/Parsers/maskSensitiveInfoInQueryForLogging.cpp @@ -1,7 +1,6 @@ -#include +#include -#include -#include +#include #include #include #include @@ -9,7 +8,7 @@ #include #include #include -#include +#include #include #include #include @@ -278,7 +277,7 @@ namespace return; } - if (FormatFactory::instance().getAllFormats().contains(format)) + if (KnownFormatNames::instance().exists(format)) return; /// The argument after 'url' is a format: s3('url', 'format', ...) /// The argument after 'url' is not a format so we do our replacement: @@ -305,7 +304,7 @@ namespace /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. const auto * table_function = arguments[arg_num]->as(); - if (table_function && TableFunctionFactory::instance().isTableFunctionName(table_function->name)) + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) { ++arg_num; } diff --git a/src/Interpreters/maskSensitiveInfoInQueryForLogging.h b/src/Parsers/maskSensitiveInfoInQueryForLogging.h similarity index 100% rename from src/Interpreters/maskSensitiveInfoInQueryForLogging.h rename to src/Parsers/maskSensitiveInfoInQueryForLogging.h diff --git a/src/Storages/TTLDescription.cpp b/src/Storages/TTLDescription.cpp index 41c9c1996b1..46df784986f 100644 --- a/src/Storages/TTLDescription.cpp +++ b/src/Storages/TTLDescription.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/WindowView/StorageWindowView.cpp b/src/Storages/WindowView/StorageWindowView.cpp index c0bc5ad8da9..c4dd9a8442b 100644 --- a/src/Storages/WindowView/StorageWindowView.cpp +++ b/src/Storages/WindowView/StorageWindowView.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/transformQueryForExternalDatabase.cpp b/src/Storages/transformQueryForExternalDatabase.cpp index 51b11680f82..1b3a00c417d 100644 --- a/src/Storages/transformQueryForExternalDatabase.cpp +++ b/src/Storages/transformQueryForExternalDatabase.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/TableFunctions/TableFunctionFactory.cpp b/src/TableFunctions/TableFunctionFactory.cpp index 3d2a72ddc9e..ff001661000 100644 --- a/src/TableFunctions/TableFunctionFactory.cpp +++ b/src/TableFunctions/TableFunctionFactory.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -27,6 +28,8 @@ void TableFunctionFactory::registerFunction( && !case_insensitive_table_functions.emplace(Poco::toLower(name), value).second) throw Exception("TableFunctionFactory: the case insensitive table function name '" + name + "' is not unique", ErrorCodes::LOGICAL_ERROR); + + KnownTableFunctionNames::instance().add(name, (case_sensitiveness == CaseInsensitive)); } TableFunctionPtr TableFunctionFactory::get( From 5d6c622e40aaa73cb1ff9c7ad08a81bcad411d1a Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Mon, 14 Nov 2022 14:20:55 -0500 Subject: [PATCH 460/526] apply suggestions and simplify --- .../settings/permissions-for-queries.md | 38 ++++++++++++------- 1 file changed, 25 insertions(+), 13 deletions(-) diff --git a/docs/en/operations/settings/permissions-for-queries.md b/docs/en/operations/settings/permissions-for-queries.md index 09797146773..c565de9b21a 100644 --- a/docs/en/operations/settings/permissions-for-queries.md +++ b/docs/en/operations/settings/permissions-for-queries.md @@ -16,16 +16,20 @@ Queries in ClickHouse can be divided into several types: The following settings regulate user permissions by the type of query: -- [readonly](#settings_readonly) — Restricts permissions for Write and DDL queries, and selectively on Change settings queries. -- [allow_ddl](#settings_allow_ddl) — Restricts permissions for DDL queries. - -`KILL QUERY` can be performed with any settings. - -## readonly {#settings_readonly} - +## readonly Restricts permissions for read data, write data, and change settings queries. -See how the queries are divided into types [above](#permissions_for_queries). +When set to 1, allows: + +- All types of read queries (like SELECT and equivalent queries). +- Queries that modify only session context (like USE). + +When set to 2, allows the above plus: +- SET and CREATE TEMPORARY TABLE + + :::tip + Queries like EXISTS, DESCRIBE, EXPLAIN, SHOW PROCESSLIST, etc are equivalent to SELECT, because they just do select from system tables. + ::: Possible values: @@ -33,25 +37,33 @@ Possible values: - 1 — Only Read data queries are allowed. - 2 — Read data and Change settings queries are allowed. +Default value: 0 + +:::note After setting `readonly = 1`, the user can’t change `readonly` and `allow_ddl` settings in the current session. When using the `GET` method in the [HTTP interface](../../interfaces/http.md), `readonly = 1` is set automatically. To modify data, use the `POST` method. Setting `readonly = 1` prohibits the user from changing settings. There is a way to prohibit the user from changing only specific settings. Also there is a way to allow changing only specific settings under `readonly = 1` restrictions. For details see [constraints on settings](../../operations/settings/constraints-on-settings.md). +::: -Default value: 0 ## allow_ddl {#settings_allow_ddl} Allows or denies [DDL](https://en.wikipedia.org/wiki/Data_definition_language) queries. -See how the queries are divided into types [above](#permissions_for_queries). - Possible values: - 0 — DDL queries are not allowed. - 1 — DDL queries are allowed. -You can’t run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. - Default value: 1 + +:::note +You cannot run `SET allow_ddl = 1` if `allow_ddl = 0` for the current session. +::: + + +:::note KILL QUERY +`KILL QUERY` can be performed with any combination of readonly and allow_ddl settings. +::: From d557bece0466db7ca574dbf70900ef6d20b80e8d Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Mon, 14 Nov 2022 20:06:41 +0000 Subject: [PATCH 461/526] fix consecutive days query --- .../example-datasets/github.md | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 6412aa61d3a..9a4fbb7da06 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -2345,23 +2345,23 @@ WITH commit_days AS ) SELECT author, - arrayMax(arrayMap(x -> length(x), arraySplit(x -> (x = 0), groupArray(consecutive_day)))) AS max_consecutive_days + arrayMax(arrayMap(x -> length(x), arraySplit(x -> (x = 0), groupArray(consecutive_day)))) - 1 AS max_consecutive_days FROM commit_days GROUP BY author ORDER BY max_consecutive_days DESC LIMIT 10 ┌─author───────────┬─max_consecutive_days─┐ -│ kssenii │ 33 │ -│ Alexey Milovidov │ 31 │ -│ alesapin │ 27 │ -│ Azat Khuzhin │ 24 │ -│ Nikolai Kochetov │ 16 │ -│ Igor Nikonov │ 12 │ -│ feng lv │ 12 │ -│ alexey-milovidov │ 12 │ -│ Maksim Kita │ 12 │ -│ Nikita Vasilev │ 12 │ +│ kssenii │ 32 │ +│ Alexey Milovidov │ 30 │ +│ alesapin │ 26 │ +│ Azat Khuzhin │ 23 │ +│ Nikolai Kochetov │ 15 │ +│ feng lv │ 11 │ +│ alexey-milovidov │ 11 │ +│ Igor Nikonov │ 11 │ +│ Maksim Kita │ 11 │ +│ Nikita Vasilev │ 11 │ └──────────────────┴──────────────────────┘ 10 rows in set. Elapsed: 0.025 sec. Processed 62.78 thousand rows, 395.47 KB (2.54 million rows/s., 16.02 MB/s.) From 28fec44ffcf7d27dfea25206538aae02cde6558f Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 14 Nov 2022 20:45:28 +0000 Subject: [PATCH 462/526] Fix merging null values in AggregateFunctionSumMap --- .../AggregateFunctionSumMap.h | 17 ++- src/Common/FieldVisitorSum.cpp | 7 +- .../02480_max_map_null_totals.reference | 119 ++++++++++++++++++ .../0_stateless/02480_max_map_null_totals.sql | 39 ++++++ 4 files changed, 178 insertions(+), 4 deletions(-) create mode 100644 tests/queries/0_stateless/02480_max_map_null_totals.reference create mode 100644 tests/queries/0_stateless/02480_max_map_null_totals.sql diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 6a8fc9e99d8..cee59fcc907 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -202,7 +202,7 @@ public: auto & merged_maps = this->data(place).merged_maps; for (size_t col = 0, size = values_types.size(); col < size; ++col) { - const auto & array_column = assert_cast(*columns[col + 1]); + const auto & array_column = assert_cast(*columns[col + 1]); const IColumn & value_column = array_column.getData(); const IColumn::Offsets & offsets = array_column.getOffsets(); const size_t values_vec_offset = offsets[row_num - 1]; @@ -532,7 +532,12 @@ private: public: explicit FieldVisitorMax(const Field & rhs_) : rhs(rhs_) {} - bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); } + bool operator() (Null &) const + { + /// Do not update current value, skip nulls + return false; + } + bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot compare AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); } bool operator() (Array & x) const { return compareImpl(x); } @@ -567,7 +572,13 @@ private: public: explicit FieldVisitorMin(const Field & rhs_) : rhs(rhs_) {} - bool operator() (Null &) const { throw Exception("Cannot compare Nulls", ErrorCodes::LOGICAL_ERROR); } + + bool operator() (Null &) const + { + /// Do not update current value, skip nulls + return false; + } + bool operator() (AggregateFunctionStateData &) const { throw Exception("Cannot sum AggregateFunctionStates", ErrorCodes::LOGICAL_ERROR); } bool operator() (Array & x) const { return compareImpl(x); } diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index 2c404c33177..332b1a4255b 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -21,7 +21,12 @@ bool FieldVisitorSum::operator() (UInt64 & x) const bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get(); return x != 0; } -bool FieldVisitorSum::operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } +bool FieldVisitorSum::operator() (Null &) const +{ + /// Do not add anything + return rhs != 0; +} + bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); } bool FieldVisitorSum::operator() (Tuple &) const { throw Exception("Cannot sum Tuples", ErrorCodes::LOGICAL_ERROR); } diff --git a/tests/queries/0_stateless/02480_max_map_null_totals.reference b/tests/queries/0_stateless/02480_max_map_null_totals.reference new file mode 100644 index 00000000000..5cc9b5a495f --- /dev/null +++ b/tests/queries/0_stateless/02480_max_map_null_totals.reference @@ -0,0 +1,119 @@ +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([0],[0]) +([2],[2]) +([1],[1]) + +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) +- +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([0],[0]) +([2],[2]) +([1],[1]) + +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) +- +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) + +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([-1,0],[0,0]) +([1,2],[0,2]) +([0,1],[0,1]) +([-1,0,1,2],[0,0,0,2]) +([0],[0]) +([2],[2]) +([1],[1]) + +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) +([0],[0]) +([2],[2]) +([1],[1]) +([0,2],[0,2]) diff --git a/tests/queries/0_stateless/02480_max_map_null_totals.sql b/tests/queries/0_stateless/02480_max_map_null_totals.sql new file mode 100644 index 00000000000..81e2a5c4243 --- /dev/null +++ b/tests/queries/0_stateless/02480_max_map_null_totals.sql @@ -0,0 +1,39 @@ +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT maxMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT minMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT sumMap([number % 3, number % 4 - 1], [number, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT '-'; + +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT maxMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT minMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT sumMap([number % 3, number % 4 - 1], [number :: Float64, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT '-'; + +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT maxMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT minMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; + +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH TOTALS; +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH ROLLUP; +SELECT sumMap([number % 3, number % 4 - 1], [number :: UInt256, NULL]) FROM numbers(3) GROUP BY number WITH CUBE; From ef45889ecb9113a522036abf6cf332145b99ef26 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 15 Nov 2022 07:15:58 +0800 Subject: [PATCH 463/526] Add function pmod which return non-negative result based on modulo (#42755) * add function positive_modulo * add ducument * fix type deduction of positive_modulo * add function positive_modulo * add ducument * fix type deduction of positive_modulo * add notice * fix typo * fix typo * fix bug * fix ub error * fix ub error --- .../functions/arithmetic-functions.md | 5 ++ src/DataTypes/NumberTraits.h | 9 ++++ src/Functions/DivisionUtils.h | 28 +++++++++++ src/Functions/FunctionBinaryArithmetic.h | 48 ++++++++++--------- src/Functions/IsOperation.h | 2 + src/Functions/modulo.cpp | 19 ++++++++ .../02475_positive_modulo.reference | 4 ++ .../0_stateless/02475_positive_modulo.sql | 4 ++ 8 files changed, 97 insertions(+), 22 deletions(-) create mode 100644 tests/queries/0_stateless/02475_positive_modulo.reference create mode 100644 tests/queries/0_stateless/02475_positive_modulo.sql diff --git a/docs/en/sql-reference/functions/arithmetic-functions.md b/docs/en/sql-reference/functions/arithmetic-functions.md index 9059facb0c6..ece50591ef9 100644 --- a/docs/en/sql-reference/functions/arithmetic-functions.md +++ b/docs/en/sql-reference/functions/arithmetic-functions.md @@ -65,6 +65,11 @@ An exception is thrown when dividing by zero or when dividing a minimal negative Differs from [modulo](#modulo) in that it returns zero when the divisor is zero. +## positive_modulo(a, b) +Calculates the remainder when dividing `a` by `b`. Similar to function `modulo` except that `positive_modulo` always return non-negative number. + +Notice that `positive_modulo` is 4-5 times slower than `modulo`. You should not use `positive_modulo` unless you want to get positive result and don't care about performance too much. + ## negate(a), -a operator Calculates a number with the reverse sign. The result is always signed. diff --git a/src/DataTypes/NumberTraits.h b/src/DataTypes/NumberTraits.h index e63c5bcdcb7..6b068b0d8b1 100644 --- a/src/DataTypes/NumberTraits.h +++ b/src/DataTypes/NumberTraits.h @@ -116,6 +116,15 @@ template struct ResultOfModulo using Type = std::conditional_t || std::is_floating_point_v, Float64, Type0>; }; +template struct ResultOfPositiveModulo +{ + /// function positive_modulo always return non-negative number. + static constexpr size_t size_of_result = sizeof(B); + using Type0 = typename Construct::Type; + using Type = std::conditional_t || std::is_floating_point_v, Float64, Type0>; +}; + + template struct ResultOfModuloLegacy { using Type0 = typename Construct || is_signed_v, false, sizeof(B)>::Type; diff --git a/src/Functions/DivisionUtils.h b/src/Functions/DivisionUtils.h index 98e5c690eb9..f8cffab9f48 100644 --- a/src/Functions/DivisionUtils.h +++ b/src/Functions/DivisionUtils.h @@ -178,4 +178,32 @@ struct ModuloLegacyImpl : ModuloImpl using ResultType = typename NumberTraits::ResultOfModuloLegacy::Type; }; +template +struct PositiveModuloImpl : ModuloImpl +{ + using OriginResultType = typename ModuloImpl::ResultType; + using ResultType = typename NumberTraits::ResultOfPositiveModulo::Type; + + template + static inline Result apply(A a, B b) + { + auto res = ModuloImpl::template apply(a, b); + if constexpr (is_signed_v) + { + if (res < 0) + { + if constexpr (is_unsigned_v) + res += static_cast(b); + else + { + if (b == std::numeric_limits::lowest()) + throw Exception("Division by the most negative number", ErrorCodes::ILLEGAL_DIVISION); + res += b >= 0 ? static_cast(b) : static_cast(-b); + } + } + } + return static_cast(res); + } +}; + } diff --git a/src/Functions/FunctionBinaryArithmetic.h b/src/Functions/FunctionBinaryArithmetic.h index 7e4dc387b66..0a79ac3b0d9 100644 --- a/src/Functions/FunctionBinaryArithmetic.h +++ b/src/Functions/FunctionBinaryArithmetic.h @@ -131,50 +131,53 @@ public: using ResultDataType = Switch< /// Decimal cases Case || IsDataTypeDecimal), InvalidType>, - Case && IsDataTypeDecimal && UseLeftDecimal, LeftDataType>, + Case< + IsDataTypeDecimal && IsDataTypeDecimal && UseLeftDecimal, + LeftDataType>, Case && IsDataTypeDecimal, RightDataType>, Case && IsIntegralOrExtended, LeftDataType>, Case && IsIntegralOrExtended, RightDataType>, /// e.g Decimal +-*/ Float, least(Decimal, Float), greatest(Decimal, Float) = Float64 - Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, - DataTypeFloat64>, - Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, - DataTypeFloat64>, + Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, + Case::allow_decimal && IsDataTypeDecimal && IsFloatingPoint, DataTypeFloat64>, - Case::bit_hamming_distance && IsIntegral && IsIntegral, - DataTypeUInt8>, + Case::bit_hamming_distance && IsIntegral && IsIntegral, DataTypeUInt8>, /// Decimal Real is not supported (traditional DBs convert Decimal Real to Real) Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, Case && !IsIntegralOrExtendedOrDecimal, InvalidType>, /// number number -> see corresponding impl - Case && !IsDateOrDateTime, - DataTypeFromFieldType>, + Case && !IsDateOrDateTime, DataTypeFromFieldType>, /// Date + Integral -> Date /// Integral + Date -> Date - Case::plus, Switch< - Case, LeftDataType>, - Case, RightDataType>>>, + Case< + IsOperation::plus, + Switch, LeftDataType>, Case, RightDataType>>>, /// Date - Date -> Int32 /// Date - Integral -> Date - Case::minus, Switch< - Case, DataTypeInt32>, - Case && IsIntegral, LeftDataType>>>, + Case< + IsOperation::minus, + Switch< + Case, DataTypeInt32>, + Case && IsIntegral, LeftDataType>>>, /// least(Date, Date) -> Date /// greatest(Date, Date) -> Date - Case && (IsOperation::least || IsOperation::greatest), + Case< + std::is_same_v && (IsOperation::least || IsOperation::greatest), LeftDataType>, /// Date % Int32 -> Int32 /// Date % Float -> Float64 - Case::modulo, Switch< - Case && IsIntegral, RightDataType>, - Case && IsFloatingPoint, DataTypeFloat64>>>>; + Case< + IsOperation::modulo || IsOperation::positive_modulo, + Switch< + Case && IsIntegral, RightDataType>, + Case && IsFloatingPoint, DataTypeFloat64>>>>; }; } @@ -1176,8 +1179,9 @@ public: bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & arguments) const override { - return ((IsOperation::div_int || IsOperation::modulo) && !arguments[1].is_const) - || (IsOperation::div_floating && (isDecimalOrNullableDecimal(arguments[0].type) || isDecimalOrNullableDecimal(arguments[1].type))); + return ((IsOperation::div_int || IsOperation::modulo || IsOperation::positive_modulo) && !arguments[1].is_const) + || (IsOperation::div_floating + && (isDecimalOrNullableDecimal(arguments[0].type) || isDecimalOrNullableDecimal(arguments[1].type))); } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override @@ -2080,7 +2084,7 @@ public: /// Check the case when operation is divide, intDiv or modulo and denominator is Nullable(Something). /// For divide operation we should check only Nullable(Decimal), because only this case can throw division by zero error. bool division_by_nullable = !arguments[0].type->onlyNull() && !arguments[1].type->onlyNull() && arguments[1].type->isNullable() - && (IsOperation::div_int || IsOperation::modulo + && (IsOperation::div_int || IsOperation::modulo || IsOperation::positive_modulo || (IsOperation::div_floating && (isDecimalOrNullableDecimal(arguments[0].type) || isDecimalOrNullableDecimal(arguments[1].type)))); diff --git a/src/Functions/IsOperation.h b/src/Functions/IsOperation.h index de7701db59a..39f9114f5e0 100644 --- a/src/Functions/IsOperation.h +++ b/src/Functions/IsOperation.h @@ -15,6 +15,7 @@ template struct DivideIntegralOrZeroImpl; template struct LeastBaseImpl; template struct GreatestBaseImpl; template struct ModuloImpl; +template struct PositiveModuloImpl; template struct EqualsOp; template struct NotEqualsOp; template struct LessOrEqualsOp; @@ -53,6 +54,7 @@ struct IsOperation static constexpr bool div_int = IsSameOperation::value; static constexpr bool div_int_or_zero = IsSameOperation::value; static constexpr bool modulo = IsSameOperation::value; + static constexpr bool positive_modulo = IsSameOperation::value; static constexpr bool least = IsSameOperation::value; static constexpr bool greatest = IsSameOperation::value; diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index b2411899160..9cd104cd1dc 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -133,6 +133,7 @@ struct ModuloLegacyByConstantImpl : ModuloByConstantImpl { using Op = ModuloLegacyImpl; }; + } /** Specializations are specified for dividing numbers of the type UInt64 and UInt32 by the numbers of the same sign. @@ -179,4 +180,22 @@ REGISTER_FUNCTION(ModuloLegacy) factory.registerFunction(); } +struct NamePositiveModulo +{ + static constexpr auto name = "positive_modulo"; +}; +using FunctionPositiveModulo = BinaryArithmeticOverloadResolver; + +REGISTER_FUNCTION(PositiveModulo) +{ + factory.registerFunction( + { + R"( +Calculates the remainder when dividing `a` by `b`. Similar to function `modulo` except that `positive_modulo` always return non-negative number. + )", + Documentation::Examples{{"positive_modulo", "SELECT positive_modulo(-1000, 32);"}}, + Documentation::Categories{"Arithmetic"}}, + FunctionFactory::CaseInsensitive); +} + } diff --git a/tests/queries/0_stateless/02475_positive_modulo.reference b/tests/queries/0_stateless/02475_positive_modulo.reference new file mode 100644 index 00000000000..1e707fe2cc8 --- /dev/null +++ b/tests/queries/0_stateless/02475_positive_modulo.reference @@ -0,0 +1,4 @@ +8 +8 +24 +24 diff --git a/tests/queries/0_stateless/02475_positive_modulo.sql b/tests/queries/0_stateless/02475_positive_modulo.sql new file mode 100644 index 00000000000..5f1fdad3c1a --- /dev/null +++ b/tests/queries/0_stateless/02475_positive_modulo.sql @@ -0,0 +1,4 @@ +SELECT positive_modulo(1000, 32); +SELECT positive_modulo(1000, -32); +SELECT positive_modulo(-1000, -32); +SELECT positive_modulo(-1000, 32); \ No newline at end of file From 1de5bb239229c7229d0a9fa793dbdb0642eefcda Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Tue, 15 Nov 2022 07:27:19 +0800 Subject: [PATCH 464/526] Add function canonicalRand (#43124) * add function canonicalRand * add perf test * revert rand.xml --- .../functions/random-functions.md | 5 ++ src/Functions/canonicalRand.cpp | 59 +++++++++++++++++++ tests/performance/rand.xml | 1 + .../0_stateless/01047_nullable_rand.reference | 6 +- .../0_stateless/01047_nullable_rand.sql | 4 ++ 5 files changed, 74 insertions(+), 1 deletion(-) create mode 100644 src/Functions/canonicalRand.cpp diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index d77cc55e5eb..08f2620a009 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -24,6 +24,11 @@ Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type Uses a linear congruential generator. +## canonicalRand +The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). + +Non-deterministic. Return type is Float64. + ## randConstant Produces a constant column with a random value. diff --git a/src/Functions/canonicalRand.cpp b/src/Functions/canonicalRand.cpp new file mode 100644 index 00000000000..d0b8c655e14 --- /dev/null +++ b/src/Functions/canonicalRand.cpp @@ -0,0 +1,59 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +struct CanonicalRandImpl +{ + static void execute(char * output, size_t size) + { + pcg64_fast rng1(randomSeed()); + pcg64_fast rng2(randomSeed()); + std::uniform_real_distribution distribution1(min, max); + std::uniform_real_distribution distribution2(min, max); + + for (const char * end = output + size; output < end; output += 16) + { + unalignedStore(output, distribution1(rng1)); + unalignedStore(output + 8, distribution2(rng2)); + } + } + /// It is guaranteed (by PaddedPODArray) that we can overwrite up to 15 bytes after end. + +private: + const static constexpr Float64 min = 0; + const static constexpr Float64 max = 1; +}; + + +struct NameCanonicalRand +{ + static constexpr auto name = "canonicalRand"; +}; + +class FunctionCanonicalRand : public FunctionRandomImpl +{ +public: + static FunctionPtr create(ContextPtr /*context*/) { return std::make_shared(); } +}; + +} + +REGISTER_FUNCTION(CanonicalRand) +{ + factory.registerFunction({ + R"( +The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). +Non-deterministic. Return type is Float64. + )", + Documentation::Examples{{"canonicalRand", "SELECT canonicalRand()"}}, + Documentation::Categories{"Mathematical"}}); +} + +} diff --git a/tests/performance/rand.xml b/tests/performance/rand.xml index 32c97bb77d6..807b811c147 100644 --- a/tests/performance/rand.xml +++ b/tests/performance/rand.xml @@ -10,5 +10,6 @@ SELECT count() FROM (SELECT rand() FROM zeros(1000000000)) SELECT count() FROM (SELECT rand64() FROM zeros(1000000000)) + SELECT count() FROM (SELECT randUniform(0, 1) FROM zeros(100000000)) SELECT count() FROM (SELECT generateUUIDv4() FROM zeros( 100000000)) diff --git a/tests/queries/0_stateless/01047_nullable_rand.reference b/tests/queries/0_stateless/01047_nullable_rand.reference index aa75dc9a81a..e9993590a84 100644 --- a/tests/queries/0_stateless/01047_nullable_rand.reference +++ b/tests/queries/0_stateless/01047_nullable_rand.reference @@ -1,8 +1,12 @@ UInt32 +Float64 +UInt32 UInt32 -UInt32 +Float64 UInt32 0 0 0 0 +0 +0 diff --git a/tests/queries/0_stateless/01047_nullable_rand.sql b/tests/queries/0_stateless/01047_nullable_rand.sql index 865647aa0fb..9d3c361c543 100644 --- a/tests/queries/0_stateless/01047_nullable_rand.sql +++ b/tests/queries/0_stateless/01047_nullable_rand.sql @@ -1,9 +1,13 @@ select toTypeName(rand(cast(4 as Nullable(UInt8)))); +select toTypeName(canonicalRand(CAST(4 as Nullable(UInt8)))); select toTypeName(randConstant(CAST(4 as Nullable(UInt8)))); select toTypeName(rand(Null)); +select toTypeName(canonicalRand(Null)); select toTypeName(randConstant(Null)); select rand(cast(4 as Nullable(UInt8))) * 0; +select canonicalRand(cast(4 as Nullable(UInt8))) * 0; select randConstant(CAST(4 as Nullable(UInt8))) * 0; select rand(Null) * 0; +select canonicalRand(Null) * 0; select randConstant(Null) * 0; From 5eba20e1198b0bcd3fa91caa2a597e53cd5432fd Mon Sep 17 00:00:00 2001 From: taiyang-li <654010905@qq.com> Date: Tue, 15 Nov 2022 09:46:24 +0800 Subject: [PATCH 465/526] modify return type from Int64 to UInt64 --- docs/en/sql-reference/functions/math-functions.md | 2 +- src/Functions/factorial.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 7f349686d8b..47c27268b09 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -553,7 +553,7 @@ Result: ## factorial(n) -Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is Int64. +Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64. The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. diff --git a/src/Functions/factorial.cpp b/src/Functions/factorial.cpp index 5c46b97c193..b76ef90a48d 100644 --- a/src/Functions/factorial.cpp +++ b/src/Functions/factorial.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes template struct FactorialImpl { - using ResultType = Int64; + using ResultType = UInt64; static const constexpr bool allow_decimal = false; static const constexpr bool allow_fixed_string = false; static const constexpr bool allow_string_integer = false; @@ -101,7 +101,7 @@ REGISTER_FUNCTION(Factorial) factory.registerFunction( { R"( -Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is Int64. +Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64. The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. )", From 7fdb2f400e97db601c2a290ab73da67f1c9695ad Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Mon, 14 Nov 2022 05:31:27 +0000 Subject: [PATCH 466/526] unhex support FixedString --- .../FunctionsBinaryRepresentation.cpp | 38 ++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 775696ded8a..9e475da87c2 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -5,6 +5,8 @@ #include #include #include +#include "Columns/IColumn.h" +#include "DataTypes/IDataType.h" #include #include #include @@ -566,7 +568,8 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (!isString(arguments[0])) + WhichDataType which(arguments[0]); + if (!which.isStringOrFixedString()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -612,6 +615,39 @@ public: return col_res; } + else if (const ColumnFixedString * col_fix_string = checkAndGetColumn(column.get())) + { + auto col_res = ColumnString::create(); + + ColumnString::Chars & out_vec = col_res->getChars(); + ColumnString::Offsets & out_offsets = col_res->getOffsets(); + + const ColumnString::Chars & in_vec = col_fix_string->getChars(); + size_t n = col_fix_string->getN(); + + size_t size = col_fix_string->size(); + out_offsets.resize(size); + out_vec.resize(in_vec.size() / word_size + size); + + char * begin = reinterpret_cast(out_vec.data()); + char * pos = begin; + size_t prev_offset = 0; + + for (size_t i = 0; i < size; ++i) + { + size_t new_offset = prev_offset + n; + + Impl::decode(reinterpret_cast(&in_vec[prev_offset]), reinterpret_cast(&in_vec[new_offset]), pos); + + out_offsets[i] = pos - begin; + + prev_offset = new_offset; + } + + out_vec.resize(pos - begin); + + return col_res; + } else { throw Exception("Illegal column " + arguments[0].column->getName() From 0fb23b05b9f129d6437117c09720ed78eb108f58 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Mon, 14 Nov 2022 05:32:02 +0000 Subject: [PATCH 467/526] add tests --- tests/queries/0_stateless/02474_unhex_in_fix_string.reference | 1 + tests/queries/0_stateless/02474_unhex_in_fix_string.sql | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 tests/queries/0_stateless/02474_unhex_in_fix_string.reference create mode 100644 tests/queries/0_stateless/02474_unhex_in_fix_string.sql diff --git a/tests/queries/0_stateless/02474_unhex_in_fix_string.reference b/tests/queries/0_stateless/02474_unhex_in_fix_string.reference new file mode 100644 index 00000000000..407a8b39948 --- /dev/null +++ b/tests/queries/0_stateless/02474_unhex_in_fix_string.reference @@ -0,0 +1 @@ +ClickHouse ClickHouse diff --git a/tests/queries/0_stateless/02474_unhex_in_fix_string.sql b/tests/queries/0_stateless/02474_unhex_in_fix_string.sql new file mode 100644 index 00000000000..288336aa4fa --- /dev/null +++ b/tests/queries/0_stateless/02474_unhex_in_fix_string.sql @@ -0,0 +1,4 @@ +drop table if exists unhex_in_fix_string_table; +create table unhex_in_fix_string_table ( dt Date, s1 FixedString(20), s2 String) engine=MergeTree partition by dt order by tuple(); +insert into unhex_in_fix_string_table values(today(), '436C69636B486F757365', '436C69636B486F757365'); +select unhex(s1), unhex(s2) from unhex_in_fix_string_table; From 5753be7b9925404ceb3b312739bb6aa44a8ee0a3 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Mon, 14 Nov 2022 05:43:56 +0000 Subject: [PATCH 468/526] remove useless head file --- src/Functions/FunctionsBinaryRepresentation.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Functions/FunctionsBinaryRepresentation.cpp b/src/Functions/FunctionsBinaryRepresentation.cpp index 9e475da87c2..187d82981db 100644 --- a/src/Functions/FunctionsBinaryRepresentation.cpp +++ b/src/Functions/FunctionsBinaryRepresentation.cpp @@ -5,8 +5,6 @@ #include #include #include -#include "Columns/IColumn.h" -#include "DataTypes/IDataType.h" #include #include #include From 37c448c6be84cb12bc9a18b64efdbd1d8def77b1 Mon Sep 17 00:00:00 2001 From: freedomDR <1640145602@qq.com> Date: Mon, 14 Nov 2022 05:54:54 +0000 Subject: [PATCH 469/526] complete unhex doc --- docs/en/sql-reference/functions/encoding-functions.md | 2 +- docs/zh/sql-reference/functions/encoding-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/encoding-functions.md b/docs/en/sql-reference/functions/encoding-functions.md index 4a6e46e1759..cccc02c2553 100644 --- a/docs/en/sql-reference/functions/encoding-functions.md +++ b/docs/en/sql-reference/functions/encoding-functions.md @@ -185,7 +185,7 @@ unhex(arg) **Arguments** -- `arg` — A string containing any number of hexadecimal digits. Type: [String](../../sql-reference/data-types/string.md). +- `arg` — A string containing any number of hexadecimal digits. Type: [String](../../sql-reference/data-types/string.md), [FixedString](../../sql-reference/data-types/fixedstring.md). Supports both uppercase and lowercase letters `A-F`. The number of hexadecimal digits does not have to be even. If it is odd, the last digit is interpreted as the least significant half of the `00-0F` byte. If the argument string contains anything other than hexadecimal digits, some implementation-defined result is returned (an exception isn’t thrown). For a numeric argument the inverse of hex(N) is not performed by unhex(). diff --git a/docs/zh/sql-reference/functions/encoding-functions.md b/docs/zh/sql-reference/functions/encoding-functions.md index 5e9260ff94a..846d6c58f40 100644 --- a/docs/zh/sql-reference/functions/encoding-functions.md +++ b/docs/zh/sql-reference/functions/encoding-functions.md @@ -181,7 +181,7 @@ unhex(arg) **参数** -- `arg` — 包含任意数量的十六进制数字的字符串。类型为:[String](../../sql-reference/data-types/string.md)。 +- `arg` — 包含任意数量的十六进制数字的字符串。类型为:[String](../../sql-reference/data-types/string.md),[FixedString](../../sql-reference/data-types/fixedstring.md)。 支持大写和小写字母A-F。十六进制数字的数量不必是偶数。如果是奇数,则最后一位数被解释为00-0F字节的低位。如果参数字符串包含除十六进制数字以外的任何内容,则返回一些实现定义的结果(不抛出异常)。对于数字参数, unhex()不执行 hex(N) 的倒数。 From 0017416069fa23e1e530d6a56197669113657489 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Tue, 15 Nov 2022 11:55:54 +0100 Subject: [PATCH 470/526] Update src/Common/FieldVisitorSum.cpp --- src/Common/FieldVisitorSum.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/FieldVisitorSum.cpp b/src/Common/FieldVisitorSum.cpp index 332b1a4255b..db7b4850204 100644 --- a/src/Common/FieldVisitorSum.cpp +++ b/src/Common/FieldVisitorSum.cpp @@ -24,7 +24,7 @@ bool FieldVisitorSum::operator() (Float64 & x) const { x += rhs.get(); bool FieldVisitorSum::operator() (Null &) const { /// Do not add anything - return rhs != 0; + return false; } bool FieldVisitorSum::operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } From 2a9ab046ddf44b040ad0a5a923b2381ed9b8778b Mon Sep 17 00:00:00 2001 From: Christoph Wurm Date: Tue, 15 Nov 2022 11:51:14 +0000 Subject: [PATCH 471/526] Unify spelling of DateTime --- docs/en/operations/system-tables/crash-log.md | 4 ++-- docs/en/operations/system-tables/mutations.md | 4 ++-- docs/en/operations/system-tables/replication_queue.md | 6 +++--- docs/en/sql-reference/data-types/date32.md | 2 +- docs/en/sql-reference/data-types/datetime.md | 2 +- docs/en/sql-reference/data-types/datetime64.md | 2 +- docs/en/sql-reference/functions/date-time-functions.md | 8 ++++---- docs/ru/operations/system-tables/crash-log.md | 4 ++-- docs/ru/operations/system-tables/mutations.md | 4 ++-- docs/ru/operations/system-tables/replication_queue.md | 6 +++--- docs/ru/sql-reference/data-types/date32.md | 2 +- docs/ru/sql-reference/functions/date-time-functions.md | 4 ++-- docs/zh/operations/system-tables/crash-log.md | 4 ++-- docs/zh/operations/system-tables/mutations.md | 4 ++-- docs/zh/operations/system-tables/replication_queue.md | 6 +++--- docs/zh/sql-reference/ansi.md | 2 +- docs/zh/sql-reference/data-types/datetime64.md | 2 +- docs/zh/sql-reference/functions/date-time-functions.md | 4 ++-- 18 files changed, 35 insertions(+), 35 deletions(-) diff --git a/docs/en/operations/system-tables/crash-log.md b/docs/en/operations/system-tables/crash-log.md index 0c0a4cd967d..a44b0db8e9b 100644 --- a/docs/en/operations/system-tables/crash-log.md +++ b/docs/en/operations/system-tables/crash-log.md @@ -7,8 +7,8 @@ Contains information about stack traces for fatal errors. The table does not exi Columns: -- `event_date` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date of the event. -- `event_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Time of the event. +- `event_date` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date of the event. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Time of the event. - `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Timestamp of the event with nanoseconds. - `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — Signal number. - `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Thread ID. diff --git a/docs/en/operations/system-tables/mutations.md b/docs/en/operations/system-tables/mutations.md index 782d7c42ad2..0d3b764846b 100644 --- a/docs/en/operations/system-tables/mutations.md +++ b/docs/en/operations/system-tables/mutations.md @@ -15,7 +15,7 @@ Columns: - `command` ([String](/docs/en/sql-reference/data-types/string.md)) — The mutation command string (the part of the query after `ALTER TABLE [db.]table`). -- `create_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. +- `create_time` ([DateTime](/docs/en/sql-reference/data-types/datetime.md)) — Date and time when the mutation command was submitted for execution. - `block_numbers.partition_id` ([Array](/docs/en/sql-reference/data-types/array.md)([String](/docs/en/sql-reference/data-types/string.md))) — For mutations of replicated tables, the array contains the partitions' IDs (one record for each partition). For mutations of non-replicated tables the array is empty. @@ -39,7 +39,7 @@ If there were problems with mutating some data parts, the following columns cont - `latest_failed_part` ([String](/docs/en/sql-reference/data-types/string.md)) — The name of the most recent part that could not be mutated. -- `latest_fail_time` ([Datetime](/docs/en/sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. +- `latest_fail_time` ([DateTime](/docs/en/sql-reference/data-types/datetime.md)) — The date and time of the most recent part mutation failure. - `latest_fail_reason` ([String](/docs/en/sql-reference/data-types/string.md)) — The exception message that caused the most recent part mutation failure. diff --git a/docs/en/operations/system-tables/replication_queue.md b/docs/en/operations/system-tables/replication_queue.md index ced20b0048a..dff3bce246a 100644 --- a/docs/en/operations/system-tables/replication_queue.md +++ b/docs/en/operations/system-tables/replication_queue.md @@ -29,7 +29,7 @@ Columns: - `MUTATE_PART` — Apply one or several mutations to the part. - `ALTER_METADATA` — Apply alter modification according to global /metadata and /columns paths. -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was submitted for execution. +- `create_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was submitted for execution. - `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of replicas waiting for the task to complete with confirmation of completion. This column is only relevant for the `GET_PARTS` task. @@ -47,13 +47,13 @@ Columns: - `last_exception` ([String](../../sql-reference/data-types/string.md)) — Text message about the last error that occurred (if any). -- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last attempted. +- `last_attempt_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last attempted. - `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — The number of postponed tasks. - `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — The reason why the task was postponed. -- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last postponed. +- `last_postpone_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Date and time when the task was last postponed. - `merge_type` ([String](../../sql-reference/data-types/string.md)) — Type of the current merge. Empty if it's a mutation. diff --git a/docs/en/sql-reference/data-types/date32.md b/docs/en/sql-reference/data-types/date32.md index ff1a745785b..c8c7470d2cb 100644 --- a/docs/en/sql-reference/data-types/date32.md +++ b/docs/en/sql-reference/data-types/date32.md @@ -6,7 +6,7 @@ sidebar_label: Date32 # Date32 -A date. Supports the date range same with [Datetime64](../../sql-reference/data-types/datetime64.md). Stored in four bytes as the number of days since 1900-01-01. Allows storing values till 2299-12-31. +A date. Supports the date range same with [DateTime64](../../sql-reference/data-types/datetime64.md). Stored in four bytes as the number of days since 1900-01-01. Allows storing values till 2299-12-31. **Examples** diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 85587882e01..7f7f21ded54 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -4,7 +4,7 @@ sidebar_position: 48 sidebar_label: DateTime --- -# Datetime +# DateTime Allows to store an instant in time, that can be expressed as a calendar date and a time of a day. diff --git a/docs/en/sql-reference/data-types/datetime64.md b/docs/en/sql-reference/data-types/datetime64.md index c7372e4b064..fa3a1eecd46 100644 --- a/docs/en/sql-reference/data-types/datetime64.md +++ b/docs/en/sql-reference/data-types/datetime64.md @@ -4,7 +4,7 @@ sidebar_position: 49 sidebar_label: DateTime64 --- -# Datetime64 +# DateTime64 Allows to store an instant in time, that can be expressed as a calendar date and a time of a day, with defined sub-second precision diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f7ea2690b21..6156a823d58 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -550,7 +550,7 @@ Alias: `dateTrunc`. - Value, truncated to the specified part of date. -Type: [Datetime](../../sql-reference/data-types/datetime.md). +Type: [DateTime](../../sql-reference/data-types/datetime.md). **Example** @@ -881,7 +881,7 @@ now([timezone]) - Current date and time. -Type: [Datetime](../../sql-reference/data-types/datetime.md). +Type: [DateTime](../../sql-reference/data-types/datetime.md). **Example** @@ -932,7 +932,7 @@ now64([scale], [timezone]) - Current date and time with sub-second precision. -Type: [Datetime64](../../sql-reference/data-types/datetime64.md). +Type: [DateTime64](../../sql-reference/data-types/datetime64.md). **Example** @@ -968,7 +968,7 @@ nowInBlock([timezone]) - Current date and time at the moment of processing of each block of data. -Type: [Datetime](../../sql-reference/data-types/datetime.md). +Type: [DateTime](../../sql-reference/data-types/datetime.md). **Example** diff --git a/docs/ru/operations/system-tables/crash-log.md b/docs/ru/operations/system-tables/crash-log.md index 4ca8be5a199..68148fec6bd 100644 --- a/docs/ru/operations/system-tables/crash-log.md +++ b/docs/ru/operations/system-tables/crash-log.md @@ -7,8 +7,8 @@ slug: /ru/operations/system-tables/crash-log Колонки: -- `event_date` ([Datetime](../../sql-reference/data-types/datetime.md)) — Дата события. -- `event_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — Время события. +- `event_date` ([DateTime](../../sql-reference/data-types/datetime.md)) — Дата события. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Время события. - `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Время события с наносекундами. - `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — Номер сигнала, пришедшего в поток. - `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Идентификатор треда. diff --git a/docs/ru/operations/system-tables/mutations.md b/docs/ru/operations/system-tables/mutations.md index 20e4ebfdaf1..bb0bd44ed7a 100644 --- a/docs/ru/operations/system-tables/mutations.md +++ b/docs/ru/operations/system-tables/mutations.md @@ -15,7 +15,7 @@ slug: /ru/operations/system-tables/mutations - `command` ([String](../../sql-reference/data-types/string.md)) — команда мутации (часть запроса после `ALTER TABLE [db.]table`). -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время создания мутации. +- `create_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время создания мутации. - `block_numbers.partition_id` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Для мутаций реплицированных таблиц массив содержит содержит номера партиций (по одной записи для каждой партиции). Для мутаций нереплицированных таблиц массив пустой. @@ -39,7 +39,7 @@ slug: /ru/operations/system-tables/mutations - `latest_failed_part` ([String](../../sql-reference/data-types/string.md)) — имя последнего куска, мутация которого не удалась. -- `latest_fail_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время последней ошибки мутации. +- `latest_fail_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время последней ошибки мутации. - `latest_fail_reason` ([String](../../sql-reference/data-types/string.md)) — причина последней ошибки мутации. diff --git a/docs/ru/operations/system-tables/replication_queue.md b/docs/ru/operations/system-tables/replication_queue.md index 25de174e98f..60d42133153 100644 --- a/docs/ru/operations/system-tables/replication_queue.md +++ b/docs/ru/operations/system-tables/replication_queue.md @@ -29,7 +29,7 @@ slug: /ru/operations/system-tables/replication_queue - `MUTATE_PART` — применить одну или несколько мутаций к куску. - `ALTER_METADATA` — применить изменения структуры таблицы в результате запросов с выражением `ALTER`. -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время отправки задачи на выполнение. +- `create_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время отправки задачи на выполнение. - `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество реплик, ожидающих завершения задачи, с подтверждением о завершении. Этот столбец актуален только для задачи `GET_PARTS`. @@ -47,13 +47,13 @@ slug: /ru/operations/system-tables/replication_queue - `last_exception` ([String](../../sql-reference/data-types/string.md)) — текст сообщения о последней возникшей ошибке, если таковые имеются. -- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время последней попытки выполнить задачу. +- `last_attempt_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время последней попытки выполнить задачу. - `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — количество отложенных задач. - `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — причина, по которой была отложена задача. -- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — дата и время, когда была отложена задача в последний раз. +- `last_postpone_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — дата и время, когда была отложена задача в последний раз. - `merge_type` ([String](../../sql-reference/data-types/string.md)) — тип текущего слияния. Пусто, если это мутация. diff --git a/docs/ru/sql-reference/data-types/date32.md b/docs/ru/sql-reference/data-types/date32.md index fcb7d688c20..958b8e9763e 100644 --- a/docs/ru/sql-reference/data-types/date32.md +++ b/docs/ru/sql-reference/data-types/date32.md @@ -6,7 +6,7 @@ sidebar_label: Date32 # Date32 {#data_type-datetime32} -Дата. Поддерживается такой же диапазон дат, как для типа [Datetime64](../../sql-reference/data-types/datetime64.md). Значение хранится в четырех байтах и соответствует числу дней с 1900-01-01 по 2299-12-31. +Дата. Поддерживается такой же диапазон дат, как для типа [DateTime64](../../sql-reference/data-types/datetime64.md). Значение хранится в четырех байтах и соответствует числу дней с 1900-01-01 по 2299-12-31. **Пример** diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index f18c2ea258a..f430f5cae51 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -602,7 +602,7 @@ date_trunc(unit, value[, timezone]) - Дата и время, отсеченные до указанной части. -Тип: [Datetime](../../sql-reference/data-types/datetime.md). +Тип: [DateTime](../../sql-reference/data-types/datetime.md). **Примеры** @@ -913,7 +913,7 @@ now([timezone]) - Текущие дата и время. -Тип: [Datetime](../../sql-reference/data-types/datetime.md). +Тип: [DateTime](../../sql-reference/data-types/datetime.md). **Пример** diff --git a/docs/zh/operations/system-tables/crash-log.md b/docs/zh/operations/system-tables/crash-log.md index d0ed406fa0c..06087a34f35 100644 --- a/docs/zh/operations/system-tables/crash-log.md +++ b/docs/zh/operations/system-tables/crash-log.md @@ -7,8 +7,8 @@ slug: /zh/operations/system-tables/crash-log 列信息: -- `event_date` ([Datetime](../../sql-reference/data-types/datetime.md)) — 事件日期. -- `event_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 事件时间. +- `event_date` ([DateTime](../../sql-reference/data-types/datetime.md)) — 事件日期. +- `event_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 事件时间. - `timestamp_ns` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 以纳秒为单位的事件时间戳. - `signal` ([Int32](../../sql-reference/data-types/int-uint.md)) — 信号编号. - `thread_id` ([UInt64](../../sql-reference/data-types/int-uint.md)) — 线程ID. diff --git a/docs/zh/operations/system-tables/mutations.md b/docs/zh/operations/system-tables/mutations.md index dbce0a59063..f5f82c1717a 100644 --- a/docs/zh/operations/system-tables/mutations.md +++ b/docs/zh/operations/system-tables/mutations.md @@ -15,7 +15,7 @@ slug: /zh/operations/system-tables/mutations - `command` ([String](../../sql-reference/data-types/string.md)) — mutation命令字符串(`ALTER TABLE [db.]table`语句之后的部分)。 -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — mutation命令提交执行的日期和时间。 +- `create_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — mutation命令提交执行的日期和时间。 - `block_numbers.partition_id` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — 对于复制表的mutation,该数组包含分区的ID(每个分区都有一条记录)。对于非复制表的mutation,该数组为空。 @@ -39,7 +39,7 @@ slug: /zh/operations/system-tables/mutations - `latest_failed_part`([String](../../sql-reference/data-types/string.md)) — 最近不能mutation的part的名称。 -- `latest_fail_time`([Datetime](../../sql-reference/data-types/datetime.md)) — 最近的一个mutation失败的时间。 +- `latest_fail_time`([DateTime](../../sql-reference/data-types/datetime.md)) — 最近的一个mutation失败的时间。 - `latest_fail_reason`([String](../../sql-reference/data-types/string.md)) — 导致最近part的mutation失败的异常消息。 diff --git a/docs/zh/operations/system-tables/replication_queue.md b/docs/zh/operations/system-tables/replication_queue.md index e82569e378d..95a183cf9f7 100644 --- a/docs/zh/operations/system-tables/replication_queue.md +++ b/docs/zh/operations/system-tables/replication_queue.md @@ -29,7 +29,7 @@ slug: /zh/operations/system-tables/replication_queue - `MUTATE_PART` — 对分片应用一个或多个突变. - `ALTER_METADATA` — 根据全局 /metadata 和 /columns 路径应用alter修改. -- `create_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 提交任务执行的日期和时间. +- `create_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 提交任务执行的日期和时间. - `required_quorum` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 等待任务完成并确认完成的副本数. 此列仅与 `GET_PARTS` 任务相关. @@ -47,13 +47,13 @@ slug: /zh/operations/system-tables/replication_queue - `last_exception` ([String](../../sql-reference/data-types/string.md)) — 发生的最后一个错误的短信(如果有). -- `last_attempt_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 上次尝试任务的日期和时间. +- `last_attempt_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 上次尝试任务的日期和时间. - `num_postponed` ([UInt32](../../sql-reference/data-types/int-uint.md)) — 延期任务数. - `postpone_reason` ([String](../../sql-reference/data-types/string.md)) — 任务延期的原因. -- `last_postpone_time` ([Datetime](../../sql-reference/data-types/datetime.md)) — 上次推迟任务的日期和时间. +- `last_postpone_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — 上次推迟任务的日期和时间. - `merge_type` ([String](../../sql-reference/data-types/string.md)) — 当前合并的类型. 如果是突变则为空. diff --git a/docs/zh/sql-reference/ansi.md b/docs/zh/sql-reference/ansi.md index 9cf335f89ef..cdccee0084f 100644 --- a/docs/zh/sql-reference/ansi.md +++ b/docs/zh/sql-reference/ansi.md @@ -152,7 +152,7 @@ sidebar_label: "ANSI\u517C\u5BB9\u6027" | F051-02 | TIME(时间)数据类型(并支持用于表达时间的字面量),小数秒精度至少为0 | 否 {.text-danger} | | | F051-03 | 时间戳数据类型(并支持用于表达时间戳的字面量),小数秒精度至少为0和6 | 是 {.text-danger} | | | F051-04 | 日期、时间和时间戳数据类型的比较谓词 | 是 {.text-success} | | -| F051-05 | Datetime 类型和字符串形式表达的时间之间的显式转换 | 是 {.text-success} | | +| F051-05 | DateTime 类型和字符串形式表达的时间之间的显式转换 | 是 {.text-success} | | | F051-06 | CURRENT_DATE | 否 {.text-danger} | 使用`today()`替代 | | F051-07 | LOCALTIME | 否 {.text-danger} | 使用`now()`替代 | | F051-08 | LOCALTIMESTAMP | 否 {.text-danger} | | diff --git a/docs/zh/sql-reference/data-types/datetime64.md b/docs/zh/sql-reference/data-types/datetime64.md index ee2d7a6f258..24888645cba 100644 --- a/docs/zh/sql-reference/data-types/datetime64.md +++ b/docs/zh/sql-reference/data-types/datetime64.md @@ -6,7 +6,7 @@ sidebar_position: 49 sidebar_label: DateTime64 --- -# Datetime64 {#data_type-datetime64} +# DateTime64 {#data_type-datetime64} 此类型允许以日期(date)加时间(time)的形式来存储一个时刻的时间值,具有定义的亚秒精度 diff --git a/docs/zh/sql-reference/functions/date-time-functions.md b/docs/zh/sql-reference/functions/date-time-functions.md index c666d01d15f..4bbd0e5b69b 100644 --- a/docs/zh/sql-reference/functions/date-time-functions.md +++ b/docs/zh/sql-reference/functions/date-time-functions.md @@ -539,7 +539,7 @@ date_trunc(unit, value[, timezone]) - 按指定的单位向前取整后的DateTime。 -类型: [Datetime](../../sql-reference/data-types/datetime.md). +类型: [DateTime](../../sql-reference/data-types/datetime.md). **示例** @@ -850,7 +850,7 @@ now([timezone]) - 当前日期和时间。 -类型: [Datetime](../../sql-reference/data-types/datetime.md). +类型: [DateTime](../../sql-reference/data-types/datetime.md). **示例** From 0e5e58bed97db8ed66b140df2dce4976bd02e1da Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 15 Nov 2022 12:18:24 +0000 Subject: [PATCH 472/526] Remove exception if shared ID already created --- .../MergeTree/ReplicatedMergeTreeAttachThread.cpp | 1 + src/Storages/StorageReplicatedMergeTree.cpp | 8 +++++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index 7f91ffee1fe..47f10acb157 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -182,6 +182,7 @@ void ReplicatedMergeTreeAttachThread::runImpl() storage.createNewZooKeeperNodes(); storage.syncPinnedPartUUIDs(); + std::lock_guard lock(storage.table_shared_id_mutex); storage.createTableSharedID(); }; diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index bbbb4b6d22c..8b4788c8d55 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7609,8 +7609,6 @@ std::unique_ptr StorageReplicatedMergeTree::getDefaultSetting String StorageReplicatedMergeTree::getTableSharedID() const { - /// Lock is not required in other places because createTableSharedID() - /// can be called only during table initialization std::lock_guard lock(table_shared_id_mutex); /// Can happen if table was partially initialized before drop by DatabaseCatalog @@ -7637,8 +7635,12 @@ String StorageReplicatedMergeTree::getTableSharedID() const void StorageReplicatedMergeTree::createTableSharedID() const { LOG_DEBUG(log, "Creating shared ID for table {}", getStorageID().getNameForLogs()); + // can be set by the call to getTableSharedID if (table_shared_id != UUIDHelpers::Nil) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Table shared id already initialized"); + { + LOG_INFO(log, "Shared ID already set to {}", table_shared_id); + return; + } auto zookeeper = getZooKeeper(); String zookeeper_table_id_path = fs::path(zookeeper_path) / "table_shared_id"; From d7c882951f1e3c5c66d744e9105a6d30a419a6e0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 15 Nov 2022 12:36:28 +0000 Subject: [PATCH 473/526] Fix nullptr dereference in collectScopeValidIdentifiersForTypoCorrection --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 20 ++++++++++++++++++- .../02480_analyzer_alias_nullptr.reference | 0 .../02480_analyzer_alias_nullptr.sql | 3 +++ 3 files changed, 22 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02480_analyzer_alias_nullptr.reference create mode 100644 tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 24b88a729be..760d036ab4f 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1517,6 +1517,7 @@ void QueryAnalyzer::collectScopeValidIdentifiersForTypoCorrection( { for (const auto & [name, expression] : scope.alias_name_to_expression_node) { + assert(expression); auto expression_identifier = Identifier(name); valid_identifiers_result.insert(expression_identifier); @@ -2170,7 +2171,24 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto & alias_identifier_node = it->second->as(); auto identifier = alias_identifier_node.getIdentifier(); auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings); - it->second = lookup_result.resolved_identifier; + if (lookup_result.isResolved()) + { + it->second = lookup_result.resolved_identifier; + } + else + { + alias_name_to_node_map.erase(it); + + std::unordered_set valid_identifiers; + collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers); + + auto hints = collectIdentifierTypoHints(identifier, valid_identifiers); + throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER, "Unknown {} identifier '{}' in scope {}{}", + toStringLowercase(IdentifierLookupContext::EXPRESSION), + identifier.getFullName(), + scope.scope_node->formatASTForErrorMessage(), + getHintsErrorMessageSuffix(hints)); + } /** During collection of aliases if node is identifier and has alias, we cannot say if it is * column or function node. Check QueryExpressionsAliasVisitor documentation for clarification. diff --git a/tests/queries/0_stateless/02480_analyzer_alias_nullptr.reference b/tests/queries/0_stateless/02480_analyzer_alias_nullptr.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql b/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql new file mode 100644 index 00000000000..f6b381e5c70 --- /dev/null +++ b/tests/queries/0_stateless/02480_analyzer_alias_nullptr.sql @@ -0,0 +1,3 @@ +SET allow_experimental_analyzer = 1; + +SELECT min(b), x AS b FROM (SELECT max(number) FROM numbers(1)); -- { serverError UNKNOWN_IDENTIFIER } From 143b67d0beecfd0f51c6d4499a0383e003baea96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 15 Nov 2022 15:40:06 +0100 Subject: [PATCH 474/526] Fix ubsan in AggregateFunctionMinMaxAny::read with high sizes --- src/AggregateFunctions/AggregateFunctionMinMaxAny.h | 7 ++++++- src/Common/Arena.h | 2 +- .../0_stateless/02481_i43247_ubsan_in_minmaxany.reference | 0 .../0_stateless/02481_i43247_ubsan_in_minmaxany.sql | 3 +++ 4 files changed, 10 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference create mode 100644 tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 46be7331195..18f065caaf9 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -29,6 +29,7 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NOT_IMPLEMENTED; + extern const int TOO_LARGE_STRING_SIZE; } /** Aggregate functions that store one of passed values. @@ -521,7 +522,11 @@ public: { if (capacity < rhs_size) { - capacity = static_cast(roundUpToPowerOfTwoOrZero(rhs_size)); + capacity = static_cast(roundUpToPowerOfTwoOrZero(rhs_size)); + /// It might happen if the size was too big and the rounded value does not fit a size_t + if (unlikely(capacity <= rhs_size)) + throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size is too big ({})", rhs_size); + /// Don't free large_data here. large_data = arena->alloc(capacity); } diff --git a/src/Common/Arena.h b/src/Common/Arena.h index 17d53acd8f7..5772dff6bca 100644 --- a/src/Common/Arena.h +++ b/src/Common/Arena.h @@ -141,7 +141,7 @@ public: /// Get piece of memory, without alignment. char * alloc(size_t size) { - if (unlikely(head->pos + size > head->end)) + if (unlikely(static_cast(size) > head->end - head->pos)) addMemoryChunk(size); char * res = head->pos; diff --git a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql new file mode 100644 index 00000000000..7204053de04 --- /dev/null +++ b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql @@ -0,0 +1,3 @@ +-- https://github.com/ClickHouse/ClickHouse/issues/43247 +SELECT finalizeAggregation(CAST('AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)', + 'AggregateFunction(min, String)')); -- { serverError 131 } From d49b65cf1ead8fbd4a5d996c2d405f18ff954aca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 15 Nov 2022 16:31:12 +0100 Subject: [PATCH 475/526] Fix capacity check --- src/AggregateFunctions/AggregateFunctionMinMaxAny.h | 2 +- .../0_stateless/02481_i43247_ubsan_in_minmaxany.reference | 1 + tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql | 4 ++++ 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h index 18f065caaf9..1f3c51c1c1c 100644 --- a/src/AggregateFunctions/AggregateFunctionMinMaxAny.h +++ b/src/AggregateFunctions/AggregateFunctionMinMaxAny.h @@ -524,7 +524,7 @@ public: { capacity = static_cast(roundUpToPowerOfTwoOrZero(rhs_size)); /// It might happen if the size was too big and the rounded value does not fit a size_t - if (unlikely(capacity <= rhs_size)) + if (unlikely(capacity < rhs_size)) throw Exception(ErrorCodes::TOO_LARGE_STRING_SIZE, "String size is too big ({})", rhs_size); /// Don't free large_data here. diff --git a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference index e69de29bb2d..3e3abfb9a41 100644 --- a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference +++ b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.reference @@ -0,0 +1 @@ +0123456789012345678901234567890123456789012345678901234567890123 diff --git a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql index 7204053de04..7dc29c2daae 100644 --- a/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql +++ b/tests/queries/0_stateless/02481_i43247_ubsan_in_minmaxany.sql @@ -1,3 +1,7 @@ -- https://github.com/ClickHouse/ClickHouse/issues/43247 SELECT finalizeAggregation(CAST('AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)AggregateFunction(categoricalInformationValue, Nullable(UInt8), UInt8)', 'AggregateFunction(min, String)')); -- { serverError 131 } + +-- Value from hex(minState('0123456789012345678901234567890123456789012345678901234567890123')). Size 63 + 1 (64) +SELECT finalizeAggregation(CAST(unhex('4000000030313233343536373839303132333435363738393031323334353637383930313233343536373839303132333435363738393031323334353637383930313233'), + 'AggregateFunction(min, String)')); From 5aae1d07246f6b1ce46b43cb8f59fe4495864185 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 15 Nov 2022 10:49:13 -0500 Subject: [PATCH 476/526] update note ot include default user --- .../external-dictionaries/external-dicts-dict-layout.md | 2 +- .../external-dictionaries/external-dicts-dict-lifetime.md | 2 +- .../external-dictionaries/external-dicts-dict-polygon.md | 2 +- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- .../external-dictionaries/external-dicts-dict-structure.md | 2 +- .../dictionaries/external-dictionaries/external-dicts-dict.md | 2 +- .../dictionaries/external-dictionaries/external-dicts.md | 4 ++-- 7 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 49a8620b609..4e379365912 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -25,7 +25,7 @@ ClickHouse generates an exception for errors with dictionaries. Examples of erro You can view the list of dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md index f0d6cb64a0b..f5b70522841 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md @@ -13,7 +13,7 @@ Dictionary updates (other than loading for first use) do not block queries. Duri Example of settings: :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 42c4a7faa73..aa85247c1be 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -11,7 +11,7 @@ For example: defining a city area by geographical coordinates. Example of a polygon dictionary configuration: :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 8218c066530..bcd5d4417a5 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -7,7 +7,7 @@ sidebar_label: Dictionary Sources # Dictionary Sources :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index 20056c6ac9c..ba8411db1ce 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -7,7 +7,7 @@ sidebar_label: Dictionary Key and Fields # Dictionary Key and Fields :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md index f95a3593ae4..1d273a9e9ed 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md @@ -7,7 +7,7 @@ sidebar_label: Configuring a Dictionary # Configuring a Dictionary :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 19bb7474d40..34d8b7b74a4 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -7,7 +7,7 @@ sidebar_label: General Description # Dictionaries :::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries. +If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). ::: @@ -36,7 +36,7 @@ Dictionaries can be created with [DDL queries](../../../sql-reference/statements ## Creating a dictionary with a configuration file :::note -Creating a dictionary with a configuration file is not applicable to ClickHouse Cloud. Please use DDL (see above). +Creating a dictionary with a configuration file is not applicable to ClickHouse Cloud. Please use DDL (see above), and create your dictionary as user `default`. ::: The dictionary configuration file has the following format: From 03968eb694e414281ac5b0529edd2785b266ef59 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 15 Nov 2022 16:16:04 +0000 Subject: [PATCH 477/526] Better message in wait_zookeeper_to_start --- tests/integration/helpers/cluster.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 666833013c8..a0e8e0b0cce 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2070,10 +2070,10 @@ class ClickHouseCluster: logging.debug("All instances of ZooKeeper started") return except Exception as ex: - logging.debug("Can't connect to ZooKeeper " + str(ex)) + logging.debug(f"Can't connect to ZooKeeper {instance}: {ex}") time.sleep(0.5) - raise Exception("Cannot wait ZooKeeper container") + raise Exception("Cannot wait ZooKeeper container (probably it's a `iptables-nft` issue, you may try to `sudo iptables -P FORWARD ACCEPT`)") def make_hdfs_api(self, timeout=180, kerberized=False): if kerberized: From aaeeeaf1fc1c58d1d96521d9c604a213cb8cd3c4 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 15 Nov 2022 16:40:01 +0000 Subject: [PATCH 478/526] Automatic style fix --- tests/integration/helpers/cluster.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index a0e8e0b0cce..a190126a8ff 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2073,7 +2073,9 @@ class ClickHouseCluster: logging.debug(f"Can't connect to ZooKeeper {instance}: {ex}") time.sleep(0.5) - raise Exception("Cannot wait ZooKeeper container (probably it's a `iptables-nft` issue, you may try to `sudo iptables -P FORWARD ACCEPT`)") + raise Exception( + "Cannot wait ZooKeeper container (probably it's a `iptables-nft` issue, you may try to `sudo iptables -P FORWARD ACCEPT`)" + ) def make_hdfs_api(self, timeout=180, kerberized=False): if kerberized: From 63ae261119da4484d02c16979b73e3231b0ef2e3 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 15 Nov 2022 12:44:54 -0500 Subject: [PATCH 479/526] move tip to snippet --- .../external-dictionaries/_snippet_dictionary_in_cloud.md | 4 ++++ .../external-dictionaries/external-dicts-dict-layout.md | 6 ++---- .../external-dictionaries/external-dicts-dict-lifetime.md | 6 ++---- .../external-dictionaries/external-dicts-dict-polygon.md | 6 ++---- .../external-dictionaries/external-dicts-dict-sources.md | 6 ++---- .../external-dictionaries/external-dicts-dict-structure.md | 6 ++---- .../external-dictionaries/external-dicts-dict.md | 6 ++---- .../dictionaries/external-dictionaries/external-dicts.md | 6 ++---- 8 files changed, 18 insertions(+), 28 deletions(-) create mode 100644 docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md b/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md new file mode 100644 index 00000000000..e6a0dac7afb --- /dev/null +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md @@ -0,0 +1,4 @@ +:::tip +If you are using a dictionary with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. +Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). +::: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 4e379365912..aac0db208c6 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -3,6 +3,7 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-l sidebar_position: 41 sidebar_label: Storing Dictionaries in Memory --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Storing Dictionaries in Memory @@ -24,10 +25,7 @@ ClickHouse generates an exception for errors with dictionaries. Examples of erro You can view the list of dictionaries and their statuses in the [system.dictionaries](../../../operations/system-tables/dictionaries.md) table. -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + The configuration looks like this: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md index f5b70522841..e4edad4d9a1 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-lifetime.md @@ -3,6 +3,7 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-l sidebar_position: 42 sidebar_label: Dictionary Updates --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Dictionary Updates @@ -12,10 +13,7 @@ Dictionary updates (other than loading for first use) do not block queries. Duri Example of settings: -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + ``` xml diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index aa85247c1be..366d88e07c7 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -4,16 +4,14 @@ sidebar_position: 46 sidebar_label: Polygon Dictionaries With Grids title: "Polygon dictionaries" --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; Polygon dictionaries allow you to efficiently search for the polygon containing specified points. For example: defining a city area by geographical coordinates. Example of a polygon dictionary configuration: -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + ``` xml diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index bcd5d4417a5..4eb96fe80a2 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -3,13 +3,11 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-s sidebar_position: 43 sidebar_label: Dictionary Sources --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Dictionary Sources -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + A dictionary can be connected to ClickHouse from many different sources. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md index ba8411db1ce..881630167e3 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-structure.md @@ -3,13 +3,11 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-s sidebar_position: 44 sidebar_label: Dictionary Key and Fields --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Dictionary Key and Fields -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + The `structure` clause describes the dictionary key and fields available for queries. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md index 1d273a9e9ed..76ca3ac978f 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict.md @@ -3,13 +3,11 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict sidebar_position: 40 sidebar_label: Configuring a Dictionary --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Configuring a Dictionary -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + If dictionary is configured using xml file, than dictionary configuration has the following structure: diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 34d8b7b74a4..7498afd8492 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -3,13 +3,11 @@ slug: /en/sql-reference/dictionaries/external-dictionaries/external-dicts sidebar_position: 39 sidebar_label: General Description --- +import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dictionaries/_snippet_dictionary_in_cloud.md'; # Dictionaries -:::tip -If you are using a DICTIONARY with ClickHouse Cloud please use the DDL query option to create your dictionaries, and create your dictionary as user `default`. -Also, verify the list of supported dictionary sources in the [Cloud Compatibility guide](/docs/en/whats-new/cloud-capabilities.md). -::: + You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a DDL query, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. From 2a48fb344d325aa3e5e8c1e818ec6cf0685deb53 Mon Sep 17 00:00:00 2001 From: Camilo Sierra Date: Tue, 15 Nov 2022 19:22:46 +0100 Subject: [PATCH 480/526] Added examples and extra info to projections I have added two examples about how we can use projections --- .../statements/alter/projection.md | 127 +++++++++++++++++- 1 file changed, 126 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 7913c7cb7e4..02dd255be52 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -2,9 +2,134 @@ slug: /en/sql-reference/statements/alter/projection sidebar_position: 49 sidebar_label: PROJECTION -title: "Manipulating Projections" +title: "Projections" --- +Projections store data in a format that optimizes query execution, this feature is useful if: +You need to run queries on a column that is not a part of the primary key, +For Pre-aggregate columns, it will reduce both computation and IO. + +You can define one or more projections for a table, and during the query analysis the projection with least data to scan will be selected by ClickHouse without modifying the query provided by the user. + +## Example filtering without using primary keys + +Creating the table: +``` +CREATE TABLE visits_order +( + `user_id` UInt64, + `user_name` String, + `pages_visited` Nullable(Float64), + `user_agent` String +) +ENGINE = MergeTree() +PRIMARY KEY user_agent +``` +Using `ALTER TABLE`, we could add the Projection to an existing table: +``` +ALTER TABLE visits_order ADD PROJECTION user_name_projection ( +SELECT +* +ORDER BY user_name +) + +ALTER TABLE visits_order MATERIALIZE PROJECTION user_name_projection +``` +Inserting the data: +``` +INSERT INTO visits_order SELECT + number, + 'test', + 1.5 * (number / 2), + 'Android' +FROM numbers(1, 100); +``` + +The Projection will allow us to filter by `user_name` fast even if in the original Table `user_name` was not defined as a `PRIMARY_KEY`. +At query time ClickHouse determined that less data will be processed if the projection is used, as the data is ordered by `user_name`. +``` +SELECT + * +FROM visits_order +WHERE user_name='test' +LIMIT 2 +``` + +To verify that a query is using the projection, we could review the `system.query_log` table. On the `projections` field we have the name of the projection used or empty if none has been used: +``` +SELECT query, projections FROM system.query_log WHERE query_id='' +``` + +## Example for pre-aggregate query + +Creating the table with the Projection: +``` +CREATE TABLE visits +( + `user_id` UInt64, + `user_name` String, + `pages_visited` Nullable(Float64), + `user_agent` String, + PROJECTION projection_visits_by_user + ( + SELECT + user_agent, + sum(pages_visited) + GROUP BY user_id, user_agent + ) +) +ENGINE = MergeTree() +ORDER BY user_agent +``` +Inserting the data: +``` +INSERT INTO visits SELECT + number, + 'test', + 1.5 * (number / 2), + 'Android' +FROM numbers(1, 100); +``` +``` +INSERT INTO visits SELECT + number, + 'test', + 1. * (number / 2), + 'IOS' +FROM numbers(100, 500); +``` +We will execute a first query using `GROUP BY` using the field `user_agent`, this query will not use the projection defined as the pre-aggregate do not match. +``` +SELECT + user_agent, + count(DISTINCT user_id) +FROM visits +GROUP BY user_agent +``` + +To use the projection we could execute queries that select partially or all the pre-aggregate and `GROUP BY` fields. +``` +SELECT + user_agent +FROM visits +WHERE user_id > 50 AND user_id < 150 +GROUP BY user_agent +``` +``` +SELECT + user_agent, + sum(pages_visited) +FROM visits +GROUP BY user_id +``` + +As mentioned before, we could review the `system.query_log` table. On the `projections` field we have the name of the projection used or empty if none has been used: +``` +SELECT query, projections FROM system.query_log WHERE query_id='' +``` + +# Manipulating Projections + The following operations with [projections](/docs/en/engines/table-engines/mergetree-family/mergetree.md/#projections) are available: ## ADD PROJECTION From 7e08798fa8b2bd4a2559eee7b1bd9ec57b1aabe4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 15 Nov 2022 13:47:50 -0500 Subject: [PATCH 481/526] edits --- .../sql-reference/statements/alter/projection.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 02dd255be52..99cb8fb8fd1 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -5,11 +5,11 @@ sidebar_label: PROJECTION title: "Projections" --- -Projections store data in a format that optimizes query execution, this feature is useful if: -You need to run queries on a column that is not a part of the primary key, -For Pre-aggregate columns, it will reduce both computation and IO. +Projections store data in a format that optimizes query execution, this feature is useful for: +- Running queries on a column that is not a part of the primary key +- Pre-aggregating columns, it will reduce both computation and IO -You can define one or more projections for a table, and during the query analysis the projection with least data to scan will be selected by ClickHouse without modifying the query provided by the user. +You can define one or more projections for a table, and during the query analysis the projection with the least data to scan will be selected by ClickHouse without modifying the query provided by the user. ## Example filtering without using primary keys @@ -60,7 +60,7 @@ To verify that a query is using the projection, we could review the `system.quer SELECT query, projections FROM system.query_log WHERE query_id='' ``` -## Example for pre-aggregate query +## Example pre-aggregation query Creating the table with the Projection: ``` @@ -98,7 +98,7 @@ INSERT INTO visits SELECT 'IOS' FROM numbers(100, 500); ``` -We will execute a first query using `GROUP BY` using the field `user_agent`, this query will not use the projection defined as the pre-aggregate do not match. +We will execute a first query using `GROUP BY` using the field `user_agent`, this query will not use the projection defined as the pre-aggregation does not match. ``` SELECT user_agent, @@ -107,7 +107,7 @@ FROM visits GROUP BY user_agent ``` -To use the projection we could execute queries that select partially or all the pre-aggregate and `GROUP BY` fields. +To use the projection we could execute queries that select part of, or all of the pre-aggregation and `GROUP BY` fields. ``` SELECT user_agent From 4778b5c13f7aeb2602047a790fcf640251b4c2b9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Tue, 15 Nov 2022 20:23:49 +0000 Subject: [PATCH 482/526] Fix IS (NOT)NULL operator --- src/Parsers/ExpressionListParsers.cpp | 60 +++++++++---------- .../02477_is_null_parser.reference | 3 + .../0_stateless/02477_is_null_parser.sql | 3 + 3 files changed, 36 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/02477_is_null_parser.reference create mode 100644 tests/queries/0_stateless/02477_is_null_parser.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index d29aa248ec4..29158254e88 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -2199,40 +2199,40 @@ std::vector> ParserExpressionImpl::operators_t {"AND", Operator("and", 4, 2, OperatorType::Mergeable)}, {"BETWEEN", Operator("", 6, 0, OperatorType::StartBetween)}, {"NOT BETWEEN", Operator("", 6, 0, OperatorType::StartNotBetween)}, - {"IS NULL", Operator("isNull", 8, 1, OperatorType::IsNull)}, - {"IS NOT NULL", Operator("isNotNull", 8, 1, OperatorType::IsNull)}, - {"==", Operator("equals", 9, 2, OperatorType::Comparison)}, - {"!=", Operator("notEquals", 9, 2, OperatorType::Comparison)}, - {"<>", Operator("notEquals", 9, 2, OperatorType::Comparison)}, - {"<=", Operator("lessOrEquals", 9, 2, OperatorType::Comparison)}, - {">=", Operator("greaterOrEquals", 9, 2, OperatorType::Comparison)}, - {"<", Operator("less", 9, 2, OperatorType::Comparison)}, - {">", Operator("greater", 9, 2, OperatorType::Comparison)}, - {"=", Operator("equals", 9, 2, OperatorType::Comparison)}, - {"LIKE", Operator("like", 9, 2)}, - {"ILIKE", Operator("ilike", 9, 2)}, - {"NOT LIKE", Operator("notLike", 9, 2)}, - {"NOT ILIKE", Operator("notILike", 9, 2)}, - {"IN", Operator("in", 9, 2)}, - {"NOT IN", Operator("notIn", 9, 2)}, - {"GLOBAL IN", Operator("globalIn", 9, 2)}, - {"GLOBAL NOT IN", Operator("globalNotIn", 9, 2)}, - {"||", Operator("concat", 10, 2, OperatorType::Mergeable)}, - {"+", Operator("plus", 11, 2)}, - {"-", Operator("minus", 11, 2)}, - {"*", Operator("multiply", 12, 2)}, - {"/", Operator("divide", 12, 2)}, - {"%", Operator("modulo", 12, 2)}, - {"MOD", Operator("modulo", 12, 2)}, - {"DIV", Operator("intDiv", 12, 2)}, - {".", Operator("tupleElement", 14, 2, OperatorType::TupleElement)}, - {"[", Operator("arrayElement", 14, 2, OperatorType::ArrayElement)}, - {"::", Operator("CAST", 14, 2, OperatorType::Cast)}, + {"==", Operator("equals", 8, 2, OperatorType::Comparison)}, + {"!=", Operator("notEquals", 8, 2, OperatorType::Comparison)}, + {"<>", Operator("notEquals", 8, 2, OperatorType::Comparison)}, + {"<=", Operator("lessOrEquals", 8, 2, OperatorType::Comparison)}, + {">=", Operator("greaterOrEquals", 8, 2, OperatorType::Comparison)}, + {"<", Operator("less", 8, 2, OperatorType::Comparison)}, + {">", Operator("greater", 8, 2, OperatorType::Comparison)}, + {"=", Operator("equals", 8, 2, OperatorType::Comparison)}, + {"LIKE", Operator("like", 8, 2)}, + {"ILIKE", Operator("ilike", 8, 2)}, + {"NOT LIKE", Operator("notLike", 8, 2)}, + {"NOT ILIKE", Operator("notILike", 8, 2)}, + {"IN", Operator("in", 8, 2)}, + {"NOT IN", Operator("notIn", 8, 2)}, + {"GLOBAL IN", Operator("globalIn", 8, 2)}, + {"GLOBAL NOT IN", Operator("globalNotIn", 8, 2)}, + {"||", Operator("concat", 9, 2, OperatorType::Mergeable)}, + {"+", Operator("plus", 10, 2)}, + {"-", Operator("minus", 10, 2)}, + {"*", Operator("multiply", 11, 2)}, + {"/", Operator("divide", 11, 2)}, + {"%", Operator("modulo", 11, 2)}, + {"MOD", Operator("modulo", 11, 2)}, + {"DIV", Operator("intDiv", 11, 2)}, + {".", Operator("tupleElement", 13, 2, OperatorType::TupleElement)}, + {"[", Operator("arrayElement", 13, 2, OperatorType::ArrayElement)}, + {"::", Operator("CAST", 13, 2, OperatorType::Cast)}, + {"IS NULL", Operator("isNull", 13, 1, OperatorType::IsNull)}, + {"IS NOT NULL", Operator("isNotNull", 13, 1, OperatorType::IsNull)}, }); std::vector> ParserExpressionImpl::unary_operators_table({ {"NOT", Operator("not", 5, 1)}, - {"-", Operator("negate", 13, 1)} + {"-", Operator("negate", 12, 1)} }); Operator ParserExpressionImpl::finish_between_operator = Operator("", 7, 0, OperatorType::FinishBetween); diff --git a/tests/queries/0_stateless/02477_is_null_parser.reference b/tests/queries/0_stateless/02477_is_null_parser.reference new file mode 100644 index 00000000000..57d96862011 --- /dev/null +++ b/tests/queries/0_stateless/02477_is_null_parser.reference @@ -0,0 +1,3 @@ +SELECT (\'a\' IS NULL) + (\'b\' IS NOT NULL) +SELECT (\'a\' IS NULL) = 0 +SELECT CAST(1 IS NULL, \'Int32\') diff --git a/tests/queries/0_stateless/02477_is_null_parser.sql b/tests/queries/0_stateless/02477_is_null_parser.sql new file mode 100644 index 00000000000..b95a35fde21 --- /dev/null +++ b/tests/queries/0_stateless/02477_is_null_parser.sql @@ -0,0 +1,3 @@ +EXPLAIN SYNTAX SELECT 'a' IS NULL + 'b' IS NOT NULL; +EXPLAIN SYNTAX SELECT 'a' IS NULL = 0; +EXPLAIN SYNTAX SELECT 1 IS NULL :: Int32; From fcdc9dfaacfb1d473c142cad7303d9c12e170a1b Mon Sep 17 00:00:00 2001 From: Dmitry Novik Date: Tue, 15 Nov 2022 20:52:18 +0000 Subject: [PATCH 483/526] Make test_global_overcommit_tracker non-parallel --- tests/integration/parallel_skip.json | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/parallel_skip.json b/tests/integration/parallel_skip.json index 3b4d1f2f29a..ba593b76bbf 100644 --- a/tests/integration/parallel_skip.json +++ b/tests/integration/parallel_skip.json @@ -48,6 +48,8 @@ "test_system_replicated_fetches/test.py::test_system_replicated_fetches", "test_zookeeper_config_load_balancing/test.py::test_round_robin", + "test_global_overcommit_tracker/test.py::test_global_overcommit", + "test_user_ip_restrictions/test.py::test_ipv4", "test_user_ip_restrictions/test.py::test_ipv6" ] From c60b98f57630015f577385640c102eb6d23cce72 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 15 Nov 2022 16:17:43 -0500 Subject: [PATCH 484/526] updates from review --- .../dictionaries/external-dictionaries/external-dicts.md | 8 ++++++-- docs/en/sql-reference/statements/create/dictionary.md | 2 +- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 7498afd8492..06b5b8a6746 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -9,7 +9,7 @@ import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dict -You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a DDL query, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. +You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. ClickHouse: @@ -29,7 +29,11 @@ The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tabl ## Creating a dictionary with a DDL query -Dictionaries can be created with [DDL queries](../../../sql-reference/statements/create/dictionary.md). This does not require any additional records in a server configuration file. This allows dictionaries to be worked with as first-class entities, like tables or views. +Dictionaries can be created with [DDL queries](../../../sql-reference/statements/create/dictionary.md), and this is the recommended method because with DDL created dictionaries: +- No additional records are added to server configuration files +- The dictionaries can be worked with as first-class entities, like tables or views +- Data can be read directly, using familiar SELECT rather than dictionary table functions +- The dictionaries can be easily renamed ## Creating a dictionary with a configuration file diff --git a/docs/en/sql-reference/statements/create/dictionary.md b/docs/en/sql-reference/statements/create/dictionary.md index 37051f8031a..a470b071971 100644 --- a/docs/en/sql-reference/statements/create/dictionary.md +++ b/docs/en/sql-reference/statements/create/dictionary.md @@ -130,4 +130,4 @@ Please see the details in [Dictionary sources](/docs/en/sql-reference/dictionari **See Also** - For more information, see the [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) section. -- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). +- [system.dictionaries](../../../operations/system-tables/dictionaries.md) — This table contains information about [Dictionaries](../../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). From 8d72f75556bbbb7dbe5675324bb3e00634480492 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 11 Nov 2022 16:26:04 +0100 Subject: [PATCH 485/526] Make the password wiping an option of IAST::format(). --- programs/copier/TaskTable.cpp | 2 +- src/Analyzer/FunctionNode.cpp | 7 +- src/Backups/BackupInfo.cpp | 5 +- src/Backups/BackupSettings.cpp | 7 +- src/Backups/BackupsWorker.cpp | 1 + src/Backups/RestoreSettings.cpp | 8 +- src/Common/SensitiveDataMasker.cpp | 25 + src/Common/SensitiveDataMasker.h | 4 + .../Access/InterpreterShowAccessQuery.cpp | 7 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 14 +- src/Interpreters/DDLTask.cpp | 5 +- src/Interpreters/SystemLog.cpp | 2 +- src/Interpreters/executeQuery.cpp | 15 +- src/Parsers/ASTBackupQuery.cpp | 8 +- src/Parsers/ASTBackupQuery.h | 5 +- src/Parsers/ASTFunction.cpp | 36 +- src/Parsers/ASTFunction.h | 14 + .../ASTFunctionWithKeyValueArguments.cpp | 17 +- .../ASTFunctionWithKeyValueArguments.h | 2 + src/Parsers/Access/ASTCreateUserQuery.cpp | 20 +- src/Parsers/Access/ASTCreateUserQuery.h | 2 +- src/Parsers/ExpressionListParsers.cpp | 8 +- src/Parsers/IAST.cpp | 25 +- src/Parsers/IAST.h | 29 +- src/Parsers/ParserBackupQuery.cpp | 16 +- src/Parsers/ParserCreateQuery.cpp | 28 +- src/Parsers/ParserCreateQuery.h | 11 + src/Parsers/findFunctionSecretArguments.cpp | 348 +++++++++++ src/Parsers/findFunctionSecretArguments.h | 16 + .../maskSensitiveInfoInQueryForLogging.cpp | 579 ------------------ .../maskSensitiveInfoInQueryForLogging.h | 19 - 31 files changed, 607 insertions(+), 678 deletions(-) create mode 100644 src/Parsers/findFunctionSecretArguments.cpp create mode 100644 src/Parsers/findFunctionSecretArguments.h delete mode 100644 src/Parsers/maskSensitiveInfoInQueryForLogging.cpp delete mode 100644 src/Parsers/maskSensitiveInfoInQueryForLogging.h diff --git a/programs/copier/TaskTable.cpp b/programs/copier/TaskTable.cpp index 5b09a9c99a7..65eaf8b7108 100644 --- a/programs/copier/TaskTable.cpp +++ b/programs/copier/TaskTable.cpp @@ -45,7 +45,7 @@ TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConfigurati engine_push_str = config.getString(table_prefix + "engine", "rand()"); { - ParserStorage parser_storage; + ParserStorage parser_storage{ParserStorage::TABLE_ENGINE}; engine_push_ast = parseQuery(parser_storage, engine_push_str, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); engine_push_partition_key_ast = extractPartitionKey(engine_push_ast); primary_key_comma_separated = boost::algorithm::join(extractPrimaryKeyColumnNames(engine_push_ast), ", "); diff --git a/src/Analyzer/FunctionNode.cpp b/src/Analyzer/FunctionNode.cpp index 55e097cc5f4..adc5782f7b7 100644 --- a/src/Analyzer/FunctionNode.cpp +++ b/src/Analyzer/FunctionNode.cpp @@ -166,7 +166,12 @@ ASTPtr FunctionNode::toASTImpl() const auto function_ast = std::make_shared(); function_ast->name = function_name; - function_ast->is_window_function = isWindowFunction(); + + if (isWindowFunction()) + { + function_ast->is_window_function = true; + function_ast->kind = ASTFunction::Kind::WINDOW_FUNCTION; + } const auto & parameters = getParameters(); if (!parameters.getNodes().empty()) diff --git a/src/Backups/BackupInfo.cpp b/src/Backups/BackupInfo.cpp index 009c262ad67..334bc5b5892 100644 --- a/src/Backups/BackupInfo.cpp +++ b/src/Backups/BackupInfo.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -36,6 +35,7 @@ ASTPtr BackupInfo::toAST() const auto func = std::make_shared(); func->name = backup_engine_name; func->no_empty_args = true; + func->kind = ASTFunction::Kind::BACKUP_NAME; auto list = std::make_shared(); func->arguments = list; @@ -95,8 +95,7 @@ BackupInfo BackupInfo::fromAST(const IAST & ast) String BackupInfo::toStringForLogging() const { - ASTPtr ast = toAST(); - return maskSensitiveInfoInBackupNameForLogging(serializeAST(*ast), ast); + return toAST()->formatForLogging(); } } diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 295ab723326..2179a572bcd 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -126,7 +126,12 @@ void BackupSettings::copySettingsToQuery(ASTBackupQuery & query) const query.settings = query_settings; - query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; + auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; + if (base_backup_name) + query.setOrReplace(query.base_backup_name, std::move(base_backup_name)); + else + query.reset(query.base_backup_name); + query.cluster_host_ids = !cluster_host_ids.empty() ? Util::clusterHostIDsToAST(cluster_host_ids) : nullptr; } diff --git a/src/Backups/BackupsWorker.cpp b/src/Backups/BackupsWorker.cpp index 5be2dde9a3d..704562488b1 100644 --- a/src/Backups/BackupsWorker.cpp +++ b/src/Backups/BackupsWorker.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 63915670fa4..322954abc2b 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -213,7 +214,12 @@ void RestoreSettings::copySettingsToQuery(ASTBackupQuery & query) const query.settings = query_settings; - query.base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; + auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; + if (base_backup_name) + query.setOrReplace(query.base_backup_name, std::move(base_backup_name)); + else + query.reset(query.base_backup_name); + query.cluster_host_ids = !cluster_host_ids.empty() ? BackupSettings::Util::clusterHostIDsToAST(cluster_host_ids) : nullptr; } diff --git a/src/Common/SensitiveDataMasker.cpp b/src/Common/SensitiveDataMasker.cpp index 15ae6961513..0c2e71a71f3 100644 --- a/src/Common/SensitiveDataMasker.cpp +++ b/src/Common/SensitiveDataMasker.cpp @@ -13,12 +13,19 @@ #include #include +#include #ifndef NDEBUG # include #endif +namespace ProfileEvents +{ + extern const Event QueryMaskingRulesMatch; +} + + namespace DB { namespace ErrorCodes @@ -165,6 +172,10 @@ size_t SensitiveDataMasker::wipeSensitiveData(std::string & data) const size_t matches = 0; for (const auto & rule : all_masking_rules) matches += rule->apply(data); + + if (matches) + ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches); + return matches; } @@ -184,4 +195,18 @@ size_t SensitiveDataMasker::rulesCount() const return all_masking_rules.size(); } + +std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length) +{ + std::string res = str; + + if (auto * masker = SensitiveDataMasker::getInstance()) + masker->wipeSensitiveData(res); + + if (max_length && (res.length() > max_length)) + res.resize(max_length); + + return res; +} + } diff --git a/src/Common/SensitiveDataMasker.h b/src/Common/SensitiveDataMasker.h index adb6f5d51e1..031956cb926 100644 --- a/src/Common/SensitiveDataMasker.h +++ b/src/Common/SensitiveDataMasker.h @@ -69,4 +69,8 @@ public: size_t rulesCount() const; }; +/// Wipes sensitive data and cuts to a specified maximum length in one function call. +/// If the maximum length is zero then the function doesn't cut to the maximum length. +std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length = 0); + } diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index d1d8ee63b8e..cafe5df104e 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -32,13 +32,8 @@ QueryPipeline InterpreterShowAccessQuery::executeImpl() const /// Build the result column. MutableColumnPtr column = ColumnString::create(); - WriteBufferFromOwnString buf; for (const auto & query : queries) - { - buf.restart(); - formatAST(*query, buf, false, true); - column->insert(buf.str()); - } + column->insert(query->formatWithHiddenSecrets()); String desc = "ACCESS"; return QueryPipeline(std::make_shared(Block{{std::move(column), std::make_shared(), desc}})); diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index a6c4df3a86d..3a36008aa0a 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -62,10 +62,7 @@ namespace } if (user.auth_data.getType() != AuthenticationType::NO_PASSWORD) - { query->auth_data = user.auth_data; - query->show_password = attach_mode; /// We don't show password unless it's an ATTACH statement. - } if (!user.settings.empty()) { @@ -256,19 +253,12 @@ QueryPipeline InterpreterShowCreateAccessEntityQuery::executeImpl() /// Build the result column. MutableColumnPtr column = ColumnString::create(); - WriteBufferFromOwnString create_query_buf; for (const auto & create_query : create_queries) - { - formatAST(*create_query, create_query_buf, false, true); - column->insert(create_query_buf.str()); - create_query_buf.restart(); - } + column->insert(create_query->formatWithHiddenSecrets()); /// Prepare description of the result column. - WriteBufferFromOwnString desc_buf; const auto & show_query = query_ptr->as(); - formatAST(show_query, desc_buf, false, true); - String desc = desc_buf.str(); + String desc = serializeAST(show_query); String prefix = "SHOW "; if (startsWith(desc, prefix)) desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 331cc5092c5..1c551dc89e0 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -15,7 +15,6 @@ #include #include #include -#include namespace DB @@ -170,11 +169,11 @@ void DDLTaskBase::parseQueryFromEntry(ContextPtr context) query = parseQuery(parser_query, begin, end, description, 0, settings.max_parser_depth); } -void DDLTaskBase::formatRewrittenQuery(ContextPtr) +void DDLTaskBase::formatRewrittenQuery(ContextPtr context) { /// Convert rewritten AST back to string. query_str = queryToString(*query); - query_for_logging = maskSensitiveInfoInQueryForLogging(query_str, query); + query_for_logging = query->formatForLogging(context->getSettingsRef().log_queries_cut_to_length); } ContextMutablePtr DDLTaskBase::makeQueryContext(ContextPtr from_context, const ZooKeeperPtr & /*zookeeper*/) diff --git a/src/Interpreters/SystemLog.cpp b/src/Interpreters/SystemLog.cpp index b5e4a0e81c7..2cd177edd40 100644 --- a/src/Interpreters/SystemLog.cpp +++ b/src/Interpreters/SystemLog.cpp @@ -75,7 +75,7 @@ namespace const char * getName() const override { return "storage definition with comment"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override { - ParserStorage storage_p; + ParserStorage storage_p{ParserStorage::TABLE_ENGINE}; ASTPtr storage; if (!storage_p.parse(pos, storage, expected)) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index e1966597e7d..b44db316f90 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -55,7 +56,6 @@ #include #include #include -#include #include #include @@ -352,6 +352,7 @@ static std::tuple executeQueryImpl( ASTPtr ast; String query; String query_for_logging; + size_t log_queries_cut_to_length = context->getSettingsRef().log_queries_cut_to_length; /// Parse the query from string. try @@ -392,15 +393,23 @@ static std::tuple executeQueryImpl( /// MUST go before any modification (except for prepared statements, /// since it substitute parameters and without them query does not contain /// parameters), to keep query as-is in query_log and server log. - query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast); + if (ast->hasSecretParts()) + { + /// IAST::formatForLogging() wipes secret parts in AST and then calls wipeSensitiveDataAndCutToLength(). + query_for_logging = ast->formatForLogging(log_queries_cut_to_length); + } + else + { + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); + } } catch (...) { /// Anyway log the query. if (query.empty()) query.assign(begin, std::min(end - begin, static_cast(max_query_size))); - query_for_logging = maskSensitiveInfoInQueryForLogging(query, ast); + query_for_logging = wipeSensitiveDataAndCutToLength(query, log_queries_cut_to_length); logQuery(query_for_logging, context, internal, stage); if (!internal) diff --git a/src/Parsers/ASTBackupQuery.cpp b/src/Parsers/ASTBackupQuery.cpp index 567b52b5669..85a0a0c7799 100644 --- a/src/Parsers/ASTBackupQuery.cpp +++ b/src/Parsers/ASTBackupQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -141,7 +142,7 @@ namespace } } - void formatSettings(const ASTPtr & settings, const ASTPtr & base_backup_name, const ASTPtr & cluster_host_ids, const IAST::FormatSettings & format) + void formatSettings(const ASTPtr & settings, const ASTFunction * base_backup_name, const ASTPtr & cluster_host_ids, const IAST::FormatSettings & format) { if (!settings && !base_backup_name && !cluster_host_ids) return; @@ -246,12 +247,13 @@ String ASTBackupQuery::getID(char) const ASTPtr ASTBackupQuery::clone() const { auto res = std::make_shared(*this); + res->children.clear(); if (backup_name) - res->backup_name = backup_name->clone(); + res->set(res->backup_name, backup_name->clone()); if (base_backup_name) - res->base_backup_name = base_backup_name->clone(); + res->set(res->base_backup_name, base_backup_name->clone()); if (cluster_host_ids) res->cluster_host_ids = cluster_host_ids->clone(); diff --git a/src/Parsers/ASTBackupQuery.h b/src/Parsers/ASTBackupQuery.h index dd094b77aae..708f5bf468f 100644 --- a/src/Parsers/ASTBackupQuery.h +++ b/src/Parsers/ASTBackupQuery.h @@ -8,6 +8,7 @@ namespace DB { using Strings = std::vector; using DatabaseAndTableName = std::pair; +class ASTFunction; /** BACKUP { TABLE [db.]table_name [AS [db.]table_name_in_backup] [PARTITION[S] partition_expr [,...]] | @@ -77,13 +78,13 @@ public: Elements elements; - ASTPtr backup_name; + ASTFunction * backup_name = nullptr; ASTPtr settings; /// Base backup. Only differences made after the base backup will be included in a newly created backup, /// so this setting allows to make an incremental backup. - ASTPtr base_backup_name; + ASTFunction * base_backup_name = nullptr; /// List of cluster's hosts' IDs if this is a BACKUP/RESTORE ON CLUSTER command. ASTPtr cluster_host_ids; diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 63dc9f6b3ac..3467b8b5ebc 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #include #include @@ -629,6 +630,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format && (name == "match" || name == "extract" || name == "extractAll" || name == "replaceRegexpOne" || name == "replaceRegexpAll"); + auto secret_arguments = std::make_pair(static_cast(-1), static_cast(-1)); + if (!settings.show_secrets) + secret_arguments = findFunctionSecretArguments(*this); + for (size_t i = 0, size = arguments->children.size(); i < size; ++i) { if (i != 0) @@ -636,12 +641,21 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; - bool special_hilite = false; - if (i == 1 && special_hilite_regexp) - special_hilite = highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-"); + if ((i == 1) && special_hilite_regexp + && highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-")) + { + continue; + } - if (!special_hilite) - arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); + if (!settings.show_secrets && (secret_arguments.first <= i) && (i < secret_arguments.second)) + { + settings.ostr << "'[HIDDEN]'"; + if (size - 1 < secret_arguments.second) + break; /// All other arguments should also be hidden. + continue; + } + + arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } } @@ -653,6 +667,18 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format return finishFormatWithWindow(settings, state, frame); } +bool ASTFunction::hasSecretParts() const +{ + if (arguments) + { + size_t num_arguments = arguments->children.size(); + auto secret_arguments = findFunctionSecretArguments(*this); + if ((secret_arguments.first < num_arguments) && (secret_arguments.first < secret_arguments.second)) + return true; + } + return childrenHaveSecretParts(); +} + String getFunctionName(const IAST * ast) { String res; diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 5756fb9ba86..052325fcf61 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -42,6 +42,18 @@ public: /// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names. bool no_empty_args = false; + /// Kind of this function-like expression, how it's actually used. + enum class Kind + { + ORDINARY_FUNCTION, + WINDOW_FUNCTION, + LAMBDA_FUNCTION, + TABLE_ENGINE, + DATABASE_ENGINE, + BACKUP_NAME, + }; + Kind kind = Kind::ORDINARY_FUNCTION; + /** Get text identifying the AST node. */ String getID(char delim) const override; @@ -55,6 +67,8 @@ public: std::string getWindowDescription() const; + bool hasSecretParts() const override; + protected: void formatImplWithoutAlias(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; void appendColumnNameImpl(WriteBuffer & ostr) const override; diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.cpp b/src/Parsers/ASTFunctionWithKeyValueArguments.cpp index d94490ab8b3..2c28e342610 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.cpp +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.cpp @@ -29,7 +29,16 @@ void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, F if (second_with_brackets) settings.ostr << (settings.hilite ? hilite_keyword : "") << "("; - second->formatImpl(settings, state, frame); + if (!settings.show_secrets && (first == "password")) + { + /// Hide password in the definition of a dictionary: + /// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password '[HIDDEN]' db 'default' table 'ids')) + settings.ostr << "'[HIDDEN]'"; + } + else + { + second->formatImpl(settings, state, frame); + } if (second_with_brackets) settings.ostr << (settings.hilite ? hilite_keyword : "") << ")"; @@ -38,6 +47,12 @@ void ASTPair::formatImpl(const FormatSettings & settings, FormatState & state, F } +bool ASTPair::hasSecretParts() const +{ + return first == "password"; +} + + void ASTPair::updateTreeHashImpl(SipHash & hash_state) const { hash_state.update(first.size()); diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.h b/src/Parsers/ASTFunctionWithKeyValueArguments.h index 4b745e2c1a2..e54bc9288f2 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.h +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.h @@ -29,6 +29,8 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; + + bool hasSecretParts() const override; void updateTreeHashImpl(SipHash & hash_state) const override; }; diff --git a/src/Parsers/Access/ASTCreateUserQuery.cpp b/src/Parsers/Access/ASTCreateUserQuery.cpp index b4eaf08856e..a59b5dd472c 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.cpp +++ b/src/Parsers/Access/ASTCreateUserQuery.cpp @@ -23,7 +23,7 @@ namespace } - void formatAuthenticationData(const AuthenticationData & auth_data, bool show_password, const IAST::FormatSettings & settings) + void formatAuthenticationData(const AuthenticationData & auth_data, const IAST::FormatSettings & settings) { auto auth_type = auth_data.getType(); if (auth_type == AuthenticationType::NO_PASSWORD) @@ -93,7 +93,7 @@ namespace throw Exception("AST: Unexpected authentication type " + toString(auth_type), ErrorCodes::LOGICAL_ERROR); } - if (password && !show_password) + if (password && !settings.show_secrets) { prefix = ""; password.reset(); @@ -324,7 +324,7 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & formatRenameTo(*new_name, format); if (auth_data) - formatAuthenticationData(*auth_data, show_password, format); + formatAuthenticationData(*auth_data, format); if (hosts) formatHosts(nullptr, *hosts, format); @@ -345,4 +345,18 @@ void ASTCreateUserQuery::formatImpl(const FormatSettings & format, FormatState & if (grantees) formatGrantees(*grantees, format); } + +bool ASTCreateUserQuery::hasSecretParts() const +{ + if (auth_data) + { + auto auth_type = auth_data->getType(); + if ((auth_type == AuthenticationType::PLAINTEXT_PASSWORD) + || (auth_type == AuthenticationType::SHA256_PASSWORD) + || (auth_type == AuthenticationType::DOUBLE_SHA1_PASSWORD)) + return true; + } + return childrenHaveSecretParts(); +} + } diff --git a/src/Parsers/Access/ASTCreateUserQuery.h b/src/Parsers/Access/ASTCreateUserQuery.h index 32d53b1bba7..8357b9f2b6c 100644 --- a/src/Parsers/Access/ASTCreateUserQuery.h +++ b/src/Parsers/Access/ASTCreateUserQuery.h @@ -45,7 +45,6 @@ public: std::optional new_name; std::optional auth_data; - bool show_password = true; /// formatImpl() shows a password or hash by default std::optional hosts; std::optional add_hosts; @@ -60,6 +59,7 @@ public: String getID(char) const override; ASTPtr clone() const override; void formatImpl(const FormatSettings & format, FormatState &, FormatStateStacked) const override; + bool hasSecretParts() const override; ASTPtr getRewrittenASTWithoutOnCluster(const WithoutOnClusterASTRewriteParams &) const override { return removeOnCluster(clone()); } }; } diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index d29aa248ec4..2d7c4892f6c 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -494,7 +494,12 @@ template static std::shared_ptr makeASTFunction(Operator & op, Args &&... args) { auto ast_function = makeASTFunction(op.function_name, std::forward(args)...); - ast_function->is_lambda_function = op.type == OperatorType::Lambda; + + if (op.type == OperatorType::Lambda) + { + ast_function->is_lambda_function = true; + ast_function->kind = ASTFunction::Kind::LAMBDA_FUNCTION; + } return ast_function; } @@ -999,6 +1004,7 @@ public: if (over.ignore(pos, expected)) { function_node->is_window_function = true; + function_node->kind = ASTFunction::Kind::WINDOW_FUNCTION; ASTPtr function_node_as_iast = function_node; diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 65740f290b5..7b1f8fe51f6 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -1,8 +1,10 @@ +#include + #include #include #include +#include #include -#include namespace DB @@ -165,11 +167,26 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -std::string IAST::formatForErrorMessage() const +String IAST::formatWithHiddenSecrets(size_t max_length, bool one_line, bool hilite) const { WriteBufferFromOwnString buf; - format(FormatSettings(buf, true /* one line */)); - return buf.str(); + + FormatSettings settings{buf, one_line}; + settings.hilite = hilite; + settings.show_secrets = false; + format(settings); + + return wipeSensitiveDataAndCutToLength(buf.str(), max_length); +} + +bool IAST::childrenHaveSecretParts() const +{ + for (const auto & child : children) + { + if (child->hasSecretParts()) + return true; + } + return false; } void IAST::cloneChildren() diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index bba878e4769..68e086b16b6 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -185,6 +185,7 @@ public: bool one_line; bool always_quote_identifiers = false; IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks; + bool show_secrets = true; /// Show secret parts of the AST (e.g. passwords, encryption keys). // Newline or whitespace. char nl_or_ws; @@ -240,9 +241,12 @@ public: } // A simple way to add some user-readable context to an error message. - std::string formatForErrorMessage() const; - template - static std::string formatForErrorMessage(const AstArray & array); + String formatWithHiddenSecrets(size_t max_length = 0, bool one_line = true, bool hilite = false) const; + String formatForLogging(size_t max_length = 0) const { return formatWithHiddenSecrets(max_length, true, false); } + String formatForErrorMessage() const { return formatWithHiddenSecrets(0, true, false); } + + /// If an AST has secret parts then formatForLogging() will replace them with the placeholder '[HIDDEN]'. + virtual bool hasSecretParts() const { return childrenHaveSecretParts(); } void cloneChildren(); @@ -272,6 +276,9 @@ public: static const char * hilite_substitution; static const char * hilite_none; +protected: + bool childrenHaveSecretParts() const; + private: size_t checkDepthImpl(size_t max_depth) const; @@ -282,20 +289,4 @@ private: ASTPtr * next_to_delete_list_head = nullptr; }; -template -std::string IAST::formatForErrorMessage(const AstArray & array) -{ - WriteBufferFromOwnString buf; - for (size_t i = 0; i < array.size(); ++i) - { - if (i > 0) - { - const char * delim = ", "; - buf.write(delim, strlen(delim)); - } - array[i]->format(IAST::FormatSettings(buf, true /* one line */)); - } - return buf.str(); -} - } diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 649304b1dab..4a8eb2b5802 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -207,7 +208,11 @@ namespace bool parseBackupName(IParser::Pos & pos, Expected & expected, ASTPtr & backup_name) { - return ParserIdentifierWithOptionalParameters{}.parse(pos, backup_name, expected); + if (!ParserIdentifierWithOptionalParameters{}.parse(pos, backup_name, expected)) + return false; + + backup_name->as().kind = ASTFunction::Kind::BACKUP_NAME; + return true; } bool parseBaseBackupSetting(IParser::Pos & pos, Expected & expected, ASTPtr & base_backup_name) @@ -358,11 +363,16 @@ bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->kind = kind; query->elements = std::move(elements); query->cluster = std::move(cluster); - query->backup_name = std::move(backup_name); + + if (backup_name) + query->set(query->backup_name, std::move(backup_name)); + query->settings = std::move(settings); - query->base_backup_name = std::move(base_backup_name); query->cluster_host_ids = std::move(cluster_host_ids); + if (base_backup_name) + query->set(query->base_backup_name, std::move(base_backup_name)); + return true; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index bf305ba4781..9c1c682ca03 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -442,6 +442,20 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!storage_like) return false; + if (engine) + { + switch (engine_kind) + { + case EngineKind::TABLE_ENGINE: + engine->as().kind = ASTFunction::Kind::TABLE_ENGINE; + break; + + case EngineKind::DATABASE_ENGINE: + engine->as().kind = ASTFunction::Kind::DATABASE_ENGINE; + break; + } + } + auto storage = std::make_shared(); storage->set(storage->engine, engine); storage->set(storage->partition_by, partition_by); @@ -449,7 +463,6 @@ bool ParserStorage::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) storage->set(storage->order_by, order_by); storage->set(storage->sample_by, sample_by); storage->set(storage->ttl_table, ttl_table); - storage->set(storage->settings, settings); node = storage; @@ -473,7 +486,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe ParserToken s_comma(TokenType::Comma); ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); - ParserStorage storage_p; + ParserStorage storage_p{ParserStorage::TABLE_ENGINE}; ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; @@ -635,6 +648,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe } } } + auto comment = parseComment(pos, expected); auto query = std::make_shared(); @@ -697,6 +711,8 @@ bool ParserCreateLiveViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserToken s_dot(TokenType::Dot); ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); + ParserStorage storage_p{ParserStorage::TABLE_ENGINE}; + ParserStorage storage_inner{ParserStorage::TABLE_ENGINE}; ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; @@ -835,8 +851,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & ParserToken s_eq(TokenType::Equals); ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); - ParserStorage storage_p; - ParserStorage storage_inner; + ParserStorage storage_p{ParserStorage::TABLE_ENGINE}; + ParserStorage storage_inner{ParserStorage::TABLE_ENGINE}; ParserTablePropertiesDeclarationList table_properties_p; ParserExpression watermark_p; ParserExpression lateness_p; @@ -1135,7 +1151,7 @@ bool ParserCreateDatabaseQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & e ParserKeyword s_attach("ATTACH"); ParserKeyword s_database("DATABASE"); ParserKeyword s_if_not_exists("IF NOT EXISTS"); - ParserStorage storage_p; + ParserStorage storage_p{ParserStorage::DATABASE_ENGINE}; ParserIdentifier name_p(true); ParserTableOverridesDeclarationList table_overrides_p; @@ -1222,7 +1238,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserToken s_dot(TokenType::Dot); ParserToken s_lparen(TokenType::OpeningRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket); - ParserStorage storage_p; + ParserStorage storage_p{ParserStorage::TABLE_ENGINE}; ParserIdentifier name_p; ParserTablePropertiesDeclarationList table_properties_p; ParserSelectWithUnionQuery select_p; diff --git a/src/Parsers/ParserCreateQuery.h b/src/Parsers/ParserCreateQuery.h index b5480fa6eae..e1573c92dab 100644 --- a/src/Parsers/ParserCreateQuery.h +++ b/src/Parsers/ParserCreateQuery.h @@ -420,9 +420,20 @@ protected: */ class ParserStorage : public IParserBase { +public: + /// What kind of engine we're going to parse. + enum EngineKind + { + TABLE_ENGINE, + DATABASE_ENGINE, + }; + + ParserStorage(EngineKind engine_kind_) : engine_kind(engine_kind_) {} + protected: const char * getName() const override { return "storage definition"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; + EngineKind engine_kind; }; /** Query like this: diff --git a/src/Parsers/findFunctionSecretArguments.cpp b/src/Parsers/findFunctionSecretArguments.cpp new file mode 100644 index 00000000000..65c06b7ec97 --- /dev/null +++ b/src/Parsers/findFunctionSecretArguments.cpp @@ -0,0 +1,348 @@ +#include + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace +{ + constexpr const std::pair npos{static_cast(-1), static_cast(-1)}; + + bool tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * res, bool allow_literal, bool allow_identifier) + { + if (!function.arguments) + return false; + + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return false; /// return false because we don't want to validate query here + + const auto & arguments = expr_list->children; + if (arg_idx >= arguments.size()) + return false; + + ASTPtr argument = arguments[arg_idx]; + if (allow_literal) + { + if (const auto * literal = argument->as()) + { + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; + } + } + + if (allow_identifier) + { + if (const auto * id = argument->as()) + { + if (res) + *res = id->name(); + return true; + } + } + + return false; + } + + + std::pair findS3FunctionSecretArguments(const ASTFunction & function, bool is_cluster_function) + { + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return npos; /// return because we don't want to validate query here + + const auto & arguments = expr_list->children; + + /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. + size_t url_arg_idx = is_cluster_function ? 1 : 0; + + /// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format']) + if (arguments.size() < url_arg_idx + 3) + return npos; + + if (arguments.size() >= url_arg_idx + 5) + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...) + return {url_arg_idx + 2, url_arg_idx + 3}; + } + else + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + /// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']). + /// So we will check whether the argument after 'url' is a format. + String format; + if (!tryGetStringFromArgument(function, url_arg_idx + 1, &format, /* allow_literal= */ true, /* allow_identifier= */ false)) + { + /// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`. + /// So it's safer to wipe the next argument just in case. + return {url_arg_idx + 2, url_arg_idx + 3}; /// Wipe either `aws_secret_access_key` or `structure`. + } + + if (KnownFormatNames::instance().exists(format)) + return npos; /// The argument after 'url' is a format: s3('url', 'format', ...) + + /// The argument after 'url' is not a format so we do our replacement: + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...) + return {url_arg_idx + 2, url_arg_idx + 3}; + } + } + + + std::pair findRemoteFunctionSecretArguments(const ASTFunction & function) + { + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return npos; /// return because we don't want to validate query here + + const auto & arguments = expr_list->children; + + /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: + /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) + /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) + + /// But we should check the number of arguments first because we don't need to do any replacements in case of + /// remote('addresses_expr', db.table) + if (arguments.size() < 3) + return npos; + + size_t arg_num = 1; + + /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. + const auto * table_function = arguments[arg_num]->as(); + if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) + { + ++arg_num; + } + else + { + String database; + if (!tryGetStringFromArgument(function, arg_num, &database, /* allow_literal= */ true, /* allow_identifier= */ true)) + { + /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. + /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' + /// before the argument 'password'. So it's safer to wipe two arguments just in case. + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + auto res = npos; + if (tryGetStringFromArgument(function, arg_num + 2, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `user`. + res = {arg_num + 2, arg_num + 3}; + } + if (tryGetStringFromArgument(function, arg_num + 3, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + { + /// Wipe either `password` or `sharding_key`. + if (res == npos) + res.first = arg_num + 3; + res.second = arg_num + 4; + } + return res; + } + + ++arg_num; + auto qualified_name = QualifiedTableName::parseFromString(database); + if (qualified_name.database.empty()) + ++arg_num; /// skip 'table' argument + } + + /// Skip username. + ++arg_num; + + /// Do our replacement: + /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) + /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string + /// before wiping it (because the `password` argument is always a literal string). + bool can_be_password = tryGetStringFromArgument(function, arg_num, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false); + if (can_be_password) + return {arg_num, arg_num + 1}; + + return npos; + } + + + std::pair findEncryptionFunctionSecretArguments(const ASTFunction & function) + { + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return npos; /// return because we don't want to validate query here + + const auto & arguments = expr_list->children; + + /// We replace all arguments after 'mode' with '[HIDDEN]': + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') + return {1, arguments.size()}; + } + + + std::pair findOrdinaryFunctionSecretArguments(const ASTFunction & function) + { + if (function.name == "mysql") + { + /// mysql('host:port', 'database', 'table', 'user', 'password', ...) + return {4, 5}; + } + else if (function.name == "postgresql") + { + /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) + return {4, 5}; + } + else if (function.name == "mongodb") + { + /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) + return {4, 5}; + } + else if (function.name == "s3" || function.name == "cosn" || function.name == "oss") + { + /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) + return findS3FunctionSecretArguments(function, /* is_cluster_function= */ false); + } + else if (function.name == "s3Cluster") + { + /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) + return findS3FunctionSecretArguments(function, /* is_cluster_function= */ true); + } + else if (function.name == "remote" || function.name == "remoteSecure") + { + /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) + return findRemoteFunctionSecretArguments(function); + } + else if ( + function.name == "encrypt" || function.name == "decrypt" || function.name == "aes_encrypt_mysql" + || function.name == "aes_decrypt_mysql" || function.name == "tryDecrypt") + { + /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) + return findEncryptionFunctionSecretArguments(function); + } + else + { + return npos; + } + } + + + std::pair findS3TableEngineSecretArguments(const ASTFunction & function) + { + const auto * expr_list = function.arguments->as(); + if (!expr_list) + return npos; /// return because we don't want to validate query here + + const auto & arguments = expr_list->children; + + /// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') + /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') + + /// But we should check the number of arguments first because we don't need to do that replacements in case of + /// S3('url' [, 'format' [, 'compression']]) + if (arguments.size() < 4) + return npos; + + return {2, 3}; + } + + + std::pair findTableEngineSecretArguments(const ASTFunction & function) + { + const String & engine_name = function.name; + if (engine_name == "ExternalDistributed") + { + /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') + return {5, 6}; + } + else if (engine_name == "MySQL") + { + /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) + return {4, 5}; + } + else if (engine_name == "PostgreSQL") + { + /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + return {4, 5}; + } + else if (engine_name == "MaterializedPostgreSQL") + { + /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) + return {4, 5}; + } + else if (engine_name == "MongoDB") + { + /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) + return {4, 5}; + } + else if (engine_name == "S3" || engine_name == "COSN" || engine_name == "OSS") + { + /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) + return findS3TableEngineSecretArguments(function); + } + else + { + return npos; + } + } + + + std::pair findDatabaseEngineSecretArguments(const ASTFunction & function) + { + const String & engine_name = function.name; + if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL") + { + /// MySQL('host:port', 'database', 'user', 'password') + return {3, 4}; + } + else if (engine_name == "PostgreSQL" || engine_name == "MaterializedPostgreSQL") + { + /// PostgreSQL('host:port', 'database', 'user', 'password', ...) + return {3, 4}; + } + else + { + return npos; + } + } + + + std::pair findBackupNameSecretArguments(const ASTFunction & function) + { + const String & engine_name = function.name; + if (engine_name == "S3") + { + /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) + return {2, 3}; + } + else + { + return npos; + } + } +} + +std::pair findFunctionSecretArguments(const ASTFunction & function) +{ + switch (function.kind) + { + case ASTFunction::Kind::ORDINARY_FUNCTION: return findOrdinaryFunctionSecretArguments(function); + case ASTFunction::Kind::WINDOW_FUNCTION: return npos; + case ASTFunction::Kind::LAMBDA_FUNCTION: return npos; + case ASTFunction::Kind::TABLE_ENGINE: return findTableEngineSecretArguments(function); + case ASTFunction::Kind::DATABASE_ENGINE: return findDatabaseEngineSecretArguments(function); + case ASTFunction::Kind::BACKUP_NAME: return findBackupNameSecretArguments(function); + } +} + +} diff --git a/src/Parsers/findFunctionSecretArguments.h b/src/Parsers/findFunctionSecretArguments.h new file mode 100644 index 00000000000..30c00c9466c --- /dev/null +++ b/src/Parsers/findFunctionSecretArguments.h @@ -0,0 +1,16 @@ +#pragma once + +#include + + +namespace DB +{ +class ASTFunction; + +/// Finds arguments of a specified function which should not be displayed for most users for security reasons. +/// That involves passwords and secret keys. +/// The function returns a pair of numbers [first, last) specifying arguments which must be hidden. +/// If the function returns {-1, -1} that means no arguments must be hidden. +std::pair findFunctionSecretArguments(const ASTFunction & function); + +} diff --git a/src/Parsers/maskSensitiveInfoInQueryForLogging.cpp b/src/Parsers/maskSensitiveInfoInQueryForLogging.cpp deleted file mode 100644 index ebadfa5425b..00000000000 --- a/src/Parsers/maskSensitiveInfoInQueryForLogging.cpp +++ /dev/null @@ -1,579 +0,0 @@ -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace ProfileEvents -{ - extern const Event QueryMaskingRulesMatch; -} - - -namespace DB -{ - -namespace -{ - enum class PasswordWipingMode - { - Query, - BackupName, - }; - - - template - class PasswordWipingVisitor - { - public: - struct Data - { - bool can_contain_password = false; - bool password_was_hidden = false; - bool is_create_table_query = false; - bool is_create_database_query = false; - bool is_create_dictionary_query = false; - PasswordWipingMode mode = PasswordWipingMode::Query; - }; - - using Visitor = std::conditional_t< - check_only, - ConstInDepthNodeVisitor, - InDepthNodeVisitor>; - - static bool needChildVisit(const ASTPtr & /* ast */, const ASTPtr & /* child */, Data & data) - { - if constexpr (check_only) - { - return !data.can_contain_password; - } - else - { - return true; - } - } - - static void visit(ASTPtr ast, Data & data) - { - if (auto * create_user_query = ast->as()) - { - visitCreateUserQuery(*create_user_query, data); - } - else if (auto * create_query = ast->as()) - { - visitCreateQuery(*create_query, data); - } - else if (auto * backup_query = ast->as()) - { - visitBackupQuery(*backup_query, data); - } - else if (auto * storage = ast->as()) - { - if (data.is_create_table_query) - visitTableEngine(*storage, data); - else if (data.is_create_database_query) - visitDatabaseEngine(*storage, data); - } - else if (auto * dictionary = ast->as()) - { - if (data.is_create_dictionary_query) - visitDictionaryDef(*dictionary, data); - } - else if (auto * function = ast->as()) - { - if (data.mode == PasswordWipingMode::BackupName) - wipePasswordFromBackupEngineArguments(*function, data); - else - visitFunction(*function, data); - } - } - - private: - static void visitCreateUserQuery(ASTCreateUserQuery & query, Data & data) - { - if (!query.auth_data) - return; - - auto auth_type = query.auth_data->getType(); - if (auth_type == AuthenticationType::NO_PASSWORD || auth_type == AuthenticationType::LDAP - || auth_type == AuthenticationType::KERBEROS || auth_type == AuthenticationType::SSL_CERTIFICATE) - return; /// No password, nothing to hide. - - if constexpr (check_only) - { - data.can_contain_password = true; - return; - } - - query.show_password = false; - data.password_was_hidden = true; - } - - static void visitCreateQuery(ASTCreateQuery & query, Data & data) - { - if (query.is_dictionary) - data.is_create_dictionary_query = true; - else if (query.table) - data.is_create_table_query = true; - else if (query.database) - data.is_create_database_query = true; - } - - static void visitTableEngine(ASTStorage & storage, Data & data) - { - if (!storage.engine) - return; - - const String & engine_name = storage.engine->name; - - if (engine_name == "ExternalDistributed") - { - /// ExternalDistributed('engine', 'host:port', 'database', 'table', 'user', 'password') - wipePasswordFromArgument(*storage.engine, data, 5); - } - else if (engine_name == "MySQL") - { - /// MySQL('host:port', 'database', 'table', 'user', 'password', ...) - wipePasswordFromArgument(*storage.engine, data, 4); - } - else if (engine_name == "PostgreSQL") - { - /// PostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - wipePasswordFromArgument(*storage.engine, data, 4); - } - else if (engine_name == "MaterializedPostgreSQL") - { - /// MaterializedPostgreSQL('host:port', 'database', 'table', 'user', 'password', ...) - wipePasswordFromArgument(*storage.engine, data, 4); - } - else if (engine_name == "MongoDB") - { - /// MongoDB('host:port', 'database', 'collection', 'user', 'password', ...) - wipePasswordFromArgument(*storage.engine, data, 4); - } - else if (engine_name == "S3" || engine_name == "COSN" || engine_name == "OSS") - { - /// S3('url', ['aws_access_key_id', 'aws_secret_access_key',] ...) - wipePasswordFromS3TableEngineArguments(*storage.engine, data); - } - } - - static void wipePasswordFromS3TableEngineArguments(ASTFunction & engine, Data & data) - { - /// We replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format') - /// S3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - - /// But we should check the number of arguments first because we don't need to do that replacements in case of - /// S3('url' [, 'format' [, 'compression']]) - size_t num_arguments; - if (!tryGetNumArguments(engine, &num_arguments) || (num_arguments < 4)) - return; - - wipePasswordFromArgument(engine, data, 2); - } - - static void visitDatabaseEngine(ASTStorage & storage, Data & data) - { - if (!storage.engine) - return; - - const String & engine_name = storage.engine->name; - - if (engine_name == "MySQL" || engine_name == "MaterializeMySQL" || engine_name == "MaterializedMySQL") - { - /// MySQL('host:port', 'database', 'user', 'password') - wipePasswordFromArgument(*storage.engine, data, 3); - } - else if (engine_name == "PostgreSQL" || engine_name == "MaterializedPostgreSQL") - { - /// PostgreSQL('host:port', 'database', 'user', 'password', ...) - wipePasswordFromArgument(*storage.engine, data, 3); - } - } - - static void visitFunction(ASTFunction & function, Data & data) - { - if (function.name == "mysql") - { - /// mysql('host:port', 'database', 'table', 'user', 'password', ...) - wipePasswordFromArgument(function, data, 4); - } - else if (function.name == "postgresql") - { - /// postgresql('host:port', 'database', 'table', 'user', 'password', ...) - wipePasswordFromArgument(function, data, 4); - } - else if (function.name == "mongodb") - { - /// mongodb('host:port', 'database', 'collection', 'user', 'password', ...) - wipePasswordFromArgument(function, data, 4); - } - else if (function.name == "s3" || function.name == "cosn" || function.name == "oss") - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ false); - } - else if (function.name == "s3Cluster") - { - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', ...) - wipePasswordFromS3FunctionArguments(function, data, /* is_cluster_function= */ true); - } - else if (function.name == "remote" || function.name == "remoteSecure") - { - /// remote('addresses_expr', 'db', 'table', 'user', 'password', ...) - wipePasswordFromRemoteFunctionArguments(function, data); - } - else if ( - function.name == "encrypt" || function.name == "decrypt" || function.name == "aes_encrypt_mysql" - || function.name == "aes_decrypt_mysql" || function.name == "tryDecrypt") - { - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) - wipePasswordFromEncryptionFunctionArguments(function, data); - } - } - - static void wipePasswordFromS3FunctionArguments(ASTFunction & function, Data & data, bool is_cluster_function) - { - /// s3Cluster('cluster_name', 'url', ...) has 'url' as its second argument. - size_t url_arg_idx = is_cluster_function ? 1 : 0; - - /// We're going to replace 'aws_secret_access_key' with '[HIDDEN'] for the following signatures: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// s3Cluster('cluster_name', 'url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'compression') - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// s3('url' [, 'format']) or s3Cluster('cluster_name', 'url' [, 'format']) - size_t num_arguments; - if (!tryGetNumArguments(function, &num_arguments) || (num_arguments < url_arg_idx + 3)) - return; - - if (num_arguments >= url_arg_idx + 5) - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', 'format', 'structure', ...) - wipePasswordFromArgument(function, data, url_arg_idx + 2); - } - else - { - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) - /// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']). - /// So we will check whether the argument after 'url' is a format. - String format; - if (!tryGetStringFromArgument(function, url_arg_idx + 1, &format, /* allow_literal= */ true, /* allow_identifier= */ false)) - { - /// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`. - /// So it's safer to wipe the next argument just in case. - wipePasswordFromArgument(function, data, url_arg_idx + 2); /// Wipe either `aws_secret_access_key` or `structure`. - return; - } - - if (KnownFormatNames::instance().exists(format)) - return; /// The argument after 'url' is a format: s3('url', 'format', ...) - - /// The argument after 'url' is not a format so we do our replacement: - /// s3('url', 'aws_access_key_id', 'aws_secret_access_key', ...) -> s3('url', 'aws_access_key_id', '[HIDDEN]', ...) - wipePasswordFromArgument(function, data, url_arg_idx + 2); - } - } - - static void wipePasswordFromRemoteFunctionArguments(ASTFunction & function, Data & data) - { - /// We're going to replace 'password' with '[HIDDEN'] for the following signatures: - /// remote('addresses_expr', db.table, 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', 'db', 'table', 'user' [, 'password'] [, sharding_key]) - /// remote('addresses_expr', table_function(), 'user' [, 'password'] [, sharding_key]) - - /// But we should check the number of arguments first because we don't need to do any replacements in case of - /// remote('addresses_expr', db.table) - size_t num_arguments; - if (!tryGetNumArguments(function, &num_arguments) || (num_arguments < 3)) - return; - - auto & arguments = assert_cast(*function.arguments).children; - size_t arg_num = 1; - - /// Skip 1 or 2 arguments with table_function() or db.table or 'db', 'table'. - const auto * table_function = arguments[arg_num]->as(); - if (table_function && KnownTableFunctionNames::instance().exists(table_function->name)) - { - ++arg_num; - } - else - { - String database; - if (!tryGetStringFromArgument(function, arg_num, &database, /* allow_literal= */ true, /* allow_identifier= */ true)) - { - /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. - /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' - /// before the argument 'password'. So it's safer to wipe two arguments just in case. - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - if (tryGetStringFromArgument(function, arg_num + 2, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) - wipePasswordFromArgument(function, data, arg_num + 2); /// Wipe either `password` or `user`. - if (tryGetStringFromArgument(function, arg_num + 3, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) - wipePasswordFromArgument(function, data, arg_num + 3); /// Wipe either `password` or `sharding_key`. - return; - } - - ++arg_num; - auto qualified_name = QualifiedTableName::parseFromString(database); - if (qualified_name.database.empty()) - ++arg_num; /// skip 'table' argument - } - - /// Skip username. - ++arg_num; - - /// Do our replacement: - /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) - /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string - /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(function, arg_num, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false); - if (can_be_password) - wipePasswordFromArgument(function, data, arg_num); - } - - static void wipePasswordFromEncryptionFunctionArguments(ASTFunction & function, Data & data) - { - /// We replace all arguments after 'mode' with '[HIDDEN]': - /// encrypt('mode', 'plaintext', 'key' [, iv, aad]) -> encrypt('mode', '[HIDDEN]') - wipePasswordFromArgument(function, data, 1); - removeArgumentsAfter(function, data, 2); - } - - static void visitBackupQuery(ASTBackupQuery & query, Data & data) - { - if (query.backup_name) - { - if (auto * backup_engine = query.backup_name->as()) - wipePasswordFromBackupEngineArguments(*backup_engine, data); - } - - if (query.base_backup_name) - { - if (auto * base_backup_engine = query.base_backup_name->as()) - wipePasswordFromBackupEngineArguments(*base_backup_engine, data); - } - } - - static void wipePasswordFromBackupEngineArguments(ASTFunction & engine, Data & data) - { - if (engine.name == "S3") - { - /// BACKUP ... TO S3(url, [aws_access_key_id, aws_secret_access_key]) - wipePasswordFromArgument(engine, data, 2); - } - } - - static void wipePasswordFromArgument(ASTFunction & function, Data & data, size_t arg_idx) - { - if (!function.arguments) - return; - - auto * expr_list = function.arguments->as(); - if (!expr_list) - return; /// return because we don't want to validate query here - - auto & arguments = expr_list->children; - if (arg_idx >= arguments.size()) - return; - - if constexpr (check_only) - { - data.can_contain_password = true; - return; - } - - arguments[arg_idx] = std::make_shared("[HIDDEN]"); - data.password_was_hidden = true; - } - - static void removeArgumentsAfter(ASTFunction & function, Data & data, size_t new_num_arguments) - { - if (!function.arguments) - return; - - auto * expr_list = function.arguments->as(); - if (!expr_list) - return; /// return because we don't want to validate query here - - auto & arguments = expr_list->children; - if (new_num_arguments >= arguments.size()) - return; - - if constexpr (check_only) - { - data.can_contain_password = true; - return; - } - - arguments.resize(new_num_arguments); - data.password_was_hidden = true; - } - - static bool tryGetNumArguments(const ASTFunction & function, size_t * res) - { - if (!function.arguments) - return false; - - auto * expr_list = function.arguments->as(); - if (!expr_list) - return false; /// return false because we don't want to validate query here - - const auto & arguments = expr_list->children; - if (res) - *res = arguments.size(); - return true; - } - - static bool - tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * res, bool allow_literal, bool allow_identifier) - { - if (!function.arguments) - return false; - - const auto * expr_list = function.arguments->as(); - if (!expr_list) - return false; /// return false because we don't want to validate query here - - const auto & arguments = expr_list->children; - if (arg_idx >= arguments.size()) - return false; - - ASTPtr argument = arguments[arg_idx]; - if (allow_literal) - { - if (const auto * literal = argument->as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } - } - - if (allow_identifier) - { - if (const auto * id = argument->as()) - { - if (res) - *res = id->name(); - return true; - } - } - - return false; - } - - static void visitDictionaryDef(ASTDictionary & dictionary, Data & data) - { - if (!dictionary.source || !dictionary.source->elements) - return; - - const auto * expr_list = dictionary.source->elements->as(); - if (!expr_list) - return; /// return because we don't want to validate query here - - const auto & elements = expr_list->children; - - /// We replace password in the dictionary's definition: - /// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password 'qwe123' db 'default' table 'ids')) -> - /// SOURCE(CLICKHOUSE(host 'example01-01-1' port 9000 user 'default' password '[HIDDEN]' db 'default' table 'ids')) - for (const auto & element : elements) - { - auto * pair = element->as(); - if (!pair) - continue; /// just skip because we don't want to validate query here - - if (pair->first == "password") - { - if constexpr (check_only) - { - data.can_contain_password = true; - return; - } - pair->set(pair->second, std::make_shared("[HIDDEN]")); - data.password_was_hidden = true; - } - } - } - }; - - /// Checks the type of a specified AST and returns true if it can contain a password. - bool canContainPassword(const IAST & ast, PasswordWipingMode mode) - { - using WipingVisitor = PasswordWipingVisitor; - WipingVisitor::Data data; - data.mode = mode; - WipingVisitor::Visitor visitor{data}; - ASTPtr ast_ptr = std::const_pointer_cast(ast.shared_from_this()); - visitor.visit(ast_ptr); - return data.can_contain_password; - } - - /// Removes a password or its hash from a query if it's specified there or replaces it with some placeholder. - /// This function is used to prepare a query for storing in logs (we don't want logs to contain sensitive information). - bool wipePasswordFromQuery(ASTPtr ast, PasswordWipingMode mode) - { - using WipingVisitor = PasswordWipingVisitor; - WipingVisitor::Data data; - data.mode = mode; - WipingVisitor::Visitor visitor{data}; - visitor.visit(ast); - return data.password_was_hidden; - } - - /// Common utility for masking sensitive information. - String maskSensitiveInfoImpl(const String & query, const ASTPtr & parsed_query, PasswordWipingMode mode) - { - String res = query; - - // Wiping a password or hash from the query because we don't want it to go to logs. - if (parsed_query && canContainPassword(*parsed_query, mode)) - { - ASTPtr ast_without_password = parsed_query->clone(); - if (wipePasswordFromQuery(ast_without_password, mode)) - res = serializeAST(*ast_without_password); - } - - // Wiping sensitive data before cropping query by log_queries_cut_to_length, - // otherwise something like credit card without last digit can go to log. - if (auto * masker = SensitiveDataMasker::getInstance()) - { - auto matches = masker->wipeSensitiveData(res); - if (matches > 0) - { - ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches); - } - } - - //res = res.substr(0, context->getSettingsRef().log_queries_cut_to_length); - - return res; - } -} - - -String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query) -{ - return maskSensitiveInfoImpl(query, parsed_query, PasswordWipingMode::Query); -} - - -String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast) -{ - return maskSensitiveInfoImpl(backup_name, ast, PasswordWipingMode::BackupName); -} - -} diff --git a/src/Parsers/maskSensitiveInfoInQueryForLogging.h b/src/Parsers/maskSensitiveInfoInQueryForLogging.h deleted file mode 100644 index 6284b9a9294..00000000000 --- a/src/Parsers/maskSensitiveInfoInQueryForLogging.h +++ /dev/null @@ -1,19 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/// Makes a version of a query without sensitive information (e.g. passwords) for logging. -/// The parameter `parsed query` is allowed to be nullptr if the query cannot be parsed. -/// Does not validate AST, works a best-effort way. -String maskSensitiveInfoInQueryForLogging(const String & query, const ASTPtr & parsed_query); - -/// Makes a version of backup name without sensitive information (e.g. passwords) for logging. -/// Does not validate AST, works a best-effort way. -String maskSensitiveInfoInBackupNameForLogging(const String & backup_name, const ASTPtr & ast); - -} From d365570b27c7ee28dcabece9cedb14d4d04e0085 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 14 Nov 2022 16:37:50 +0100 Subject: [PATCH 486/526] Hide passwords in SHOW CREATE TABLE and system.tables. --- src/Interpreters/InterpreterShowCreateQuery.cpp | 4 +--- src/Storages/System/StorageSystemTables.cpp | 17 ++++++++--------- 2 files changed, 9 insertions(+), 12 deletions(-) diff --git a/src/Interpreters/InterpreterShowCreateQuery.cpp b/src/Interpreters/InterpreterShowCreateQuery.cpp index 6ae51c1ff23..6853b3c1be1 100644 --- a/src/Interpreters/InterpreterShowCreateQuery.cpp +++ b/src/Interpreters/InterpreterShowCreateQuery.cpp @@ -92,9 +92,7 @@ QueryPipeline InterpreterShowCreateQuery::executeImpl() create.to_inner_uuid = UUIDHelpers::Nil; } - WriteBufferFromOwnString buf; - formatAST(*create_query, buf, false, false); - String res = buf.str(); + String res = create_query->formatWithHiddenSecrets(/* max_length= */ 0, /* one_line= */ false, /* hilite= */ false); MutableColumnPtr column = ColumnString::create(); column->insert(res); diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 83f922850a3..3f47c1ddb8e 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -10,7 +10,6 @@ #include #include #include -#include #include #include #include @@ -232,7 +231,7 @@ protected: { auto temp_db = DatabaseCatalog::instance().getDatabaseForTemporaryTables(); ASTPtr ast = temp_db ? temp_db->tryGetCreateTableQuery(table.second->getStorageID().getTableName(), context) : nullptr; - res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); + res_columns[res_index++]->insert(ast ? ast->formatWithHiddenSecrets() : ""); } // engine_full @@ -383,7 +382,7 @@ protected: } if (columns_mask[src_index++]) - res_columns[res_index++]->insert(ast ? queryToString(ast) : ""); + res_columns[res_index++]->insert(ast ? ast->formatWithHiddenSecrets() : ""); if (columns_mask[src_index++]) { @@ -391,7 +390,7 @@ protected: if (ast_create && ast_create->storage) { - engine_full = queryToString(*ast_create->storage); + engine_full = ast_create->storage->formatWithHiddenSecrets(); static const char * const extra_head = " ENGINE = "; if (startsWith(engine_full, extra_head)) @@ -405,7 +404,7 @@ protected: { String as_select; if (ast_create && ast_create->select) - as_select = queryToString(*ast_create->select); + as_select = ast_create->select->formatWithHiddenSecrets(); res_columns[res_index++]->insert(as_select); } } @@ -420,7 +419,7 @@ protected: if (columns_mask[src_index++]) { if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPartitionKeyAST())) - res_columns[res_index++]->insert(queryToString(expression_ptr)); + res_columns[res_index++]->insert(expression_ptr->formatWithHiddenSecrets()); else res_columns[res_index++]->insertDefault(); } @@ -428,7 +427,7 @@ protected: if (columns_mask[src_index++]) { if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSortingKey().expression_list_ast)) - res_columns[res_index++]->insert(queryToString(expression_ptr)); + res_columns[res_index++]->insert(expression_ptr->formatWithHiddenSecrets()); else res_columns[res_index++]->insertDefault(); } @@ -436,7 +435,7 @@ protected: if (columns_mask[src_index++]) { if (metadata_snapshot && (expression_ptr = metadata_snapshot->getPrimaryKey().expression_list_ast)) - res_columns[res_index++]->insert(queryToString(expression_ptr)); + res_columns[res_index++]->insert(expression_ptr->formatWithHiddenSecrets()); else res_columns[res_index++]->insertDefault(); } @@ -444,7 +443,7 @@ protected: if (columns_mask[src_index++]) { if (metadata_snapshot && (expression_ptr = metadata_snapshot->getSamplingKeyAST())) - res_columns[res_index++]->insert(queryToString(expression_ptr)); + res_columns[res_index++]->insert(expression_ptr->formatWithHiddenSecrets()); else res_columns[res_index++]->insertDefault(); } From 2e242fa99104506e92b44f08a0f5d84c34ddd649 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 14 Nov 2022 18:42:46 +0100 Subject: [PATCH 487/526] Add tests. --- .../__init__.py | 0 .../test.py | 43 +++++++++++++++++++ 2 files changed, 43 insertions(+) rename tests/integration/{test_mask_sensitive_info_in_logs => test_mask_sensitive_info}/__init__.py (100%) rename tests/integration/{test_mask_sensitive_info_in_logs => test_mask_sensitive_info}/test.py (88%) diff --git a/tests/integration/test_mask_sensitive_info_in_logs/__init__.py b/tests/integration/test_mask_sensitive_info/__init__.py similarity index 100% rename from tests/integration/test_mask_sensitive_info_in_logs/__init__.py rename to tests/integration/test_mask_sensitive_info/__init__.py diff --git a/tests/integration/test_mask_sensitive_info_in_logs/test.py b/tests/integration/test_mask_sensitive_info/test.py similarity index 88% rename from tests/integration/test_mask_sensitive_info_in_logs/test.py rename to tests/integration/test_mask_sensitive_info/test.py index 48f11fbf7a1..f546c559f66 100644 --- a/tests/integration/test_mask_sensitive_info_in_logs/test.py +++ b/tests/integration/test_mask_sensitive_info/test.py @@ -1,6 +1,7 @@ import pytest import random, string from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", with_zookeeper=True) @@ -110,6 +111,22 @@ def test_create_table(): for i, table_engine in enumerate(table_engines): node.query(f"CREATE TABLE table{i} (x int) ENGINE = {table_engine}") + assert ( + node.query("SHOW CREATE TABLE table0") + == "CREATE TABLE default.table0\\n(\\n `x` Int32\\n)\\nENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n" + ) + + assert node.query( + "SELECT create_table_query, engine_full FROM system.tables WHERE name = 'table0'" + ) == TSV( + [ + [ + "CREATE TABLE default.table0 (`x` Int32) ENGINE = MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')", + "MySQL(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')", + ], + ] + ) + check_logs( must_contain=[ "CREATE TABLE table0 (`x` int) ENGINE = MySQL('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')", @@ -189,6 +206,22 @@ def test_table_functions(): for i, table_function in enumerate(table_functions): node.query(f"CREATE TABLE tablefunc{i} (x int) AS {table_function}") + assert ( + node.query("SHOW CREATE TABLE tablefunc0") + == "CREATE TABLE default.tablefunc0\\n(\\n `x` Int32\\n) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')\n" + ) + + assert node.query( + "SELECT create_table_query, engine_full FROM system.tables WHERE name = 'tablefunc0'" + ) == TSV( + [ + [ + "CREATE TABLE default.tablefunc0 (`x` Int32) AS mysql(\\'mysql57:3306\\', \\'mysql_db\\', \\'mysql_table\\', \\'mysql_user\\', \\'[HIDDEN]\\')", + "", + ], + ] + ) + check_logs( must_contain=[ "CREATE TABLE tablefunc0 (`x` int) AS mysql('mysql57:3306', 'mysql_db', 'mysql_table', 'mysql_user', '[HIDDEN]')", @@ -268,6 +301,16 @@ def test_create_dictionary(): f"LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())" ) + assert ( + node.query("SHOW CREATE TABLE dict1") + == "CREATE DICTIONARY default.dict1\\n(\\n `n` int DEFAULT 0,\\n `m` int DEFAULT 1\\n)\\nPRIMARY KEY n\\nSOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\'))\\nLIFETIME(MIN 0 MAX 10)\\nLAYOUT(FLAT())\n" + ) + + assert ( + node.query("SELECT create_table_query FROM system.tables WHERE name = 'dict1'") + == "CREATE DICTIONARY default.dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST \\'localhost\\' PORT 9000 USER \\'user1\\' TABLE \\'test\\' PASSWORD \\'[HIDDEN]\\' DB \\'default\\')) LIFETIME(MIN 0 MAX 10) LAYOUT(FLAT())\n" + ) + check_logs( must_contain=[ "CREATE DICTIONARY dict1 (`n` int DEFAULT 0, `m` int DEFAULT 1) PRIMARY KEY n " From ce81166c7edab32c99679cf961520aeababdad4d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 14 Nov 2022 19:28:19 +0100 Subject: [PATCH 488/526] Fix style. --- src/Backups/BackupSettings.cpp | 2 +- src/Common/KnownObjectNames.cpp | 4 ++-- src/Common/SensitiveDataMasker.h | 2 +- src/Formats/FormatFactory.cpp | 4 ++-- src/Parsers/ASTFunction.cpp | 12 ++++++------ src/Parsers/ASTFunction.h | 2 +- src/Parsers/ASTFunctionWithKeyValueArguments.h | 2 +- src/Parsers/ParserBackupQuery.cpp | 2 +- 8 files changed, 15 insertions(+), 15 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index 2179a572bcd..bed34f2fbcc 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -2,8 +2,8 @@ #include #include #include -#include #include +#include #include #include diff --git a/src/Common/KnownObjectNames.cpp b/src/Common/KnownObjectNames.cpp index 45066fc93d4..0862c52d892 100644 --- a/src/Common/KnownObjectNames.cpp +++ b/src/Common/KnownObjectNames.cpp @@ -10,8 +10,8 @@ bool KnownObjectNames::exists(const String & name) const std::lock_guard lock{mutex}; if (names.contains(name)) return true; - - if (case_insensitive_names.empty()) + + if (!case_insensitive_names.empty()) { String lower_name = Poco::toLower(name); if (case_insensitive_names.contains(lower_name)) diff --git a/src/Common/SensitiveDataMasker.h b/src/Common/SensitiveDataMasker.h index 031956cb926..de5cc125dcc 100644 --- a/src/Common/SensitiveDataMasker.h +++ b/src/Common/SensitiveDataMasker.h @@ -71,6 +71,6 @@ public: /// Wipes sensitive data and cuts to a specified maximum length in one function call. /// If the maximum length is zero then the function doesn't cut to the maximum length. -std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length = 0); +std::string wipeSensitiveDataAndCutToLength(const std::string & str, size_t max_length); } diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index bf1168dd8a7..868e275fcdc 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -445,8 +445,8 @@ void FormatFactory::registerInputFormat(const String & name, InputCreator input_ if (target) throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(input_creator); - KnownFormatNames::instance().add(name); registerFileExtension(name, name); + KnownFormatNames::instance().add(name); } void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker) @@ -455,7 +455,6 @@ void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name if (target) throw Exception("FormatFactory: Non trivial prefix and suffix checker " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(non_trivial_prefix_and_suffix_checker); - KnownFormatNames::instance().add(name); } void FormatFactory::registerAppendSupportChecker(const String & name, AppendSupportChecker append_support_checker) @@ -486,6 +485,7 @@ void FormatFactory::registerOutputFormat(const String & name, OutputCreator outp throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); target = std::move(output_creator); registerFileExtension(name, name); + KnownFormatNames::instance().add(name); } void FormatFactory::registerFileExtension(const String & extension, const String & format_name) diff --git a/src/Parsers/ASTFunction.cpp b/src/Parsers/ASTFunction.cpp index 3467b8b5ebc..8eeb06871a8 100644 --- a/src/Parsers/ASTFunction.cpp +++ b/src/Parsers/ASTFunction.cpp @@ -641,12 +641,6 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format if (arguments->children[i]->as()) settings.ostr << "SETTINGS "; - if ((i == 1) && special_hilite_regexp - && highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-")) - { - continue; - } - if (!settings.show_secrets && (secret_arguments.first <= i) && (i < secret_arguments.second)) { settings.ostr << "'[HIDDEN]'"; @@ -655,6 +649,12 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format continue; } + if ((i == 1) && special_hilite_regexp + && highlightStringLiteralWithMetacharacters(arguments->children[i], settings, "|()^$.[]?*+{:-")) + { + continue; + } + arguments->children[i]->formatImpl(settings, state, nested_dont_need_parens); } } diff --git a/src/Parsers/ASTFunction.h b/src/Parsers/ASTFunction.h index 052325fcf61..83b06bd26ec 100644 --- a/src/Parsers/ASTFunction.h +++ b/src/Parsers/ASTFunction.h @@ -42,7 +42,7 @@ public: /// do not print empty parentheses if there are no args - compatibility with new AST for data types and engine names. bool no_empty_args = false; - /// Kind of this function-like expression, how it's actually used. + /// Specifies where this function-like expression is used. enum class Kind { ORDINARY_FUNCTION, diff --git a/src/Parsers/ASTFunctionWithKeyValueArguments.h b/src/Parsers/ASTFunctionWithKeyValueArguments.h index e54bc9288f2..67d591dfcdc 100644 --- a/src/Parsers/ASTFunctionWithKeyValueArguments.h +++ b/src/Parsers/ASTFunctionWithKeyValueArguments.h @@ -29,7 +29,7 @@ public: ASTPtr clone() const override; void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; - + bool hasSecretParts() const override; void updateTreeHashImpl(SipHash & hash_state) const override; diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 4a8eb2b5802..78bc10511b2 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -210,7 +210,7 @@ namespace { if (!ParserIdentifierWithOptionalParameters{}.parse(pos, backup_name, expected)) return false; - + backup_name->as().kind = ASTFunction::Kind::BACKUP_NAME; return true; } From 4a25fcc056ae4bf03bec114525b1eaced6dec672 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 15 Nov 2022 17:11:26 +0100 Subject: [PATCH 489/526] Fix procesing '' as a database name in remote() --- src/Parsers/findFunctionSecretArguments.cpp | 73 +++++++++++++++------ 1 file changed, 54 insertions(+), 19 deletions(-) diff --git a/src/Parsers/findFunctionSecretArguments.cpp b/src/Parsers/findFunctionSecretArguments.cpp index 65c06b7ec97..e40c25bd83b 100644 --- a/src/Parsers/findFunctionSecretArguments.cpp +++ b/src/Parsers/findFunctionSecretArguments.cpp @@ -15,7 +15,7 @@ namespace { constexpr const std::pair npos{static_cast(-1), static_cast(-1)}; - bool tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * res, bool allow_literal, bool allow_identifier) + bool tryGetStringFromArgument(const ASTFunction & function, size_t arg_idx, String * res, bool allow_identifier = true) { if (!function.arguments) return false; @@ -29,16 +29,13 @@ namespace return false; ASTPtr argument = arguments[arg_idx]; - if (allow_literal) + if (const auto * literal = argument->as()) { - if (const auto * literal = argument->as()) - { - if (literal->value.getType() != Field::Types::String) - return false; - if (res) - *res = literal->value.safeGet(); - return true; - } + if (literal->value.getType() != Field::Types::String) + return false; + if (res) + *res = literal->value.safeGet(); + return true; } if (allow_identifier) @@ -86,7 +83,7 @@ namespace /// We need to distinguish that from s3('url', 'format', 'structure' [, 'compression_method']). /// So we will check whether the argument after 'url' is a format. String format; - if (!tryGetStringFromArgument(function, url_arg_idx + 1, &format, /* allow_literal= */ true, /* allow_identifier= */ false)) + if (!tryGetStringFromArgument(function, url_arg_idx + 1, &format, /* allow_identifier= */ false)) { /// We couldn't evaluate the argument after 'url' so we don't know whether it is a format or `aws_access_key_id`. /// So it's safer to wipe the next argument just in case. @@ -103,6 +100,40 @@ namespace } + /// Tries to get either a database name or a qualified table name from an argument. + /// Empty string is also allowed (it means the default database). + /// The function is used by findRemoteFunctionSecretArguments() to determine how many arguments to skip before a password. + bool tryGetDatabaseNameOrQualifiedTableName( + const ASTFunction & function, + size_t arg_idx, + std::optional & res_database, + std::optional & res_qualified_table_name) + { + res_database.reset(); + res_qualified_table_name.reset(); + + String str; + if (!tryGetStringFromArgument(function, arg_idx, &str, /* allow_identifier= */ true)) + return false; + + if (str.empty()) + { + res_database = ""; + return true; + } + + auto qualified_table_name = QualifiedTableName::tryParseFromString(str); + if (!qualified_table_name) + return false; + + if (qualified_table_name->database.empty()) + res_database = std::move(qualified_table_name->table); + else + res_qualified_table_name = std::move(qualified_table_name); + return true; + } + + std::pair findRemoteFunctionSecretArguments(const ASTFunction & function) { const auto * expr_list = function.arguments->as(); @@ -131,8 +162,9 @@ namespace } else { - String database; - if (!tryGetStringFromArgument(function, arg_num, &database, /* allow_literal= */ true, /* allow_identifier= */ true)) + std::optional database; + std::optional qualified_table_name; + if (!tryGetDatabaseNameOrQualifiedTableName(function, arg_num, database, qualified_table_name)) { /// We couldn't evaluate the argument so we don't know whether it is 'db.table' or just 'db'. /// Hence we can't figure out whether we should skip one argument 'user' or two arguments 'table', 'user' @@ -140,12 +172,12 @@ namespace /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string /// before wiping it (because the `password` argument is always a literal string). auto res = npos; - if (tryGetStringFromArgument(function, arg_num + 2, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + if (tryGetStringFromArgument(function, arg_num + 2, nullptr, /* allow_identifier= */ false)) { /// Wipe either `password` or `user`. res = {arg_num + 2, arg_num + 3}; } - if (tryGetStringFromArgument(function, arg_num + 3, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false)) + if (tryGetStringFromArgument(function, arg_num + 3, nullptr, /* allow_identifier= */ false)) { /// Wipe either `password` or `sharding_key`. if (res == npos) @@ -155,10 +187,13 @@ namespace return res; } + /// Skip the current argument (which is either a database name or a qualified table name). ++arg_num; - auto qualified_name = QualifiedTableName::parseFromString(database); - if (qualified_name.database.empty()) - ++arg_num; /// skip 'table' argument + if (database) + { + /// Skip the 'table' argument if the previous argument was a database name. + ++arg_num; + } } /// Skip username. @@ -168,7 +203,7 @@ namespace /// remote('addresses_expr', db.table, 'user', 'password', ...) -> remote('addresses_expr', db.table, 'user', '[HIDDEN]', ...) /// The last argument can be also a `sharding_key`, so we need to check that argument is a literal string /// before wiping it (because the `password` argument is always a literal string). - bool can_be_password = tryGetStringFromArgument(function, arg_num, nullptr, /* allow_literal= */ true, /* allow_identifier= */ false); + bool can_be_password = tryGetStringFromArgument(function, arg_num, nullptr, /* allow_identifier= */ false); if (can_be_password) return {arg_num, arg_num + 1}; From 63be12adca23c5b227bf85a330143263c2316550 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Tue, 15 Nov 2022 17:11:35 +0100 Subject: [PATCH 490/526] Fix tests. --- .../0_stateless/01018_ddl_dictionaries_create.reference | 4 ++-- tests/queries/0_stateless/01190_full_attach_syntax.reference | 4 ++-- .../0_stateless/01224_no_superfluous_dict_reload.reference | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference b/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference index 9b130f11df6..450eaee4b2f 100644 --- a/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference +++ b/tests/queries/0_stateless/01018_ddl_dictionaries_create.reference @@ -1,5 +1,5 @@ =DICTIONARY in Ordinary DB -CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY db_01018.dict1\n(\n `key_column` UInt64 DEFAULT 0,\n `second_column` UInt8 DEFAULT 1,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) dict1 1 db_01018 dict1 @@ -12,7 +12,7 @@ db_01018 dict1 ==DROP DICTIONARY 0 =DICTIONARY in Memory DB -CREATE DICTIONARY memory_db.dict2\n(\n `key_column` UInt64 DEFAULT 0 INJECTIVE,\n `second_column` UInt8 DEFAULT 1 EXPRESSION rand() % 222,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY memory_db.dict2\n(\n `key_column` UInt64 DEFAULT 0 INJECTIVE,\n `second_column` UInt8 DEFAULT 1 EXPRESSION rand() % 222,\n `third_column` String DEFAULT \'qqq\'\n)\nPRIMARY KEY key_column\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'database_for_dict_01018\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) dict2 1 memory_db dict2 diff --git a/tests/queries/0_stateless/01190_full_attach_syntax.reference b/tests/queries/0_stateless/01190_full_attach_syntax.reference index f924c2ec780..61a0603975a 100644 --- a/tests/queries/0_stateless/01190_full_attach_syntax.reference +++ b/tests/queries/0_stateless/01190_full_attach_syntax.reference @@ -1,5 +1,5 @@ -CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) -CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) +CREATE DICTIONARY test_01190.dict\n(\n `key` UInt64 DEFAULT 0,\n `col` UInt8 DEFAULT 1\n)\nPRIMARY KEY key\nSOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'table_for_dict\' PASSWORD \'[HIDDEN]\' DB \'test_01190\'))\nLIFETIME(MIN 1 MAX 10)\nLAYOUT(FLAT()) CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log CREATE TABLE test_01190.log\n(\n `s` String\n)\nENGINE = Log test diff --git a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference index d80501b3f4d..e5b8ffd7732 100644 --- a/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference +++ b/tests/queries/0_stateless/01224_no_superfluous_dict_reload.reference @@ -6,7 +6,7 @@ CREATE DICTIONARY dict_db_01224.dict `val` UInt64 DEFAULT 10 ) PRIMARY KEY key -SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01224')) +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '[HIDDEN]' DB 'dict_db_01224')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()) NOT_LOADED @@ -17,7 +17,7 @@ CREATE TABLE dict_db_01224_dictionary.`dict_db_01224.dict` ) ENGINE = Dictionary(`dict_db_01224.dict`) NOT_LOADED -Dictionary 1 CREATE DICTIONARY dict_db_01224.dict (`key` UInt64 DEFAULT 0, `val` UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'dict_data\' PASSWORD \'\' DB \'dict_db_01224\')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()) +Dictionary 1 CREATE DICTIONARY dict_db_01224.dict (`key` UInt64 DEFAULT 0, `val` UInt64 DEFAULT 10) PRIMARY KEY key SOURCE(CLICKHOUSE(HOST \'localhost\' PORT 9000 USER \'default\' TABLE \'dict_data\' PASSWORD \'[HIDDEN]\' DB \'dict_db_01224\')) LIFETIME(MIN 0 MAX 0) LAYOUT(FLAT()) NOT_LOADED key UInt64 val UInt64 From 11c28abbed324a0f9b27fdd82e939402c93a4aa8 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 16 Nov 2022 00:46:57 +0000 Subject: [PATCH 491/526] fix race in IColumn::dumpStructure --- src/Columns/ColumnAggregateFunction.cpp | 6 ++--- src/Columns/ColumnArray.h | 8 +++--- src/Columns/ColumnConst.h | 6 ++--- src/Columns/ColumnLowCardinality.h | 9 ++++--- src/Columns/ColumnMap.cpp | 6 ++--- src/Columns/ColumnMap.h | 4 +-- src/Columns/ColumnNullable.h | 8 +++--- src/Columns/ColumnObject.cpp | 8 +++--- src/Columns/ColumnObject.h | 4 +-- src/Columns/ColumnSparse.cpp | 8 +++--- src/Columns/ColumnSparse.h | 4 +-- src/Columns/ColumnTuple.cpp | 6 ++--- src/Columns/ColumnTuple.h | 4 +-- src/Columns/ColumnUnique.h | 18 ++++++++++--- src/Columns/IColumn.cpp | 22 ++++++++++++--- src/Columns/IColumn.h | 17 +++++++++--- .../tests/gtest_column_dump_structure.cpp | 27 +++++++++++++++++++ src/Interpreters/AggregationUtils.cpp | 7 ++--- .../02482_insert_into_dist_race.reference | 0 .../02482_insert_into_dist_race.sql | 26 ++++++++++++++++++ 20 files changed, 145 insertions(+), 53 deletions(-) create mode 100644 src/Columns/tests/gtest_column_dump_structure.cpp create mode 100644 tests/queries/0_stateless/02482_insert_into_dist_race.reference create mode 100644 tests/queries/0_stateless/02482_insert_into_dist_race.sql diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp index b51b21307f3..f51a0426199 100644 --- a/src/Columns/ColumnAggregateFunction.cpp +++ b/src/Columns/ColumnAggregateFunction.cpp @@ -152,16 +152,16 @@ MutableColumnPtr ColumnAggregateFunction::convertToValues(MutableColumnPtr colum /// If there are references to states in final column, we must hold their ownership /// by holding arenas and source. - auto callback = [&](auto & subcolumn) + auto callback = [&](IColumn & subcolumn) { - if (auto * aggregate_subcolumn = typeid_cast(subcolumn.get())) + if (auto * aggregate_subcolumn = typeid_cast(&subcolumn)) { aggregate_subcolumn->foreign_arenas = concatArenas(column_aggregate_func.foreign_arenas, column_aggregate_func.my_arena); aggregate_subcolumn->src = column_aggregate_func.getPtr(); } }; - callback(res); + callback(*res); res->forEachSubcolumnRecursively(callback); for (auto * val : data) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 5970802f598..44652fd0c4b 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -151,17 +151,17 @@ public: ColumnPtr compress() const override; - void forEachSubcolumn(ColumnCallback callback) override + void forEachSubcolumn(ColumnCallback callback) const override { callback(offsets); callback(data); } - void forEachSubcolumnRecursively(ColumnCallback callback) override + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override { - callback(offsets); + callback(*offsets); offsets->forEachSubcolumnRecursively(callback); - callback(data); + callback(*data); data->forEachSubcolumnRecursively(callback); } diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index e1f335460fd..0c9245885c4 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -230,14 +230,14 @@ public: data->getExtremes(min, max); } - void forEachSubcolumn(ColumnCallback callback) override + void forEachSubcolumn(ColumnCallback callback) const override { callback(data); } - void forEachSubcolumnRecursively(ColumnCallback callback) override + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override { - callback(data); + callback(*data); data->forEachSubcolumnRecursively(callback); } diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index 20bc45eb569..e7310e70fcb 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -164,7 +164,7 @@ public: size_t byteSizeAt(size_t n) const override { return getDictionary().byteSizeAt(getIndexes().getUInt(n)); } size_t allocatedBytes() const override { return idx.getPositions()->allocatedBytes() + getDictionary().allocatedBytes(); } - void forEachSubcolumn(ColumnCallback callback) override + void forEachSubcolumn(ColumnCallback callback) const override { callback(idx.getPositionsPtr()); @@ -173,15 +173,15 @@ public: callback(dictionary.getColumnUniquePtr()); } - void forEachSubcolumnRecursively(ColumnCallback callback) override + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override { - callback(idx.getPositionsPtr()); + callback(*idx.getPositionsPtr()); idx.getPositionsPtr()->forEachSubcolumnRecursively(callback); /// Column doesn't own dictionary if it's shared. if (!dictionary.isShared()) { - callback(dictionary.getColumnUniquePtr()); + callback(*dictionary.getColumnUniquePtr()); dictionary.getColumnUniquePtr()->forEachSubcolumnRecursively(callback); } } @@ -278,6 +278,7 @@ public: const ColumnPtr & getPositions() const { return positions; } WrappedPtr & getPositionsPtr() { return positions; } + const WrappedPtr & getPositionsPtr() const { return positions; } size_t getPositionAt(size_t row) const; void insertPosition(UInt64 position); void insertPositionsRange(const IColumn & column, UInt64 offset, UInt64 limit); diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index 7377707bdb2..0fe6d4c361b 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -273,14 +273,14 @@ void ColumnMap::getExtremes(Field & min, Field & max) const max = std::move(map_max_value); } -void ColumnMap::forEachSubcolumn(ColumnCallback callback) +void ColumnMap::forEachSubcolumn(ColumnCallback callback) const { callback(nested); } -void ColumnMap::forEachSubcolumnRecursively(ColumnCallback callback) +void ColumnMap::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { - callback(nested); + callback(*nested); nested->forEachSubcolumnRecursively(callback); } diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 1e03633ced7..db918c3db50 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -88,8 +88,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; - void forEachSubcolumn(ColumnCallback callback) override; - void forEachSubcolumnRecursively(ColumnCallback callback) override; + void forEachSubcolumn(ColumnCallback callback) const override; + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override; bool structureEquals(const IColumn & rhs) const override; double getRatioOfDefaultRows(double sample_ratio) const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 2d15442e583..85bf095a9d1 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -130,17 +130,17 @@ public: ColumnPtr compress() const override; - void forEachSubcolumn(ColumnCallback callback) override + void forEachSubcolumn(ColumnCallback callback) const override { callback(nested_column); callback(null_map); } - void forEachSubcolumnRecursively(ColumnCallback callback) override + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override { - callback(nested_column); + callback(*nested_column); nested_column->forEachSubcolumnRecursively(callback); - callback(null_map); + callback(*null_map); null_map->forEachSubcolumnRecursively(callback); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index bf4630137d5..c3e8af5a8cf 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -664,20 +664,20 @@ size_t ColumnObject::allocatedBytes() const return res; } -void ColumnObject::forEachSubcolumn(ColumnCallback callback) +void ColumnObject::forEachSubcolumn(ColumnCallback callback) const { for (auto & entry : subcolumns) for (auto & part : entry->data.data) callback(part); } -void ColumnObject::forEachSubcolumnRecursively(ColumnCallback callback) +void ColumnObject::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { for (auto & entry : subcolumns) { - for (auto & part : entry->data.data) + for (const auto & part : entry->data.data) { - callback(part); + callback(*part); part->forEachSubcolumnRecursively(callback); } } diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 8fcf3d41fba..aa31cf54413 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -206,8 +206,8 @@ public: size_t size() const override; size_t byteSize() const override; size_t allocatedBytes() const override; - void forEachSubcolumn(ColumnCallback callback) override; - void forEachSubcolumnRecursively(ColumnCallback callback) override; + void forEachSubcolumn(ColumnCallback callback) const override; + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override; void insert(const Field & field) override; void insertDefault() override; void insertFrom(const IColumn & src, size_t n) override; diff --git a/src/Columns/ColumnSparse.cpp b/src/Columns/ColumnSparse.cpp index 0c3f8d11adc..0f44d2994fb 100644 --- a/src/Columns/ColumnSparse.cpp +++ b/src/Columns/ColumnSparse.cpp @@ -744,17 +744,17 @@ bool ColumnSparse::structureEquals(const IColumn & rhs) const return false; } -void ColumnSparse::forEachSubcolumn(ColumnCallback callback) +void ColumnSparse::forEachSubcolumn(ColumnCallback callback) const { callback(values); callback(offsets); } -void ColumnSparse::forEachSubcolumnRecursively(ColumnCallback callback) +void ColumnSparse::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { - callback(values); + callback(*values); values->forEachSubcolumnRecursively(callback); - callback(offsets); + callback(*offsets); offsets->forEachSubcolumnRecursively(callback); } diff --git a/src/Columns/ColumnSparse.h b/src/Columns/ColumnSparse.h index 5814cd77637..ffe392a83c1 100644 --- a/src/Columns/ColumnSparse.h +++ b/src/Columns/ColumnSparse.h @@ -139,8 +139,8 @@ public: ColumnPtr compress() const override; - void forEachSubcolumn(ColumnCallback callback) override; - void forEachSubcolumnRecursively(ColumnCallback callback) override; + void forEachSubcolumn(ColumnCallback callback) const override; + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override; bool structureEquals(const IColumn & rhs) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index d8a43bf510d..c8ac2c67c36 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -495,17 +495,17 @@ void ColumnTuple::getExtremes(Field & min, Field & max) const max = max_tuple; } -void ColumnTuple::forEachSubcolumn(ColumnCallback callback) +void ColumnTuple::forEachSubcolumn(ColumnCallback callback) const { for (auto & column : columns) callback(column); } -void ColumnTuple::forEachSubcolumnRecursively(ColumnCallback callback) +void ColumnTuple::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { for (auto & column : columns) { - callback(column); + callback(*column); column->forEachSubcolumnRecursively(callback); } } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 96395d4edfb..25f6328b3fc 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -96,8 +96,8 @@ public: size_t byteSizeAt(size_t n) const override; size_t allocatedBytes() const override; void protect() override; - void forEachSubcolumn(ColumnCallback callback) override; - void forEachSubcolumnRecursively(ColumnCallback callback) override; + void forEachSubcolumn(ColumnCallback callback) const override; + void forEachSubcolumnRecursively(RecursiveColumnCallback callback) const override; bool structureEquals(const IColumn & rhs) const override; bool isCollationSupported() const override; ColumnPtr compress() const override; diff --git a/src/Columns/ColumnUnique.h b/src/Columns/ColumnUnique.h index a1579a46ae0..27faf4bd2ad 100644 --- a/src/Columns/ColumnUnique.h +++ b/src/Columns/ColumnUnique.h @@ -105,7 +105,13 @@ public: return column_holder->allocatedBytes() + reverse_index.allocatedBytes() + (nested_null_mask ? nested_null_mask->allocatedBytes() : 0); } - void forEachSubcolumn(IColumn::ColumnCallback callback) override + + void forEachSubcolumn(IColumn::ColumnCallback callback) const override + { + callback(column_holder); + } + + void forEachSubcolumn(IColumn::MutableColumnCallback callback) override { callback(column_holder); reverse_index.setColumn(getRawColumnPtr()); @@ -113,9 +119,15 @@ public: nested_column_nullable = ColumnNullable::create(column_holder, nested_null_mask); } - void forEachSubcolumnRecursively(IColumn::ColumnCallback callback) override + void forEachSubcolumnRecursively(IColumn::RecursiveColumnCallback callback) const override { - callback(column_holder); + callback(*column_holder); + column_holder->forEachSubcolumnRecursively(callback); + } + + void forEachSubcolumnRecursively(IColumn::RecursiveMutableColumnCallback callback) override + { + callback(*column_holder); column_holder->forEachSubcolumnRecursively(callback); reverse_index.setColumn(getRawColumnPtr()); if (is_nullable) diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 2158adb86a8..46f07cffa9e 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -20,12 +20,10 @@ String IColumn::dumpStructure() const WriteBufferFromOwnString res; res << getFamilyName() << "(size = " << size(); - ColumnCallback callback = [&](ColumnPtr & subcolumn) + forEachSubcolumn([&](const auto & subcolumn) { res << ", " << subcolumn->dumpStructure(); - }; - - const_cast(this)->forEachSubcolumn(callback); + }); res << ")"; return res.str(); @@ -64,6 +62,22 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & defa return res; } +void IColumn::forEachSubcolumn(MutableColumnCallback callback) +{ + std::as_const(*this).forEachSubcolumn([&callback](const WrappedPtr & subcolumn) + { + callback(const_cast(subcolumn)); + }); +} + +void IColumn::forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback) +{ + std::as_const(*this).forEachSubcolumnRecursively([&callback](const IColumn & subcolumn) + { + callback(const_cast(subcolumn)); + }); +} + bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 461e41e3eec..94e87508619 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -411,11 +411,22 @@ public: /// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them. /// Shallow: doesn't do recursive calls; don't do call for itself. - using ColumnCallback = std::function; - virtual void forEachSubcolumn(ColumnCallback) {} + + using ColumnCallback = std::function; + virtual void forEachSubcolumn(ColumnCallback) const {} + + using MutableColumnCallback = std::function; + virtual void forEachSubcolumn(MutableColumnCallback callback); /// Similar to forEachSubcolumn but it also do recursive calls. - virtual void forEachSubcolumnRecursively(ColumnCallback) {} + /// In recursive calls it's prohibited to replace pointers + /// to subcolumns, so we use another callback function. + + using RecursiveColumnCallback = std::function; + virtual void forEachSubcolumnRecursively(RecursiveColumnCallback) const {} + + using RecursiveMutableColumnCallback = std::function; + virtual void forEachSubcolumnRecursively(RecursiveMutableColumnCallback callback); /// Columns have equal structure. /// If true - you can use "compareAt", "insertFrom", etc. methods. diff --git a/src/Columns/tests/gtest_column_dump_structure.cpp b/src/Columns/tests/gtest_column_dump_structure.cpp new file mode 100644 index 00000000000..69769ba0db9 --- /dev/null +++ b/src/Columns/tests/gtest_column_dump_structure.cpp @@ -0,0 +1,27 @@ +#include +#include +#include +#include +#include + +using namespace DB; + +TEST(IColumn, dumpStructure) +{ + auto type_lc = std::make_shared(std::make_shared()); + ColumnPtr column_lc = type_lc->createColumn(); + auto expected_structure = "ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1)))"; + + std::vector threads; + for (size_t i = 0; i < 6; ++i) + { + threads.emplace_back([&] + { + for (size_t j = 0; j < 10000; ++j) + ASSERT_EQ(column_lc->dumpStructure(), expected_structure); + }); + } + + for (auto & t : threads) + t.join(); +} diff --git a/src/Interpreters/AggregationUtils.cpp b/src/Interpreters/AggregationUtils.cpp index ed5e1512a1f..4e870e8152b 100644 --- a/src/Interpreters/AggregationUtils.cpp +++ b/src/Interpreters/AggregationUtils.cpp @@ -50,14 +50,15 @@ OutputBlockColumns prepareOutputBlockColumns( if (aggregate_functions[i]->isState()) { - auto callback = [&](auto & subcolumn) + auto callback = [&](IColumn & subcolumn) { /// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states. - if (auto * column_aggregate_func = typeid_cast(subcolumn.get())) + if (auto * column_aggregate_func = typeid_cast(&subcolumn)) for (auto & pool : aggregates_pools) column_aggregate_func->addArena(pool); }; - callback(final_aggregate_columns[i]); + + callback(*final_aggregate_columns[i]); final_aggregate_columns[i]->forEachSubcolumnRecursively(callback); } } diff --git a/tests/queries/0_stateless/02482_insert_into_dist_race.reference b/tests/queries/0_stateless/02482_insert_into_dist_race.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02482_insert_into_dist_race.sql b/tests/queries/0_stateless/02482_insert_into_dist_race.sql new file mode 100644 index 00000000000..ee412a87d33 --- /dev/null +++ b/tests/queries/0_stateless/02482_insert_into_dist_race.sql @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS tmp_02482; +DROP TABLE IF EXISTS dist_02482; + +-- This test produces warning +SET send_logs_level = 'error'; +SET prefer_localhost_replica=0; + +CREATE TABLE tmp_02482 (i UInt64, n LowCardinality(String)) ENGINE = Memory; +CREATE TABLE dist_02482(i UInt64, n LowCardinality(Nullable(String))) ENGINE = Distributed(test_cluster_two_shards, currentDatabase(), tmp_02482, i); + +SET insert_distributed_sync=1; + +INSERT INTO dist_02482 VALUES (1, '1'), (2, '2'); +INSERT INTO dist_02482 SELECT number, number FROM numbers(1000); + +SET insert_distributed_sync=0; + +SYSTEM STOP DISTRIBUTED SENDS dist_02482; + +INSERT INTO dist_02482 VALUES (1, '1'),(2, '2'); +INSERT INTO dist_02482 SELECT number, number FROM numbers(1000); + +SYSTEM FLUSH DISTRIBUTED dist_02482; + +DROP TABLE tmp_02482; +DROP TABLE dist_02482; From d9adf2f02d50a6c15229150516f90d94aa97a65f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 16 Nov 2022 09:34:04 +0800 Subject: [PATCH 492/526] fix doc --- docs/en/sql-reference/functions/math-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 47c27268b09..bcd118ce0be 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -555,7 +555,7 @@ Result: Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64. -The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater will cause exception throw. **Syntax** From 6393d11dabe76d8e7f73a8fa5c6cc054f670a91b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=9D=8E=E6=89=AC?= <654010905@qq.com> Date: Wed, 16 Nov 2022 09:34:45 +0800 Subject: [PATCH 493/526] fix doc --- src/Functions/factorial.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/factorial.cpp b/src/Functions/factorial.cpp index b76ef90a48d..4e96391bccd 100644 --- a/src/Functions/factorial.cpp +++ b/src/Functions/factorial.cpp @@ -103,7 +103,7 @@ REGISTER_FUNCTION(Factorial) R"( Computes the factorial of an integer value. It works with any native integer type including UInt(8|16|32|64) and Int(8|16|32|64). The return type is UInt64. -The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater overflows the range for Int64 and will cause exception throw. +The factorial of 0 is 1. Likewise, the factorial() function returns 1 for any negative value. The maximum positive value for the input argument is 20, a value of 21 or greater will cause exception throw. )", Documentation::Examples{{"factorial", "SELECT factorial(10)"}}, Documentation::Categories{"Mathematical"}}, From 81971acf3551069aa87fceb757da146af24986ed Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 16 Nov 2022 02:32:44 +0000 Subject: [PATCH 494/526] Fix docs --- docs/en/engines/table-engines/log-family/index.md | 2 +- .../table-engines/mergetree-family/mergetree.md | 4 ++-- .../table-engines/mergetree-family/replication.md | 2 +- docs/en/engines/table-engines/special/join.md | 2 +- docs/en/operations/settings/index.md | 2 +- docs/en/operations/settings/settings.md | 14 +++++++------- docs/en/operations/system-tables/mutations.md | 4 ++-- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/system-tables/parts_columns.md | 2 +- docs/en/sql-reference/statements/alter/column.md | 2 +- docs/en/sql-reference/statements/alter/delete.md | 6 +++--- .../en/sql-reference/statements/alter/partition.md | 4 ++-- .../sql-reference/statements/alter/projection.md | 6 +++--- .../statements/alter/skipping-index.md | 2 +- docs/en/sql-reference/statements/alter/update.md | 6 +++--- docs/ru/operations/settings/index.md | 2 +- docs/ru/operations/settings/settings.md | 2 +- docs/ru/sql-reference/statements/alter/column.md | 2 +- docs/ru/sql-reference/statements/insert-into.md | 2 +- docs/ru/sql-reference/statements/select/index.md | 2 +- docs/zh/sql-reference/statements/alter.md | 2 +- docs/zh/sql-reference/statements/insert-into.md | 2 +- 22 files changed, 37 insertions(+), 37 deletions(-) diff --git a/docs/en/engines/table-engines/log-family/index.md b/docs/en/engines/table-engines/log-family/index.md index 486c41c2496..21f857510f7 100644 --- a/docs/en/engines/table-engines/log-family/index.md +++ b/docs/en/engines/table-engines/log-family/index.md @@ -28,7 +28,7 @@ Engines: During `INSERT` queries, the table is locked, and other queries for reading and writing data both wait for the table to unlock. If there are no data writing queries, any number of data reading queries can be performed concurrently. -- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). +- Do not support [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations). - Do not support indexes. diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7dfb5a9fed7..ce6cec079a3 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -537,7 +537,7 @@ TTL time_column TTL time_column + interval ``` -To define `interval`, use [time interval](/docs/en/sql-reference/operators/index.md/#operators-datetime) operators, for example: +To define `interval`, use [time interval](/docs/en/sql-reference/operators/index.md#operators-datetime) operators, for example: ``` sql TTL date_time + INTERVAL 1 MONTH @@ -860,7 +860,7 @@ The number of threads performing background moves of data parts can be changed b In the case of `MergeTree` tables, data is getting to disk in different ways: - As a result of an insert (`INSERT` query). -- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md/#alter-mutations). +- During background merges and [mutations](/docs/en/sql-reference/statements/alter/index.md#alter-mutations). - When downloading from another replica. - As a result of partition freezing [ALTER TABLE … FREEZE PARTITION](/docs/en/sql-reference/statements/alter/partition.md/#alter_freeze-partition). diff --git a/docs/en/engines/table-engines/mergetree-family/replication.md b/docs/en/engines/table-engines/mergetree-family/replication.md index 67b595d0fa0..ead1a76992e 100644 --- a/docs/en/engines/table-engines/mergetree-family/replication.md +++ b/docs/en/engines/table-engines/mergetree-family/replication.md @@ -20,7 +20,7 @@ Replication works at the level of an individual table, not the entire server. A Replication does not depend on sharding. Each shard has its own independent replication. -Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/index.md/#query_language_queries_alter)). +Compressed data for `INSERT` and `ALTER` queries is replicated (for more information, see the documentation for [ALTER](/docs/en/sql-reference/statements/alter/index.md#query_language_queries_alter)). `CREATE`, `DROP`, `ATTACH`, `DETACH` and `RENAME` queries are executed on a single server and are not replicated: diff --git a/docs/en/engines/table-engines/special/join.md b/docs/en/engines/table-engines/special/join.md index 0e51a8b7696..a49214bd00a 100644 --- a/docs/en/engines/table-engines/special/join.md +++ b/docs/en/engines/table-engines/special/join.md @@ -59,7 +59,7 @@ Main use-cases for `Join`-engine tables are following: ### Deleting Data {#deleting-data} -`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. +`ALTER DELETE` queries for `Join`-engine tables are implemented as [mutations](/docs/en/sql-reference/statements/alter/index.md#mutations). `DELETE` mutation reads filtered data and overwrites data of memory and disk. ### Limitations and Settings {#join-limitations-and-settings} diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md index 8603257ea55..eee4058c230 100644 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -26,7 +26,7 @@ Ways to configure settings, in order of priority: - When starting the ClickHouse console client in non-interactive mode, set the startup parameter `--setting=value`. - When using the HTTP API, pass CGI parameters (`URL?setting_1=value&setting_2=value...`). - - Make settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) clause of the SELECT query. The setting value is applied only to that query and is reset to default or previous value after the query is executed. + - Make settings in the [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) clause of the SELECT query. The setting value is applied only to that query and is reset to default or previous value after the query is executed. Settings that can only be made in the server config file are not covered in this section. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 7abe4affbd1..a15a6e9bf4a 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -276,7 +276,7 @@ Default value: 0. Enables or disables the insertion of [default values](../../sql-reference/statements/create/table.md/#create-default-values) instead of [NULL](../../sql-reference/syntax.md/#null-literal) into columns with not [nullable](../../sql-reference/data-types/nullable.md/#data_type-nullable) data type. If column type is not nullable and this setting is disabled, then inserting `NULL` causes an exception. If column type is nullable, then `NULL` values are inserted as is, regardless of this setting. -This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md/#insert_query_insert-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. +This setting is applicable to [INSERT ... SELECT](../../sql-reference/statements/insert-into.md/#inserting-the-results-of-select) queries. Note that `SELECT` subqueries may be concatenated with `UNION ALL` clause. Possible values: @@ -1619,8 +1619,8 @@ These functions can be transformed: - [length](../../sql-reference/functions/array-functions.md/#array_functions-length) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. - [empty](../../sql-reference/functions/array-functions.md/#function-empty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. - [notEmpty](../../sql-reference/functions/array-functions.md/#function-notempty) to read the [size0](../../sql-reference/data-types/array.md/#array-size) subcolumn. -- [isNull](../../sql-reference/operators/index.md/#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. -- [isNotNull](../../sql-reference/operators/index.md/#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [isNull](../../sql-reference/operators/index.md#operator-is-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. +- [isNotNull](../../sql-reference/operators/index.md#is-not-null) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. - [count](../../sql-reference/aggregate-functions/reference/count.md) to read the [null](../../sql-reference/data-types/nullable.md/#finding-null) subcolumn. - [mapKeys](../../sql-reference/functions/tuple-map-functions.md/#mapkeys) to read the [keys](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. - [mapValues](../../sql-reference/functions/tuple-map-functions.md/#mapvalues) to read the [values](../../sql-reference/data-types/map.md/#map-subcolumns) subcolumn. @@ -2041,7 +2041,7 @@ Default value: 16. ## validate_polygons {#validate_polygons} -Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md/#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. +Enables or disables throwing an exception in the [pointInPolygon](../../sql-reference/functions/geo/index.md#pointinpolygon) function, if the polygon is self-intersecting or self-tangent. Possible values: @@ -2227,7 +2227,7 @@ Default value: `0`. ## mutations_sync {#mutations_sync} -Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md/#mutations)) synchronously. +Allows to execute `ALTER TABLE ... UPDATE|DELETE` queries ([mutations](../../sql-reference/statements/alter/index.md#mutations)) synchronously. Possible values: @@ -2239,8 +2239,8 @@ Default value: `0`. **See Also** -- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) -- [Mutations](../../sql-reference/statements/alter/index.md/#mutations) +- [Synchronicity of ALTER Queries](../../sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) +- [Mutations](../../sql-reference/statements/alter/index.md#mutations) ## ttl_only_drop_parts {#ttl_only_drop_parts} diff --git a/docs/en/operations/system-tables/mutations.md b/docs/en/operations/system-tables/mutations.md index 0d3b764846b..d8fb91a63f5 100644 --- a/docs/en/operations/system-tables/mutations.md +++ b/docs/en/operations/system-tables/mutations.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/mutations --- # mutations -The table contains information about [mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. +The table contains information about [mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) of [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) tables and their progress. Each mutation command is represented by a single row. Columns: @@ -45,7 +45,7 @@ If there were problems with mutating some data parts, the following columns cont **See Also** -- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) - [MergeTree](/docs/en/engines/table-engines/mergetree-family/mergetree.md) table engine - [ReplicatedMergeTree](/docs/en/engines/table-engines/mergetree-family/replication.md) family diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index cbabd9b27b1..bbd5385f44b 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) – The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. Formats: diff --git a/docs/en/operations/system-tables/parts_columns.md b/docs/en/operations/system-tables/parts_columns.md index d934e01f245..68757ddfbff 100644 --- a/docs/en/operations/system-tables/parts_columns.md +++ b/docs/en/operations/system-tables/parts_columns.md @@ -9,7 +9,7 @@ Each row describes one data part. Columns: -- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md/#query_language_queries_alter) query. +- `partition` ([String](../../sql-reference/data-types/string.md)) — The partition name. To learn what a partition is, see the description of the [ALTER](../../sql-reference/statements/alter/index.md#query_language_queries_alter) query. Formats: diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 6bca0dbff42..ae8671ffa9d 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -254,7 +254,7 @@ The `ALTER` query lets you create and delete separate elements (columns) in nest There is no support for deleting columns in the primary key or the sampling key (columns that are used in the `ENGINE` expression). Changing the type for columns that are included in the primary key is only possible if this change does not cause the data to be modified (for example, you are allowed to add values to an Enum or to change a type from `DateTime` to `UInt32`). -If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](/docs/en/sql-reference/statements/insert-into.md/#insert_query_insert-select) query, then switch the tables using the [RENAME](/docs/en/sql-reference/statements/rename.md/#rename-table) query and delete the old table. You can use the [clickhouse-copier](/docs/en/operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. +If the `ALTER` query is not sufficient to make the table changes you need, you can create a new table, copy the data to it using the [INSERT SELECT](/docs/en/sql-reference/statements/insert-into.md/#inserting-the-results-of-select) query, then switch the tables using the [RENAME](/docs/en/sql-reference/statements/rename.md/#rename-table) query and delete the old table. You can use the [clickhouse-copier](/docs/en/operations/utilities/clickhouse-copier.md) as an alternative to the `INSERT SELECT` query. The `ALTER` query blocks all reads and writes for the table. In other words, if a long `SELECT` is running at the time of the `ALTER` query, the `ALTER` query will wait for it to complete. At the same time, all new queries to the same table will wait while this `ALTER` is running. diff --git a/docs/en/sql-reference/statements/alter/delete.md b/docs/en/sql-reference/statements/alter/delete.md index 4dcab030d13..30ed96c0b9c 100644 --- a/docs/en/sql-reference/statements/alter/delete.md +++ b/docs/en/sql-reference/statements/alter/delete.md @@ -10,7 +10,7 @@ sidebar_label: DELETE ALTER TABLE [db.]table [ON CLUSTER cluster] DELETE WHERE filter_expr ``` -Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +Deletes data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). :::note @@ -25,6 +25,6 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do **See also** -- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) -- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index 146c15e776e..a8cea63380c 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -270,7 +270,7 @@ ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' ## UPDATE IN PARTITION -Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +Manipulates data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). Syntax: @@ -290,7 +290,7 @@ ALTER TABLE mt UPDATE x = x + 1 IN PARTITION 2 WHERE p = 2; ## DELETE IN PARTITION -Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +Deletes data in the specifies partition matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). Syntax: diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index 99cb8fb8fd1..3f6f493aa89 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -138,15 +138,15 @@ The following operations with [projections](/docs/en/engines/table-engines/merge ## DROP PROJECTION -`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). ## MATERIALIZE PROJECTION -`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). ## CLEAR PROJECTION -`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. diff --git a/docs/en/sql-reference/statements/alter/skipping-index.md b/docs/en/sql-reference/statements/alter/skipping-index.md index 2dadffc4527..037e4bc38c5 100644 --- a/docs/en/sql-reference/statements/alter/skipping-index.md +++ b/docs/en/sql-reference/statements/alter/skipping-index.md @@ -14,7 +14,7 @@ The following operations are available: - `ALTER TABLE [db].table_name [ON CLUSTER cluster] DROP INDEX name` - Removes index description from tables metadata and deletes index files from disk. -- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. +- `ALTER TABLE [db.]table_name [ON CLUSTER cluster] MATERIALIZE INDEX name [IN PARTITION partition_name]` - Rebuilds the secondary index `name` for the specified `partition_name`. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). If `IN PARTITION` part is omitted then it rebuilds the index for the whole table data. The first two commands are lightweight in a sense that they only change metadata or remove files. diff --git a/docs/en/sql-reference/statements/alter/update.md b/docs/en/sql-reference/statements/alter/update.md index f40b72f7ab3..5d27c382982 100644 --- a/docs/en/sql-reference/statements/alter/update.md +++ b/docs/en/sql-reference/statements/alter/update.md @@ -10,7 +10,7 @@ sidebar_label: UPDATE ALTER TABLE [db.]table [ON CLUSTER cluster] UPDATE column1 = expr1 [, ...] WHERE filter_expr ``` -Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md/#mutations). +Manipulates data matching the specified filtering expression. Implemented as a [mutation](/docs/en/sql-reference/statements/alter/index.md#mutations). :::note The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use. @@ -24,7 +24,7 @@ The synchronicity of the query processing is defined by the [mutations_sync](/do **See also** -- [Mutations](/docs/en/sql-reference/statements/alter/index.md/#mutations) -- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md/#synchronicity-of-alter-queries) +- [Mutations](/docs/en/sql-reference/statements/alter/index.md#mutations) +- [Synchronicity of ALTER Queries](/docs/en/sql-reference/statements/alter/index.md#synchronicity-of-alter-queries) - [mutations_sync](/docs/en/operations/settings/settings.md/#mutations_sync) setting diff --git a/docs/ru/operations/settings/index.md b/docs/ru/operations/settings/index.md index 4e055405847..6806aea5135 100644 --- a/docs/ru/operations/settings/index.md +++ b/docs/ru/operations/settings/index.md @@ -24,7 +24,7 @@ slug: /ru/operations/settings/ - При запуске консольного клиента ClickHouse в не интерактивном режиме установите параметр запуска `--setting=value`. - При использовании HTTP API передавайте cgi-параметры (`URL?setting_1=value&setting_2=value...`). - - Укажите необходимые настройки в секции [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select) запроса SELECT. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. + - Укажите необходимые настройки в секции [SETTINGS](../../sql-reference/statements/select/index.md#settings-in-select-query) запроса SELECT. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. Настройки, которые можно задать только в конфигурационном файле сервера, в разделе не рассматриваются. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a070dbd5e10..58894611386 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -479,7 +479,7 @@ SELECT * FROM table_with_enum_column_for_tsv_insert; Включает или отключает вставку [значений по умолчанию](../../sql-reference/statements/create/table.md#create-default-values) вместо [NULL](../../sql-reference/syntax.md#null-literal) в столбцы, которые не позволяют [хранить NULL](../../sql-reference/data-types/nullable.md#data_type-nullable). Если столбец не позволяет хранить `NULL` и эта настройка отключена, то вставка `NULL` приведет к возникновению исключения. Если столбец позволяет хранить `NULL`, то значения `NULL` вставляются независимо от этой настройки. -Эта настройка используется для запросов [INSERT ... SELECT](../../sql-reference/statements/insert-into.md#insert_query_insert-select). При этом подзапросы `SELECT` могут объединяться с помощью `UNION ALL`. +Эта настройка используется для запросов [INSERT ... SELECT](../../sql-reference/statements/insert-into.md#inserting-the-results-of-select). При этом подзапросы `SELECT` могут объединяться с помощью `UNION ALL`. Возможные значения: diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md index 11ec72596c4..a8ace213075 100644 --- a/docs/ru/sql-reference/statements/alter/column.md +++ b/docs/ru/sql-reference/statements/alter/column.md @@ -254,7 +254,7 @@ SELECT groupArray(x), groupArray(s) FROM tmp; Отсутствует возможность удалять столбцы, входящие в первичный ключ или ключ для сэмплирования (в общем, входящие в выражение `ENGINE`). Изменение типа у столбцов, входящих в первичный ключ возможно только в том случае, если это изменение не приводит к изменению данных (например, разрешено добавление значения в Enum или изменение типа с `DateTime` на `UInt32`). -Если возможностей запроса `ALTER` не хватает для нужного изменения таблицы, вы можете создать новую таблицу, скопировать туда данные с помощью запроса [INSERT SELECT](../insert-into.md#insert_query_insert-select), затем поменять таблицы местами с помощью запроса [RENAME](../rename.md#rename-table), и удалить старую таблицу. В качестве альтернативы для запроса `INSERT SELECT`, можно использовать инструмент [clickhouse-copier](../../../sql-reference/statements/alter/index.md). +Если возможностей запроса `ALTER` не хватает для нужного изменения таблицы, вы можете создать новую таблицу, скопировать туда данные с помощью запроса [INSERT SELECT](../insert-into.md#inserting-the-results-of-select), затем поменять таблицы местами с помощью запроса [RENAME](../rename.md#rename-table), и удалить старую таблицу. В качестве альтернативы для запроса `INSERT SELECT`, можно использовать инструмент [clickhouse-copier](../../../sql-reference/statements/alter/index.md). Запрос `ALTER` блокирует все чтения и записи для таблицы. То есть если на момент запроса `ALTER` выполнялся долгий `SELECT`, то запрос `ALTER` сначала дождётся его выполнения. И в это время все новые запросы к той же таблице будут ждать, пока завершится этот `ALTER`. diff --git a/docs/ru/sql-reference/statements/insert-into.md b/docs/ru/sql-reference/statements/insert-into.md index 573b8d39926..4fa6ac4ce66 100644 --- a/docs/ru/sql-reference/statements/insert-into.md +++ b/docs/ru/sql-reference/statements/insert-into.md @@ -95,7 +95,7 @@ INSERT INTO t FORMAT TabSeparated Если в таблице объявлены [ограничения](../../sql-reference/statements/create/table.md#constraints), то их выполнимость будет проверена для каждой вставляемой строки. Если для хотя бы одной строки ограничения не будут выполнены, запрос будет остановлен. -### Вставка результатов `SELECT` {#insert_query_insert-select} +### Вставка результатов `SELECT` {#inserting-the-results-of-select} **Синтаксис** diff --git a/docs/ru/sql-reference/statements/select/index.md b/docs/ru/sql-reference/statements/select/index.md index 4479e24000b..f360a09eb10 100644 --- a/docs/ru/sql-reference/statements/select/index.md +++ b/docs/ru/sql-reference/statements/select/index.md @@ -270,7 +270,7 @@ SELECT * REPLACE(i + 1 AS i) EXCEPT (j) APPLY(sum) from columns_transformers; └─────────────────┴────────┘ ``` -## SETTINGS в запросе SELECT {#settings-in-select} +## SETTINGS в запросе SELECT {#settings-in-select-query} Вы можете задать значения необходимых настроек непосредственно в запросе `SELECT` в секции `SETTINGS`. Эти настройки действуют только в рамках данного запроса, а после его выполнения сбрасываются до предыдущего значения или значения по умолчанию. diff --git a/docs/zh/sql-reference/statements/alter.md b/docs/zh/sql-reference/statements/alter.md index 23edfd633db..fd73be4fd93 100644 --- a/docs/zh/sql-reference/statements/alter.md +++ b/docs/zh/sql-reference/statements/alter.md @@ -150,7 +150,7 @@ ALTER TABLE visits MODIFY COLUMN browser Array(String) 不支持对primary key或者sampling key中的列(在 `ENGINE` 表达式中用到的列)进行删除操作。改变包含在primary key中的列的类型时,如果操作不会导致数据的变化(例如,往Enum中添加一个值,或者将`DateTime` 类型改成 `UInt32`),那么这种操作是可行的。 -如果 `ALTER` 操作不足以完成你想要的表变动操作,你可以创建一张新的表,通过 [INSERT SELECT](../../sql-reference/statements/insert-into.md#insert_query_insert-select)将数据拷贝进去,然后通过 [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename)将新的表改成和原有表一样的名称,并删除原有的表。你可以使用 [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) 代替 `INSERT SELECT`。 +如果 `ALTER` 操作不足以完成你想要的表变动操作,你可以创建一张新的表,通过 [INSERT SELECT](../../sql-reference/statements/insert-into.md#inserting-the-results-of-select)将数据拷贝进去,然后通过 [RENAME](../../sql-reference/statements/misc.md#misc_operations-rename)将新的表改成和原有表一样的名称,并删除原有的表。你可以使用 [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) 代替 `INSERT SELECT`。 `ALTER` 操作会阻塞对表的所有读写操作。换句话说,当一个大的 `SELECT` 语句和 `ALTER`同时执行时,`ALTER`会等待,直到 `SELECT` 执行结束。与此同时,当 `ALTER` 运行时,新的 sql 语句将会等待。 diff --git a/docs/zh/sql-reference/statements/insert-into.md b/docs/zh/sql-reference/statements/insert-into.md index 69762bf43bc..f199329829c 100644 --- a/docs/zh/sql-reference/statements/insert-into.md +++ b/docs/zh/sql-reference/statements/insert-into.md @@ -90,7 +90,7 @@ INSERT INTO t FORMAT TabSeparated 如果表中有一些[限制](../../sql-reference/statements/create/table.mdx#constraints),,数据插入时会逐行进行数据校验,如果这里面包含了不符合限制条件的数据,服务将会抛出包含限制信息的异常,这个语句也会被停止执行。 -### 使用`SELECT`的结果写入 {#insert_query_insert-select} +### 使用`SELECT`的结果写入 {#inserting-the-results-of-select} ``` sql INSERT INTO [db.]table [(c1, c2, c3)] SELECT ... From ce8066e90e5c708899f05b2f0544dde4bd324718 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 16 Nov 2022 07:31:24 +0100 Subject: [PATCH 495/526] Add changelog for 21.11 --- CHANGELOG.md | 103 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 103 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 68767612892..ac1f0793c04 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v22.11, 2022-11-17](#2211)**
**[ClickHouse release v22.10, 2022-10-25](#2210)**
**[ClickHouse release v22.9, 2022-09-22](#229)**
**[ClickHouse release v22.8-lts, 2022-08-18](#228)**
@@ -11,6 +12,108 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
+###
ClickHouse release 22.11, 2022-11-17 + +#### Backward Incompatible Change +* `JSONExtract` family of functions will now attempt to coerce to the requested type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). + +#### New Feature +* Support for retries during INSERTs into ReplicatedMergeTree if a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). +* Add `Hudi` and `DeltaLake` table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add table function `hudi` and `deltaLake`. [#43080](https://github.com/ClickHouse/ClickHouse/pull/43080) ([flynn](https://github.com/ucasfl)). +* Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In case when the types of Intervals are different they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added `**` glob support for recursive directory traversal to filesystem and S3. resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Introduce `s3_plain` disk type for write-once-read-many operations. Implement `ATTACH` of `MergeTree` table for `s3_plain` disk. [#42628](https://github.com/ClickHouse/ClickHouse/pull/42628) ([Azat Khuzhin](https://github.com/azat)). +* Added applied row-level policies to `system.query_log`. [#39819](https://github.com/ClickHouse/ClickHouse/pull/39819) ([Vladimir Chebotaryov](https://github.com/quickhouse)). +* Add four-letter command `csnp` for manually creating snapshots in ClickHouse Keeper. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). +* Add function `ascii` like in spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). +* Add function `pmod` which return non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). +* Add function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). +* Add function `randCanonical`, which is similar to the `rand` function in spark or impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). +* Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). +* Add `min_age_to_force_merge_on_partition_only` setting to optimize old parts for the entire partition only. [#42659](https://github.com/ClickHouse/ClickHouse/pull/42659) ([Antonio Andelic](https://github.com/antonio2368)). +* Add generic implementation for arbitrary structured named collections, access type and `system.named_collections`. [#43147](https://github.com/ClickHouse/ClickHouse/pull/43147) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Performance Improvement +* `match` function can use the index if it's a condition on string prefix. This closes [#37333](https://github.com/ClickHouse/ClickHouse/issues/37333). [#42458](https://github.com/ClickHouse/ClickHouse/pull/42458) ([clarkcaoliu](https://github.com/Clark0)). +* Speed up AND and OR operators when they are sequenced. [#42214](https://github.com/ClickHouse/ClickHouse/pull/42214) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Support parallel parsing for `LineAsString` input format. This improves performance just slightly. This closes [#42502](https://github.com/ClickHouse/ClickHouse/issues/42502). [#42780](https://github.com/ClickHouse/ClickHouse/pull/42780) ([Kruglov Pavel](https://github.com/Avogar)). +* ClickHouse Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)). +* A condition like `NOT LIKE 'prefix%'` can use primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). + +#### Experimental Feature +* Support type `Object` inside other types, e.g. `Array(JSON)`. [#36969](https://github.com/ClickHouse/ClickHouse/pull/36969) ([Anton Popov](https://github.com/CurtizJ)). +* Ignore MySQL binlog SAVEPOINT event for MaterializedMySQL. [#42931](https://github.com/ClickHouse/ClickHouse/pull/42931) ([zzsmdfj](https://github.com/zzsmdfj)). Handle (ignore) SAVEPOINT queries in MaterializedMySQL. [#43086](https://github.com/ClickHouse/ClickHouse/pull/43086) ([Stig Bakken](https://github.com/stigsb)). + +#### Improvement +* Trivial queries with small LIMIT will properly determine the number of estimated rows to read, so the threshold will be checked properly. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). +* Add support for interactive parameters in INSERT VALUES queries. [#43077](https://github.com/ClickHouse/ClickHouse/pull/43077) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added new field `allow_readonly` in `system.table_functions` to allow using table functions in readonly mode resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* The `system.asynchronous_metrics` gets embedded documentation. This documentation is also exported to Prometheus. Fixed an error with the metrics about `cache` disks - they were calculated only for one arbitrary cache disk instead all of them. This closes [#7644](https://github.com/ClickHouse/ClickHouse/issues/7644). [#43194](https://github.com/ClickHouse/ClickHouse/pull/43194) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Throttling algorithm changed to token bucket. [#42665](https://github.com/ClickHouse/ClickHouse/pull/42665) ([Sergei Trifonov](https://github.com/serxa)). +* Mask passwords and secret keys both in `system.query_log` and `/var/log/clickhouse-server/*.log` and also in error messages. [#42484](https://github.com/ClickHouse/ClickHouse/pull/42484) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove covered parts for fetched part (to avoid possible replication delay grows). [#39737](https://github.com/ClickHouse/ClickHouse/pull/39737) ([Azat Khuzhin](https://github.com/azat)). +* If `/dev/tty` is available, the progress in clickhouse-client and clickhouse-local will be rendered directly to the terminal, without writing to stderr. It allows to get progress even if stderr is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add support for `FixedString` input to base64 coding functions. [#42285](https://github.com/ClickHouse/ClickHouse/pull/42285) ([ltrk2](https://github.com/ltrk2)). +* Add columns `bytes_on_disk` and `path` to `system.detached_parts`. Closes [#42264](https://github.com/ClickHouse/ClickHouse/issues/42264). [#42303](https://github.com/ClickHouse/ClickHouse/pull/42303) ([chen](https://github.com/xiedeyantu)). +* Improve using structure from insertion table in table functions, now setting `use_structure_from_insertion_table_in_table_functions` has new possible value - `2` that means that ClickHouse will try to determine if we can use structure from insertion table or not automatically. Closes [#40028](https://github.com/ClickHouse/ClickHouse/issues/40028). [#42320](https://github.com/ClickHouse/ClickHouse/pull/42320) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix no progress indication on INSERT FROM INFILE. Closes [#42548](https://github.com/ClickHouse/ClickHouse/issues/42548). [#42634](https://github.com/ClickHouse/ClickHouse/pull/42634) ([chen](https://github.com/xiedeyantu)). +* Refactor function `tokens` to enable max tokens returned for related functions (disabled by default). [#42673](https://github.com/ClickHouse/ClickHouse/pull/42673) ([李扬](https://github.com/taiyang-li)). +* Allow to use `Date32` arguments for `formatDateTime` and `FROM_UNIXTIME` functions. [#42737](https://github.com/ClickHouse/ClickHouse/pull/42737) ([Roman Vasin](https://github.com/rvasin)). +* Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `FailedAsyncInsertQuery` event metric for async inserts. [#42814](https://github.com/ClickHouse/ClickHouse/pull/42814) ([Krzysztof Góralski](https://github.com/kgoralski)). +* Implement `read-in-order` optimization on top of query plan. It is enabled by default. Set `query_plan_read_in_order = 0` to use previous AST-based version. [#42829](https://github.com/ClickHouse/ClickHouse/pull/42829) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Increase the size of upload part exponentially for backup to S3 to avoid errors about max 10 000 parts limit of the multipart upload to s3. [#42833](https://github.com/ClickHouse/ClickHouse/pull/42833) ([Vitaly Baranov](https://github.com/vitlibar)). +* When the merge task is continuously busy and the disk space is insufficient, the completely expired parts cannot be selected and dropped, resulting in insufficient disk space. My idea is that when the entire Part expires, there is no need for additional disk space to guarantee, ensure the normal execution of TTL. [#42869](https://github.com/ClickHouse/ClickHouse/pull/42869) ([zhongyuankai](https://github.com/zhongyuankai)). +* Add `oss` function and `OSS` table engine (this is convenient for users). oss is fully compatible with s3. [#43155](https://github.com/ClickHouse/ClickHouse/pull/43155) ([zzsmdfj](https://github.com/zzsmdfj)). +* Improve error reporting in the collection of OS-related info for the `system.asynchronous_metrics` table. [#43192](https://github.com/ClickHouse/ClickHouse/pull/43192) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Modify the `INFORMATION_SCHEMA` tables in a way so that now ClickHouse can connect to itself using the MySQL compatibility protocol. Add columns instead of aliases (related to [#9769](https://github.com/ClickHouse/ClickHouse/issues/9769)). It will improve the compatibility with various MySQL clients. [#43198](https://github.com/ClickHouse/ClickHouse/pull/43198) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Add some functions for compatibility with PowerBI, when it connects using MySQL protocol [#42612](https://github.com/ClickHouse/ClickHouse/pull/42612) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Better usability for Dashboard on changes [#42872](https://github.com/ClickHouse/ClickHouse/pull/42872) ([Vladimir C](https://github.com/vdimir)). + +#### Build/Testing/Packaging Improvement +* Run SQLancer for each pull request and commit to master. [SQLancer](https://github.com/sqlancer/sqlancer) is an OpenSource fuzzer that focuses on automatic detection of logical bugs. [#42397](https://github.com/ClickHouse/ClickHouse/pull/42397) ([Ilya Yatsishin](https://github.com/qoega)). +* Update to latest zlib-ng. [#42463](https://github.com/ClickHouse/ClickHouse/pull/42463) ([Boris Kuschel](https://github.com/bkuschel)). +* Add support for testing ClickHouse server with Jepsen. By the way, we already have support for testing ClickHouse Keeper with Jepsen. This pull request extends it to Replicated tables. [#42619](https://github.com/ClickHouse/ClickHouse/pull/42619) ([Antonio Andelic](https://github.com/antonio2368)). +* Use https://github.com/matus-chochlik/ctcache for clang-tidy results caching. [#42913](https://github.com/ClickHouse/ClickHouse/pull/42913) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove some libraries from Ubuntu Docker image. [#42622](https://github.com/ClickHouse/ClickHouse/pull/42622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Updated normaliser to clone the alias ast. resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix race for backup of tables in `Lazy` databases. [#43104](https://github.com/ClickHouse/ClickHouse/pull/43104) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix for `skip_unavailable_shards`: it did not work with the `s3Cluster` table function. [#43131](https://github.com/ClickHouse/ClickHouse/pull/43131) ([chen](https://github.com/xiedeyantu)). +* Fix schema inference in `s3Cluster` and improvement in `hdfsCluster`. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix retries while reading from URL table engines / table function. (retrtiable errors could be retries more times than needed, non-retrialble errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). +* A segmentation fault related to DNS & c-ares has been reported and fixed. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix `LOGICAL_ERROR` `Arguments of 'plus' have incorrect data types` which may happen in PK analysis (monotonicity check). Fix invalid PK analysis for monotonic binary functions with first constant argument. [#42410](https://github.com/ClickHouse/ClickHouse/pull/42410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix incorrect key analysis when key types cannot be inside Nullable. This fixes [#42456](https://github.com/ClickHouse/ClickHouse/issues/42456). [#42469](https://github.com/ClickHouse/ClickHouse/pull/42469) ([Amos Bird](https://github.com/amosbird)). +* Fix typo in setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix creating a Set with wrong header when data type is LowCardinality. Closes [#42460](https://github.com/ClickHouse/ClickHouse/issues/42460). [#42579](https://github.com/ClickHouse/ClickHouse/pull/42579) ([flynn](https://github.com/ucasfl)). +* `(U)Int128` and `(U)Int256` values were correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a bug in functions parser that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix the locking in `truncate table`. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)). +* Fix possible crash in `web` disks when file does not exist (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)). +* Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). +* Fix stack-use-after-return under ASAN build in the Create User query parser. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `lowerUTF8`/`upperUTF8` in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). +* Additional bound check was added to lz4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). +* Fix rare possible hung on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect behavior with multiple disjuncts in hash join, close [#42832](https://github.com/ClickHouse/ClickHouse/issues/42832). [#42876](https://github.com/ClickHouse/ClickHouse/pull/42876) ([Vladimir C](https://github.com/vdimir)). +* A null pointer will be generated when select if as from ‘three table join’ , For example, this SQL query: [#42883](https://github.com/ClickHouse/ClickHouse/pull/42883) ([zzsmdfj](https://github.com/zzsmdfj)). +* Fix memory sanitizer report in Cluster Discovery, close [#42763](https://github.com/ClickHouse/ClickHouse/issues/42763). [#42905](https://github.com/ClickHouse/ClickHouse/pull/42905) ([Vladimir C](https://github.com/vdimir)). +* Improve datetime schema inference in case of empty string. [#42911](https://github.com/ClickHouse/ClickHouse/pull/42911) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)). +* Fix ATTACH TABLE in `PostgreSQL` database engine if the table contains DATETIME data type. Closes [#42817](https://github.com/ClickHouse/ClickHouse/issues/42817). [#42960](https://github.com/ClickHouse/ClickHouse/pull/42960) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix lambda parsing. Closes [#41848](https://github.com/ClickHouse/ClickHouse/issues/41848). [#42979](https://github.com/ClickHouse/ClickHouse/pull/42979) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix incorrect key analysis when nullable keys appear in the middle of a hyperrectangle. This fixes [#43111](https://github.com/ClickHouse/ClickHouse/issues/43111) . [#43133](https://github.com/ClickHouse/ClickHouse/pull/43133) ([Amos Bird](https://github.com/amosbird)). +* Fix several buffer over-reads in deserialization of carefully crafted aggregate function states. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). +* Fix function `if` in case of NULL and const Nullable arguments. Closes [#43069](https://github.com/ClickHouse/ClickHouse/issues/43069). [#43178](https://github.com/ClickHouse/ClickHouse/pull/43178) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix decimal math overflow in parsing datetime with the 'best effort' algorithm. Closes [#43061](https://github.com/ClickHouse/ClickHouse/issues/43061). [#43180](https://github.com/ClickHouse/ClickHouse/pull/43180) ([Kruglov Pavel](https://github.com/Avogar)). +* The `indent` field produced by the `git-import` tool was miscalculated. See https://clickhouse.com/docs/en/getting-started/example-datasets/github/. [#43191](https://github.com/ClickHouse/ClickHouse/pull/43191) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed unexpected behaviour of `Interval` types with subquery and casting. [#43193](https://github.com/ClickHouse/ClickHouse/pull/43193) ([jh0x](https://github.com/jh0x)). + ### ClickHouse release 22.10, 2022-10-26 #### Backward Incompatible Change From 122508d844b44283679ff6753e16f665f23d1726 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 16 Nov 2022 10:22:28 +0000 Subject: [PATCH 496/526] Fix cloning of ASTIdentifier --- src/Parsers/ASTIdentifier.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Parsers/ASTIdentifier.cpp b/src/Parsers/ASTIdentifier.cpp index ca8ac0e8e64..341ac44b56e 100644 --- a/src/Parsers/ASTIdentifier.cpp +++ b/src/Parsers/ASTIdentifier.cpp @@ -63,6 +63,7 @@ ASTPtr ASTIdentifier::clone() const { auto ret = std::make_shared(*this); ret->semantic = std::make_shared(*ret->semantic); + ret->cloneChildren(); return ret; } From 981e6e2c24b96ef3e3516273d04864658fda1d72 Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 16 Nov 2022 12:44:54 +0100 Subject: [PATCH 497/526] Update src/Analyzer/Passes/QueryAnalysisPass.cpp --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 760d036ab4f..2b76376c4c9 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2177,8 +2177,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier } else { - alias_name_to_node_map.erase(it); - std::unordered_set valid_identifiers; collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers); From 2d265523bbaf5dd1aff1f60ddae52640098da5cf Mon Sep 17 00:00:00 2001 From: Vladimir C Date: Wed, 16 Nov 2022 12:45:54 +0100 Subject: [PATCH 498/526] Update src/Analyzer/Passes/QueryAnalysisPass.cpp --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 2b76376c4c9..138ff721f99 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -2171,11 +2171,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier auto & alias_identifier_node = it->second->as(); auto identifier = alias_identifier_node.getIdentifier(); auto lookup_result = tryResolveIdentifier(IdentifierLookup{identifier, identifier_lookup.lookup_context}, scope, identifier_resolve_settings); - if (lookup_result.isResolved()) - { - it->second = lookup_result.resolved_identifier; - } - else + if (!lookup_result.isResolved()) { std::unordered_set valid_identifiers; collectScopeWithParentScopesValidIdentifiersForTypoCorrection(identifier, scope, true, false, false, valid_identifiers); @@ -2187,6 +2183,7 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveIdentifierFromAliases(const Identifier scope.scope_node->formatASTForErrorMessage(), getHintsErrorMessageSuffix(hints)); } + it->second = lookup_result.resolved_identifier; /** During collection of aliases if node is identifier and has alias, we cannot say if it is * column or function node. Check QueryExpressionsAliasVisitor documentation for clarification. From b13a8d478bb9c1d1e9f333c26acb6b33a22c6d9b Mon Sep 17 00:00:00 2001 From: Sema Checherinda Date: Wed, 16 Nov 2022 11:53:59 +0100 Subject: [PATCH 499/526] check limits for an AST in select parser fuzzer --- src/Parsers/fuzzers/select_parser_fuzzer.cpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Parsers/fuzzers/select_parser_fuzzer.cpp b/src/Parsers/fuzzers/select_parser_fuzzer.cpp index caa6c586cd6..3f712834c55 100644 --- a/src/Parsers/fuzzers/select_parser_fuzzer.cpp +++ b/src/Parsers/fuzzers/select_parser_fuzzer.cpp @@ -12,7 +12,15 @@ try std::string input = std::string(reinterpret_cast(data), size); DB::ParserQueryWithOutput parser(input.data() + input.size()); - DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, 1000); + + const UInt64 max_parser_depth = 1000; + DB::ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0, max_parser_depth); + + const UInt64 max_ast_depth = 1000; + ast->checkDepth(max_ast_depth); + + const UInt64 max_ast_elements = 50000; + ast->checkSize(max_ast_elements); DB::WriteBufferFromOwnString wb; DB::formatAST(*ast, wb); From dcb76f6bbf1db37b15da40bab450760e7115bcea Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Wed, 16 Nov 2022 14:16:42 +0100 Subject: [PATCH 500/526] Allow autoremoval of old parts if detach_not_byte_identical_parts enabled Allow autoremoval of old & detached parts if detach_not_byte_identical_parts enabled. See also #28708 #37975 --- src/Storages/MergeTree/MergeTreePartInfo.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index c19cc55e74e..60c7e4e8822 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -154,6 +154,8 @@ struct DetachedPartInfo : public MergeTreePartInfo "deleting", "tmp-fetch", "covered-by-broken", + "merge-not-byte-identical", + "mutate-not-byte-identical" }); static constexpr auto DETACHED_REASONS_REMOVABLE_BY_TIMEOUT = std::to_array({ @@ -163,7 +165,9 @@ struct DetachedPartInfo : public MergeTreePartInfo "ignored", "broken-on-start", "deleting", - "clone" + "clone", + "merge-not-byte-identical", + "mutate-not-byte-identical" }); /// NOTE: It may parse part info incorrectly. From 11aba7597e1287166b1396fefc56d9ac468101e8 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 16 Nov 2022 13:33:22 +0000 Subject: [PATCH 501/526] Check if initialization is done on write --- src/Storages/StorageReplicatedMergeTree.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8b4788c8d55..9eacec2351e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4507,6 +4507,9 @@ void StorageReplicatedMergeTree::assertNotReadonly() const SinkToStoragePtr StorageReplicatedMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) { + if (!initialization_done) + throw Exception(ErrorCodes::NOT_INITIALIZED, "Table is not initialized yet"); + /// If table is read-only because it doesn't have metadata in zk yet, then it's not possible to insert into it /// Without this check, we'll write data parts on disk, and afterwards will remove them since we'll fail to commit them into zk /// In case of remote storage like s3, it'll generate unnecessary PUT requests From ea8c7df296deeac1c5f65416d03c73f2b044089f Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 16 Nov 2022 09:32:13 -0500 Subject: [PATCH 502/526] edits --- CHANGELOG.md | 42 +++++++++++++++++++++--------------------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ac1f0793c04..051bde44dd2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,18 +18,18 @@ * `JSONExtract` family of functions will now attempt to coerce to the requested type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). #### New Feature -* Support for retries during INSERTs into ReplicatedMergeTree if a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). +* Adds support for retries during INSERTs into ReplicatedMergeTree when a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). * Add `Hudi` and `DeltaLake` table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do), [Kseniia Sumarokova](https://github.com/kssenii)). * Add table function `hudi` and `deltaLake`. [#43080](https://github.com/ClickHouse/ClickHouse/pull/43080) ([flynn](https://github.com/ucasfl)). -* Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In case when the types of Intervals are different they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). -* Added `**` glob support for recursive directory traversal to filesystem and S3. resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In the case where the types of Intervals are different, they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added `**` glob support for recursive directory traversal of the filesystem and S3. Resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Introduce `s3_plain` disk type for write-once-read-many operations. Implement `ATTACH` of `MergeTree` table for `s3_plain` disk. [#42628](https://github.com/ClickHouse/ClickHouse/pull/42628) ([Azat Khuzhin](https://github.com/azat)). * Added applied row-level policies to `system.query_log`. [#39819](https://github.com/ClickHouse/ClickHouse/pull/39819) ([Vladimir Chebotaryov](https://github.com/quickhouse)). * Add four-letter command `csnp` for manually creating snapshots in ClickHouse Keeper. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). -* Add function `ascii` like in spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). -* Add function `pmod` which return non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). +* Add function `ascii` like in Apache Spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). +* Add function `pmod` which returns non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). * Add function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). -* Add function `randCanonical`, which is similar to the `rand` function in spark or impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). +* Add function `randCanonical`, which is similar to the `rand` function in Apache Spark or Impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). * Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). * Add `min_age_to_force_merge_on_partition_only` setting to optimize old parts for the entire partition only. [#42659](https://github.com/ClickHouse/ClickHouse/pull/42659) ([Antonio Andelic](https://github.com/antonio2368)). * Add generic implementation for arbitrary structured named collections, access type and `system.named_collections`. [#43147](https://github.com/ClickHouse/ClickHouse/pull/43147) ([Kseniia Sumarokova](https://github.com/kssenii)). @@ -39,21 +39,21 @@ * Speed up AND and OR operators when they are sequenced. [#42214](https://github.com/ClickHouse/ClickHouse/pull/42214) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). * Support parallel parsing for `LineAsString` input format. This improves performance just slightly. This closes [#42502](https://github.com/ClickHouse/ClickHouse/issues/42502). [#42780](https://github.com/ClickHouse/ClickHouse/pull/42780) ([Kruglov Pavel](https://github.com/Avogar)). * ClickHouse Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)). -* A condition like `NOT LIKE 'prefix%'` can use primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). +* A condition like `NOT LIKE 'prefix%'` can use the primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). #### Experimental Feature * Support type `Object` inside other types, e.g. `Array(JSON)`. [#36969](https://github.com/ClickHouse/ClickHouse/pull/36969) ([Anton Popov](https://github.com/CurtizJ)). * Ignore MySQL binlog SAVEPOINT event for MaterializedMySQL. [#42931](https://github.com/ClickHouse/ClickHouse/pull/42931) ([zzsmdfj](https://github.com/zzsmdfj)). Handle (ignore) SAVEPOINT queries in MaterializedMySQL. [#43086](https://github.com/ClickHouse/ClickHouse/pull/43086) ([Stig Bakken](https://github.com/stigsb)). #### Improvement -* Trivial queries with small LIMIT will properly determine the number of estimated rows to read, so the threshold will be checked properly. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). +* Trivial queries with small LIMIT will properly determine the number of estimated rows to read, so that the threshold will be checked properly. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). * Add support for interactive parameters in INSERT VALUES queries. [#43077](https://github.com/ClickHouse/ClickHouse/pull/43077) ([Nikolay Degterinsky](https://github.com/evillique)). -* Added new field `allow_readonly` in `system.table_functions` to allow using table functions in readonly mode resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Added new field `allow_readonly` in `system.table_functions` to allow using table functions in readonly mode. Resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * The `system.asynchronous_metrics` gets embedded documentation. This documentation is also exported to Prometheus. Fixed an error with the metrics about `cache` disks - they were calculated only for one arbitrary cache disk instead all of them. This closes [#7644](https://github.com/ClickHouse/ClickHouse/issues/7644). [#43194](https://github.com/ClickHouse/ClickHouse/pull/43194) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Throttling algorithm changed to token bucket. [#42665](https://github.com/ClickHouse/ClickHouse/pull/42665) ([Sergei Trifonov](https://github.com/serxa)). * Mask passwords and secret keys both in `system.query_log` and `/var/log/clickhouse-server/*.log` and also in error messages. [#42484](https://github.com/ClickHouse/ClickHouse/pull/42484) ([Vitaly Baranov](https://github.com/vitlibar)). * Remove covered parts for fetched part (to avoid possible replication delay grows). [#39737](https://github.com/ClickHouse/ClickHouse/pull/39737) ([Azat Khuzhin](https://github.com/azat)). -* If `/dev/tty` is available, the progress in clickhouse-client and clickhouse-local will be rendered directly to the terminal, without writing to stderr. It allows to get progress even if stderr is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* If `/dev/tty` is available, the progress in clickhouse-client and clickhouse-local will be rendered directly to the terminal, without writing to STDERR. It allows getting progress even if STDERR is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Add support for `FixedString` input to base64 coding functions. [#42285](https://github.com/ClickHouse/ClickHouse/pull/42285) ([ltrk2](https://github.com/ltrk2)). * Add columns `bytes_on_disk` and `path` to `system.detached_parts`. Closes [#42264](https://github.com/ClickHouse/ClickHouse/issues/42264). [#42303](https://github.com/ClickHouse/ClickHouse/pull/42303) ([chen](https://github.com/xiedeyantu)). * Improve using structure from insertion table in table functions, now setting `use_structure_from_insertion_table_in_table_functions` has new possible value - `2` that means that ClickHouse will try to determine if we can use structure from insertion table or not automatically. Closes [#40028](https://github.com/ClickHouse/ClickHouse/issues/40028). [#42320](https://github.com/ClickHouse/ClickHouse/pull/42320) ([Kruglov Pavel](https://github.com/Avogar)). @@ -67,7 +67,7 @@ * When the merge task is continuously busy and the disk space is insufficient, the completely expired parts cannot be selected and dropped, resulting in insufficient disk space. My idea is that when the entire Part expires, there is no need for additional disk space to guarantee, ensure the normal execution of TTL. [#42869](https://github.com/ClickHouse/ClickHouse/pull/42869) ([zhongyuankai](https://github.com/zhongyuankai)). * Add `oss` function and `OSS` table engine (this is convenient for users). oss is fully compatible with s3. [#43155](https://github.com/ClickHouse/ClickHouse/pull/43155) ([zzsmdfj](https://github.com/zzsmdfj)). * Improve error reporting in the collection of OS-related info for the `system.asynchronous_metrics` table. [#43192](https://github.com/ClickHouse/ClickHouse/pull/43192) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Modify the `INFORMATION_SCHEMA` tables in a way so that now ClickHouse can connect to itself using the MySQL compatibility protocol. Add columns instead of aliases (related to [#9769](https://github.com/ClickHouse/ClickHouse/issues/9769)). It will improve the compatibility with various MySQL clients. [#43198](https://github.com/ClickHouse/ClickHouse/pull/43198) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Modify the `INFORMATION_SCHEMA` tables in a way so that ClickHouse can connect to itself using the MySQL compatibility protocol. Add columns instead of aliases (related to [#9769](https://github.com/ClickHouse/ClickHouse/issues/9769)). It will improve the compatibility with various MySQL clients. [#43198](https://github.com/ClickHouse/ClickHouse/pull/43198) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Add some functions for compatibility with PowerBI, when it connects using MySQL protocol [#42612](https://github.com/ClickHouse/ClickHouse/pull/42612) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Better usability for Dashboard on changes [#42872](https://github.com/ClickHouse/ClickHouse/pull/42872) ([Vladimir C](https://github.com/vdimir)). @@ -81,15 +81,15 @@ #### Bug Fix (user-visible misbehavior in official stable or prestable release) -* Updated normaliser to clone the alias ast. resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Updated normaliser to clone the alias ast. Resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). * Fix race for backup of tables in `Lazy` databases. [#43104](https://github.com/ClickHouse/ClickHouse/pull/43104) ([Vitaly Baranov](https://github.com/vitlibar)). * Fix for `skip_unavailable_shards`: it did not work with the `s3Cluster` table function. [#43131](https://github.com/ClickHouse/ClickHouse/pull/43131) ([chen](https://github.com/xiedeyantu)). * Fix schema inference in `s3Cluster` and improvement in `hdfsCluster`. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix retries while reading from URL table engines / table function. (retrtiable errors could be retries more times than needed, non-retrialble errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix retries while reading from URL table engines / table function. (retriable errors could be retries more times than needed, non-retriable errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). * A segmentation fault related to DNS & c-ares has been reported and fixed. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). * Fix `LOGICAL_ERROR` `Arguments of 'plus' have incorrect data types` which may happen in PK analysis (monotonicity check). Fix invalid PK analysis for monotonic binary functions with first constant argument. [#42410](https://github.com/ClickHouse/ClickHouse/pull/42410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). * Fix incorrect key analysis when key types cannot be inside Nullable. This fixes [#42456](https://github.com/ClickHouse/ClickHouse/issues/42456). [#42469](https://github.com/ClickHouse/ClickHouse/pull/42469) ([Amos Bird](https://github.com/amosbird)). -* Fix typo in setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix typo in a setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). * Fix creating a Set with wrong header when data type is LowCardinality. Closes [#42460](https://github.com/ClickHouse/ClickHouse/issues/42460). [#42579](https://github.com/ClickHouse/ClickHouse/pull/42579) ([flynn](https://github.com/ucasfl)). * `(U)Int128` and `(U)Int256` values were correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). * Fix a bug in functions parser that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). @@ -98,19 +98,19 @@ * Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). * Fix stack-use-after-return under ASAN build in the Create User query parser. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). * Fix `lowerUTF8`/`upperUTF8` in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). -* Additional bound check was added to lz4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). -* Fix rare possible hung on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). +* Additional bound check was added to LZ4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). +* Fix rare possible hang on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). * Fix incorrect behavior with multiple disjuncts in hash join, close [#42832](https://github.com/ClickHouse/ClickHouse/issues/42832). [#42876](https://github.com/ClickHouse/ClickHouse/pull/42876) ([Vladimir C](https://github.com/vdimir)). * A null pointer will be generated when select if as from ‘three table join’ , For example, this SQL query: [#42883](https://github.com/ClickHouse/ClickHouse/pull/42883) ([zzsmdfj](https://github.com/zzsmdfj)). * Fix memory sanitizer report in Cluster Discovery, close [#42763](https://github.com/ClickHouse/ClickHouse/issues/42763). [#42905](https://github.com/ClickHouse/ClickHouse/pull/42905) ([Vladimir C](https://github.com/vdimir)). -* Improve datetime schema inference in case of empty string. [#42911](https://github.com/ClickHouse/ClickHouse/pull/42911) ([Kruglov Pavel](https://github.com/Avogar)). +* Improve DateTime schema inference in case of empty string. [#42911](https://github.com/ClickHouse/ClickHouse/pull/42911) ([Kruglov Pavel](https://github.com/Avogar)). * Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)). * Fix ATTACH TABLE in `PostgreSQL` database engine if the table contains DATETIME data type. Closes [#42817](https://github.com/ClickHouse/ClickHouse/issues/42817). [#42960](https://github.com/ClickHouse/ClickHouse/pull/42960) ([Kseniia Sumarokova](https://github.com/kssenii)). * Fix lambda parsing. Closes [#41848](https://github.com/ClickHouse/ClickHouse/issues/41848). [#42979](https://github.com/ClickHouse/ClickHouse/pull/42979) ([Nikolay Degterinsky](https://github.com/evillique)). * Fix incorrect key analysis when nullable keys appear in the middle of a hyperrectangle. This fixes [#43111](https://github.com/ClickHouse/ClickHouse/issues/43111) . [#43133](https://github.com/ClickHouse/ClickHouse/pull/43133) ([Amos Bird](https://github.com/amosbird)). * Fix several buffer over-reads in deserialization of carefully crafted aggregate function states. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). * Fix function `if` in case of NULL and const Nullable arguments. Closes [#43069](https://github.com/ClickHouse/ClickHouse/issues/43069). [#43178](https://github.com/ClickHouse/ClickHouse/pull/43178) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix decimal math overflow in parsing datetime with the 'best effort' algorithm. Closes [#43061](https://github.com/ClickHouse/ClickHouse/issues/43061). [#43180](https://github.com/ClickHouse/ClickHouse/pull/43180) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix decimal math overflow in parsing DateTime with the 'best effort' algorithm. Closes [#43061](https://github.com/ClickHouse/ClickHouse/issues/43061). [#43180](https://github.com/ClickHouse/ClickHouse/pull/43180) ([Kruglov Pavel](https://github.com/Avogar)). * The `indent` field produced by the `git-import` tool was miscalculated. See https://clickhouse.com/docs/en/getting-started/example-datasets/github/. [#43191](https://github.com/ClickHouse/ClickHouse/pull/43191) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Fixed unexpected behaviour of `Interval` types with subquery and casting. [#43193](https://github.com/ClickHouse/ClickHouse/pull/43193) ([jh0x](https://github.com/jh0x)). @@ -673,7 +673,7 @@ * Support SQL standard CREATE INDEX and DROP INDEX syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). * Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). * Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). -* Remove subprocess run for kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* Remove subprocess run for Kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). * * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). * Improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). * Enable stack trace collection and query profiler for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). @@ -953,8 +953,8 @@ #### Upgrade Notes -* Now, background merges, mutations and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values, and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes of metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. +* Now, background merges, mutations, and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes to the metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant in this area. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. * `max_memory_usage` setting is removed from the default user profile in `users.xml`. This enables flexible memory limits for queries instead of the old rigid limit of 10 GB. * Disable `log_query_threads` setting by default. It controls the logging of statistics about every thread participating in query execution. After supporting asynchronous reads, the total number of distinct thread ids became too large, and logging into the `query_thread_log` has become too heavy. [#37077](https://github.com/ClickHouse/ClickHouse/pull/37077) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove function `groupArraySorted` which has a bug. [#36822](https://github.com/ClickHouse/ClickHouse/pull/36822) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From b280b683332bcd1e0689eec0be9c684f03d1dd3f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 16 Nov 2022 15:57:50 +0100 Subject: [PATCH 503/526] Fix style. --- src/Backups/BackupSettings.cpp | 2 +- src/Backups/RestoreSettings.cpp | 2 +- src/Parsers/ParserBackupQuery.cpp | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Backups/BackupSettings.cpp b/src/Backups/BackupSettings.cpp index bed34f2fbcc..5266296c248 100644 --- a/src/Backups/BackupSettings.cpp +++ b/src/Backups/BackupSettings.cpp @@ -128,7 +128,7 @@ void BackupSettings::copySettingsToQuery(ASTBackupQuery & query) const auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; if (base_backup_name) - query.setOrReplace(query.base_backup_name, std::move(base_backup_name)); + query.setOrReplace(query.base_backup_name, base_backup_name); else query.reset(query.base_backup_name); diff --git a/src/Backups/RestoreSettings.cpp b/src/Backups/RestoreSettings.cpp index 322954abc2b..2c06ee907b5 100644 --- a/src/Backups/RestoreSettings.cpp +++ b/src/Backups/RestoreSettings.cpp @@ -216,7 +216,7 @@ void RestoreSettings::copySettingsToQuery(ASTBackupQuery & query) const auto base_backup_name = base_backup_info ? base_backup_info->toAST() : nullptr; if (base_backup_name) - query.setOrReplace(query.base_backup_name, std::move(base_backup_name)); + query.setOrReplace(query.base_backup_name, base_backup_name); else query.reset(query.base_backup_name); diff --git a/src/Parsers/ParserBackupQuery.cpp b/src/Parsers/ParserBackupQuery.cpp index 78bc10511b2..cead1a98c1a 100644 --- a/src/Parsers/ParserBackupQuery.cpp +++ b/src/Parsers/ParserBackupQuery.cpp @@ -365,13 +365,13 @@ bool ParserBackupQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->cluster = std::move(cluster); if (backup_name) - query->set(query->backup_name, std::move(backup_name)); + query->set(query->backup_name, backup_name); query->settings = std::move(settings); query->cluster_host_ids = std::move(cluster_host_ids); if (base_backup_name) - query->set(query->base_backup_name, std::move(base_backup_name)); + query->set(query->base_backup_name, base_backup_name); return true; } From bc19d350090bb37e446c27d17d04f1707f901522 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 16 Nov 2022 15:34:36 +0000 Subject: [PATCH 504/526] fix clang-tidy --- src/Columns/ColumnObject.cpp | 6 +++--- src/Columns/ColumnTuple.cpp | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index c3e8af5a8cf..b2adf9e0059 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -666,14 +666,14 @@ size_t ColumnObject::allocatedBytes() const void ColumnObject::forEachSubcolumn(ColumnCallback callback) const { - for (auto & entry : subcolumns) - for (auto & part : entry->data.data) + for (const auto & entry : subcolumns) + for (const auto & part : entry->data.data) callback(part); } void ColumnObject::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { - for (auto & entry : subcolumns) + for (const auto & entry : subcolumns) { for (const auto & part : entry->data.data) { diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index c8ac2c67c36..8827b7b9850 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -497,13 +497,13 @@ void ColumnTuple::getExtremes(Field & min, Field & max) const void ColumnTuple::forEachSubcolumn(ColumnCallback callback) const { - for (auto & column : columns) + for (const auto & column : columns) callback(column); } void ColumnTuple::forEachSubcolumnRecursively(RecursiveColumnCallback callback) const { - for (auto & column : columns) + for (const auto & column : columns) { callback(*column); column->forEachSubcolumnRecursively(callback); From 5e087ae967db4b4cd1e2642689a971c8e7d69c28 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 16 Nov 2022 11:54:06 -0500 Subject: [PATCH 505/526] link to tutorial --- .../dictionaries/external-dictionaries/external-dicts.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md index 06b5b8a6746..9f922a2cccb 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts.md @@ -7,7 +7,9 @@ import CloudDetails from '@site/docs/en/sql-reference/dictionaries/external-dict # Dictionaries - +:::tip Tutorial +If you are getting started with Dictionaries in ClickHouse we have a tutorial that covers that topic. Take a look [here](/docs/en/tutorial.md). +::: You can add your own dictionaries from various data sources. The source for a dictionary can be a ClickHouse table, a local text or executable file, an HTTP(s) resource, or another DBMS. For more information, see “[Dictionary Sources](../../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md)”. @@ -27,6 +29,8 @@ The [dictionaries](../../../operations/system-tables/dictionaries.md#system_tabl - Configuration parameters. - Metrics like amount of RAM allocated for the dictionary or a number of queries since the dictionary was successfully loaded. + + ## Creating a dictionary with a DDL query Dictionaries can be created with [DDL queries](../../../sql-reference/statements/create/dictionary.md), and this is the recommended method because with DDL created dictionaries: From 4016fd2efcb196a97bfc157f661acf6c165f1fa8 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 16 Nov 2022 11:58:10 -0500 Subject: [PATCH 506/526] remove changes --- CHANGELOG.md | 109 ++------------------------------------------------- 1 file changed, 3 insertions(+), 106 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 051bde44dd2..68767612892 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,4 @@ ### Table of Contents -**[ClickHouse release v22.11, 2022-11-17](#2211)**
**[ClickHouse release v22.10, 2022-10-25](#2210)**
**[ClickHouse release v22.9, 2022-09-22](#229)**
**[ClickHouse release v22.8-lts, 2022-08-18](#228)**
@@ -12,108 +11,6 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
-### ClickHouse release 22.11, 2022-11-17 - -#### Backward Incompatible Change -* `JSONExtract` family of functions will now attempt to coerce to the requested type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). - -#### New Feature -* Adds support for retries during INSERTs into ReplicatedMergeTree when a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). -* Add `Hudi` and `DeltaLake` table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do), [Kseniia Sumarokova](https://github.com/kssenii)). -* Add table function `hudi` and `deltaLake`. [#43080](https://github.com/ClickHouse/ClickHouse/pull/43080) ([flynn](https://github.com/ucasfl)). -* Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In the case where the types of Intervals are different, they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). -* Added `**` glob support for recursive directory traversal of the filesystem and S3. Resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Introduce `s3_plain` disk type for write-once-read-many operations. Implement `ATTACH` of `MergeTree` table for `s3_plain` disk. [#42628](https://github.com/ClickHouse/ClickHouse/pull/42628) ([Azat Khuzhin](https://github.com/azat)). -* Added applied row-level policies to `system.query_log`. [#39819](https://github.com/ClickHouse/ClickHouse/pull/39819) ([Vladimir Chebotaryov](https://github.com/quickhouse)). -* Add four-letter command `csnp` for manually creating snapshots in ClickHouse Keeper. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). -* Add function `ascii` like in Apache Spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). -* Add function `pmod` which returns non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). -* Add function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). -* Add function `randCanonical`, which is similar to the `rand` function in Apache Spark or Impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). -* Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). -* Add `min_age_to_force_merge_on_partition_only` setting to optimize old parts for the entire partition only. [#42659](https://github.com/ClickHouse/ClickHouse/pull/42659) ([Antonio Andelic](https://github.com/antonio2368)). -* Add generic implementation for arbitrary structured named collections, access type and `system.named_collections`. [#43147](https://github.com/ClickHouse/ClickHouse/pull/43147) ([Kseniia Sumarokova](https://github.com/kssenii)). - -#### Performance Improvement -* `match` function can use the index if it's a condition on string prefix. This closes [#37333](https://github.com/ClickHouse/ClickHouse/issues/37333). [#42458](https://github.com/ClickHouse/ClickHouse/pull/42458) ([clarkcaoliu](https://github.com/Clark0)). -* Speed up AND and OR operators when they are sequenced. [#42214](https://github.com/ClickHouse/ClickHouse/pull/42214) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). -* Support parallel parsing for `LineAsString` input format. This improves performance just slightly. This closes [#42502](https://github.com/ClickHouse/ClickHouse/issues/42502). [#42780](https://github.com/ClickHouse/ClickHouse/pull/42780) ([Kruglov Pavel](https://github.com/Avogar)). -* ClickHouse Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)). -* A condition like `NOT LIKE 'prefix%'` can use the primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). - -#### Experimental Feature -* Support type `Object` inside other types, e.g. `Array(JSON)`. [#36969](https://github.com/ClickHouse/ClickHouse/pull/36969) ([Anton Popov](https://github.com/CurtizJ)). -* Ignore MySQL binlog SAVEPOINT event for MaterializedMySQL. [#42931](https://github.com/ClickHouse/ClickHouse/pull/42931) ([zzsmdfj](https://github.com/zzsmdfj)). Handle (ignore) SAVEPOINT queries in MaterializedMySQL. [#43086](https://github.com/ClickHouse/ClickHouse/pull/43086) ([Stig Bakken](https://github.com/stigsb)). - -#### Improvement -* Trivial queries with small LIMIT will properly determine the number of estimated rows to read, so that the threshold will be checked properly. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). -* Add support for interactive parameters in INSERT VALUES queries. [#43077](https://github.com/ClickHouse/ClickHouse/pull/43077) ([Nikolay Degterinsky](https://github.com/evillique)). -* Added new field `allow_readonly` in `system.table_functions` to allow using table functions in readonly mode. Resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* The `system.asynchronous_metrics` gets embedded documentation. This documentation is also exported to Prometheus. Fixed an error with the metrics about `cache` disks - they were calculated only for one arbitrary cache disk instead all of them. This closes [#7644](https://github.com/ClickHouse/ClickHouse/issues/7644). [#43194](https://github.com/ClickHouse/ClickHouse/pull/43194) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Throttling algorithm changed to token bucket. [#42665](https://github.com/ClickHouse/ClickHouse/pull/42665) ([Sergei Trifonov](https://github.com/serxa)). -* Mask passwords and secret keys both in `system.query_log` and `/var/log/clickhouse-server/*.log` and also in error messages. [#42484](https://github.com/ClickHouse/ClickHouse/pull/42484) ([Vitaly Baranov](https://github.com/vitlibar)). -* Remove covered parts for fetched part (to avoid possible replication delay grows). [#39737](https://github.com/ClickHouse/ClickHouse/pull/39737) ([Azat Khuzhin](https://github.com/azat)). -* If `/dev/tty` is available, the progress in clickhouse-client and clickhouse-local will be rendered directly to the terminal, without writing to STDERR. It allows getting progress even if STDERR is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add support for `FixedString` input to base64 coding functions. [#42285](https://github.com/ClickHouse/ClickHouse/pull/42285) ([ltrk2](https://github.com/ltrk2)). -* Add columns `bytes_on_disk` and `path` to `system.detached_parts`. Closes [#42264](https://github.com/ClickHouse/ClickHouse/issues/42264). [#42303](https://github.com/ClickHouse/ClickHouse/pull/42303) ([chen](https://github.com/xiedeyantu)). -* Improve using structure from insertion table in table functions, now setting `use_structure_from_insertion_table_in_table_functions` has new possible value - `2` that means that ClickHouse will try to determine if we can use structure from insertion table or not automatically. Closes [#40028](https://github.com/ClickHouse/ClickHouse/issues/40028). [#42320](https://github.com/ClickHouse/ClickHouse/pull/42320) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix no progress indication on INSERT FROM INFILE. Closes [#42548](https://github.com/ClickHouse/ClickHouse/issues/42548). [#42634](https://github.com/ClickHouse/ClickHouse/pull/42634) ([chen](https://github.com/xiedeyantu)). -* Refactor function `tokens` to enable max tokens returned for related functions (disabled by default). [#42673](https://github.com/ClickHouse/ClickHouse/pull/42673) ([李扬](https://github.com/taiyang-li)). -* Allow to use `Date32` arguments for `formatDateTime` and `FROM_UNIXTIME` functions. [#42737](https://github.com/ClickHouse/ClickHouse/pull/42737) ([Roman Vasin](https://github.com/rvasin)). -* Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Add `FailedAsyncInsertQuery` event metric for async inserts. [#42814](https://github.com/ClickHouse/ClickHouse/pull/42814) ([Krzysztof Góralski](https://github.com/kgoralski)). -* Implement `read-in-order` optimization on top of query plan. It is enabled by default. Set `query_plan_read_in_order = 0` to use previous AST-based version. [#42829](https://github.com/ClickHouse/ClickHouse/pull/42829) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Increase the size of upload part exponentially for backup to S3 to avoid errors about max 10 000 parts limit of the multipart upload to s3. [#42833](https://github.com/ClickHouse/ClickHouse/pull/42833) ([Vitaly Baranov](https://github.com/vitlibar)). -* When the merge task is continuously busy and the disk space is insufficient, the completely expired parts cannot be selected and dropped, resulting in insufficient disk space. My idea is that when the entire Part expires, there is no need for additional disk space to guarantee, ensure the normal execution of TTL. [#42869](https://github.com/ClickHouse/ClickHouse/pull/42869) ([zhongyuankai](https://github.com/zhongyuankai)). -* Add `oss` function and `OSS` table engine (this is convenient for users). oss is fully compatible with s3. [#43155](https://github.com/ClickHouse/ClickHouse/pull/43155) ([zzsmdfj](https://github.com/zzsmdfj)). -* Improve error reporting in the collection of OS-related info for the `system.asynchronous_metrics` table. [#43192](https://github.com/ClickHouse/ClickHouse/pull/43192) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Modify the `INFORMATION_SCHEMA` tables in a way so that ClickHouse can connect to itself using the MySQL compatibility protocol. Add columns instead of aliases (related to [#9769](https://github.com/ClickHouse/ClickHouse/issues/9769)). It will improve the compatibility with various MySQL clients. [#43198](https://github.com/ClickHouse/ClickHouse/pull/43198) ([Filatenkov Artur](https://github.com/FArthur-cmd)). -* Add some functions for compatibility with PowerBI, when it connects using MySQL protocol [#42612](https://github.com/ClickHouse/ClickHouse/pull/42612) ([Filatenkov Artur](https://github.com/FArthur-cmd)). -* Better usability for Dashboard on changes [#42872](https://github.com/ClickHouse/ClickHouse/pull/42872) ([Vladimir C](https://github.com/vdimir)). - -#### Build/Testing/Packaging Improvement -* Run SQLancer for each pull request and commit to master. [SQLancer](https://github.com/sqlancer/sqlancer) is an OpenSource fuzzer that focuses on automatic detection of logical bugs. [#42397](https://github.com/ClickHouse/ClickHouse/pull/42397) ([Ilya Yatsishin](https://github.com/qoega)). -* Update to latest zlib-ng. [#42463](https://github.com/ClickHouse/ClickHouse/pull/42463) ([Boris Kuschel](https://github.com/bkuschel)). -* Add support for testing ClickHouse server with Jepsen. By the way, we already have support for testing ClickHouse Keeper with Jepsen. This pull request extends it to Replicated tables. [#42619](https://github.com/ClickHouse/ClickHouse/pull/42619) ([Antonio Andelic](https://github.com/antonio2368)). -* Use https://github.com/matus-chochlik/ctcache for clang-tidy results caching. [#42913](https://github.com/ClickHouse/ClickHouse/pull/42913) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). -* Remove some libraries from Ubuntu Docker image. [#42622](https://github.com/ClickHouse/ClickHouse/pull/42622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). - -#### Bug Fix (user-visible misbehavior in official stable or prestable release) - -* Updated normaliser to clone the alias ast. Resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). -* Fix race for backup of tables in `Lazy` databases. [#43104](https://github.com/ClickHouse/ClickHouse/pull/43104) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fix for `skip_unavailable_shards`: it did not work with the `s3Cluster` table function. [#43131](https://github.com/ClickHouse/ClickHouse/pull/43131) ([chen](https://github.com/xiedeyantu)). -* Fix schema inference in `s3Cluster` and improvement in `hdfsCluster`. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix retries while reading from URL table engines / table function. (retriable errors could be retries more times than needed, non-retriable errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). -* A segmentation fault related to DNS & c-ares has been reported and fixed. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). -* Fix `LOGICAL_ERROR` `Arguments of 'plus' have incorrect data types` which may happen in PK analysis (monotonicity check). Fix invalid PK analysis for monotonic binary functions with first constant argument. [#42410](https://github.com/ClickHouse/ClickHouse/pull/42410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix incorrect key analysis when key types cannot be inside Nullable. This fixes [#42456](https://github.com/ClickHouse/ClickHouse/issues/42456). [#42469](https://github.com/ClickHouse/ClickHouse/pull/42469) ([Amos Bird](https://github.com/amosbird)). -* Fix typo in a setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix creating a Set with wrong header when data type is LowCardinality. Closes [#42460](https://github.com/ClickHouse/ClickHouse/issues/42460). [#42579](https://github.com/ClickHouse/ClickHouse/pull/42579) ([flynn](https://github.com/ucasfl)). -* `(U)Int128` and `(U)Int256` values were correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). -* Fix a bug in functions parser that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix the locking in `truncate table`. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)). -* Fix possible crash in `web` disks when file does not exist (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)). -* Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). -* Fix stack-use-after-return under ASAN build in the Create User query parser. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix `lowerUTF8`/`upperUTF8` in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). -* Additional bound check was added to LZ4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). -* Fix rare possible hang on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). -* Fix incorrect behavior with multiple disjuncts in hash join, close [#42832](https://github.com/ClickHouse/ClickHouse/issues/42832). [#42876](https://github.com/ClickHouse/ClickHouse/pull/42876) ([Vladimir C](https://github.com/vdimir)). -* A null pointer will be generated when select if as from ‘three table join’ , For example, this SQL query: [#42883](https://github.com/ClickHouse/ClickHouse/pull/42883) ([zzsmdfj](https://github.com/zzsmdfj)). -* Fix memory sanitizer report in Cluster Discovery, close [#42763](https://github.com/ClickHouse/ClickHouse/issues/42763). [#42905](https://github.com/ClickHouse/ClickHouse/pull/42905) ([Vladimir C](https://github.com/vdimir)). -* Improve DateTime schema inference in case of empty string. [#42911](https://github.com/ClickHouse/ClickHouse/pull/42911) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)). -* Fix ATTACH TABLE in `PostgreSQL` database engine if the table contains DATETIME data type. Closes [#42817](https://github.com/ClickHouse/ClickHouse/issues/42817). [#42960](https://github.com/ClickHouse/ClickHouse/pull/42960) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fix lambda parsing. Closes [#41848](https://github.com/ClickHouse/ClickHouse/issues/41848). [#42979](https://github.com/ClickHouse/ClickHouse/pull/42979) ([Nikolay Degterinsky](https://github.com/evillique)). -* Fix incorrect key analysis when nullable keys appear in the middle of a hyperrectangle. This fixes [#43111](https://github.com/ClickHouse/ClickHouse/issues/43111) . [#43133](https://github.com/ClickHouse/ClickHouse/pull/43133) ([Amos Bird](https://github.com/amosbird)). -* Fix several buffer over-reads in deserialization of carefully crafted aggregate function states. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). -* Fix function `if` in case of NULL and const Nullable arguments. Closes [#43069](https://github.com/ClickHouse/ClickHouse/issues/43069). [#43178](https://github.com/ClickHouse/ClickHouse/pull/43178) ([Kruglov Pavel](https://github.com/Avogar)). -* Fix decimal math overflow in parsing DateTime with the 'best effort' algorithm. Closes [#43061](https://github.com/ClickHouse/ClickHouse/issues/43061). [#43180](https://github.com/ClickHouse/ClickHouse/pull/43180) ([Kruglov Pavel](https://github.com/Avogar)). -* The `indent` field produced by the `git-import` tool was miscalculated. See https://clickhouse.com/docs/en/getting-started/example-datasets/github/. [#43191](https://github.com/ClickHouse/ClickHouse/pull/43191) ([Alexey Milovidov](https://github.com/alexey-milovidov)). -* Fixed unexpected behaviour of `Interval` types with subquery and casting. [#43193](https://github.com/ClickHouse/ClickHouse/pull/43193) ([jh0x](https://github.com/jh0x)). - ### ClickHouse release 22.10, 2022-10-26 #### Backward Incompatible Change @@ -673,7 +570,7 @@ * Support SQL standard CREATE INDEX and DROP INDEX syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). * Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). * Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). -* Remove subprocess run for Kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* Remove subprocess run for kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). * * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). * Improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). * Enable stack trace collection and query profiler for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). @@ -953,8 +850,8 @@ #### Upgrade Notes -* Now, background merges, mutations, and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes to the metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant in this area. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. +* Now, background merges, mutations and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values, and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes of metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. * `max_memory_usage` setting is removed from the default user profile in `users.xml`. This enables flexible memory limits for queries instead of the old rigid limit of 10 GB. * Disable `log_query_threads` setting by default. It controls the logging of statistics about every thread participating in query execution. After supporting asynchronous reads, the total number of distinct thread ids became too large, and logging into the `query_thread_log` has become too heavy. [#37077](https://github.com/ClickHouse/ClickHouse/pull/37077) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove function `groupArraySorted` which has a bug. [#36822](https://github.com/ClickHouse/ClickHouse/pull/36822) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From 9011a18234f26914104a3762e3e7e8a2e5f99661 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 16 Nov 2022 22:33:48 +0300 Subject: [PATCH 507/526] Revert "S3 request per second rate throttling" --- .../engines/table-engines/integrations/s3.md | 5 -- .../mergetree-family/mergetree.md | 4 -- src/Backups/BackupIO_S3.cpp | 27 +++++---- src/Backups/BackupIO_S3.h | 5 +- src/Common/ProfileEvents.cpp | 2 +- src/Common/Throttler.cpp | 2 + src/Common/Throttler.h | 2 - src/Coordination/KeeperSnapshotManagerS3.cpp | 12 ++-- src/Core/Settings.h | 4 -- .../ObjectStorages/S3/S3ObjectStorage.cpp | 23 ++++---- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 6 +- src/Disks/ObjectStorages/S3/diskSettings.cpp | 38 ++++--------- src/Disks/ObjectStorages/S3/diskSettings.h | 2 +- src/Disks/ObjectStorages/S3/parseConfig.h | 27 +++++++++ .../ObjectStorages/S3/registerDiskS3.cpp | 13 +++-- src/IO/ReadBufferFromS3.cpp | 10 ++-- src/IO/ReadBufferFromS3.h | 11 ++-- src/IO/S3/PocoHTTPClient.cpp | 26 +-------- src/IO/S3/PocoHTTPClient.h | 17 +----- src/IO/S3/tests/gtest_aws_s3_client.cpp | 8 +-- src/IO/S3Common.cpp | 31 ++-------- src/IO/S3Common.h | 5 +- src/IO/WriteBufferFromS3.cpp | 20 +++---- src/IO/WriteBufferFromS3.h | 4 +- .../ExternalDataSourceConfiguration.h | 2 +- src/Storages/StorageDelta.cpp | 6 +- src/Storages/StorageHudi.cpp | 2 +- src/Storages/StorageS3.cpp | 56 +++++++++---------- src/Storages/StorageS3.h | 13 ++--- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageS3Settings.cpp | 47 +++++----------- src/Storages/StorageS3Settings.h | 19 +++---- .../02477_s3_request_throttler.reference | 2 - .../02477_s3_request_throttler.sql | 26 --------- 34 files changed, 173 insertions(+), 306 deletions(-) create mode 100644 src/Disks/ObjectStorages/S3/parseConfig.h delete mode 100644 tests/queries/0_stateless/02477_s3_request_throttler.reference delete mode 100644 tests/queries/0_stateless/02477_s3_request_throttler.sql diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 484fd265c3d..db983ab9c68 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -127,10 +127,6 @@ The following settings can be set before query execution or placed into configur - `s3_min_upload_part_size` — The minimum size of part to upload during multipart upload to [S3 Multipart upload](https://docs.aws.amazon.com/AmazonS3/latest/dev/uploadobjusingmpu.html). Default value is `512Mb`. - `s3_max_redirects` — Max number of S3 redirects hops allowed. Default value is `10`. - `s3_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. -- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited). -- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. -- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). -- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. @@ -146,7 +142,6 @@ The following settings can be specified in configuration file for given endpoint - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. -- `max_put_rps`, `max_put_burst`, `max_get_rps` and `max_get_burst` - Throttling settings (see description above) to use for specific endpoint instead of per query. Optional. **Example:** diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 7614a09c018..ce6cec079a3 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -940,10 +940,6 @@ Optional parameters: - `cache_path` — Path on local FS where to store cached mark and index files. Default value is `/var/lib/clickhouse/disks//cache/`. - `skip_access_check` — If true, disk access checks will not be performed on disk start-up. Default value is `false`. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. -- `s3_max_put_rps` — Maximum PUT requests per second rate before throttling. Default value is `0` (unlimited). -- `s3_max_put_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_put_rps`. -- `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). -- `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. S3 disk can be configured as `main` or `cold` storage: ``` xml diff --git a/src/Backups/BackupIO_S3.cpp b/src/Backups/BackupIO_S3.cpp index 8342749e230..f7f7643a6e3 100644 --- a/src/Backups/BackupIO_S3.cpp +++ b/src/Backups/BackupIO_S3.cpp @@ -46,7 +46,7 @@ namespace context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}); + /* for_disk_s3 = */ false); client_configuration.endpointOverride = s3_uri.endpoint; client_configuration.maxConnections = static_cast(context->getSettingsRef().s3_max_connections); @@ -86,10 +86,9 @@ BackupReaderS3::BackupReaderS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) + , max_single_read_retries(context_->getSettingsRef().s3_max_single_read_retries) , read_settings(context_->getReadSettings()) - , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) { - request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint } DataSourceDescription BackupReaderS3::getDataSourceDescription() const @@ -116,7 +115,7 @@ UInt64 BackupReaderS3::getFileSize(const String & file_name) std::unique_ptr BackupReaderS3::readFile(const String & file_name) { return std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, max_single_read_retries, read_settings); } @@ -124,12 +123,12 @@ BackupWriterS3::BackupWriterS3( const S3::URI & s3_uri_, const String & access_key_id_, const String & secret_access_key_, const ContextPtr & context_) : s3_uri(s3_uri_) , client(makeS3Client(s3_uri_, access_key_id_, secret_access_key_, context_)) + , max_single_read_retries(context_->getSettingsRef().s3_max_single_read_retries) , read_settings(context_->getReadSettings()) - , request_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).request_settings) + , rw_settings(context_->getStorageS3Settings().getSettings(s3_uri.uri.toString()).rw_settings) , log(&Poco::Logger::get("BackupWriterS3")) { - request_settings.updateFromSettingsIfEmpty(context_->getSettingsRef()); - request_settings.max_single_read_retries = context_->getSettingsRef().s3_max_single_read_retries; // FIXME: Avoid taking value for endpoint + rw_settings.updateFromSettingsIfEmpty(context_->getSettingsRef()); } DataSourceDescription BackupWriterS3::getDataSourceDescription() const @@ -217,7 +216,7 @@ void BackupWriterS3::copyObjectMultipartImpl( std::vector part_tags; size_t position = 0; - size_t upload_part_size = request_settings.min_upload_part_size; + size_t upload_part_size = rw_settings.min_upload_part_size; for (size_t part_number = 1; position < size; ++part_number) { @@ -249,10 +248,10 @@ void BackupWriterS3::copyObjectMultipartImpl( position = next_position; - if (part_number % request_settings.upload_part_size_multiply_parts_count_threshold == 0) + if (part_number % rw_settings.upload_part_size_multiply_parts_count_threshold == 0) { - upload_part_size *= request_settings.upload_part_size_multiply_factor; - upload_part_size = std::min(upload_part_size, request_settings.max_upload_part_size); + upload_part_size *= rw_settings.upload_part_size_multiply_factor; + upload_part_size = std::min(upload_part_size, rw_settings.max_upload_part_size); } } @@ -295,7 +294,7 @@ void BackupWriterS3::copyFileNative(DiskPtr from_disk, const String & file_name_ auto file_path = fs::path(s3_uri.key) / file_name_to; auto head = requestObjectHeadData(source_bucket, objects[0].absolute_path).GetResult(); - if (static_cast(head.GetContentLength()) < request_settings.max_single_operation_copy_size) + if (static_cast(head.GetContentLength()) < rw_settings.max_single_operation_copy_size) { copyObjectImpl( source_bucket, objects[0].absolute_path, s3_uri.bucket, file_path, head); @@ -332,7 +331,7 @@ bool BackupWriterS3::fileContentsEqual(const String & file_name, const String & try { auto in = std::make_unique( - client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, request_settings, read_settings); + client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, s3_uri.version_id, max_single_read_retries, read_settings); String actual_file_contents(expected_file_contents.size(), ' '); return (in->read(actual_file_contents.data(), actual_file_contents.size()) == actual_file_contents.size()) && (actual_file_contents == expected_file_contents) && in->eof(); @@ -350,7 +349,7 @@ std::unique_ptr BackupWriterS3::writeFile(const String & file_name) client, s3_uri.bucket, fs::path(s3_uri.key) / file_name, - request_settings, + rw_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get(), "BackupWriterS3")); diff --git a/src/Backups/BackupIO_S3.h b/src/Backups/BackupIO_S3.h index deaf7979ff8..b52de23e262 100644 --- a/src/Backups/BackupIO_S3.h +++ b/src/Backups/BackupIO_S3.h @@ -39,8 +39,8 @@ public: private: S3::URI s3_uri; std::shared_ptr client; + UInt64 max_single_read_retries; ReadSettings read_settings; - S3Settings::RequestSettings request_settings; }; @@ -81,8 +81,9 @@ private: S3::URI s3_uri; std::shared_ptr client; + UInt64 max_single_read_retries; ReadSettings read_settings; - S3Settings::RequestSettings request_settings; + S3Settings::ReadWriteSettings rw_settings; Poco::Logger * log; }; diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 90d24ec027e..e30a6bb6aaf 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -62,7 +62,7 @@ M(NetworkSendElapsedMicroseconds, "Total time spent waiting for data to send to network or sending data to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries..") \ M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ - M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform 'max_network_bandwidth' and other throttling settings.") \ + M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform the 'max_network_bandwidth' setting.") \ \ M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \ \ diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 79625d4894c..b38777efc03 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -20,6 +20,8 @@ namespace ErrorCodes /// Just 10^9. static constexpr auto NS = 1000000000UL; +static const size_t default_burst_seconds = 1; + Throttler::Throttler(size_t max_speed_, const std::shared_ptr & parent_) : max_speed(max_speed_) , max_burst(max_speed_ * default_burst_seconds) diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 708e9b939fa..9b6eff13506 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -17,8 +17,6 @@ namespace DB class Throttler { public: - static const size_t default_burst_seconds = 1; - Throttler(size_t max_speed_, size_t max_burst_, const std::shared_ptr & parent_ = nullptr) : max_speed(max_speed_), max_burst(max_burst_), limit_exceeded_exception_message(""), tokens(max_burst), parent(parent_) {} diff --git a/src/Coordination/KeeperSnapshotManagerS3.cpp b/src/Coordination/KeeperSnapshotManagerS3.cpp index cc837f5f496..2e19d496407 100644 --- a/src/Coordination/KeeperSnapshotManagerS3.cpp +++ b/src/Coordination/KeeperSnapshotManagerS3.cpp @@ -93,7 +93,7 @@ void KeeperSnapshotManagerS3::updateS3Configuration(const Poco::Util::AbstractCo auth_settings.region, RemoteHostFilter(), s3_max_redirects, enable_s3_requests_logging, - /* for_disk_s3 = */ false, /* get_request_throttler = */ {}, /* put_request_throttler = */ {}); + /* for_disk_s3 = */ false); client_configuration.endpointOverride = new_uri.endpoint; @@ -135,8 +135,8 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa if (s3_client == nullptr) return; - S3Settings::RequestSettings request_settings_1; - request_settings_1.upload_part_size_multiply_parts_count_threshold = 10000; + S3Settings::ReadWriteSettings read_write_settings; + read_write_settings.upload_part_size_multiply_parts_count_threshold = 10000; const auto create_writer = [&](const auto & key) { @@ -145,7 +145,7 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa s3_client->client, s3_client->uri.bucket, key, - request_settings_1 + read_write_settings }; }; @@ -194,15 +194,13 @@ void KeeperSnapshotManagerS3::uploadSnapshotImpl(const std::string & snapshot_pa lock_writer.finalize(); // We read back the written UUID, if it's the same we can upload the file - S3Settings::RequestSettings request_settings_2; - request_settings_2.max_single_read_retries = 1; ReadBufferFromS3 lock_reader { s3_client->client, s3_client->uri.bucket, lock_file, "", - request_settings_2, + 1, {} }; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index bcaea96512d..83252b6f0a9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -90,10 +90,6 @@ static constexpr UInt64 operator""_GiB(unsigned long long value) M(UInt64, s3_max_unexpected_write_error_retries, 4, "The maximum number of retries in case of unexpected errors during S3 write.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ - M(UInt64, s3_max_get_rps, 0, "Limit on S3 GET request per second rate before throttling. Zero means unlimited.", 0) \ - M(UInt64, s3_max_get_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_get_rps`", 0) \ - M(UInt64, s3_max_put_rps, 0, "Limit on S3 PUT request per second rate before throttling. Zero means unlimited.", 0) \ - M(UInt64, s3_max_put_burst, 0, "Max number of requests that can be issued simultaneously before hitting request per second limit. By default (0) equals to `s3_max_put_rps`", 0) \ M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(Bool, s3_check_objects_after_upload, false, "Check each uploaded object to s3 with head request to be sure that upload was successful", 0) \ diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 11c9345d4a1..099a7d458d0 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -175,7 +175,7 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT bucket, path, version_id, - settings_ptr->request_settings, + settings_ptr->s3_settings.max_single_read_retries, disk_read_settings, /* use_external_buffer */true, /* offset */0, @@ -212,7 +212,7 @@ std::unique_ptr S3ObjectStorage::readObject( /// NOLINT bucket, object.absolute_path, version_id, - settings_ptr->request_settings, + settings_ptr->s3_settings.max_single_read_retries, patchSettings(read_settings)); } @@ -238,7 +238,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN client.get(), bucket, object.absolute_path, - settings_ptr->request_settings, + settings_ptr->s3_settings, attributes, buf_size, std::move(scheduler), @@ -489,7 +489,7 @@ void S3ObjectStorage::copyObjectImpl( throwIfError(outcome); auto settings_ptr = s3_settings.get(); - if (settings_ptr->request_settings.check_objects_after_upload) + if (settings_ptr->s3_settings.check_objects_after_upload) { auto object_head = requestObjectHeadData(dst_bucket, dst_key); if (!object_head.IsSuccess()) @@ -533,7 +533,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( std::vector part_tags; - size_t upload_part_size = settings_ptr->request_settings.min_upload_part_size; + size_t upload_part_size = settings_ptr->s3_settings.min_upload_part_size; for (size_t position = 0, part_number = 1; position < size; ++part_number, position += upload_part_size) { ProfileEvents::increment(ProfileEvents::S3UploadPartCopy); @@ -586,7 +586,7 @@ void S3ObjectStorage::copyObjectMultipartImpl( throwIfError(outcome); } - if (settings_ptr->request_settings.check_objects_after_upload) + if (settings_ptr->s3_settings.check_objects_after_upload) { auto object_head = requestObjectHeadData(dst_bucket, dst_key); if (!object_head.IsSuccess()) @@ -643,20 +643,17 @@ void S3ObjectStorage::startup() void S3ObjectStorage::applyNewSettings(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { - auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); - s3_settings.set(std::move(new_s3_settings)); - client.set(std::move(new_client)); + s3_settings.set(getSettings(config, config_prefix, context)); + client.set(getClient(config, config_prefix, context)); applyRemoteThrottlingSettings(context); } std::unique_ptr S3ObjectStorage::cloneObjectStorage( const std::string & new_namespace, const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, ContextPtr context) { - auto new_s3_settings = getSettings(config, config_prefix, context); - auto new_client = getClient(config, config_prefix, context, *new_s3_settings); return std::make_unique( - std::move(new_client), std::move(new_s3_settings), + getClient(config, config_prefix, context), + getSettings(config, config_prefix, context), version_id, s3_capabilities, new_namespace, S3::URI(Poco::URI(config.getString(config_prefix + ".endpoint"))).endpoint); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 57ceaa679e0..447ca034aac 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -23,17 +23,17 @@ struct S3ObjectStorageSettings S3ObjectStorageSettings() = default; S3ObjectStorageSettings( - const S3Settings::RequestSettings & request_settings_, + const S3Settings::ReadWriteSettings & s3_settings_, uint64_t min_bytes_for_seek_, int32_t list_object_keys_size_, int32_t objects_chunk_size_to_delete_) - : request_settings(request_settings_) + : s3_settings(s3_settings_) , min_bytes_for_seek(min_bytes_for_seek_) , list_object_keys_size(list_object_keys_size_) , objects_chunk_size_to_delete(objects_chunk_size_to_delete_) {} - S3Settings::RequestSettings request_settings; + S3Settings::ReadWriteSettings s3_settings; uint64_t min_bytes_for_seek; int32_t list_object_keys_size; diff --git a/src/Disks/ObjectStorages/S3/diskSettings.cpp b/src/Disks/ObjectStorages/S3/diskSettings.cpp index ca2e9d04926..e61987163d2 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.cpp +++ b/src/Disks/ObjectStorages/S3/diskSettings.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include @@ -33,26 +32,17 @@ namespace ErrorCodes std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { - const Settings & settings = context->getSettingsRef(); - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", settings.s3_max_single_read_retries); - request_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", settings.s3_min_upload_part_size); - request_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", settings.s3_upload_part_size_multiply_factor); - request_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", settings.s3_upload_part_size_multiply_parts_count_threshold); - request_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", settings.s3_max_single_part_upload_size); - request_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", settings.s3_check_objects_after_upload); - request_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", settings.s3_max_unexpected_write_error_retries); - - // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = config.getUInt64(config_prefix + ".s3_max_get_rps", settings.s3_max_get_rps)) - request_settings.get_request_throttler = std::make_shared( - max_get_rps, config.getUInt64(config_prefix + ".s3_max_get_burst", settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); - if (UInt64 max_put_rps = config.getUInt64(config_prefix + ".s3_max_put_rps", settings.s3_max_put_rps)) - request_settings.put_request_throttler = std::make_shared( - max_put_rps, config.getUInt64(config_prefix + ".s3_max_put_burst", settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); + S3Settings::ReadWriteSettings rw_settings; + rw_settings.max_single_read_retries = config.getUInt64(config_prefix + ".s3_max_single_read_retries", context->getSettingsRef().s3_max_single_read_retries); + rw_settings.min_upload_part_size = config.getUInt64(config_prefix + ".s3_min_upload_part_size", context->getSettingsRef().s3_min_upload_part_size); + rw_settings.upload_part_size_multiply_factor = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_factor", context->getSettingsRef().s3_upload_part_size_multiply_factor); + rw_settings.upload_part_size_multiply_parts_count_threshold = config.getUInt64(config_prefix + ".s3_upload_part_size_multiply_parts_count_threshold", context->getSettingsRef().s3_upload_part_size_multiply_parts_count_threshold); + rw_settings.max_single_part_upload_size = config.getUInt64(config_prefix + ".s3_max_single_part_upload_size", context->getSettingsRef().s3_max_single_part_upload_size); + rw_settings.check_objects_after_upload = config.getUInt64(config_prefix + ".s3_check_objects_after_upload", context->getSettingsRef().s3_check_objects_after_upload); + rw_settings.max_unexpected_write_error_retries = config.getUInt64(config_prefix + ".s3_max_unexpected_write_error_retries", context->getSettingsRef().s3_max_unexpected_write_error_retries); return std::make_unique( - request_settings, + rw_settings, config.getUInt64(config_prefix + ".min_bytes_for_seek", 1024 * 1024), config.getInt(config_prefix + ".list_object_keys_size", 1000), config.getInt(config_prefix + ".objects_chunk_size_to_delete", 1000)); @@ -122,20 +112,14 @@ std::shared_ptr getProxyConfiguration(const String & pre } -std::unique_ptr getClient( - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - ContextPtr context, - const S3ObjectStorageSettings & settings) +std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context) { S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( config.getString(config_prefix + ".region", ""), context->getRemoteHostFilter(), static_cast(context->getGlobalContext()->getSettingsRef().s3_max_redirects), context->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ true, - settings.request_settings.get_request_throttler, - settings.request_settings.put_request_throttler); + /* for_disk_s3 = */ true); S3::URI uri(Poco::URI(config.getString(config_prefix + ".endpoint"))); if (uri.key.back() != '/') diff --git a/src/Disks/ObjectStorages/S3/diskSettings.h b/src/Disks/ObjectStorages/S3/diskSettings.h index 04eb7aced8e..05ba8819f83 100644 --- a/src/Disks/ObjectStorages/S3/diskSettings.h +++ b/src/Disks/ObjectStorages/S3/diskSettings.h @@ -22,7 +22,7 @@ struct S3ObjectStorageSettings; std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); -std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context, const S3ObjectStorageSettings & settings); +std::unique_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); } diff --git a/src/Disks/ObjectStorages/S3/parseConfig.h b/src/Disks/ObjectStorages/S3/parseConfig.h new file mode 100644 index 00000000000..1defc673c2e --- /dev/null +++ b/src/Disks/ObjectStorages/S3/parseConfig.h @@ -0,0 +1,27 @@ +#pragma once + +#include "config.h" + +#if USE_AWS_S3 + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + + +std::unique_ptr getSettings(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + +std::shared_ptr getClient(const Poco::Util::AbstractConfiguration & config, const String & config_prefix, ContextPtr context); + + +} diff --git a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp index e73accbb956..e09aef22122 100644 --- a/src/Disks/ObjectStorages/S3/registerDiskS3.cpp +++ b/src/Disks/ObjectStorages/S3/registerDiskS3.cpp @@ -130,16 +130,21 @@ void registerDiskS3(DiskFactory & factory) chassert(type == "s3" || type == "s3_plain"); MetadataStoragePtr metadata_storage; - auto settings = getSettings(config, config_prefix, context); - auto client = getClient(config, config_prefix, context, *settings); if (type == "s3_plain") { - s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); + s3_storage = std::make_shared( + getClient(config, config_prefix, context), + getSettings(config, config_prefix, context), + uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); metadata_storage = std::make_shared(s3_storage, uri.key); } else { - s3_storage = std::make_shared(std::move(client), std::move(settings), uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); + s3_storage = std::make_shared( + getClient(config, config_prefix, context), + getSettings(config, config_prefix, context), + uri.version_id, s3_capabilities, uri.bucket, uri.endpoint); + auto [metadata_path, metadata_disk] = prepareForLocalMetadata(name, config, config_prefix, context); metadata_storage = std::make_shared(metadata_disk, uri.key); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index c14fbecf223..c49941b025d 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -45,7 +45,7 @@ ReadBufferFromS3::ReadBufferFromS3( const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::RequestSettings & request_settings_, + UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer_, size_t offset_, @@ -56,7 +56,7 @@ ReadBufferFromS3::ReadBufferFromS3( , bucket(bucket_) , key(key_) , version_id(version_id_) - , request_settings(request_settings_) + , max_single_read_retries(max_single_read_retries_) , offset(offset_) , read_until_position(read_until_position_) , read_settings(settings_) @@ -105,7 +105,7 @@ bool ReadBufferFromS3::nextImpl() } size_t sleep_time_with_backoff_milliseconds = 100; - for (size_t attempt = 0; attempt < request_settings.max_single_read_retries && !next_result; ++attempt) + for (size_t attempt = 0; (attempt < max_single_read_retries) && !next_result; ++attempt) { Stopwatch watch; try @@ -166,7 +166,7 @@ bool ReadBufferFromS3::nextImpl() attempt, e.message()); - if (attempt + 1 == request_settings.max_single_read_retries) + if (attempt + 1 == max_single_read_retries) throw; /// Pause before next attempt. @@ -349,7 +349,7 @@ SeekableReadBufferPtr ReadBufferS3Factory::getReader() bucket, key, version_id, - request_settings, + s3_max_single_read_retries, read_settings, false /*use_external_buffer*/, next_range->first, diff --git a/src/IO/ReadBufferFromS3.h b/src/IO/ReadBufferFromS3.h index 17b13bf7d62..cc836bba495 100644 --- a/src/IO/ReadBufferFromS3.h +++ b/src/IO/ReadBufferFromS3.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include "config.h" #if USE_AWS_S3 @@ -34,7 +33,7 @@ private: String bucket; String key; String version_id; - const S3Settings::RequestSettings request_settings; + UInt64 max_single_read_retries; /// These variables are atomic because they can be used for `logging only` /// (where it is not important to get consistent result) @@ -53,7 +52,7 @@ public: const String & bucket_, const String & key_, const String & version_id_, - const S3Settings::RequestSettings & request_settings_, + UInt64 max_single_read_retries_, const ReadSettings & settings_, bool use_external_buffer = false, size_t offset_ = 0, @@ -101,7 +100,7 @@ public: const String & version_id_, size_t range_step_, size_t object_size_, - const S3Settings::RequestSettings & request_settings_, + UInt64 s3_max_single_read_retries_, const ReadSettings & read_settings_) : client_ptr(client_ptr_) , bucket(bucket_) @@ -111,7 +110,7 @@ public: , range_generator(object_size_, range_step_) , range_step(range_step_) , object_size(object_size_) - , request_settings(request_settings_) + , s3_max_single_read_retries(s3_max_single_read_retries_) { assert(range_step > 0); assert(range_step < object_size); @@ -136,7 +135,7 @@ private: size_t range_step; size_t object_size; - const S3Settings::RequestSettings request_settings; + UInt64 s3_max_single_read_retries; }; } diff --git a/src/IO/S3/PocoHTTPClient.cpp b/src/IO/S3/PocoHTTPClient.cpp index 33dd3250c9f..7d053bebe4a 100644 --- a/src/IO/S3/PocoHTTPClient.cpp +++ b/src/IO/S3/PocoHTTPClient.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include #include @@ -77,16 +76,12 @@ PocoHTTPClientConfiguration::PocoHTTPClientConfiguration( const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, bool enable_s3_requests_logging_, - bool for_disk_s3_, - const ThrottlerPtr & get_request_throttler_, - const ThrottlerPtr & put_request_throttler_) + bool for_disk_s3_) : force_region(force_region_) , remote_host_filter(remote_host_filter_) , s3_max_redirects(s3_max_redirects_) , enable_s3_requests_logging(enable_s3_requests_logging_) , for_disk_s3(for_disk_s3_) - , get_request_throttler(get_request_throttler_) - , put_request_throttler(put_request_throttler_) { } @@ -133,8 +128,6 @@ PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_config , s3_max_redirects(client_configuration.s3_max_redirects) , enable_s3_requests_logging(client_configuration.enable_s3_requests_logging) , for_disk_s3(client_configuration.for_disk_s3) - , get_request_throttler(client_configuration.get_request_throttler) - , put_request_throttler(client_configuration.put_request_throttler) , extra_headers(client_configuration.extra_headers) { } @@ -252,23 +245,6 @@ void PocoHTTPClient::makeRequestInternal( if (enable_s3_requests_logging) LOG_TEST(log, "Make request to: {}", uri); - switch (request.GetMethod()) - { - case Aws::Http::HttpMethod::HTTP_GET: - case Aws::Http::HttpMethod::HTTP_HEAD: - if (get_request_throttler) - get_request_throttler->add(1); - break; - case Aws::Http::HttpMethod::HTTP_PUT: - case Aws::Http::HttpMethod::HTTP_POST: - case Aws::Http::HttpMethod::HTTP_PATCH: - if (put_request_throttler) - put_request_throttler->add(1); - break; - case Aws::Http::HttpMethod::HTTP_DELETE: - break; // Not throttled - } - addMetric(request, S3MetricType::Count); CurrentMetrics::Increment metric_increment{CurrentMetrics::S3Requests}; diff --git a/src/IO/S3/PocoHTTPClient.h b/src/IO/S3/PocoHTTPClient.h index ed6e1793c2c..5649638285d 100644 --- a/src/IO/S3/PocoHTTPClient.h +++ b/src/IO/S3/PocoHTTPClient.h @@ -8,7 +8,6 @@ #if USE_AWS_S3 #include -#include #include #include #include @@ -49,8 +48,6 @@ struct PocoHTTPClientConfiguration : public Aws::Client::ClientConfiguration unsigned int s3_max_redirects; bool enable_s3_requests_logging; bool for_disk_s3; - ThrottlerPtr get_request_throttler; - ThrottlerPtr put_request_throttler; HeaderCollection extra_headers; void updateSchemeAndRegion(); @@ -63,9 +60,7 @@ private: const RemoteHostFilter & remote_host_filter_, unsigned int s3_max_redirects_, bool enable_s3_requests_logging_, - bool for_disk_s3_, - const ThrottlerPtr & get_request_throttler_, - const ThrottlerPtr & put_request_throttler_ + bool for_disk_s3_ ); /// Constructor of Aws::Client::ClientConfiguration must be called after AWS SDK initialization. @@ -159,16 +154,6 @@ private: unsigned int s3_max_redirects; bool enable_s3_requests_logging; bool for_disk_s3; - - /// Limits get request per second rate for GET, SELECT and all other requests, excluding throttled by put throttler - /// (i.e. throttles GetObject, HeadObject) - ThrottlerPtr get_request_throttler; - - /// Limits put request per second rate for PUT, COPY, POST, LIST requests - /// (i.e. throttles PutObject, CopyObject, ListObjects, CreateMultipartUpload, UploadPartCopy, UploadPart, CompleteMultipartUpload) - /// NOTE: DELETE and CANCEL requests are not throttled by either put or get throttler - ThrottlerPtr put_request_throttler; - const HeaderCollection extra_headers; }; diff --git a/src/IO/S3/tests/gtest_aws_s3_client.cpp b/src/IO/S3/tests/gtest_aws_s3_client.cpp index 2d298ca5df2..9b2a65d84fc 100644 --- a/src/IO/S3/tests/gtest_aws_s3_client.cpp +++ b/src/IO/S3/tests/gtest_aws_s3_client.cpp @@ -88,9 +88,7 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) remote_host_filter, s3_max_redirects, enable_s3_requests_logging, - /* for_disk_s3 = */ false, - /* get_request_throttler = */ {}, - /* put_request_throttler = */ {} + /* for_disk_s3 = */ false ); client_configuration.endpointOverride = uri.endpoint; @@ -115,14 +113,12 @@ TEST(IOTestAwsS3Client, AppendExtraSSECHeaders) ASSERT_TRUE(client); DB::ReadSettings read_settings; - DB::S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = max_single_read_retries; DB::ReadBufferFromS3 read_buffer( client, uri.bucket, uri.key, version_id, - request_settings, + max_single_read_retries, read_settings ); diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 5af09275df4..859f5ce796b 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -573,14 +573,7 @@ public: /// AWS API tries credentials providers one by one. Some of providers (like ProfileConfigFileAWSCredentialsProvider) can be /// quite verbose even if nobody configured them. So we use our provider first and only after it use default providers. { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( - configuration.region, - configuration.remote_host_filter, - configuration.s3_max_redirects, - configuration.enable_s3_requests_logging, - configuration.for_disk_s3, - configuration.get_request_throttler, - configuration.put_request_throttler); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); AddProvider(std::make_shared(aws_client_configuration)); } @@ -617,14 +610,7 @@ public: } else if (Aws::Utils::StringUtils::ToLower(ec2_metadata_disabled.c_str()) != "true") { - DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration( - configuration.region, - configuration.remote_host_filter, - configuration.s3_max_redirects, - configuration.enable_s3_requests_logging, - configuration.for_disk_s3, - configuration.get_request_throttler, - configuration.put_request_throttler); + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects, configuration.enable_s3_requests_logging, configuration.for_disk_s3); /// See MakeDefaultHttpResourceClientConfiguration(). /// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside @@ -745,18 +731,9 @@ namespace S3 const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, bool enable_s3_requests_logging, - bool for_disk_s3, - const ThrottlerPtr & get_request_throttler, - const ThrottlerPtr & put_request_throttler) + bool for_disk_s3) { - return PocoHTTPClientConfiguration( - force_region, - remote_host_filter, - s3_max_redirects, - enable_s3_requests_logging, - for_disk_s3, - get_request_throttler, - put_request_throttler); + return PocoHTTPClientConfiguration(force_region, remote_host_filter, s3_max_redirects, enable_s3_requests_logging, for_disk_s3); } URI::URI(const Poco::URI & uri_) diff --git a/src/IO/S3Common.h b/src/IO/S3Common.h index 01a6b8d5d82..93e5eb78c7f 100644 --- a/src/IO/S3Common.h +++ b/src/IO/S3Common.h @@ -17,7 +17,6 @@ #include #include -#include namespace Aws::S3 { @@ -89,9 +88,7 @@ public: const RemoteHostFilter & remote_host_filter, unsigned int s3_max_redirects, bool enable_s3_requests_logging, - bool for_disk_s3, - const ThrottlerPtr & get_request_throttler, - const ThrottlerPtr & put_request_throttler); + bool for_disk_s3); private: ClientFactory(); diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index 56d487f165b..9ed2c41fd01 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -71,7 +71,7 @@ WriteBufferFromS3::WriteBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::RequestSettings & request_settings_, + const S3Settings::ReadWriteSettings & s3_settings_, std::optional> object_metadata_, size_t buffer_size_, ThreadPoolCallbackRunner schedule_, @@ -79,10 +79,10 @@ WriteBufferFromS3::WriteBufferFromS3( : BufferWithOwnMemory(buffer_size_, nullptr, 0) , bucket(bucket_) , key(key_) - , request_settings(request_settings_) + , s3_settings(s3_settings_) , client_ptr(std::move(client_ptr_)) , object_metadata(std::move(object_metadata_)) - , upload_part_size(request_settings_.min_upload_part_size) + , upload_part_size(s3_settings_.min_upload_part_size) , schedule(std::move(schedule_)) , write_settings(write_settings_) { @@ -107,7 +107,7 @@ void WriteBufferFromS3::nextImpl() write_settings.remote_throttler->add(offset()); /// Data size exceeds singlepart upload threshold, need to use multipart upload. - if (multipart_upload_id.empty() && last_part_size > request_settings.max_single_part_upload_size) + if (multipart_upload_id.empty() && last_part_size > s3_settings.max_single_part_upload_size) createMultipartUpload(); if (!multipart_upload_id.empty() && last_part_size > upload_part_size) @@ -122,10 +122,10 @@ void WriteBufferFromS3::nextImpl() void WriteBufferFromS3::allocateBuffer() { - if (total_parts_uploaded != 0 && total_parts_uploaded % request_settings.upload_part_size_multiply_parts_count_threshold == 0) + if (total_parts_uploaded != 0 && total_parts_uploaded % s3_settings.upload_part_size_multiply_parts_count_threshold == 0) { - upload_part_size *= request_settings.upload_part_size_multiply_factor; - upload_part_size = std::min(upload_part_size, request_settings.max_upload_part_size); + upload_part_size *= s3_settings.upload_part_size_multiply_factor; + upload_part_size = std::min(upload_part_size, s3_settings.max_upload_part_size); } temporary_buffer = Aws::MakeShared("temporary buffer"); @@ -180,7 +180,7 @@ void WriteBufferFromS3::finalizeImpl() if (!multipart_upload_id.empty()) completeMultipartUpload(); - if (request_settings.check_objects_after_upload) + if (s3_settings.check_objects_after_upload) { LOG_TRACE(log, "Checking object {} exists after upload", key); @@ -370,7 +370,7 @@ void WriteBufferFromS3::completeMultipartUpload() req.SetMultipartUpload(multipart_upload); - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(s3_settings.max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3CompleteMultipartUpload); @@ -476,7 +476,7 @@ void WriteBufferFromS3::fillPutRequest(Aws::S3::Model::PutObjectRequest & req) void WriteBufferFromS3::processPutRequest(const PutObjectTask & task) { - size_t max_retry = std::max(request_settings.max_unexpected_write_error_retries, 1UL); + size_t max_retry = std::max(s3_settings.max_unexpected_write_error_retries, 1UL); for (size_t i = 0; i < max_retry; ++i) { ProfileEvents::increment(ProfileEvents::S3PutObject); diff --git a/src/IO/WriteBufferFromS3.h b/src/IO/WriteBufferFromS3.h index b4b5a6d37a3..28f831856d7 100644 --- a/src/IO/WriteBufferFromS3.h +++ b/src/IO/WriteBufferFromS3.h @@ -50,7 +50,7 @@ public: std::shared_ptr client_ptr_, const String & bucket_, const String & key_, - const S3Settings::RequestSettings & request_settings_, + const S3Settings::ReadWriteSettings & s3_settings_, std::optional> object_metadata_ = std::nullopt, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, ThreadPoolCallbackRunner schedule_ = {}, @@ -88,7 +88,7 @@ private: const String bucket; const String key; - const S3Settings::RequestSettings request_settings; + const S3Settings::ReadWriteSettings s3_settings; const std::shared_ptr client_ptr; const std::optional> object_metadata; diff --git a/src/Storages/ExternalDataSourceConfiguration.h b/src/Storages/ExternalDataSourceConfiguration.h index a8c27e3d1d4..5736336983a 100644 --- a/src/Storages/ExternalDataSourceConfiguration.h +++ b/src/Storages/ExternalDataSourceConfiguration.h @@ -118,7 +118,7 @@ struct URLBasedDataSourceConfiguration struct StorageS3Configuration : URLBasedDataSourceConfiguration { S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; + S3Settings::ReadWriteSettings rw_settings; }; diff --git a/src/Storages/StorageDelta.cpp b/src/Storages/StorageDelta.cpp index c077b24c610..e8287a2fd61 100644 --- a/src/Storages/StorageDelta.cpp +++ b/src/Storages/StorageDelta.cpp @@ -151,14 +151,12 @@ std::vector JsonMetadataGetter::getJsonLogFiles() std::shared_ptr JsonMetadataGetter::createS3ReadBuffer(const String & key, ContextPtr context) { /// TODO: add parallel downloads - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = 10; return std::make_shared( base_configuration.client, base_configuration.uri.bucket, key, base_configuration.uri.version_id, - request_settings, + /* max single read retries */10, context->getReadSettings()); } @@ -189,7 +187,7 @@ StorageDelta::StorageDelta( ContextPtr context_, std::optional format_settings_) : IStorage(table_id_) - , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageDeltaLake (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) { diff --git a/src/Storages/StorageHudi.cpp b/src/Storages/StorageHudi.cpp index 4b20e4cbd22..121856c4a57 100644 --- a/src/Storages/StorageHudi.cpp +++ b/src/Storages/StorageHudi.cpp @@ -37,7 +37,7 @@ StorageHudi::StorageHudi( ContextPtr context_, std::optional format_settings_) : IStorage(table_id_) - , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} + , base_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , log(&Poco::Logger::get("StorageHudi (" + table_id_.table_name + ")")) , table_path(base_configuration.uri.key) { diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 7b575d0d12f..ab9b71f5ff3 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -100,8 +100,7 @@ public: const Block & virtual_header_, ContextPtr context_, std::unordered_map * object_infos_, - Strings * read_keys_, - const S3Settings::RequestSettings & request_settings_) + Strings * read_keys_) : WithContext(context_) , client(client_) , globbed_uri(globbed_uri_) @@ -109,7 +108,6 @@ public: , virtual_header(virtual_header_) , object_infos(object_infos_) , read_keys(read_keys_) - , request_settings(request_settings_) { if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -260,7 +258,6 @@ private: bool is_finished{false}; std::unordered_map * object_infos; Strings * read_keys; - S3Settings::RequestSettings request_settings; }; StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( @@ -270,9 +267,8 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator( const Block & virtual_header, ContextPtr context, std::unordered_map * object_infos_, - Strings * read_keys_, - const S3Settings::RequestSettings & request_settings_) - : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_, request_settings_)) + Strings * read_keys_) + : pimpl(std::make_shared(client_, globbed_uri_, query, virtual_header, context, object_infos_, read_keys_)) { } @@ -385,7 +381,7 @@ StorageS3Source::StorageS3Source( std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, + UInt64 max_single_read_retries_, String compression_hint_, const std::shared_ptr & client_, const String & bucket_, @@ -401,7 +397,7 @@ StorageS3Source::StorageS3Source( , format(format_) , columns_desc(columns_) , max_block_size(max_block_size_) - , request_settings(request_settings_) + , max_single_read_retries(max_single_read_retries_) , compression_hint(std::move(compression_hint_)) , client(client_) , sample_block(sample_block_) @@ -467,7 +463,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k if (!use_parallel_download || object_too_small) { LOG_TRACE(log, "Downloading object of size {} from S3 in single thread", object_size); - return std::make_unique(client, bucket, key, version_id, request_settings, getContext()->getReadSettings()); + return std::make_unique(client, bucket, key, version_id, max_single_read_retries, getContext()->getReadSettings()); } assert(object_size > 0); @@ -479,7 +475,7 @@ std::unique_ptr StorageS3Source::createS3ReadBuffer(const String & k } auto factory = std::make_unique( - client, bucket, key, version_id, download_buffer_size, object_size, request_settings, getContext()->getReadSettings()); + client, bucket, key, version_id, download_buffer_size, object_size, max_single_read_retries, getContext()->getReadSettings()); LOG_TRACE( log, "Downloading from S3 in {} threads. Object size: {}, Range size: {}.", download_thread_num, object_size, download_buffer_size); @@ -589,7 +585,7 @@ public: s3_configuration_.client, bucket, key, - s3_configuration_.request_settings, + s3_configuration_.rw_settings, std::nullopt, DBMS_DEFAULT_BUFFER_SIZE, threadPoolCallbackRunner(IOThreadPool::get(), "S3ParallelRead"), @@ -753,7 +749,7 @@ StorageS3::StorageS3( bool distributed_processing_, ASTPtr partition_by_) : IStorage(table_id_) - , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} + , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , keys({s3_configuration.uri.key}) , format_name(configuration_.format) , compression_method(configuration_.compression_method) @@ -819,7 +815,7 @@ std::shared_ptr StorageS3::createFileIterator( { /// Iterate through disclosed globs and make a source for each file auto glob_iterator = std::make_shared( - *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys, s3_configuration.request_settings); + *s3_configuration.client, s3_configuration.uri, query, virtual_block, local_context, object_infos, read_keys); return std::make_shared([glob_iterator]() { return glob_iterator->next(); }); } else @@ -909,7 +905,7 @@ Pipe StorageS3::read( format_settings, columns_description, max_block_size, - s3_configuration.request_settings, + s3_configuration.rw_settings.max_single_read_retries, compression_method, s3_configuration.client, s3_configuration.uri.bucket, @@ -1026,10 +1022,12 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration & upd) { auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString()); - if (upd.request_settings != settings.request_settings) - upd.request_settings = settings.request_settings; + const auto & config_rw_settings = settings.rw_settings; - upd.request_settings.updateFromSettingsIfEmpty(ctx->getSettings()); + if (upd.rw_settings != config_rw_settings) + upd.rw_settings = settings.rw_settings; + + upd.rw_settings.updateFromSettingsIfEmpty(ctx->getSettings()); if (upd.client) { @@ -1047,12 +1045,10 @@ void StorageS3::updateS3Configuration(ContextPtr ctx, StorageS3::S3Configuration ctx->getRemoteHostFilter(), static_cast(ctx->getGlobalContext()->getSettingsRef().s3_max_redirects), ctx->getGlobalContext()->getSettingsRef().enable_s3_requests_logging, - /* for_disk_s3 = */ false, - upd.request_settings.get_request_throttler, - upd.request_settings.put_request_throttler); + /* for_disk_s3 = */ false); client_configuration.endpointOverride = upd.uri.endpoint; - client_configuration.maxConnections = static_cast(upd.request_settings.max_connections); + client_configuration.maxConnections = static_cast(upd.rw_settings.max_connections); auto credentials = Aws::Auth::AWSCredentials(upd.auth_settings.access_key_id, upd.auth_settings.secret_access_key); auto headers = upd.auth_settings.headers; @@ -1084,17 +1080,17 @@ void StorageS3::processNamedCollectionResult(StorageS3Configuration & configurat else if (arg_name == "use_environment_credentials") configuration.auth_settings.use_environment_credentials = checkAndGetLiteralArgument(arg_value, "use_environment_credentials"); else if (arg_name == "max_single_read_retries") - configuration.request_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_read_retries"); else if (arg_name == "min_upload_part_size") - configuration.request_settings.min_upload_part_size = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "min_upload_part_size"); else if (arg_name == "upload_part_size_multiply_factor") - configuration.request_settings.upload_part_size_multiply_factor = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_factor"); else if (arg_name == "upload_part_size_multiply_parts_count_threshold") - configuration.request_settings.upload_part_size_multiply_parts_count_threshold = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "upload_part_size_multiply_parts_count_threshold"); else if (arg_name == "max_single_part_upload_size") - configuration.request_settings.max_single_part_upload_size = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_single_part_upload_size"); else if (arg_name == "max_connections") - configuration.request_settings.max_connections = checkAndGetLiteralArgument(arg_value, "max_connections"); + configuration.rw_settings.max_single_read_retries = checkAndGetLiteralArgument(arg_value, "max_connections"); else throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Unknown key-value argument `{}` for StorageS3, expected: url, [access_key_id, secret_access_key], name of used format and [compression_method].", @@ -1170,7 +1166,7 @@ ColumnsDescription StorageS3::getTableStructureFromData( S3Configuration s3_configuration{ configuration.url, configuration.auth_settings, - S3Settings::RequestSettings(ctx->getSettingsRef()), + S3Settings::ReadWriteSettings(ctx->getSettingsRef()), configuration.headers}; updateS3Configuration(ctx, s3_configuration); @@ -1232,7 +1228,7 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( int zstd_window_log_max = static_cast(ctx->getSettingsRef().zstd_window_log_max); return wrapReadBufferWithCompressionMethod( std::make_unique( - s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.request_settings, ctx->getReadSettings()), + s3_configuration.client, s3_configuration.uri.bucket, key, s3_configuration.uri.version_id, s3_configuration.rw_settings.max_single_read_retries, ctx->getReadSettings()), chooseCompressionMethod(key, compression_method), zstd_window_log_max); }; diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 47ac26abccb..81bbe2c86ae 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -43,8 +43,7 @@ public: const Block & virtual_header, ContextPtr context, std::unordered_map * object_infos = nullptr, - Strings * read_keys_ = nullptr, - const S3Settings::RequestSettings & request_settings_ = {}); + Strings * read_keys_ = nullptr); String next(); @@ -80,7 +79,7 @@ public: std::optional format_settings_, const ColumnsDescription & columns_, UInt64 max_block_size_, - const S3Settings::RequestSettings & request_settings_, + UInt64 max_single_read_retries_, String compression_hint_, const std::shared_ptr & client_, const String & bucket, @@ -103,7 +102,7 @@ private: String format; ColumnsDescription columns_desc; UInt64 max_block_size; - S3Settings::RequestSettings request_settings; + UInt64 max_single_read_retries; String compression_hint; std::shared_ptr client; Block sample_block; @@ -187,7 +186,7 @@ public: std::shared_ptr client; S3::AuthSettings auth_settings; - S3Settings::RequestSettings request_settings; + S3Settings::ReadWriteSettings rw_settings; /// If s3 configuration was passed from ast, then it is static. /// If from config - it can be changed with config reload. @@ -199,11 +198,11 @@ public: S3Configuration( const String & url_, const S3::AuthSettings & auth_settings_, - const S3Settings::RequestSettings & request_settings_, + const S3Settings::ReadWriteSettings & rw_settings_, const HeaderCollection & headers_from_ast_) : uri(S3::URI(url_)) , auth_settings(auth_settings_) - , request_settings(request_settings_) + , rw_settings(rw_settings_) , static_configuration(!auth_settings_.access_key_id.empty()) , headers_from_ast(headers_from_ast_) {} }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index ec970654b6e..350e942f972 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -46,7 +46,7 @@ StorageS3Cluster::StorageS3Cluster( const ConstraintsDescription & constraints_, ContextPtr context_) : IStorage(table_id_) - , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.request_settings, configuration_.headers} + , s3_configuration{configuration_.url, configuration_.auth_settings, configuration_.rw_settings, configuration_.headers} , filename(configuration_.url) , cluster_name(configuration_.cluster_name) , format_name(configuration_.format) diff --git a/src/Storages/StorageS3Settings.cpp b/src/Storages/StorageS3Settings.cpp index 962f7c113bd..68e15d10f52 100644 --- a/src/Storages/StorageS3Settings.cpp +++ b/src/Storages/StorageS3Settings.cpp @@ -4,7 +4,6 @@ #include #include -#include #include #include #include @@ -58,26 +57,18 @@ void StorageS3Settings::loadFromConfig(const String & config_elem, const Poco::U auto auth_settings = S3::AuthSettings::loadFromConfig(config_elem + "." + key, config); - S3Settings::RequestSettings request_settings; - request_settings.max_single_read_retries = get_uint_for_key(key, "max_single_read_retries", true, settings.s3_max_single_read_retries); - request_settings.min_upload_part_size = get_uint_for_key(key, "min_upload_part_size", true, settings.s3_min_upload_part_size); - request_settings.max_upload_part_size = get_uint_for_key(key, "max_upload_part_size", true, DEFAULT_MAX_UPLOAD_PART_SIZE); - request_settings.upload_part_size_multiply_factor = get_uint_for_key(key, "upload_part_size_multiply_factor", true, settings.s3_upload_part_size_multiply_factor); - request_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); - request_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); - request_settings.max_single_operation_copy_size = get_uint_for_key(key, "max_single_operation_copy_size", true, DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE); - request_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); - request_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); + S3Settings::ReadWriteSettings rw_settings; + rw_settings.max_single_read_retries = get_uint_for_key(key, "max_single_read_retries", true, settings.s3_max_single_read_retries); + rw_settings.min_upload_part_size = get_uint_for_key(key, "min_upload_part_size", true, settings.s3_min_upload_part_size); + rw_settings.max_upload_part_size = get_uint_for_key(key, "max_upload_part_size", true, DEFAULT_MAX_UPLOAD_PART_SIZE); + rw_settings.upload_part_size_multiply_factor = get_uint_for_key(key, "upload_part_size_multiply_factor", true, settings.s3_upload_part_size_multiply_factor); + rw_settings.upload_part_size_multiply_parts_count_threshold = get_uint_for_key(key, "upload_part_size_multiply_parts_count_threshold", true, settings.s3_upload_part_size_multiply_parts_count_threshold); + rw_settings.max_single_part_upload_size = get_uint_for_key(key, "max_single_part_upload_size", true, settings.s3_max_single_part_upload_size); + rw_settings.max_single_operation_copy_size = get_uint_for_key(key, "max_single_operation_copy_size", true, DEFAULT_MAX_SINGLE_OPERATION_COPY_SIZE); + rw_settings.max_connections = get_uint_for_key(key, "max_connections", true, settings.s3_max_connections); + rw_settings.check_objects_after_upload = get_bool_for_key(key, "check_objects_after_upload", true, false); - // NOTE: it would be better to reuse old throttlers to avoid losing token bucket state on every config reload, which could lead to exceeding limit for short time. But it is good enough unless very high `burst` values are used. - if (UInt64 max_get_rps = get_uint_for_key(key, "max_get_rps", true, settings.s3_max_get_rps)) - request_settings.get_request_throttler = std::make_shared( - max_get_rps, get_uint_for_key(key, "max_get_burst", true, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * max_get_rps)); - if (UInt64 max_put_rps = get_uint_for_key(key, "max_put_rps", true, settings.s3_max_put_rps)) - request_settings.put_request_throttler = std::make_shared( - max_put_rps, get_uint_for_key(key, "max_put_burst", true, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * max_put_rps)); - - s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(request_settings)}); + s3_settings.emplace(endpoint, S3Settings{std::move(auth_settings), std::move(rw_settings)}); } } } @@ -98,7 +89,7 @@ S3Settings StorageS3Settings::getSettings(const String & endpoint) const return {}; } -S3Settings::RequestSettings::RequestSettings(const Settings & settings) +S3Settings::ReadWriteSettings::ReadWriteSettings(const Settings & settings) { max_single_read_retries = settings.s3_max_single_read_retries; min_upload_part_size = settings.s3_min_upload_part_size; @@ -108,15 +99,9 @@ S3Settings::RequestSettings::RequestSettings(const Settings & settings) max_connections = settings.s3_max_connections; check_objects_after_upload = settings.s3_check_objects_after_upload; max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; - if (settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - if (settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); } -void S3Settings::RequestSettings::updateFromSettingsIfEmpty(const Settings & settings) +void S3Settings::ReadWriteSettings::updateFromSettingsIfEmpty(const Settings & settings) { if (!max_single_read_retries) max_single_read_retries = settings.s3_max_single_read_retries; @@ -137,12 +122,6 @@ void S3Settings::RequestSettings::updateFromSettingsIfEmpty(const Settings & set if (!max_unexpected_write_error_retries) max_unexpected_write_error_retries = settings.s3_max_unexpected_write_error_retries; check_objects_after_upload = settings.s3_check_objects_after_upload; - if (!get_request_throttler && settings.s3_max_get_rps) - get_request_throttler = std::make_shared( - settings.s3_max_get_rps, settings.s3_max_get_burst ? settings.s3_max_get_burst : Throttler::default_burst_seconds * settings.s3_max_get_rps); - if (!put_request_throttler && settings.s3_max_put_rps) - put_request_throttler = std::make_shared( - settings.s3_max_put_rps, settings.s3_max_put_burst ? settings.s3_max_put_burst : Throttler::default_burst_seconds * settings.s3_max_put_rps); } } diff --git a/src/Storages/StorageS3Settings.h b/src/Storages/StorageS3Settings.h index 955cd2d025b..bd90ba569d8 100644 --- a/src/Storages/StorageS3Settings.h +++ b/src/Storages/StorageS3Settings.h @@ -7,7 +7,6 @@ #include #include #include -#include #include #include @@ -24,7 +23,7 @@ struct Settings; struct S3Settings { - struct RequestSettings + struct ReadWriteSettings { size_t max_single_read_retries = 0; size_t min_upload_part_size = 0; @@ -36,13 +35,11 @@ struct S3Settings size_t max_connections = 0; bool check_objects_after_upload = false; size_t max_unexpected_write_error_retries = 0; - ThrottlerPtr get_request_throttler; - ThrottlerPtr put_request_throttler; - RequestSettings() = default; - explicit RequestSettings(const Settings & settings); + ReadWriteSettings() = default; + explicit ReadWriteSettings(const Settings & settings); - inline bool operator==(const RequestSettings & other) const + inline bool operator==(const ReadWriteSettings & other) const { return max_single_read_retries == other.max_single_read_retries && min_upload_part_size == other.min_upload_part_size @@ -53,20 +50,18 @@ struct S3Settings && max_single_operation_copy_size == other.max_single_operation_copy_size && max_connections == other.max_connections && check_objects_after_upload == other.check_objects_after_upload - && max_unexpected_write_error_retries == other.max_unexpected_write_error_retries - && get_request_throttler == other.get_request_throttler - && put_request_throttler == other.put_request_throttler; + && max_unexpected_write_error_retries == other.max_unexpected_write_error_retries; } void updateFromSettingsIfEmpty(const Settings & settings); }; S3::AuthSettings auth_settings; - RequestSettings request_settings; + ReadWriteSettings rw_settings; inline bool operator==(const S3Settings & other) const { - return auth_settings == other.auth_settings && request_settings == other.request_settings; + return auth_settings == other.auth_settings && rw_settings == other.rw_settings; } }; diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.reference b/tests/queries/0_stateless/02477_s3_request_throttler.reference deleted file mode 100644 index 9315e86b328..00000000000 --- a/tests/queries/0_stateless/02477_s3_request_throttler.reference +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 1 1 diff --git a/tests/queries/0_stateless/02477_s3_request_throttler.sql b/tests/queries/0_stateless/02477_s3_request_throttler.sql deleted file mode 100644 index 25653b1bab0..00000000000 --- a/tests/queries/0_stateless/02477_s3_request_throttler.sql +++ /dev/null @@ -1,26 +0,0 @@ --- Tags: no-fasttest --- Tag no-fasttest: needs s3 - --- Limit S3 PUT request per second rate -SET s3_max_put_rps = 2; -SET s3_max_put_burst = 1; - -CREATE TEMPORARY TABLE times (t DateTime); - --- INSERT query requires 3 PUT requests and 1/rps = 0.5 second in between, the first query is not throttled due to burst -INSERT INTO times SELECT now(); -INSERT INTO TABLE FUNCTION s3('http://localhost:11111/test/request-throttler.csv', 'test', 'testtest', 'CSV', 'number UInt64') SELECT number FROM numbers(1000000) SETTINGS s3_max_single_part_upload_size = 10000, s3_truncate_on_insert = 1; -INSERT INTO times SELECT now(); - -SELECT max(t) - min(t) >= 1 FROM times; - -SYSTEM FLUSH LOGS; -SELECT ProfileEvents['S3CreateMultipartUpload'] == 1, - ProfileEvents['S3UploadPart'] == 1, - ProfileEvents['S3CompleteMultipartUpload'] == 1 -FROM system.query_log -WHERE query LIKE '%request-throttler.csv%' -AND type = 'QueryFinish' -AND current_database = currentDatabase() -ORDER BY query_start_time DESC -LIMIT 1; From 93dde0b3f922efdb9882eb7c9f2b50eda693a1a3 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Wed, 16 Nov 2022 16:18:20 -0500 Subject: [PATCH 508/526] Update docs for Date32 --- docs/en/sql-reference/data-types/date32.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/date32.md b/docs/en/sql-reference/data-types/date32.md index c8c7470d2cb..db41fdf2fc3 100644 --- a/docs/en/sql-reference/data-types/date32.md +++ b/docs/en/sql-reference/data-types/date32.md @@ -6,7 +6,7 @@ sidebar_label: Date32 # Date32 -A date. Supports the date range same with [DateTime64](../../sql-reference/data-types/datetime64.md). Stored in four bytes as the number of days since 1900-01-01. Allows storing values till 2299-12-31. +A date. Supports the date range same with [DateTime64](../../sql-reference/data-types/datetime64.md). Stored as a signed 32-bit integer in native byte order with the value representing the days since 1970-01-01 (0 represents 1970-01-01 and negative values represent the days before 1970). **Examples** From f4eb430caad8d5e5410b29ab0d5267e0048e8961 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Nov 2022 03:14:22 +0100 Subject: [PATCH 509/526] Revert "remove changes" This reverts commit 4016fd2efcb196a97bfc157f661acf6c165f1fa8. --- CHANGELOG.md | 109 +++++++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 106 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 68767612892..051bde44dd2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,4 +1,5 @@ ### Table of Contents +**[ClickHouse release v22.11, 2022-11-17](#2211)**
**[ClickHouse release v22.10, 2022-10-25](#2210)**
**[ClickHouse release v22.9, 2022-09-22](#229)**
**[ClickHouse release v22.8-lts, 2022-08-18](#228)**
@@ -11,6 +12,108 @@ **[ClickHouse release v22.1, 2022-01-18](#221)**
**[Changelog for 2021](https://clickhouse.com/docs/en/whats-new/changelog/2021/)**
+### ClickHouse release 22.11, 2022-11-17 + +#### Backward Incompatible Change +* `JSONExtract` family of functions will now attempt to coerce to the requested type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). + +#### New Feature +* Adds support for retries during INSERTs into ReplicatedMergeTree when a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). +* Add `Hudi` and `DeltaLake` table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do), [Kseniia Sumarokova](https://github.com/kssenii)). +* Add table function `hudi` and `deltaLake`. [#43080](https://github.com/ClickHouse/ClickHouse/pull/43080) ([flynn](https://github.com/ucasfl)). +* Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In the case where the types of Intervals are different, they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added `**` glob support for recursive directory traversal of the filesystem and S3. Resolves [#36316](https://github.com/ClickHouse/ClickHouse/issues/36316). [#42376](https://github.com/ClickHouse/ClickHouse/pull/42376) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Introduce `s3_plain` disk type for write-once-read-many operations. Implement `ATTACH` of `MergeTree` table for `s3_plain` disk. [#42628](https://github.com/ClickHouse/ClickHouse/pull/42628) ([Azat Khuzhin](https://github.com/azat)). +* Added applied row-level policies to `system.query_log`. [#39819](https://github.com/ClickHouse/ClickHouse/pull/39819) ([Vladimir Chebotaryov](https://github.com/quickhouse)). +* Add four-letter command `csnp` for manually creating snapshots in ClickHouse Keeper. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). +* Add function `ascii` like in Apache Spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). +* Add function `pmod` which returns non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). +* Add function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). +* Add function `randCanonical`, which is similar to the `rand` function in Apache Spark or Impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). +* Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). +* Add `min_age_to_force_merge_on_partition_only` setting to optimize old parts for the entire partition only. [#42659](https://github.com/ClickHouse/ClickHouse/pull/42659) ([Antonio Andelic](https://github.com/antonio2368)). +* Add generic implementation for arbitrary structured named collections, access type and `system.named_collections`. [#43147](https://github.com/ClickHouse/ClickHouse/pull/43147) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Performance Improvement +* `match` function can use the index if it's a condition on string prefix. This closes [#37333](https://github.com/ClickHouse/ClickHouse/issues/37333). [#42458](https://github.com/ClickHouse/ClickHouse/pull/42458) ([clarkcaoliu](https://github.com/Clark0)). +* Speed up AND and OR operators when they are sequenced. [#42214](https://github.com/ClickHouse/ClickHouse/pull/42214) ([Zhiguo Zhou](https://github.com/ZhiguoZh)). +* Support parallel parsing for `LineAsString` input format. This improves performance just slightly. This closes [#42502](https://github.com/ClickHouse/ClickHouse/issues/42502). [#42780](https://github.com/ClickHouse/ClickHouse/pull/42780) ([Kruglov Pavel](https://github.com/Avogar)). +* ClickHouse Keeper performance improvement: improve commit performance for cases when many different nodes have uncommitted states. This should help with cases when a follower node can't sync fast enough. [#42926](https://github.com/ClickHouse/ClickHouse/pull/42926) ([Antonio Andelic](https://github.com/antonio2368)). +* A condition like `NOT LIKE 'prefix%'` can use the primary index. [#42209](https://github.com/ClickHouse/ClickHouse/pull/42209) ([Duc Canh Le](https://github.com/canhld94)). + +#### Experimental Feature +* Support type `Object` inside other types, e.g. `Array(JSON)`. [#36969](https://github.com/ClickHouse/ClickHouse/pull/36969) ([Anton Popov](https://github.com/CurtizJ)). +* Ignore MySQL binlog SAVEPOINT event for MaterializedMySQL. [#42931](https://github.com/ClickHouse/ClickHouse/pull/42931) ([zzsmdfj](https://github.com/zzsmdfj)). Handle (ignore) SAVEPOINT queries in MaterializedMySQL. [#43086](https://github.com/ClickHouse/ClickHouse/pull/43086) ([Stig Bakken](https://github.com/stigsb)). + +#### Improvement +* Trivial queries with small LIMIT will properly determine the number of estimated rows to read, so that the threshold will be checked properly. Closes [#7071](https://github.com/ClickHouse/ClickHouse/issues/7071). [#42580](https://github.com/ClickHouse/ClickHouse/pull/42580) ([Han Fei](https://github.com/hanfei1991)). +* Add support for interactive parameters in INSERT VALUES queries. [#43077](https://github.com/ClickHouse/ClickHouse/pull/43077) ([Nikolay Degterinsky](https://github.com/evillique)). +* Added new field `allow_readonly` in `system.table_functions` to allow using table functions in readonly mode. Resolves [#42414](https://github.com/ClickHouse/ClickHouse/issues/42414) Implementation: * Added a new field allow_readonly to table system.table_functions. * Updated to use new field allow_readonly to allow using table functions in readonly mode. Testing: * Added a test for filesystem tests/queries/0_stateless/02473_functions_in_readonly_mode.sh Documentation: * Updated the english documentation for Table Functions. [#42708](https://github.com/ClickHouse/ClickHouse/pull/42708) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* The `system.asynchronous_metrics` gets embedded documentation. This documentation is also exported to Prometheus. Fixed an error with the metrics about `cache` disks - they were calculated only for one arbitrary cache disk instead all of them. This closes [#7644](https://github.com/ClickHouse/ClickHouse/issues/7644). [#43194](https://github.com/ClickHouse/ClickHouse/pull/43194) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Throttling algorithm changed to token bucket. [#42665](https://github.com/ClickHouse/ClickHouse/pull/42665) ([Sergei Trifonov](https://github.com/serxa)). +* Mask passwords and secret keys both in `system.query_log` and `/var/log/clickhouse-server/*.log` and also in error messages. [#42484](https://github.com/ClickHouse/ClickHouse/pull/42484) ([Vitaly Baranov](https://github.com/vitlibar)). +* Remove covered parts for fetched part (to avoid possible replication delay grows). [#39737](https://github.com/ClickHouse/ClickHouse/pull/39737) ([Azat Khuzhin](https://github.com/azat)). +* If `/dev/tty` is available, the progress in clickhouse-client and clickhouse-local will be rendered directly to the terminal, without writing to STDERR. It allows getting progress even if STDERR is redirected to a file, and the file will not be polluted by terminal escape sequences. The progress can be disabled by `--progress false`. This closes [#32238](https://github.com/ClickHouse/ClickHouse/issues/32238). [#42003](https://github.com/ClickHouse/ClickHouse/pull/42003) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add support for `FixedString` input to base64 coding functions. [#42285](https://github.com/ClickHouse/ClickHouse/pull/42285) ([ltrk2](https://github.com/ltrk2)). +* Add columns `bytes_on_disk` and `path` to `system.detached_parts`. Closes [#42264](https://github.com/ClickHouse/ClickHouse/issues/42264). [#42303](https://github.com/ClickHouse/ClickHouse/pull/42303) ([chen](https://github.com/xiedeyantu)). +* Improve using structure from insertion table in table functions, now setting `use_structure_from_insertion_table_in_table_functions` has new possible value - `2` that means that ClickHouse will try to determine if we can use structure from insertion table or not automatically. Closes [#40028](https://github.com/ClickHouse/ClickHouse/issues/40028). [#42320](https://github.com/ClickHouse/ClickHouse/pull/42320) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix no progress indication on INSERT FROM INFILE. Closes [#42548](https://github.com/ClickHouse/ClickHouse/issues/42548). [#42634](https://github.com/ClickHouse/ClickHouse/pull/42634) ([chen](https://github.com/xiedeyantu)). +* Refactor function `tokens` to enable max tokens returned for related functions (disabled by default). [#42673](https://github.com/ClickHouse/ClickHouse/pull/42673) ([李扬](https://github.com/taiyang-li)). +* Allow to use `Date32` arguments for `formatDateTime` and `FROM_UNIXTIME` functions. [#42737](https://github.com/ClickHouse/ClickHouse/pull/42737) ([Roman Vasin](https://github.com/rvasin)). +* Update tzdata to 2022f. Mexico will no longer observe DST except near the US border: https://www.timeanddate.com/news/time/mexico-abolishes-dst-2022.html. Chihuahua moves to year-round UTC-6 on 2022-10-30. Fiji no longer observes DST. See https://github.com/google/cctz/pull/235 and https://bugs.launchpad.net/ubuntu/+source/tzdata/+bug/1995209. [#42796](https://github.com/ClickHouse/ClickHouse/pull/42796) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Add `FailedAsyncInsertQuery` event metric for async inserts. [#42814](https://github.com/ClickHouse/ClickHouse/pull/42814) ([Krzysztof Góralski](https://github.com/kgoralski)). +* Implement `read-in-order` optimization on top of query plan. It is enabled by default. Set `query_plan_read_in_order = 0` to use previous AST-based version. [#42829](https://github.com/ClickHouse/ClickHouse/pull/42829) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Increase the size of upload part exponentially for backup to S3 to avoid errors about max 10 000 parts limit of the multipart upload to s3. [#42833](https://github.com/ClickHouse/ClickHouse/pull/42833) ([Vitaly Baranov](https://github.com/vitlibar)). +* When the merge task is continuously busy and the disk space is insufficient, the completely expired parts cannot be selected and dropped, resulting in insufficient disk space. My idea is that when the entire Part expires, there is no need for additional disk space to guarantee, ensure the normal execution of TTL. [#42869](https://github.com/ClickHouse/ClickHouse/pull/42869) ([zhongyuankai](https://github.com/zhongyuankai)). +* Add `oss` function and `OSS` table engine (this is convenient for users). oss is fully compatible with s3. [#43155](https://github.com/ClickHouse/ClickHouse/pull/43155) ([zzsmdfj](https://github.com/zzsmdfj)). +* Improve error reporting in the collection of OS-related info for the `system.asynchronous_metrics` table. [#43192](https://github.com/ClickHouse/ClickHouse/pull/43192) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Modify the `INFORMATION_SCHEMA` tables in a way so that ClickHouse can connect to itself using the MySQL compatibility protocol. Add columns instead of aliases (related to [#9769](https://github.com/ClickHouse/ClickHouse/issues/9769)). It will improve the compatibility with various MySQL clients. [#43198](https://github.com/ClickHouse/ClickHouse/pull/43198) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Add some functions for compatibility with PowerBI, when it connects using MySQL protocol [#42612](https://github.com/ClickHouse/ClickHouse/pull/42612) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Better usability for Dashboard on changes [#42872](https://github.com/ClickHouse/ClickHouse/pull/42872) ([Vladimir C](https://github.com/vdimir)). + +#### Build/Testing/Packaging Improvement +* Run SQLancer for each pull request and commit to master. [SQLancer](https://github.com/sqlancer/sqlancer) is an OpenSource fuzzer that focuses on automatic detection of logical bugs. [#42397](https://github.com/ClickHouse/ClickHouse/pull/42397) ([Ilya Yatsishin](https://github.com/qoega)). +* Update to latest zlib-ng. [#42463](https://github.com/ClickHouse/ClickHouse/pull/42463) ([Boris Kuschel](https://github.com/bkuschel)). +* Add support for testing ClickHouse server with Jepsen. By the way, we already have support for testing ClickHouse Keeper with Jepsen. This pull request extends it to Replicated tables. [#42619](https://github.com/ClickHouse/ClickHouse/pull/42619) ([Antonio Andelic](https://github.com/antonio2368)). +* Use https://github.com/matus-chochlik/ctcache for clang-tidy results caching. [#42913](https://github.com/ClickHouse/ClickHouse/pull/42913) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Before the fix, the user-defined config was preserved by RPM in `$file.rpmsave`. The PR fixes it and won't replace the user's files from packages. [#42936](https://github.com/ClickHouse/ClickHouse/pull/42936) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Remove some libraries from Ubuntu Docker image. [#42622](https://github.com/ClickHouse/ClickHouse/pull/42622) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Updated normaliser to clone the alias ast. Resolves [#42452](https://github.com/ClickHouse/ClickHouse/issues/42452) Implementation: * Updated QueryNormalizer to clone alias ast, when its replaced. Previously just assigning the same leads to exception in LogicalExpressinsOptimizer as it would be the same parent being inserted again. * This bug is not seen with new analyser (allow_experimental_analyzer), so no changes for it. I added a test for the same. [#42827](https://github.com/ClickHouse/ClickHouse/pull/42827) ([SmitaRKulkarni](https://github.com/SmitaRKulkarni)). +* Fix race for backup of tables in `Lazy` databases. [#43104](https://github.com/ClickHouse/ClickHouse/pull/43104) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fix for `skip_unavailable_shards`: it did not work with the `s3Cluster` table function. [#43131](https://github.com/ClickHouse/ClickHouse/pull/43131) ([chen](https://github.com/xiedeyantu)). +* Fix schema inference in `s3Cluster` and improvement in `hdfsCluster`. [#41979](https://github.com/ClickHouse/ClickHouse/pull/41979) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix retries while reading from URL table engines / table function. (retriable errors could be retries more times than needed, non-retriable errors resulted in failed assertion in code). [#42224](https://github.com/ClickHouse/ClickHouse/pull/42224) ([Kseniia Sumarokova](https://github.com/kssenii)). +* A segmentation fault related to DNS & c-ares has been reported and fixed. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). +* Fix `LOGICAL_ERROR` `Arguments of 'plus' have incorrect data types` which may happen in PK analysis (monotonicity check). Fix invalid PK analysis for monotonic binary functions with first constant argument. [#42410](https://github.com/ClickHouse/ClickHouse/pull/42410) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix incorrect key analysis when key types cannot be inside Nullable. This fixes [#42456](https://github.com/ClickHouse/ClickHouse/issues/42456). [#42469](https://github.com/ClickHouse/ClickHouse/pull/42469) ([Amos Bird](https://github.com/amosbird)). +* Fix typo in a setting name that led to bad usage of schema inference cache while using setting `input_format_csv_use_best_effort_in_schema_inference`. Closes [#41735](https://github.com/ClickHouse/ClickHouse/issues/41735). [#42536](https://github.com/ClickHouse/ClickHouse/pull/42536) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix creating a Set with wrong header when data type is LowCardinality. Closes [#42460](https://github.com/ClickHouse/ClickHouse/issues/42460). [#42579](https://github.com/ClickHouse/ClickHouse/pull/42579) ([flynn](https://github.com/ucasfl)). +* `(U)Int128` and `(U)Int256` values were correctly checked in `PREWHERE`. [#42605](https://github.com/ClickHouse/ClickHouse/pull/42605) ([Antonio Andelic](https://github.com/antonio2368)). +* Fix a bug in functions parser that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix the locking in `truncate table`. [#42728](https://github.com/ClickHouse/ClickHouse/pull/42728) ([flynn](https://github.com/ucasfl)). +* Fix possible crash in `web` disks when file does not exist (or `OPTIMIZE TABLE FINAL`, that also can got the same error eventually). [#42767](https://github.com/ClickHouse/ClickHouse/pull/42767) ([Azat Khuzhin](https://github.com/azat)). +* Fix `auth_type` mapping in `system.session_log`, by including `SSL_CERTIFICATE` for the enum values. [#42782](https://github.com/ClickHouse/ClickHouse/pull/42782) ([Miel Donkers](https://github.com/mdonkers)). +* Fix stack-use-after-return under ASAN build in the Create User query parser. [#42804](https://github.com/ClickHouse/ClickHouse/pull/42804) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix `lowerUTF8`/`upperUTF8` in case of symbol was in between 16-byte boundary (very frequent case of you have strings > 16 bytes long). [#42812](https://github.com/ClickHouse/ClickHouse/pull/42812) ([Azat Khuzhin](https://github.com/azat)). +* Additional bound check was added to LZ4 decompression routine to fix misbehaviour in case of malformed input. [#42868](https://github.com/ClickHouse/ClickHouse/pull/42868) ([Nikita Taranov](https://github.com/nickitat)). +* Fix rare possible hang on query cancellation. [#42874](https://github.com/ClickHouse/ClickHouse/pull/42874) ([Azat Khuzhin](https://github.com/azat)). +* Fix incorrect behavior with multiple disjuncts in hash join, close [#42832](https://github.com/ClickHouse/ClickHouse/issues/42832). [#42876](https://github.com/ClickHouse/ClickHouse/pull/42876) ([Vladimir C](https://github.com/vdimir)). +* A null pointer will be generated when select if as from ‘three table join’ , For example, this SQL query: [#42883](https://github.com/ClickHouse/ClickHouse/pull/42883) ([zzsmdfj](https://github.com/zzsmdfj)). +* Fix memory sanitizer report in Cluster Discovery, close [#42763](https://github.com/ClickHouse/ClickHouse/issues/42763). [#42905](https://github.com/ClickHouse/ClickHouse/pull/42905) ([Vladimir C](https://github.com/vdimir)). +* Improve DateTime schema inference in case of empty string. [#42911](https://github.com/ClickHouse/ClickHouse/pull/42911) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix rare NOT_FOUND_COLUMN_IN_BLOCK error when projection is possible to use but there is no projection available. This fixes [#42771](https://github.com/ClickHouse/ClickHouse/issues/42771) . The bug was introduced in https://github.com/ClickHouse/ClickHouse/pull/25563. [#42938](https://github.com/ClickHouse/ClickHouse/pull/42938) ([Amos Bird](https://github.com/amosbird)). +* Fix ATTACH TABLE in `PostgreSQL` database engine if the table contains DATETIME data type. Closes [#42817](https://github.com/ClickHouse/ClickHouse/issues/42817). [#42960](https://github.com/ClickHouse/ClickHouse/pull/42960) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fix lambda parsing. Closes [#41848](https://github.com/ClickHouse/ClickHouse/issues/41848). [#42979](https://github.com/ClickHouse/ClickHouse/pull/42979) ([Nikolay Degterinsky](https://github.com/evillique)). +* Fix incorrect key analysis when nullable keys appear in the middle of a hyperrectangle. This fixes [#43111](https://github.com/ClickHouse/ClickHouse/issues/43111) . [#43133](https://github.com/ClickHouse/ClickHouse/pull/43133) ([Amos Bird](https://github.com/amosbird)). +* Fix several buffer over-reads in deserialization of carefully crafted aggregate function states. [#43159](https://github.com/ClickHouse/ClickHouse/pull/43159) ([Raúl Marín](https://github.com/Algunenano)). +* Fix function `if` in case of NULL and const Nullable arguments. Closes [#43069](https://github.com/ClickHouse/ClickHouse/issues/43069). [#43178](https://github.com/ClickHouse/ClickHouse/pull/43178) ([Kruglov Pavel](https://github.com/Avogar)). +* Fix decimal math overflow in parsing DateTime with the 'best effort' algorithm. Closes [#43061](https://github.com/ClickHouse/ClickHouse/issues/43061). [#43180](https://github.com/ClickHouse/ClickHouse/pull/43180) ([Kruglov Pavel](https://github.com/Avogar)). +* The `indent` field produced by the `git-import` tool was miscalculated. See https://clickhouse.com/docs/en/getting-started/example-datasets/github/. [#43191](https://github.com/ClickHouse/ClickHouse/pull/43191) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Fixed unexpected behaviour of `Interval` types with subquery and casting. [#43193](https://github.com/ClickHouse/ClickHouse/pull/43193) ([jh0x](https://github.com/jh0x)). + ### ClickHouse release 22.10, 2022-10-26 #### Backward Incompatible Change @@ -570,7 +673,7 @@ * Support SQL standard CREATE INDEX and DROP INDEX syntax. [#35166](https://github.com/ClickHouse/ClickHouse/pull/35166) ([Jianmei Zhang](https://github.com/zhangjmruc)). * Send profile events for INSERT queries (previously only SELECT was supported). [#37391](https://github.com/ClickHouse/ClickHouse/pull/37391) ([Azat Khuzhin](https://github.com/azat)). * Implement in order aggregation (`optimize_aggregation_in_order`) for fully materialized projections. [#37469](https://github.com/ClickHouse/ClickHouse/pull/37469) ([Azat Khuzhin](https://github.com/azat)). -* Remove subprocess run for kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). +* Remove subprocess run for Kerberos initialization. Added new integration test. Closes [#27651](https://github.com/ClickHouse/ClickHouse/issues/27651). [#38105](https://github.com/ClickHouse/ClickHouse/pull/38105) ([Roman Vasin](https://github.com/rvasin)). * * Add setting `multiple_joins_try_to_keep_original_names` to not rewrite identifier name on multiple JOINs rewrite, close [#34697](https://github.com/ClickHouse/ClickHouse/issues/34697). [#38149](https://github.com/ClickHouse/ClickHouse/pull/38149) ([Vladimir C](https://github.com/vdimir)). * Improved trace-visualizer UX. [#38169](https://github.com/ClickHouse/ClickHouse/pull/38169) ([Sergei Trifonov](https://github.com/serxa)). * Enable stack trace collection and query profiler for AArch64. [#38181](https://github.com/ClickHouse/ClickHouse/pull/38181) ([Maksim Kita](https://github.com/kitaisreal)). @@ -850,8 +953,8 @@ #### Upgrade Notes -* Now, background merges, mutations and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values, and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes of metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. +* Now, background merges, mutations, and `OPTIMIZE` will not increment `SelectedRows` and `SelectedBytes` metrics. They (still) will increment `MergedRows` and `MergedUncompressedBytes` as it was before. This only affects the metric values and makes them better. This change does not introduce any incompatibility, but you may wonder about the changes to the metrics, so we put in this category. [#37040](https://github.com/ClickHouse/ClickHouse/pull/37040) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Updated the BoringSSL module to the official FIPS compliant version. This makes ClickHouse FIPS compliant in this area. [#35914](https://github.com/ClickHouse/ClickHouse/pull/35914) ([Meena-Renganathan](https://github.com/Meena-Renganathan)). The ciphers `aes-192-cfb128` and `aes-256-cfb128` were removed, because they are not included in the FIPS certified version of BoringSSL. * `max_memory_usage` setting is removed from the default user profile in `users.xml`. This enables flexible memory limits for queries instead of the old rigid limit of 10 GB. * Disable `log_query_threads` setting by default. It controls the logging of statistics about every thread participating in query execution. After supporting asynchronous reads, the total number of distinct thread ids became too large, and logging into the `query_thread_log` has become too heavy. [#37077](https://github.com/ClickHouse/ClickHouse/pull/37077) ([Alexey Milovidov](https://github.com/alexey-milovidov)). * Remove function `groupArraySorted` which has a bug. [#36822](https://github.com/ClickHouse/ClickHouse/pull/36822) ([Alexey Milovidov](https://github.com/alexey-milovidov)). From aa855d86a12c048359d68fb88ea9c55b5c5168a4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Nov 2022 05:51:20 +0300 Subject: [PATCH 510/526] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 051bde44dd2..9fc3a589374 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -18,7 +18,7 @@ * `JSONExtract` family of functions will now attempt to coerce to the requested type. [#41502](https://github.com/ClickHouse/ClickHouse/pull/41502) ([Márcio Martins](https://github.com/marcioapm)). #### New Feature -* Adds support for retries during INSERTs into ReplicatedMergeTree when a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). +* Adds support for retries during INSERTs into ReplicatedMergeTree when a session with ClickHouse Keeper is lost. Apart from fault tolerance, it aims to provide better user experience, - avoid returning a user an error during insert if keeper is restarted (for example, due to upgrade). This is controlled by the `insert_keeper_max_retries` setting, which is disabled by default. [#42607](https://github.com/ClickHouse/ClickHouse/pull/42607) ([Igor Nikonov](https://github.com/devcrafter)). * Add `Hudi` and `DeltaLake` table engines, read-only, only for tables on S3. [#41054](https://github.com/ClickHouse/ClickHouse/pull/41054) ([Daniil Rubin](https://github.com/rubin-do), [Kseniia Sumarokova](https://github.com/kssenii)). * Add table function `hudi` and `deltaLake`. [#43080](https://github.com/ClickHouse/ClickHouse/pull/43080) ([flynn](https://github.com/ucasfl)). * Support for composite time intervals. 1. Add, subtract and negate operations are now available on Intervals. In the case where the types of Intervals are different, they will be transformed into the Tuple of those types. 2. A tuple of intervals can be added to or subtracted from a Date/DateTime field. 3. Added parsing of Intervals with different types, for example: `INTERVAL '1 HOUR 1 MINUTE 1 SECOND'`. [#42195](https://github.com/ClickHouse/ClickHouse/pull/42195) ([Nikolay Degterinsky](https://github.com/evillique)). From c67f5e26002a0deaeb4d35c0734f8507eab04031 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 17 Nov 2022 04:40:09 +0100 Subject: [PATCH 511/526] Update gtest_column_dump_structure.cpp --- src/Columns/tests/gtest_column_dump_structure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/tests/gtest_column_dump_structure.cpp b/src/Columns/tests/gtest_column_dump_structure.cpp index 69769ba0db9..e00c77798c8 100644 --- a/src/Columns/tests/gtest_column_dump_structure.cpp +++ b/src/Columns/tests/gtest_column_dump_structure.cpp @@ -10,7 +10,7 @@ TEST(IColumn, dumpStructure) { auto type_lc = std::make_shared(std::make_shared()); ColumnPtr column_lc = type_lc->createColumn(); - auto expected_structure = "ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1)))"; + String expected_structure = "ColumnLowCardinality(size = 0, UInt8(size = 0), ColumnUnique(size = 1, String(size = 1)))"; std::vector threads; for (size_t i = 0; i < 6; ++i) From 10d7ce98154e3532f36072f331dd90973571f1a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 17 Nov 2022 05:41:03 +0100 Subject: [PATCH 512/526] pmod: compatibility with Spark, better documentation --- CHANGELOG.md | 2 +- src/Functions/modulo.cpp | 12 +++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 051bde44dd2..034ba26897e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -27,7 +27,7 @@ * Added applied row-level policies to `system.query_log`. [#39819](https://github.com/ClickHouse/ClickHouse/pull/39819) ([Vladimir Chebotaryov](https://github.com/quickhouse)). * Add four-letter command `csnp` for manually creating snapshots in ClickHouse Keeper. Additionally, `lgif` was added to get Raft information for a specific node (e.g. index of last created snapshot, last committed log index). [#41766](https://github.com/ClickHouse/ClickHouse/pull/41766) ([JackyWoo](https://github.com/JackyWoo)). * Add function `ascii` like in Apache Spark: https://spark.apache.org/docs/latest/api/sql/#ascii. [#42670](https://github.com/ClickHouse/ClickHouse/pull/42670) ([李扬](https://github.com/taiyang-li)). -* Add function `pmod` which returns non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). +* Add function `positive_modulo` (`pmod`) which returns non-negative result based on modulo. [#42755](https://github.com/ClickHouse/ClickHouse/pull/42755) ([李扬](https://github.com/taiyang-li)). * Add function `formatReadableDecimalSize`. [#42774](https://github.com/ClickHouse/ClickHouse/pull/42774) ([Alejandro](https://github.com/alexon1234)). * Add function `randCanonical`, which is similar to the `rand` function in Apache Spark or Impala. The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). [#43124](https://github.com/ClickHouse/ClickHouse/pull/43124) ([李扬](https://github.com/taiyang-li)). * Add function `displayName`, closes [#36770](https://github.com/ClickHouse/ClickHouse/issues/36770). [#37681](https://github.com/ClickHouse/ClickHouse/pull/37681) ([hongbin](https://github.com/xlwh)). diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index 9cd104cd1dc..be052b25af4 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -182,7 +182,7 @@ REGISTER_FUNCTION(ModuloLegacy) struct NamePositiveModulo { - static constexpr auto name = "positive_modulo"; + static constexpr auto name = "positiveModulo"; }; using FunctionPositiveModulo = BinaryArithmeticOverloadResolver; @@ -191,11 +191,17 @@ REGISTER_FUNCTION(PositiveModulo) factory.registerFunction( { R"( -Calculates the remainder when dividing `a` by `b`. Similar to function `modulo` except that `positive_modulo` always return non-negative number. +Calculates the remainder when dividing `a` by `b`. Similar to function `modulo` except that `positiveModulo` always return non-negative number. +Returns the difference between `a` and the nearest integer not greater than `a` divisible by `b`. +In other words, the function returning the modulus (modulo) in the terms of Modular Arithmetic. )", - Documentation::Examples{{"positive_modulo", "SELECT positive_modulo(-1000, 32);"}}, + Documentation::Examples{{"positiveModulo", "SELECT positiveModulo(-1, 10);"}}, Documentation::Categories{"Arithmetic"}}, FunctionFactory::CaseInsensitive); + + factory.registerAlias("positive_modulo", "positiveModulo", FunctionFactory::CaseInsensitive); + /// Compatibility with Spark: + factory.registerAlias("pmod", "positiveModulo", FunctionFactory::CaseInsensitive); } } From 41b557f926cadbed6e62020064ae6d419d33d001 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 17 Nov 2022 09:16:47 +0000 Subject: [PATCH 513/526] Disable DeltaLake and hudi table functions in readonly mode --- .../{TableFunctionDelta.cpp => TableFunctionDeltaLake.cpp} | 6 +++--- .../{TableFunctionDelta.h => TableFunctionDeltaLake.h} | 0 src/TableFunctions/TableFunctionHudi.cpp | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) rename src/TableFunctions/{TableFunctionDelta.cpp => TableFunctionDeltaLake.cpp} (96%) rename src/TableFunctions/{TableFunctionDelta.h => TableFunctionDeltaLake.h} (100%) diff --git a/src/TableFunctions/TableFunctionDelta.cpp b/src/TableFunctions/TableFunctionDeltaLake.cpp similarity index 96% rename from src/TableFunctions/TableFunctionDelta.cpp rename to src/TableFunctions/TableFunctionDeltaLake.cpp index 25ea2aaa77f..0f5801d57ac 100644 --- a/src/TableFunctions/TableFunctionDelta.cpp +++ b/src/TableFunctions/TableFunctionDeltaLake.cpp @@ -13,7 +13,7 @@ # include # include # include -# include +# include # include # include "registerTableFunctions.h" @@ -160,9 +160,9 @@ void registerTableFunctionDelta(TableFunctionFactory & factory) factory.registerFunction( {.documentation = {R"(The table function can be used to read the DeltaLake table stored on object store.)", - Documentation::Examples{{"hudi", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)"}}, + Documentation::Examples{{"deltaLake", "SELECT * FROM deltaLake(url, access_key_id, secret_access_key)"}}, Documentation::Categories{"DataLake"}}, - .allow_readonly = true}); + .allow_readonly = false}); } } diff --git a/src/TableFunctions/TableFunctionDelta.h b/src/TableFunctions/TableFunctionDeltaLake.h similarity index 100% rename from src/TableFunctions/TableFunctionDelta.h rename to src/TableFunctions/TableFunctionDeltaLake.h diff --git a/src/TableFunctions/TableFunctionHudi.cpp b/src/TableFunctions/TableFunctionHudi.cpp index b1db90da550..2e27d192b58 100644 --- a/src/TableFunctions/TableFunctionHudi.cpp +++ b/src/TableFunctions/TableFunctionHudi.cpp @@ -162,7 +162,7 @@ void registerTableFunctionHudi(TableFunctionFactory & factory) = {R"(The table function can be used to read the Hudi table stored on object store.)", Documentation::Examples{{"hudi", "SELECT * FROM hudi(url, access_key_id, secret_access_key)"}}, Documentation::Categories{"DataLake"}}, - .allow_readonly = true}); + .allow_readonly = false}); } } From 11b535d443f12504c79a48203e86d029005f04e6 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Nov 2022 11:17:27 +0100 Subject: [PATCH 514/526] impl (#43283) --- docs/en/sql-reference/functions/random-functions.md | 2 +- src/Functions/canonicalRand.cpp | 4 ++-- tests/queries/0_stateless/01047_nullable_rand.sql | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 08f2620a009..4efa2131eb6 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -24,7 +24,7 @@ Returns a pseudo-random UInt64 number, evenly distributed among all UInt64-type Uses a linear congruential generator. -## canonicalRand +## randCanonical The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). Non-deterministic. Return type is Float64. diff --git a/src/Functions/canonicalRand.cpp b/src/Functions/canonicalRand.cpp index d0b8c655e14..0f168142177 100644 --- a/src/Functions/canonicalRand.cpp +++ b/src/Functions/canonicalRand.cpp @@ -34,7 +34,7 @@ private: struct NameCanonicalRand { - static constexpr auto name = "canonicalRand"; + static constexpr auto name = "randCanonical"; }; class FunctionCanonicalRand : public FunctionRandomImpl @@ -52,7 +52,7 @@ REGISTER_FUNCTION(CanonicalRand) The function generates pseudo random results with independent and identically distributed uniformly distributed values in [0, 1). Non-deterministic. Return type is Float64. )", - Documentation::Examples{{"canonicalRand", "SELECT canonicalRand()"}}, + Documentation::Examples{{"randCanonical", "SELECT randCanonical()"}}, Documentation::Categories{"Mathematical"}}); } diff --git a/tests/queries/0_stateless/01047_nullable_rand.sql b/tests/queries/0_stateless/01047_nullable_rand.sql index 9d3c361c543..e5633637db6 100644 --- a/tests/queries/0_stateless/01047_nullable_rand.sql +++ b/tests/queries/0_stateless/01047_nullable_rand.sql @@ -1,13 +1,13 @@ select toTypeName(rand(cast(4 as Nullable(UInt8)))); -select toTypeName(canonicalRand(CAST(4 as Nullable(UInt8)))); +select toTypeName(randCanonical(CAST(4 as Nullable(UInt8)))); select toTypeName(randConstant(CAST(4 as Nullable(UInt8)))); select toTypeName(rand(Null)); -select toTypeName(canonicalRand(Null)); +select toTypeName(randCanonical(Null)); select toTypeName(randConstant(Null)); select rand(cast(4 as Nullable(UInt8))) * 0; -select canonicalRand(cast(4 as Nullable(UInt8))) * 0; +select randCanonical(cast(4 as Nullable(UInt8))) * 0; select randConstant(CAST(4 as Nullable(UInt8))) * 0; select rand(Null) * 0; -select canonicalRand(Null) * 0; +select randCanonical(Null) * 0; select randConstant(Null) * 0; From 7beb58b0cf2cd04fdf7548ce0574bbb1ec6280a5 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Thu, 17 Nov 2022 13:19:02 +0100 Subject: [PATCH 515/526] Optimize merge of uniqExact without_key (#43072) * impl for uniqExact * rm unused (read|write)Text methods * fix style * small fixes * impl for variadic uniqExact * refactor * fix style * more agressive inlining * disable if max_threads=1 * small improvements * review fixes * Revert "rm unused (read|write)Text methods" This reverts commit a7e74805842572f6fa2c28ea111ab8ca7c19ad21. * encapsulate is_able_to_parallelize_merge in Data * encapsulate is_exact & argument_is_tuple in Data --- .../AggregateFunctionUniq.cpp | 56 ++-- .../AggregateFunctionUniq.h | 307 +++++++++++++++--- src/AggregateFunctions/Helpers.h | 13 + src/AggregateFunctions/IAggregateFunction.h | 15 +- src/AggregateFunctions/UniqExactSet.h | 112 +++++++ src/AggregateFunctions/UniquesHashSet.h | 2 +- src/Common/HashTable/HashSet.h | 57 ++++ src/Common/HashTable/HashTable.h | 20 +- src/Common/HashTable/TwoLevelHashTable.h | 8 +- src/Common/examples/small_table.cpp | 2 +- src/Common/tests/gtest_hash_table.cpp | 56 ++++ src/Interpreters/Aggregator.cpp | 12 +- .../test_aggregate_function_state.py | 228 +++++++++++++ .../test_aggregate_function_state_avg.py | 82 ----- tests/performance/uniq_without_key.xml | 33 ++ 15 files changed, 832 insertions(+), 171 deletions(-) create mode 100644 src/AggregateFunctions/UniqExactSet.h create mode 100644 tests/integration/test_backward_compatibility/test_aggregate_function_state.py delete mode 100644 tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py create mode 100644 tests/performance/uniq_without_key.xml diff --git a/src/AggregateFunctions/AggregateFunctionUniq.cpp b/src/AggregateFunctions/AggregateFunctionUniq.cpp index 0d1c831c839..1c90767131c 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniq.cpp @@ -9,6 +9,7 @@ #include #include +#include namespace DB { @@ -28,8 +29,9 @@ namespace /** `DataForVariadic` is a data structure that will be used for `uniq` aggregate function of multiple arguments. * It differs, for example, in that it uses a trivial hash function, since `uniq` of many arguments first hashes them out itself. */ -template -AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) +template typename DataForVariadic> +AggregateFunctionPtr +createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { assertNoParameters(name, params); @@ -61,21 +63,22 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const else if (which.isTuple()) { if (use_exact_hash_function) - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); else - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); } } /// "Variadic" method also works as a fallback generic case for single argument. if (use_exact_hash_function) - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); else - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); } -template class Data, typename DataForVariadic> -AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) +template typename Data, template typename DataForVariadic, bool is_able_to_parallelize_merge> +AggregateFunctionPtr +createAggregateFunctionUniq(const std::string & name, const DataTypes & argument_types, const Array & params, const Settings *) { assertNoParameters(name, params); @@ -91,35 +94,35 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const { const IDataType & argument_type = *argument_types[0]; - AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types)); + AggregateFunctionPtr res(createWithNumericType(*argument_types[0], argument_types)); WhichDataType which(argument_type); if (res) return res; else if (which.isDate()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isDate32()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isDateTime()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isStringOrFixedString()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isUUID()) - return std::make_shared>>(argument_types); + return std::make_shared>>(argument_types); else if (which.isTuple()) { if (use_exact_hash_function) - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); else - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); } } /// "Variadic" method also works as a fallback generic case for single argument. if (use_exact_hash_function) - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); else - return std::make_shared>(argument_types); + return std::make_shared>>(argument_types); } } @@ -132,14 +135,23 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory & factory) {createAggregateFunctionUniq, properties}); factory.registerFunction("uniqHLL12", - {createAggregateFunctionUniq, properties}); + {createAggregateFunctionUniq, properties}); - factory.registerFunction("uniqExact", - {createAggregateFunctionUniq>, properties}); + auto assign_bool_param = [](const std::string & name, const DataTypes & argument_types, const Array & params, const Settings * settings) + { + /// Using two level hash set if we wouldn't be able to merge in parallel can cause ~10% slowdown. + if (settings && settings->max_threads > 1) + return createAggregateFunctionUniq< + true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactDataForVariadic, true /* is_able_to_parallelize_merge */>(name, argument_types, params, settings); + else + return createAggregateFunctionUniq< + true, AggregateFunctionUniqExactData, AggregateFunctionUniqExactDataForVariadic, false /* is_able_to_parallelize_merge */>(name, argument_types, params, settings); + }; + factory.registerFunction("uniqExact", {assign_bool_param, properties}); #if USE_DATASKETCHES factory.registerFunction("uniqTheta", - {createAggregateFunctionUniq, properties}); + {createAggregateFunctionUniq, properties}); #endif } diff --git a/src/AggregateFunctions/AggregateFunctionUniq.h b/src/AggregateFunctions/AggregateFunctionUniq.h index fe2530800cc..1a98bfc8456 100644 --- a/src/AggregateFunctions/AggregateFunctionUniq.h +++ b/src/AggregateFunctions/AggregateFunctionUniq.h @@ -1,7 +1,10 @@ #pragma once -#include +#include +#include #include +#include +#include #include @@ -13,17 +16,18 @@ #include +#include #include #include #include -#include -#include #include +#include -#include #include #include +#include #include +#include namespace DB @@ -37,94 +41,128 @@ struct AggregateFunctionUniqUniquesHashSetData using Set = UniquesHashSet>; Set set; + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + static String getName() { return "uniq"; } }; /// For a function that takes multiple arguments. Such a function pre-hashes them in advance, so TrivialHash is used here. +template struct AggregateFunctionUniqUniquesHashSetDataForVariadic { using Set = UniquesHashSet; Set set; + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = true; + constexpr static bool is_exact = is_exact_; + constexpr static bool argument_is_tuple = argument_is_tuple_; + static String getName() { return "uniq"; } }; /// uniqHLL12 -template +template struct AggregateFunctionUniqHLL12Data { using Set = HyperLogLogWithSmallSetOptimization; Set set; - static String getName() { return "uniqHLL12"; } -}; - -template <> -struct AggregateFunctionUniqHLL12Data -{ - using Set = HyperLogLogWithSmallSetOptimization; - Set set; + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; static String getName() { return "uniqHLL12"; } }; template <> -struct AggregateFunctionUniqHLL12Data +struct AggregateFunctionUniqHLL12Data { using Set = HyperLogLogWithSmallSetOptimization; Set set; + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + static String getName() { return "uniqHLL12"; } }; +template <> +struct AggregateFunctionUniqHLL12Data +{ + using Set = HyperLogLogWithSmallSetOptimization; + Set set; + + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + + static String getName() { return "uniqHLL12"; } +}; + +template struct AggregateFunctionUniqHLL12DataForVariadic { using Set = HyperLogLogWithSmallSetOptimization; Set set; + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = true; + constexpr static bool is_exact = is_exact_; + constexpr static bool argument_is_tuple = argument_is_tuple_; + static String getName() { return "uniqHLL12"; } }; /// uniqExact -template +template struct AggregateFunctionUniqExactData { using Key = T; /// When creating, the hash table must be small. - using Set = HashSet< - Key, - HashCRC32, - HashTableGrower<4>, - HashTableAllocatorWithStackMemory>; + using SingleLevelSet = HashSet, HashTableGrower<4>, HashTableAllocatorWithStackMemory>; + using TwoLevelSet = TwoLevelHashSet>; + using Set = UniqExactSet; Set set; + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; + static String getName() { return "uniqExact"; } }; /// For rows, we put the SipHash values (128 bits) into the hash table. -template <> -struct AggregateFunctionUniqExactData +template +struct AggregateFunctionUniqExactData { using Key = UInt128; /// When creating, the hash table must be small. - using Set = HashSet< - Key, - UInt128TrivialHash, - HashTableGrower<3>, - HashTableAllocatorWithStackMemory>; + using SingleLevelSet = HashSet, HashTableAllocatorWithStackMemory>; + using TwoLevelSet = TwoLevelHashSet; + using Set = UniqExactSet; Set set; + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = false; + static String getName() { return "uniqExact"; } }; +template +struct AggregateFunctionUniqExactDataForVariadic : AggregateFunctionUniqExactData +{ + constexpr static bool is_able_to_parallelize_merge = is_able_to_parallelize_merge_; + constexpr static bool is_variadic = true; + constexpr static bool is_exact = is_exact_; + constexpr static bool argument_is_tuple = argument_is_tuple_; +}; /// uniqTheta #if USE_DATASKETCHES @@ -134,14 +172,37 @@ struct AggregateFunctionUniqThetaData using Set = ThetaSketchData; Set set; + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = false; + static String getName() { return "uniqTheta"; } }; +template +struct AggregateFunctionUniqThetaDataForVariadic : AggregateFunctionUniqThetaData +{ + constexpr static bool is_able_to_parallelize_merge = false; + constexpr static bool is_variadic = true; + constexpr static bool is_exact = is_exact_; + constexpr static bool argument_is_tuple = argument_is_tuple_; +}; + #endif namespace detail { +template +struct IsUniqExactSet : std::false_type +{ +}; + +template +struct IsUniqExactSet> : std::true_type +{ +}; + + /** Hash function for uniq. */ template struct AggregateFunctionUniqTraits @@ -162,17 +223,31 @@ template struct AggregateFunctionUniqTraits }; -/** The structure for the delegation work to add one element to the `uniq` aggregate functions. +/** The structure for the delegation work to add elements to the `uniq` aggregate functions. * Used for partial specialization to add strings. */ template -struct OneAdder +struct Adder { - static void ALWAYS_INLINE add(Data & data, const IColumn & column, size_t row_num) + /// We have to introduce this template parameter (and a bunch of ugly code dealing with it), because we cannot + /// add runtime branches in whatever_hash_set::insert - it will immediately pop up in the perf top. + template + static void ALWAYS_INLINE add(Data & data, const IColumn ** columns, size_t num_args, size_t row_num) { - if constexpr (std::is_same_v - || std::is_same_v>) + if constexpr (Data::is_variadic) { + if constexpr (IsUniqExactSet::value) + data.set.template insert( + UniqVariadicHash::apply(num_args, columns, row_num)); + else + data.set.insert(T{UniqVariadicHash::apply(num_args, columns, row_num)}); + } + else if constexpr ( + std::is_same_v< + Data, + AggregateFunctionUniqUniquesHashSetData> || std::is_same_v>) + { + const auto & column = *columns[0]; if constexpr (!std::is_same_v) { using ValueType = typename decltype(data.set)::value_type; @@ -185,11 +260,13 @@ struct OneAdder data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size)); } } - else if constexpr (std::is_same_v>) + else if constexpr (std::is_same_v>) { + const auto & column = *columns[0]; if constexpr (!std::is_same_v) { - data.set.insert(assert_cast &>(column).getData()[row_num]); + data.set.template insert( + assert_cast &>(column).getData()[row_num]); } else { @@ -200,16 +277,72 @@ struct OneAdder hash.update(value.data, value.size); hash.get128(key); - data.set.insert(key); + data.set.template insert(key); } } #if USE_DATASKETCHES else if constexpr (std::is_same_v) { + const auto & column = *columns[0]; data.set.insertOriginal(column.getDataAt(row_num)); } #endif } + + static void ALWAYS_INLINE + add(Data & data, const IColumn ** columns, size_t num_args, size_t row_begin, size_t row_end, const char8_t * flags, const UInt8 * null_map) + { + bool use_single_level_hash_table = true; + if constexpr (Data::is_able_to_parallelize_merge) + use_single_level_hash_table = data.set.isSingleLevel(); + + if (use_single_level_hash_table) + addImpl(data, columns, num_args, row_begin, row_end, flags, null_map); + else + addImpl(data, columns, num_args, row_begin, row_end, flags, null_map); + + if constexpr (Data::is_able_to_parallelize_merge) + { + if (data.set.isSingleLevel() && data.set.size() > 100'000) + data.set.convertToTwoLevel(); + } + } + +private: + template + static void ALWAYS_INLINE + addImpl(Data & data, const IColumn ** columns, size_t num_args, size_t row_begin, size_t row_end, const char8_t * flags, const UInt8 * null_map) + { + if (!flags) + { + if (!null_map) + { + for (size_t row = row_begin; row < row_end; ++row) + add(data, columns, num_args, row); + } + else + { + for (size_t row = row_begin; row < row_end; ++row) + if (!null_map[row]) + add(data, columns, num_args, row); + } + } + else + { + if (!null_map) + { + for (size_t row = row_begin; row < row_end; ++row) + if (flags[row]) + add(data, columns, num_args, row); + } + else + { + for (size_t row = row_begin; row < row_end; ++row) + if (!null_map[row] && flags[row]) + add(data, columns, num_args, row); + } + } + } }; } @@ -219,9 +352,15 @@ struct OneAdder template class AggregateFunctionUniq final : public IAggregateFunctionDataHelper> { +private: + static constexpr size_t num_args = 1; + static constexpr bool is_able_to_parallelize_merge = Data::is_able_to_parallelize_merge; + public: - AggregateFunctionUniq(const DataTypes & argument_types_) - : IAggregateFunctionDataHelper>(argument_types_, {}) {} + explicit AggregateFunctionUniq(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper>(argument_types_, {}) + { + } String getName() const override { return Data::getName(); } @@ -235,7 +374,18 @@ public: /// ALWAYS_INLINE is required to have better code layout for uniqHLL12 function void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - detail::OneAdder::add(this->data(place), *columns[0], row_num); + detail::Adder::add(this->data(place), columns, num_args, row_num); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) + const override + { + const char8_t * flags = nullptr; + if (if_argument_pos >= 0) + flags = assert_cast(*columns[if_argument_pos]).getData().data(); + + detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, nullptr /* null_map */); } void addManyDefaults( @@ -244,7 +394,23 @@ public: size_t /*length*/, Arena * /*arena*/) const override { - detail::OneAdder::add(this->data(place), *columns[0], 0); + detail::Adder::add(this->data(place), columns, num_args, 0); + } + + void addBatchSinglePlaceNotNull( + size_t row_begin, + size_t row_end, + AggregateDataPtr __restrict place, + const IColumn ** columns, + const UInt8 * null_map, + Arena *, + ssize_t if_argument_pos) const override + { + const char8_t * flags = nullptr; + if (if_argument_pos >= 0) + flags = assert_cast(*columns[if_argument_pos]).getData().data(); + + detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, null_map); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override @@ -252,6 +418,16 @@ public: this->data(place).set.merge(this->data(rhs).set); } + bool isAbleToParallelizeMerge() const override { return is_able_to_parallelize_merge; } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena *) const override + { + if constexpr (is_able_to_parallelize_merge) + this->data(place).set.merge(this->data(rhs).set, &thread_pool); + else + this->data(place).set.merge(this->data(rhs).set); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); @@ -273,15 +449,20 @@ public: * You can pass multiple arguments as is; You can also pass one argument - a tuple. * But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples. */ -template -class AggregateFunctionUniqVariadic final : public IAggregateFunctionDataHelper> +template +class AggregateFunctionUniqVariadic final : public IAggregateFunctionDataHelper> { private: + using T = typename Data::Set::value_type; + + static constexpr size_t is_able_to_parallelize_merge = Data::is_able_to_parallelize_merge; + static constexpr size_t argument_is_tuple = Data::argument_is_tuple; + size_t num_args = 0; public: - AggregateFunctionUniqVariadic(const DataTypes & arguments) - : IAggregateFunctionDataHelper>(arguments, {}) + explicit AggregateFunctionUniqVariadic(const DataTypes & arguments) + : IAggregateFunctionDataHelper>(arguments, {}) { if (argument_is_tuple) num_args = typeid_cast(*arguments[0]).getElements().size(); @@ -300,8 +481,34 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override { - this->data(place).set.insert(typename Data::Set::value_type( - UniqVariadicHash::apply(num_args, columns, row_num))); + detail::Adder::add(this->data(place), columns, num_args, row_num); + } + + void addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena *, ssize_t if_argument_pos) + const override + { + const char8_t * flags = nullptr; + if (if_argument_pos >= 0) + flags = assert_cast(*columns[if_argument_pos]).getData().data(); + + detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, nullptr /* null_map */); + } + + void addBatchSinglePlaceNotNull( + size_t row_begin, + size_t row_end, + AggregateDataPtr __restrict place, + const IColumn ** columns, + const UInt8 * null_map, + Arena *, + ssize_t if_argument_pos) const override + { + const char8_t * flags = nullptr; + if (if_argument_pos >= 0) + flags = assert_cast(*columns[if_argument_pos]).getData().data(); + + detail::Adder::add(this->data(place), columns, num_args, row_begin, row_end, flags, null_map); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override @@ -309,6 +516,16 @@ public: this->data(place).set.merge(this->data(rhs).set); } + bool isAbleToParallelizeMerge() const override { return is_able_to_parallelize_merge; } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, ThreadPool & thread_pool, Arena *) const override + { + if constexpr (is_able_to_parallelize_merge) + this->data(place).set.merge(this->data(rhs).set, &thread_pool); + else + this->data(place).set.merge(this->data(rhs).set); + } + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { this->data(place).set.write(buf); diff --git a/src/AggregateFunctions/Helpers.h b/src/AggregateFunctions/Helpers.h index 6e140f4b9cf..c97733571a3 100644 --- a/src/AggregateFunctions/Helpers.h +++ b/src/AggregateFunctions/Helpers.h @@ -74,6 +74,19 @@ static IAggregateFunction * createWithNumericType(const IDataType & argument_typ return nullptr; } +template

p52M6S^K(d|4+F+JI5$CPu<*I=j${Vs57*b$$0Jr@7p)9 z(4dvZ7`(W>sMom626_%>77V<$^2)zQ0*KVo)i}5P*WJ(~mAXH@$Euu_N9np*eGc%L zHum;v=!CtGgcj;&jY`Bkrmw_2&O*aA3xI58<>x!5w~E{nrBbD$?jyrLky0S@cnehE zFdV@YaJSofPfq^aAU_xLueh8nLdBBSSa^Q;QvM0sm^-$ z%flqRvtqd+MVE@&oe%erE&Ap0!q*7r00^UQZpU~HSbWWsoR z-6x1|{)!WL%bC7)M;F`$?NBdtKh442ARm{#^es{EJoJJs&_@u)5K`3u2evnqSs@GZ z;$ncnwn-^aE$yNr!l==ZNwXN6u8Y3Ciij7Kfa59?Ng?xSEJZ6pQER22a4CRbrD&F5 z{`;UE8KMd9LuOz_T;>9ab7c!OrB(J*SS#PJPPMxPAT{*_>vFuaLg+gfV^B?t4`^o# zT|>J@cIKsY%zhwsA;UkX?*bpC=qT>#7l}C2)K$~nxeNMur!eVXm~`jF6q)vS-EVqv zhtg#=827knq0myG--%&|337zhBUy7xpfJ$c>05+;8xHfJTx zrgtRoYvnZ}oRjHTKk9%al(Jr!58c`2vQ3lr(sPpTF4^w#W~ICSGe^-S_`DMmhQ8*i zElJt1AvofGgdam^vJ|>)QZCSQjM;-h9BWuM1bxi(bIaq6nrX%G>Z3d@U@)TpAb@{& zGP?aRn9!p=&5bf|dfEYbOAWY@d%V3Ni6^BaFMsrg=aUQSD}i}RZzdD{8&Wmg_SV>k zLVTdj?aFOF#LowFKN>#}D{+)P?lUi6dW>jr&8()=Q~LnA`HndNm4tnn=0&N_PcMM> zw*W#)wc>G9?LI&>ybY$5Fn@Q$4Zw$DKrdTcYyg4Wq44|67<{kL$D2ZYA#()}=muVX z-V}~$xaag(Dg3Ua;jI_G@70f+6KMpGqdwr|j0>7p?X8=UXhw1W8A1O#^oI*BYmY(D zGxdspuR2eVwsQ;y#&WvYrnek-+x&8D2gIa%mpTC^>013vi8m7zz2;pHNYSbgOBdz2 z87n0LFkc!g5*+TJB%7h;>ot!3W=!rP{lvHep|INZ0#-UkSjio$f3Y}o!O zCSo`k31TqxVJ{!3Gmf8+9p(EhicZy$YT0WT_|52q01=zWjFF1~1WJRxOPf?~U&ASu zeZu@y!CLBctS>`<&F{?%2J9X zvmj9196+1vSk44Pw{*@7b$B_Hby0|`XIY{e6gsM7?1p-!wl@|Q(rt+AWQ=!#y1k0! z?D%LCkib6Rhv;!sZYIVliHNxfr_cz6ZQFN;1@9v`-iGD`88nakS^S29@jUOu>Nt#I zMfq-zn7;bNy|U%_ObE!$Yb(=_u4=N%Cl1xJJo_@)nl$%Jg5A=M4ofk(s!yeo87Dpw z-3+8o`;WKAf`n{I|N9^+Vb{_Qf20`-pX4QcqeGbeJyE}j64plZ@}3ppQxn@sn$rnM zlewE9)FJ0Y`4#)G^x*rojB`Vn_sz3YXw$hUdJ$X^o}#txqy*2Z`(f_VZkpTRQ5Qke z39bQk2aytPdXZ_99Ixr_Ie{hmoX0VWJZwS+I?l|`sn%s+mHzBg^woBh9Xxg=?un!u z*YX&vqxTafMNoU=s!WKO9n9|Xolhi>hS|%Y+@#(BdpByoO&K*>F;caUBV|0r(bB=# zFC(~Bi?AYDZ!XJ63D!maRTYd@y>Acv>{h(_b-=4;dl zy&&IDVp#z??#S?pKTe#x2hI?-cjDy#U}PaEWiA**%sBnX(iZ2gm50}Bx%lCy)q5aA zqsg6S^VzQ`Ua?OG%>VcnA&OaWiuzTc>-X`D|at~ODa{X6HnI?}hRcmM28XMgj zZSUqB!(tOzM~&uTBq6N*EUosao$q#H@pcZ4AfpCjWTxOhUjd*B?319|wEGlX5Szi& zpT3QBijwDin59x=wuxUs9!PAzCmk`| zwzI0;26ZQ(XE+s&Xf$93Qwiub7|i4gXA0y`9r(IyX~0EdbuBTmuP!X^P0kae5*S5Q zc5V$InLd4e|=!0Ct^wTFuutwAPo(~dF|=lKnl$3gQh zc3MU5Shu>lz(V+28P@^0W!RfC)k|_VBtj2|bwi>xk?vlxauD}YM+s-BVq>WPs4s80 zJQ*X4-zE>hzY5o{B`R0_f$fxFj(78sb`YBvkR;W}r(`xdV zxG^5!>24Q@lw|b>bsn8?#`VQ6fj-*3{#K+c{#P1KCEJKiM>vey=~Tw9Dc^jt{jQY9 z9xibsVh^fV1#;qUY!PcVVc+$1@)G}b)Q3^ixWCG|ABljPh;Vgh@xTL#AT!47XMUGe zt{qRK(nUP%*IARsz|*w%k)O-{)&zxx6mJ?R8M>e|g@>2LKpn&R#E5(xr~l6a9R!iH^9$e^e0SV5PvoxTD+ z2|GJrvr%A5$7wS!Mi(k&B8#h1|4>r*lTNPf#FgTadB}yDQcx7A?%sr(VH|@mxg;QkxNF<7 z`Z3-ao9sl=mj-TrBP29`80|sZ>;p)j`47-UIB4dQTn7!j>J|I6)>w^V{Sr3}O=D0s zqSzbYu|pZhXc@2davIMV<LwQuG&C?UxiI%@uPsLS{9A zaFX|StG{CiYp=65<-yA31DA8YGb*i$x^Ll&DKQdVc@n#_wWbY_vfa{AVJ6O zdYSv|kDF=yUhw4H*D0x)7nG70KgZkTd{5Tz=K+n05SePvRmRK`8u-Q|(SruiVaV`U)w7&FywzvS^pM?ltCqO3y8?6dt;4FC z!DR&;q}GmL>6Xcv^8`nyPx;C3PbVZsw=EGkE4!^`6?BT~pIAvWo;uP)#e00B`<%Ah zUJ(liZ_+mjJyJhU4;-;{u4H+o;DC-?7TzEc!!fE_Y6J0&EcfjS!3GO-7orpn&vG*!xGV9{-I%7QeuCZ@U-njq%`xmy|}Z z&|4n#4SX=$kiRAtBP#}qu~>mMs7s2$ET}(ek2xvhy1Cyz4~V!AB*j~|iLz!vng7Z- zn-0x+GQnHpo@(Xh0S{f8!g{A0tjxG$7Pulfp%p>a+?IqU?qlC{@q;~gSM?(R6)?4d zz0BUM&>?h`tNk`I*KB#5Fo3l`y&z?OjPhWyr5kZk#1kf=IVMh-ywQxz@d~N}x+;Z^ zG{)PL&V<;W>mLYuKbY3^6$P^* zP}AS)D37n0FLCAjG*QKGpaz!I zW&ieUKLUcV4FkTXt?x8drNrga+oRmN2^Zl?mtDcSjff3Vt{jz92cKSCd}oR%6G_&#Mp@&C&dZ)v zE9ue2mARI+(ZWJ7daM1VkDH$wlN+Dytbt#LN{N|fNvlG^5->sfa5>6HbVc+~VF$}H zhEE8IUG`HeY);2$qpo$*`S1gwB@0lDbwCLm`g8(5rl{m<8J(_}wDRR77e4WiT;|V^ zi6aF_zDV|-D4!Bsul}o-d3x+00ZbF$M~J13K}R@%LfIz7WSqti@LtOhpgQ^QeDMk6QGx>c;$_rgN%dDM8IGH2ZpLZ~Mza$xQ(y$$NXy{uHRQ*c^~{0cB6#k&{>6 zsudZHFSHV;%b;{CU|M<2{{BYQ8yO=Y#J*KnhC-oYypc68!Z|Q#CJX%j@{nBSQ5^H$ z5;&*4P((X(TJicj7_1Z}PnQeMDz06z0#0RUdBAwAZUXxj193SGmT^1Q1m*3PFd6=( zZ&}S&^RIXG^@1+`{rLRcVwTvGv@h=krEN-^hUt)!X%@z#R;&RADw-JFB(w?UR6RY~nQYTF!NsbKx&? zAUbxHTHZxG(U_OxObSzI#=N$W>QC%!`BX}jWV%BVN|zTk>VF!0j%X?u@`(N0x~d9$ z1fzzg$w!`3N=06_Jm{@x*va6w+ zmsiX=d($RIUytH93tUGVzp>Hk&EIOH_=LpNnBS4Sq>~FE|FpA^D(L&zr`pQz{Gg|I?{^VkO>!Dl#FR6{8VaQ>jYM+%1T~3<#Wj&7?b<9IseD#HG+D-`Z;!@IOmI;$JeO{N zw2_=4zH4@IUDCG8B3EH6)FfwH=`v5&igAm1%j+!Ct_D4`o~unKjJ;et@38fe^rgQq zRL)pZyu0Kw=9(Nuy4(0NFgNC7v`F8G_PD{j-1|$J(>oChYqxVRYq<@0x0T{m`)}^~ zXdmM<`i^G{qVJKCyVNIHkdush?WajiDv%f9ac3a-${Kxq%l!uqlShUp`A_+*D{0f% zpZv}0QDb2yXfQaTOj|%(4pQhe|IO#;)t`S{#o=x(VmW@Doe2Aal5gMwU?0U3u}+SK zZza0|b6C-nzKPX6Ze41Q;oH$K0GX6R;Tr=}mm3->sm^{bezuVq=4@sk#WV8nSxtHi z+t56n9)6;Dm-jAnPqWt2rsmUPvO`RO|W!;a#aO1BScTJCQ4KI!H)E$8U884z^LwoMOfSk9bYQM5Y{ zA3VsYbj$iz;nErB-Zt}G9$Rkz!P2d`aR$NzLxE&^_TJj}ij?9h z=Q0^SypzHrdg5MrTprt|ARu$c{)$SORRC`*>?v}xk+;dish2q zdPrK0BPx7A68@mNs$r0PQ%S~ktI|A6?=yQ7uJ8~3n$x`sfATDBdVj$nj-y8S@F z@gA>o`)f-LP-nfIyXx zHF?5xm0BpV`(vNn8w0+uidpD43eQpDW3pwuqb0ni#T(UkEq6Ue{=hEHvuyp;NG*OE z1!hgvoNu`l7sHM`1&7|Sc#Njfj>_Odz~4t}q{=g6{cn4>hj zS4kHntjT9ZV(JO!1V-Us!wi)`LY_lI*8);xzNUPHs zc~@Syv6fC;`b;vbc-R&9aj*33pbeX3HzJu8CI3cWAV&$|;Ab5jZv?yxE%>!;y=2?e zvg+8A6#?RDA(l+zOY-a41tys*y&10-KDwTqJfydIG@SapFr^o!{;>DYf8xg{cp4~- z-{8gD0~vCrP+^h{_j)T=4gcZ2l8!am_-K;WXCcugft<3vNZgvgQ|rk;xGrfYFjrp( zil-UBi?~XQ$s$VSob%M^MDI(MCo1G|)+RREZM}-N<@io7SMu=%?-h8PFyZ?Ud^Rqb z1@7Qd)%(DcpZ`&^$cA8HUW{obGKEj%oo_%+?q94&SS5YAB>+K-sAO+pY~wSAcozq2 zYNV0GutrV(2el+#6|-g{qpmN`g<6&n%d_R6Ee2(qE?#E&NlIT~cFB2DHG(ULcZKPV zb*<`oxjgyP`|@ve_zF(_EkBVp#`4576Ulz$T|5mu7Pcqeyn>9HRqqdxN1SU^&S}KN zUFjVAPdOrb=pt2otn!Zi?dN|vf3ka6o(n1&aTE5@co=$)=$?4#EBRE$e?JGw*Qg>? z+3iW*zq2WD>@Cn6pAxSjt+&s9?YsRf`xbIPdWaDE;7&i&Y|OoP_gZOqeah#FJ3_yM z+kXCPcfpcof3!tk#vA&m6+hNMNrSj)+yhV+t>#nt6KV{>-M( zFj-`TtL(gnD-un!qtk?pLbAx?NOLaTmMLbbUhig&MOvbv6dw0z-q=_1t7xZ<0Wm~} zaQecxki*8@&tLwJlO}<^3(Q;BWf)c(PPhqXVy4IxA;3JFPqS+n0E4{bM@L)UEa^RB zZcNT+TL^-6#Yq{ve56kfoIEdws@-M(h1RUwm(&aH-D+?l+vSaON2)QkLW4U$aq)|vBydK81RYgx>i6Wm;*x6HWs`l%6r zr>fpJ<<7FOCFah7>2Ky4uU?X}CNEhJjI(|}d*b23n|2yDh2;FeCtC1t#+|#j^3Fbk z3(6@kn>1uiI__ayyPnE>{jL9cdiFB|wpBy^R!2iga=*Sg2ZEZkwZfE97{YhxItAfJgSgCx%AH-g1E6m*1T+Cw#BiY?;yyCm9#Z}s zPJI~sI4=Ris7*oeZwtiGEM0}Hzye*z$;ko(#rn^6A5!jkgp8uH$h+bXPrOG5 zW{p!(X(>C_f=HDz_T2TNS;Ag>#{Ynv4#SpgHg-q0dN=SF=4y(dpxnIO(9oc99BFGK z#>8LUzLF>Ip4YA%@N!2K*D(NqQjjl#wD_{`G~m|)$b@VY5Jxgxix;#S0yO;mIn4{s z-`H9;6w&OVhq5!&$2eHIEE9~)2sI)(uQkHj-5U7OZ|BDu4&<9Rh;vzwVCL066P_n4 zC_Z_nHqh_ECUgWGu8a6$fPIXU6THYhGBqIf3Kao&qxUM}QPq-M?+Dp(NQ*-D&4+3- zJCdL!LdMh};-MYISw>9vy2>N7aI1&Xq3<0F%XmpD!xet{sX}ReM)k0fm0T5H;2Yd3RNgG%?C@2Q8H$>z{HWYC z;kc#2dg_fxGfxSdTUCWGZKCSZn}D{UQ+^hUd0Vq-)a;Ru;&8t%iT483C_1 zu8F)yFhfxI&O-CK1cByld-X_2m=VKphA*SgdmRft%B77;_k{21^aJj%G0Ee&YCyty z^4#TmFIRB|=T0S++Xwr5*)}s>U1@T(B%*%ruodb)Vy%UoLKV^@ z`|s5Amnbam;Z8HnoSbvrNxY8KZ#yaYAdr9gCQLF)jd9(cDmG|MT&{N~ILgwd1?2Ja zSfB#=f1#W02U@|q2gF9cvveIHXtG6JPx0lDWql3eBvSzLHtL;V&cSnXy8+7-EH}U7 zJ;1zng1!mD`Yr|*!ww9Fn*}!yJCC~|3{sZ|8v&Bi0d;8A33A?!mc&Uf1UjCIj^w!Z zva1{zD6^k$7Ib!?nonP|b2D3g^PLxxv7Xy>0>K7p>2KlzH|@OD&m z!jQ8RG5BCi&N)PqLl~gf7j6`w@qu$t-S&`q<{^xa@|v-aG2=IDv6-X6*P^ss(x}Wu zam^}8*0dfZ5pKyfjD;;_b6V9gckX7KktoqGTT&EEj*2oW&*+jWN`0@$*~=I6DVKHW z_`hr2?{^ZBzq{Z&b=Z<~fOi%n@*C_y+3NxVV1hM1T9p>u1}uW)_OJ`{X-R?0SZ?Yj z`6!2u@~6pcO(7R#gO3v5T0r11SeQHwv@<4(jQxQ6o#xOo`4q*o;W*uq4g>LoXJFFa z1~XSK%`)@!FfwlWkO;FA+RQ`by9Z<~ZP(a9a=t!rHU*2VldoXKhdi)0sfnGLJcC#<22jVUEjmuEezjElg2_Bb$uh!5`Qg5;}6|3 z_rtS<+P5P5#>-@^W7iXZq!&h<=oPn*n`?;jb$k)HSDIs@8FKY~3xhJ`LraQQXCa!? zJ)O3jh0@$u?|r6`nO>KaT}ne%jG0++ZjANJ#r>+#o0qINAeT%dNVk=cg>kx(+?SY|Ls&tIW?&eagYbije z>=*bEMaXT`LCDPlBEL#?Cf1{3i|80!F~S9IZxbijxk$I3H>H|v6ZSrr*DkQSKYEuY zsv#@f$gH5^viA?ltGn_aaBF@tKG*()Y7(aVfLxj~&Al4-hVE zIeAzdw7W=;YiHzJTN&4sHpf2;e3z?SSJK$gGQz?2_pH$)@}_X>&OQ)&oro$nw}{YX zPOTAT%V6+e?}`LdS->Dg_HS=MoK%rbRGOJx9O*yxP&19VLI$G}SOI+~63u&v$aUrd zMYh5mX_3OVZZZ|crTb6BW}urKfd}1`9_}8UBQjiMG@D)y(DI7H-3pyn*}u740#R`1 zDu{m;41>8;Cd@k0g2m45LO(`h%f#0IoXHkqbx${BN{y?UojgeX ztjvV>{Lx?jxe*l%;7hJ8u%AA!_LZDq0txZ$qbvn+m}Us0vJ&RNsqbyC3DnKQd=e=d zzU1mKduc0xyS@dlc63EENJYM=6pgr~Fa7$hOg9WlEQSPNs}E4-3)#t7E|5gpsQWH* z_NqLr%=5nPNU(FOHto_!MqdIYM4ia3TZ@H&ym4|)MmZ*(&t;``QCn11N`;S`Gj6&k zJb__dq4S$hMNcldaEa0T5QROqhm;Ejbmc8uSKrB*jymA(VJBaTvf$)>o^hVCmiL}6 zFg)d1tY*5hqk{ZDE+}JcYQK2#K-sUFxY057jHZo2^Fwu&xtGWOst%ER`|Sat56)x< zaZY9#`MR(F+Wzo)mfw^K8CN~5Yp%Ul_z;+WJg%XsvoR-h0-(96p5&E^$b!l_=l>3WOt{sk(z#5PPjU}`25wNbJv<;@a= z>0+TZvas$;SU=5}4xxE<=iTF+rcjnb=%g5yI0BI^3(`)8BBOF=;IL2>=L3^)iCQI6 zyfuUW^Dhi!NUCiy9Eb5qK6q^qwC*dKUX`m$l6~^A%b+tO%&-bBPN4+*3zet;*2k_t=24m^ZG6Maa?8ZKXw20(h-wpe zzt*($XSVS(`TFw$GZOT$PVBKBWN zyn2#|u6SCdte|Ye&J)a{6gHJ@1$=j&`5qJ&%VUMhx7s<-)1r|Hxt6a@w!W*g1$V*# zMB6UPoFM?Gy5%5CqX8%GXHtEyF`T0W$f8dBV=Y1l14!zn@B%d+Is`L%B)2kJu1=5z z5p6>Jb)?Ym>=Sz(P5dPn+AOYjL#-S^Js!K_0FCz5@=qW`e zA&!sruvpT|x{pPnfANkA`+C?F0)_u3-6D4y6Q{=_`87_SwgAQ`F(FLg-Aj5tnlfn% z@Hgg-Y6v759m-bCuW9&dz`RR)35YoAP)SLRzCjxPM9(ad%L!UjJ>Wnz?LvT&0itKp z^)?BPUZl)|zf%|_!DNs0u^P04YR*Huz3qflfJ#{MLzo&Ap!Z_7E|3PW5V z?I|WZC7;8q3c4{P+1Hz+K6rNQ zJLvs)7~mfujS(-12kvB8-3PUct5Sd_uda#G`55cL0(p;C@Wlzil1(fJ)@S|xR*TzV z`>>`QX`ka!U5odlLGMae_|X203mRuIT=aej8j!GfR+(IZJoyUpwgJ%mEz+oeI*VZb zM}f$p4g;54kdZ#?Yh~6`y1=-!YP|#h*W_nH*MT3J`v*x-I05TAZHHmY;*J`-4ZGE- z6N4Hpv^2a%-wFQXF@6o`zW@X}IdQBCL!*oIRu>bj7lsC<`IoLofj8Yp#^i5dR2D#J z&IhLRYvXBu65rL+Y^ozx5D-U2gkvV49qSjR2BF*(WQ;=`*Wzi8W*|lfP(4q!9zj@J zci6QJ_-KB1<>&!6=kY#>6Fom6!q?%uEgwK%Ak#t8RWC-aExPk7`tcth_4}ueAb?C^ zDK0wfj7+*Kh~^KQzH{D`u8T~rKq}5%12y`$<{0B#n2*$Eb?b6kBCm4<1yA5NJ+tA; z3a`)_aYlY1n3M9`=OHBb6al6CpCZTQj*iV({Gf<4aii~Plk8XF7CGb0*~iSO3eK)n zJnl;RpS$GO&%DTrZaCi{e)Mtl)yut?x8WC zx1H^2J)GqPIhqOVfLF-TcyaMAzmf$wML1KU{unTOGq+sKhrZ&?Z`O4X94(hgWO|N6Q z0%d-Qs2WCzE_YN8yY6J?W^RGKPeJPbAdJJxf=*}3#W{|PqX0b`!k9}>WO63xWp_OH z+pwYZK8fy|M1cKrGLz{!t{q#PMxrsjvGMS|yRG{pl>bQ3{tU{$F8Yr>opv7afi+h6 zT!znfs0R%i0wYUG?K121Ju|(AfdeIF=U@(W9nM^vyr-i)5mM}X1na2w-5VRX^K?9L z5BiFXRp^}6*Y$M6IY{TCa*J?YOWUlydHRIql}h64h;{lmJiSfb=l*t6 z{(m=!qCUoJm@>AzB*vN$N(cXM=NX8WC@O;+ZC5q>XXTgbLs3dOayWvM&S6VQoK{EFyRhh3>^N#z zB9{5Tn2XNtlkO9z5Wi%{y>o>1Rey4LOLo(0Vv7|0yaboB^#{wp_RIf#G-c|iQ{k&i zzOS2DJIJ=wI-T^&To04Q`fpsp9~~~`f82;wdaac*5sU}xMa6^RCzvV^mK)1Q!7_o= z7~vt=DDfcqV!G8zQA>ed<1FIBFlaYiQS*#9)7lnHF!SKss{O^LtWQX0bhnnrw|vo5 zvyS9_FKPe#)1@(EKG=H38-8^(tg&3=)5p3hk*(*Ag0G`fWvz_*9BQTL!>x#QrP-ZA zmmSD#DeNa2=x1eqN=1a1u%ZD|QBIX6C%vfa$+rafNgs^p(gE77#{YnzjC=CFuoxq4{74>iGq{b5V5(?8SGUG1KdFnnbmDBZHJTSczheA* z(edY=Hu}RuqcZP&2iYn14z1q#kZc$GqOY8}vQ{{CZ&G3yRm@&DH$K-cUVptEHdC&< z8QvJn`)h9kd1b;VF@lCnj; zPa+}-L<;YMpmW?XM4oa6alr$2Ns5mXNM4qcQ$WQ80-jjN__6zb@0;7h4q+ zk+h05eY6eADCX#RZPlBa0qErR>uBomtI4A(i?o{Ac-;RL6mN=+fD3IQP}chWh*@WW z*jo%Dr>vS#u>b4_rj0H#9s=}GJzxsTZu~jk1Mvz*m_NNh?bU}1r}R81*(cWFz5y^y z)CuxeQ=lu_Hea3Sd50%tB5vsf{I?#E_^91+Src0QEE=1ml33j?&G!lA409P@Otk_C zQH4zE$O34r7;x0h*hpIH1m`{kMe}6R1|K?J1_?h$|OQnwG$x-!6z?^*rhf7SKxmL@ohjt)bWG)O& z!D6#s92|;qey`||MlCPhaO7k#Qd&J&Q*!!W>QFQckGF_;hM;cBY63E^EE=@(%~5}3 zvK=Sts0L(v0^-An^2L^*D;?x2>#yXJZuG(s!Nb6hyFix{^4Sa;EbDb7+CAW@KOejc zS%t39!LC)$5ZK2MBfb%uc_r{ZN`);zF}1vhxb+-rYqp7~Wu_d7kIJqPpOqC{0MM6# zFTi+zw)fQ>IFu|+dJ>QrnPX$ni-hihnOx(>^Y~wIuxv@0>u79U=XJD^1enX;eP8C| z%MQ#pkrsapW=b{(`%3LrhA&4;wrD=NG+vV(+Cq5d+k>3R=zj~yejjp(yxnQmq#y}` z9?QhZn~EE)VwATbYVE0s@Fn+zElLls-NHnilN{AR`!v`MV|1$`Fg*!+ZrR&GGChe>VII4{2*?tkSUIW7h!1DZeESsT zq&5sfFx8#)IjMAiC0<~*O}FzW?e#bjvnP@TY?jhi{JZcf}DGnPZqT^v5az{ z{ubIt;k77Ma;4v;peT1*pkRqnp+8bbg|HdlW$vCM#ixYdKk9G(RF)5$A9f=9JRfKU z$PfvYDYT3zuw}{p=1ai+xEBUpPL1-SW)NC@KMPL$EU}A>p(pX4jBcgd!Ok%IGU?R^ zFA*Ik9g(8n5F)f{U&%O$3f@NeXomJPotYEd1eD^kX$0Lu&|~7blYzT|q{(1%RC3EC z?J6OIWFCyZY->NuS`a7x6K~>Hb-MW^ZMBZ2V5&1Syu^R+<*Vo8?|k=m$Rav94=VSz zAM9LNT=pOe5NTpwrFcCXj&Vx#|9IK{8MT+ih0moTfdh(C11P1}ijdJ?GVM+SmLIl? zcBwTsD`z4)KS z?{D&~!<5ovl;CMI61>3oX!pULSj-0m>*+E@ZQJ6q-7jwuk{oaub<{(1aZ*<}c`Sy< zJeJG2t4ZNVX7*Kr)8b7j1}@^C-pIG#CpaAlT^i_TrrOsaft z82^N8R9$Z%#7j;y|M&oj^zPYPNEGWk&|)BV)6p6E2~fy z>Fz|=ksJ-k5)Wl~uHGH_hO@uLRR4Ulzk7D!qZkC) z>7HdZI?w#&^Rl0`5Oho(C)oEQ*rprGgpiIqMdzKH}}{XcG8k7*QD z28U1aB>_~zxB57^4(0@V`;HYKrf~KLj^RcG3ntC-gC?5=PNG<$w47P8T+-=5|*OHm%2?R6!ktanaxqIY5fQuzYff{PO5-b5HY3+ z(vn-7>+e2!Wd!XF+@Dgmz73Er7$IQ{+7Uvn7B^hFwcRR^w5b??`Otn)q)Z|5Xr#As z>$)A1UqGpt1xegEcLH^O{H`-7u0>R;rvX6St^~Y(gXMMV_;rwP{_H3dH8FDklY|Yg z68s)oIm!W!Q>nL=E?!?w!nZ0IOv;#(SQ~h~o$SyXD!F+dI9!M8Aj-W#6V;7V{I_TQ zlWVX=8;@oU9VnSn9l$MES4mK4X?P8xj#t#CyRwa;qCRZXHe(cr*@6UTk&v zA?{xE@$}=q0tWjteMOnzkERK}+Cn4;zCr8VT8OS}7?=}Zi@}0!X=d`Q;jev0syNI< z?I&XW5%Y3sSC>;|o%G(w6t8=jA{pATp{k+e$oG8xEBB1G)4HV+c5#F<|4ZfgJq9ac zLd~k*_L!O0tnj8=O1|vsmTiQm{cGpXvdO;O#2MZ99CyK28P)@A*6 z37}Q`|F5#^j;H$V`w|rmB~r>rWJXfI z*Eu7K2O$7J7C%Bk@k40D{fYUA57U1sKm;ui*<{ioxcVv=sOqch6ZaV7jDfmOIDSC3 z{1Za_Y^{^(oI#Cl<<6j{l3EM&-8Uq{9tD`LE?h4+pv`!*l%60|O2JKLobk|i5MKN^ zLt0dwe2!haxDD}C^dbQV9F=RqxEh672fqh`oC#*o1$-Dt>!W|O!5EQimqCc zFgH|+X^&nPId}h$AK`y4>;JXbZLZQLjX>t)kJ_2k7-h5Gw@K!2YmZQ;va_)*=qnOD zU7mKYO@i~cA0pmJc0fZSXcM12f+RFjGjI5JGlMg&qvIgX^O}J7V zG|P>OO3a6lK%lq3AL#^&NsCAPFdHlDYVj-rHg+%_fqwYVW*Q0FE=`8X5UHPluHtUi z%_14t%*hFKCy@%B^i`=rrbFMMcGmb{cS^)nhEsV>^o=n1F0N9j!XG82CR4EBCnhtw zL2v=J407yk>paH3_jGGleJ6RXRy>QeT1Lr~!*k9WTZCm_ugy@-6RBjUy zs=}3fkLA?EGzAVV_KKgPD4^XKs~hU@@aV(uiQiG#o8WL?e&i9&NMoF>Whh2;g5js~ z85eRQyl>$hEBb+8ek5tJ3;hfBy~!^;fw%c8(tr7Q#gxieo*q3$y{k+?0~JBVF~ zMK9PcvqlJgNgz9Zj4b^2jb~48a1<8F7lQ`!cUCKMhk?WkX^!l+)H_R|&CO}~O0REy zYLyHUmcG-MYDz4@pAExAxtH{vj+=QWay(gD%CjKQOYp_M0?{=7+dgo7m z`^EV;4h&geOS-=tzxHxVMFfyni_KEssGL`XJR8!qk(@#0f8a5Rd(|TXaT~>x!aju$1D5 z8swlq1-EZ!u=OnO&+)b`suCA6j>&)@>5MOh?d zq_3;rWelwW>vUgx3{S^bJbN%&bfnN!A@mE}9>Q+)8}M@d1w0aYJm7|s8lWHg21D)C zh5KAWN%+Cc)6vfhCQ_tUF-RNHougDLRk4L>NYQ)zoeel1ZOpqNV?|R~@0&k(c}mn+ z2^@~^_`@w(!Uqtg^%F%sI<~UsSsq^#(t_RQjodZW8Q3-F&Mc7CbLi0*mP7KbkZXQd zs?#9s$MFF?7TI$!r}rY@RjyXiij;keM~+G@5?nmv#q>Nui9kj?B4=8H<6HKZv&g_+ z4Bw!fL|TN6hTQQd{3OhQ^BEZWX{4@R_gJQom|17C=K@Tm>@85(mS7I)yu|?dM!g7h zIyAGZl*=;u{q2K zx)i_|G)?!>O7%DMsS5gWVrv&_PPqJdTu)Kp8-jOopWB2(n zvz4|A9urlhZ!*4gOxm%hU;Ger@wy606TZ*j3Y8nAU4On)Ov?U(=~w1?S7@+$z3P`d0#9D##zGW9Xtl;I{WAyuuq(BskUiFySoq3Ul+PNqc zjHU)x&od+~H$F`zxdyTHB8FmTfJFqK_I*>=U$~%_=BQM2>)|ccnp^AOW*aakTf2Wj zB`iv{_6ySFpf6V2insugbZFh!jQMQRlhU|Uy0|bjHv}M%1Yhm^-WJ8R0BO-!Som$j zPIK4F!C$E{wd;6c8VT`VZQv4{HUnU4%tgt%C~U+ZhLZdrhp7B zOD93JrxF^Z>MS$O=o*e+$y`2X5@SB8`7ps@l&HrT+5L-Q1}{4&b7<~~MV$I}Hspzx zpYEi1B{<}AH!xNS4-#s6+PSo;60oR;eR95=F+J(xkCUW6Q}~jlB}wgO&^4~VUNX?3 z6A}`Qx`hho`B;KZah77r6egM#B^s@@%GDCBP@r!+E}Y4+N!ytW((uK=1_s(Lu_?dB zYX$MtvwnFTBkKz8yx!eGkqUv2G(oGgSIVqpuMNe5GW8)4ULf`;n9*S>eZ)Uvahn zEkr3lRZR$X2AVEC{S6B1t z&q-AQ$@}-v87Hz~`>xnwh=JSu$I#Idn3vc!LPtnRq-&U{R8>0C@8?lQTrcRAs0z~| z{*F!*uXmmS(Z6h0ghkw@12<|O&ZhL(E{NL6p%pxVx~k2p=^mM(g)ho%&LACn4Bp{R zlCmt6ZEj3Z)IzX9oP5V;x@vbt<`?i;cQq&`8TUrYI0K!2`BVOl zRB2SI_wz6_K`m_h^JpO`Vyl|mTpnwcu={fT-ucYwXVTwTYjwd|SCneczuu)RM4XU( z6p3V>z^|FsYk5gom?llYL={^13ZdvKA7jeer{>K3YZRW>ecmAS-K5ud)34h@4R!lAF)r(~ONf-rP-}^> z?;dx)M|)l`F)8pr2QU8X760MFlDS3jd~q63!2+MNjQs!s>i2LW0nl*s&F91OnKh_X}BNx%9+5NOy-;d0tD)#0x|8z3{86M(0 zt*}-BJW@XZ^{u8dsbHKI3Za!NQW|E-&7nI$TI(opKKgFR>yN4;a7JRyip>Z50m0Yp z%?PapT_?{&rZ9h{HLv^WN2whW{fEI|9em;PzNcX;9-$XWJ5%ZJo8hJk$$n~ z6(t>t7t>hP>O|l{N^FKCXvvuQsbZ^=FR_q5U;;#BHPamlc3wL3F%jjPTads@ohE+`Z8o}cDbN&Qx06XkE;|B=T5jxAE3m8||%!+FhbrYv3Cz($js z733($6?Rw>WGTH?4?`W;;cNY+Z~FB18&au^%UQ$)dcE&hy4h9R>o;Vg+|S^h`_BG? z!oI4w1?S|m3(%j>MjFXq`^02Q(T*gSm2fj9%y}2BmYPUV>JZ`^>ZUV3I7+{j{qZ;- z4eO*)u1ii(;Jdd+{sW)-@9Of@rE*C!A!h0~!7J#WGdWAY$YVTK; z8f@z3h+Y;qV^;4=&W-A3TMCqv+y$~2u?ARN)O(>NhO#W~GnqiWXo%&rMIsJhjE?xM zYGC2nPDWvnPj;2u3rMQ#d>{Qp=Yx`1Bmb8j?hl*ozxi!q!|5iq4u;CFz#V{--h}-8 zF0kGka1R-vMCBL&l~{#=L$Hf6S4C4(vosgI1}S)V9Q7a7xpoR8g$x@D13U4s6Q4@g zL&YKK_DG!zqD}WKuv%{AsO|-3?jejJxTicRZCt@gtR-AN9$T6Jg0 zS1$s(t0xJ}G2dY3a{iU~V}h{t!v_fW4z>+6lHiHRjy8bcb~pfqI~plCgaJ?k=C1oa zNjkQlZgqpq!b&r#+Yg!%IWUSzN9y>Mhii+o<~>q6T?@^;xBg$$@y|=~m*`;sm~qOF zpar;M3*GJmVqdG4XeJ!yig8yYFaM|n5(2m2NukHo!)dmUz}z4q(Df|+K{xBSjR*`7 zUW^39=|9+yoG8vpOLGDYftA46F95X z-g8-bjeuDq1!F7&uRwS~?Y3zU^4%ItRN1d6vc+;%NDp17IT{Iih;RA&BPt+G*v<>? z@{?&%X1=RRW+v~7IDx7kr9D6x2BaA;EP-?Z{ww=*iWi|-K%l*YGKD$>JAm@D_us$; z)zf)=3}8`g?)%|>M0Q^e`4FHYHrOBS8U%E|710bhLq^U#BGILo1aQv+lHQ7e2H#po zPGV^lKsJO)DOqg3Lzt+}L@98>Wh|Bf6ud4s;hj!@dH}>q?OZ} zoI@Dp?OfGhvN8vG8V*BdwE*&mx3b)Qb%13XBcP%=u}EI*l|w8}iTKrKmV~IY9!MTmk9^vIpNG0-Q)?I>%-UaYRzK5rfC;F%fn5FHL3iKjrcy0jC#1<&6 z3j^Yme(V{s#04cB0+PZ&P+m%tg~?q3hCk|s+lm;+8WNm>E!H~*^PhLdg>XCt9wF6E z|6_sTb_Q%+QN5{!Cs#dWi!qV^)EE7?3*+DSu3}0 zJRu$Qq}%%nA8|Ai>1RI#GY@5IY3a1 zW1q?RCA@QDN+ayvwI6nDtoMK@U}pWhLF@FB<-n{jfTVb1i}Y-ZrCBR7g_C?noLe}_ znU|_$Lr76mI-aGk9Mcejql`tf_g!-NSsRb9>4J?<-kcF!-a_PmX6A%xU$;{A&q~H^ z0RX;HNk(M_?5DMEjD2X0acpQkhF_(DM=_5O#~#LCjD zMEGfiq5R4Z(xko}*W`GeB<3AWmQFe~LwKG;OPyyr(c@Gh7@UUm@d|_ZRXAuTth+<8A%JkqU}LK7?jI zuowz-7VQh{jOZEWm(9r~XE;BL`dk)09ejj`xi|5mI;ALg-ibryQnY@-s%#d9eqqJ* zo^}SxSFFjq66xvPpLQ<|3U2ep*$%G`4_{m#UV4ga6I>mOU3CmB_l`m~vz&Un;8%z; zt<}3VuxXoqb@}qDbr4pHj?R-_BA7S<-bzGeQ@U`t{oulw2YU#myVv$qe@}@l(aGE% zp2iY07_j;<3+x1C-hI>aBT|o8a_sI6n#Mjiz}RBIbp{i6?~+2)U?hotnn9dH0QQXk!?S>rHx4n951J^Ko2*Za-pXUe%%-* zYH@DM5W!Gw-NwT?!^Ll}G&0fStgH8Sa1k!oxu5yjfOOA>?1tp7QnBTTUW1M30z?i= zpWvAcIexC=(~b@Ob@djIj=QZAXrXO{H7;0r@Bh5Sbt$SeWDg#uqhQK|3zi|_kw zVkvS}WP8A68uJz08sX|T)X_JS-vjV9wzL#et#v8mRC2Cyh;8oH+H|bos zb+LR6vBep!@{recg{L@3Z_wfpV zU92=TBJ8Gmt0|CgT@`!z(?$sWnZ@3M6ldrdli_A+#8c*l2IjEv2iqYcZDfnGi!pY) zk+)VOlbl8>y(fD!es4J=kHz+E6Q8`JXKdqHjNP#FOj-ngK+8~tqb(mk&t%>C{`v97 zQ%>j)8<4pes=N8JF|wqvxgYLm{J|3rr{rX*l+VkE4nr;bSETBGjQ2Z}_bDnCfsKNz zj{1@Tmzas{Iz={G$``>+dnL*m?pO>xesR0Jm0@R1n+a3Uz|F)G$ON1hpl(pCnOAE% zVl!{ks7>XX~RH#@)srOMq04n$=YdACfXIAI2rUa{b_%-)3=~ z&u-V!{r#A`7jy=uOn6Vt8@H!U!rGvtqEW3`v3?tOMTG8v8j^Jv;T%~|%hS=kq0z_u zn&!%6o&V+|-h(JU`pp7%kxM=vuve>=j(|+GW$^q*p$i{30xr$9aw!Hb@0I|g@c2ED z;TAxHGYH&_k3#oCWZ39r-m{!2nO(^!vXtsucI?Y8`AJ*rC0l4*Wvk2Z$09O?+lL5$ z=gVdl;r_^_oKj(qQiX9L-AbpTap8h@)Xf+~U)F4z++I*~$y7#?G0482D}+XZ znv748tl@??#?W4Pv39v~T)ti-;JnGHIYkY>uR6^MhL^S;jO$C>pf=K21w@HOaxZwP zzOs3*rPOHL3rtuVZWi26jL^I^!Y(D~?+xkZWW(O3sZaJ|n?7pz6L*RTsRuT)3bE&S}bK& z@Gzh4db(rC)zzw23;HbG1UtjZpJg=WZ^S$n+;>+4J@&-nc28qSHCef_ch|nH8V>PO zk7=tH`;U7t9(qk`tLevkU_A{LbY`y&YgTZMBp0(f`X7SD7<_cY^Lm(}(F zAQ*znsQAb*TK>90#fJ`Id9n#a;JuHzPHKOq60wvl`dxy++q2zG!jb zM*P-G7xARbyN270@2$MUN-d-u;zss7$tL)PSCEW8M@(%Y_k z7{TC~6)g-p&-{r7&!o2+Yc~(~R(ed{4^Ngv^Hg@k7kn7XCHj0 z$A#kv<3GY>*tS2od|7U14EL#ZS;us5uzfwLD#?0Xus_sEPCw_w{-QieZt;EI?=J!S ze7t)Gm{SU@YSnkB#o|cku>HAJ-6a}~+bthPa&;TG8yaF=CiZi;9NP@@jUM2CZ_?)z z?iF;o&lcE>>26*yn_J)ITf3Q$cKMZ@9SOGfHqPicdlUb+=d-xD8|o|g&n*iE zL{5a)6rp;`veonKxZoI{SKG5%H_v?Ub!}g3-$C1QU^am)d&kyU$yG_DgV*G~5c}}8 z4u6f2-oX*|CGC963hUq6;gBEqTu6<}M92M*`39kk3kj7`J-D_~%V;M-8MnMD{*K_C z(Y}TT!(FB7(e;Um)}gkHgMI8ybB9|Qz0mE*V(jF&hJM_RxW0}pTaWf$u136aPb2oi zS%!R?0++b1ghnm(-`s2!Za6lp5UFRz$rz+9H%f-E*&CP(Chr zs<{85f{{+0_vAQd_wPO*GKB|j1UBvR^c_WNCOX;G9K%{3Th`T8PHm`l$yclli@Nzy zI~O7b6gyVmXnPS}7v78bI@I4f@0i6L;FKXv6mIafx<6K%Juhh`OfP48a?vU`yf+3IHtlU@8u zVKmypB_Wi@$E}jVAsv5_GtYHd(4=b4el&h{?^NFJ-dEnP5zxIcPZ|+iL22Pq^2q~D zTlPw+_d@b=s8&mgrBU|-I2pUoXZ5&ZnH?9=I#|eONxJJe24r@u_iJ5g^qR%!XUnohIJ93XrqqC7bHx;Zsb!$6v_Bwh)KXq^J!;yD2fyw~$cg8PK6w5=4%68* diff --git a/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v2.png b/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v2.png new file mode 100644 index 0000000000000000000000000000000000000000..aad98b5b07715c9dfb671d4dc98df58ba69d2d15 GIT binary patch literal 322584 zcmbSzc_5T)`@fXZNK`~*J5FWEk}PE#QIvg`J;`ne*^RAm%GRkQ>sYd{S;jsjS+g$_ zV@YHmV;jSa;dkr%c24KK=Y8M${ecG$&-2{(wS2D6=enN9n(9il5Ecjp1qJP`o7c4| zDE1EzwNoDkUy_(*48WHIZrV!n6op-9=fOAEt#044R#T(k1IN@92g2+q4(*=;K3Kp9 z1;xRX0~80r*8}@MOQHPsTL?wU!EeWh+V@Yi)1zLdppc`ub^V&I*MX(6BZ-WI9lBbu-Kd3wC3bTAH|vrwbI}Ga`WWHY@Xm>`0gIy zzx!Lrd1~!LugTs~d$<|l53m}-a-Ao^&8*03rYrHP+l-7G)8_KKs~rMn+7ZDNl+>Jo zKmI9FP}t@3^~Ijlhl2wHkKE`uWThQAsvpds-0V)0LffAfFnUogB6UKJ;=m#LAO4h5 za_F?6iLF?NK!u4F{>k{y+z_~XIsG7TU3!WSd`;>BkN@%$i$~3QC;YTP3Cr0ZUz)G$m{NV{rLogh%D_SQjyZv!+c4`x6yA`Z zyI!_@PzVi^QRnCP;d#Z256=9|R#F@&J;ANW&?M|Oqe@zNM@J)>d6FMJFmO9E^uX(% z+gD8(C1fw@LEkNp`Ha|^LHCe7AN+!q4Fc|AhJPND?7e!f=5wZ$YKJn9SWM4R;!o{E zpj;{?#<{-E9XCF9_!u+2d>~{iBa(Ux7qEws-jVR%@v7MzaiEdW%Z9oR_^Jpj4&FRL zmy2d-VA)+MtXVEO!r>=e&QB&0=txRYXXC2tWmysaL_3V<1ROi&IZ@C(c_zA7y0y-i zfUF~oV|+H|EWpmU->zVmT1JtaNa*@qy@<>weTr4YbW)B@_Q3d!Ys#Y9YH);zmGXW6 zsYr>F2EC$Y$ANp9OiZ^uPiUsfFbP2^vAHyz@EG@=%~&c}QXAad(K8Mx9V2U_BxH$m z+$7AzuC;+Vyj+Gb>ZZ31mrh_K{lL!WYRIMUc~h?h9z0%LmjA&j*3^0=Yh?mkG2xWp zRBcx$_@IWv4|;aC!$(Z}^+QpTb6qBdelc#y*qN5Ml`u{c?45(hi`qRSPtpZYlg#~A zZ%ib*={`~ktzVies^QjuT&{S>x7p6QO&Kw8d*(A?} z_}cB61njskuE2Nf761F?=t31Vq2VAMgWs9GtrlqG<&ryZgM*_52nT5AeNo-M@FFc- zI)c1pGgG*~?)Uw7w`UhI_Z~*6Y-U}H%s9LA;d1w*MbhVRA?;1;@fVwBttK_Kz8iIR zpPL>}3UT0HbKz>V&2O#w$C9=?-0KF1CQQ($a78#@ZYOS->4CX-bkI?!sxPN~wQKv9c$dGQQ9AON;^1u+%#4aK$bDk*pI37g=|85=ak7Z<0Jm z9$UnDL^Y3|rVON&nIsb?>UB@WDN>o#dc9G5CHz))>(fEfOoH=5)b>(ox9TKY2o-d1 zXPy3+SC=dmZ|`kloYjIF_jU>O$O(UvVZA@u5d*6zpYP8KBfHIfIEv-b&9Kyy(9s&Z zWZ8U?QCsuX%CHTBHtekL(i^U(ryjM9x(L6u$B}GO;)kf}q7=i|WF0W0Rc`Mb%JUq& zu;u8+@ZwQw<-DK0d8o^lc9EFK)Xw($l)}Og zYafTR@$6=r@A@tY7q-}cQzvf8eZoC)8J_vfqrgl8i)y>*5gTUKVSa-==wMpP#FkFV z+I<~EEX(l_!Ph;Vl_`KiY8G~L$(3_;Cyhzvi~fGwUt%}cI)oCE`KgMg>6z*@WOuiw ztdM!?nQ%MY4d=_TcF5g1KcZsZVnleM29BSx4)>f?ZrWX$>|Q@Ha+(Q?_LV{Ht?ruO zLR1L@nAS~MT{gV&Eti0y0S=jHA}ZrE?y=W^{Fn~iC3{|}q!uOVIvk4BXzNfl?pb|- zXqI7F8dF9nTObic3rN25i*3)WUts+gw^X}dRU|#=->igI7fzGjMyvVOHxb$v+e`@Z z8C>Mf2Li*yghldaxGu)WUu^om(XCQ~Pt`bUZtTobi$ji$K~R5X>xX)rNdOr?ly-y0 zclviF$GM(V-R&Ah+FaBaD@os0b_ zOIorF*j-1wyEIa`>Ay#s5OVhq;q8(kn6OHF-TzqVhPdB&baGKBU^|?aRs5o3{bVTX zQV5fou-aqR*jO_k-m^u!Ir){JpIA`0g2+qqXg*`}h(0mO0fU6Sk|I{rlMC3qr+>Hj z^;!tGuBdsLysN~xm!+4C^LTBK0o5g4(Vf?!uXMF;PFGGv(QpaY4Q-2d-i;MBT>`P| zICs?Z6LA-JHx-HK&WUoHjdvH81$*t?EM=Mv@ zoRul>Yc(j3&IAp79Bm!|cNVFodmyuGVjs4gRe0mEX=G((GmFD3#X{X)axV|Cl|}Q1 z1!24XV6DSuJ)WzHAH_2y_mNr=mgQNq>NmKx?L~#?B6G&w(WW-#<>iz9wz=I~b=d?n z+04~C!tTymY`w{hUkwgb(`JrSGo7ejEMz5SaHcg#hRz5=ad~w@vj56!!@JkMJx8WKqzA3ZAruaIwYrHbC z5D#|vM(?DLIUGMoZ#x>WM^?kpQ9ZNj$_bGsYdyS$gRBGx{H3k|nw;o>qWhHL}BTx`dP+M%W3`KOM-nnhU@SC=#vPwvl zwD4y%JzA}h|JPKv9G)GwcHXQ3z1}%xR>$HX-iWjW;X4f!r0#&-Q=9yW*`J;hJwW6V!Sh~aUxUn;NL$9g(_oo8vDt3cDGjU%LNzUAJ6FC5`no@aAy5^C6?x>=3zkMv(I z%LuMCV~Eg;9}cI~P&(^!o%)#_2C=)5RqW9El#5Hcav?V_)W&oYdUau|vI%!XEiNhA z&M1IEUhlTnHSIbA0bnxvTD_IF@Z^BHdc0W=Ng-7>QZ_U2_j(R|vgRj)c}F1@vMOxB zHz;#th;T5{>($v9(SBbp1R)ur(y;~4rbtHB=WG!mGTo5b!n(~7uFFfH|>^;mBG&TXs2IZ@JcdXJb2{z zu^O_p-2|dN-$d>tkU!%hkF!ZWYDklh_7Pej6)207 zOmLBYN$%`antBY52E7xdd_EIAkMip)HVDyA6nO_oD!6_&kB?lAQ*=L42E|tgHU{va zF3#flr1Ylxb%(OJq1$a5@*fq`kvUrr8rm9?6{mfGWxTW1jYpQcFAk-mGVg4Bic#$7 zt5ytHPiqJZc4M*NG4y*Rr8btQl_I)k?w5}?;L(~q6q3V0xqjz4lK;{4tidNf_Bi#H z#Ve7m_ZwxEHUof)F{oGDQrx(9ejyAI(u=_5ZpmZ}cIB+hqD?=_UOn{+8>!!ln z6X(q!`M#SPEN6uF?NiZ!_0rwz z%!Y+l9WUKy`)pmYBju$43|~%^_h3mCHWbZ5K52BQo_VVh{K^yfyY?W%$A#fi8|L>a zQqny!SQPK;%(Rq5c|C~s{5b#34Bm%UIg5H65ZBblU)d zD2uZ>RBp;}3p-QELPwstD=BmH_ zj$@a8oC9LEP({F6%szvDCZrK~V}Y&uwZrO|9PI5E#<|(W`?fdoZ3iw>MedyGmtni= zdT%aDk*EJ=Gz+bI`hvHX??O$XT6b_)(TU9{=&o1M$(MGx5Wfju7ll>Q3Lb4v*F6aq z*@{2ESe`jfvx_B8d%{*`G^;0NxrlY)%q9W39LIEKt}cxbnrta#hm%5_)NkPt&0cqy zR=LXankCi|H(!~YY>{M4Fg=IBY1k1km=7B}UtHIx7<0*kMgEK^38J_FKc0M&fX2BO z62j|e8M+3uLx@r;yVY4$$=zQK)hgF5&ugXo9fC-b6@bv_JK z+|^mNx>{uTaC=}Ons_>-k)ff7`bM<$+D+;No@1GI2dB~kQyP!)LFT;L7-b_4s`^I= zi7}Nb*7Z}H+w17)pAQd~fWYk-_|;Og*XnR^p^NS&g=tfauh#B_9=Ynu7V8ioofaE? zGMhdjtX5Pqg$eOlck%?zcj=ddYDM|QPc`7VnlEalJeKA<@0nUJM9W`*@bbOL)}F&V z*sD`9iUlX39v~PTBeGUDdVZ7Q*?75#;ysk18POOZTks}Wge9(5$#-2iMzKJ9PWkQS zF_oAy-e)v+mqkmaqsveZ<{-|o<#9%5{=WG;hG9Xw4R3Zdrex<0aUl?Zdd@X?sN zs@c1q5m-d2%YxN1uP%a!omYyYIdVN<+6Q+-Z&AO_dE`?RWZgTZn}1IU8`JBv)_|w_S^=}1EG>+L+dAd(t zt&Ied$b^H=D~)sPNO@ZE_foXl7=-)Ea^jRxTcFAl>ZgL9H%RC>%M7!fTNkt()4}Vdd)P4`E9blovO8#HL>bV%J}AP+34q>L zuGz}Z>XRTTyvJVYLoVT`>Y)CzRH2OaeQ3*QnU{EPytwqvbPyCqAcPGK3e|-%G%TTu z%)Ok%B;zvux^!#J@!q)iFutexX?9`cYEjqh;4Fy2_C_|H^orb;e}ey_xo^>=GBsU$i2=&AFs#dJ>Km;#))E*IDsq(|7tPnm3j8Yx7Q#1qJ5`ao+%yJsSN zS8$Lg5wd4@XndMF!7TdRkI-I`0x9{;VC6VaqIYynHb_pL1G(h*s=m^DE=l&L&i5(q zC%F&35vUjyF{y9#u;YgDmc7<0WDuzaWxz`acB?FjWlW|2yiYMNYj*Hi7DM$L*CZm> z+yN#Vd422o%d4vCbvd@mg8)cNsj}vxNx1s@(}v+kb{IGZZtBEaUlEigXUjuJ2!P%Z z8#k4Z9|g(hR3_QP-lm<#v;~y(?203jxZ3q4a462kUth-SG59ERZcz7aRCQY1l6*2XFOxD2IR{< z$jkSUv(?VB8M(oZ5eyowxTXC`??B?j&Wr@t$`?}E#_zSFENMq{rpRsXbn0fHaJ2Bbhj5jI zT~~u5>HF^mYy!{ei69mV$7iw+t>>lK7M6^8U_yNJ?jtB$Sg>768U1hK`)+q*g#1^- zg{g`gBly@PA5eE}MNj*t%HrxOWvF&G{I%|{13XYO=8LQIXgF}>qeU3o=|-!Otfz>8 z03EoDb9F2qyKE%pLs_!l#Jr(?wo&?A+YN5ocHN07Q6FOy*Sf5!>NdPz#501_u;QA# zV8uQ1^r9nZ&Y-#cW{zUi7aWCf)g7T4iW zojZf!@@+lJd9xBvW?mSuy#X1>Np5tP!qvK638_f%P!F*etxl%AUwxQrNm?%bq1f8* z3p|HXbEL)k5}#D$P;`wr6v2R8n5`uG69ZbP40?p!>rSNs3wdRq`{G?X!$jriF>Dnp+?Yn` zA^S_*LaUt4Nr!2^khC)Mx91|80Vgx=f~i_gp!D|_l1}`5099yOGgKZhVwV+?qqon{ z)l5HAX{^(}uiC9vGL5lDg8Y}-?K?_`H|Y0XFs$2I>*!R-G*iNCL@IZeCM%6`Gvc08 z`nhfcPhwOo*q9O#5X0sxZfmzHbtfyP>x^tiT<)R{^Qu{04S98*l+Kx#+z)`9F>buB zAYgBIa<}zjLDMXHpb2f_rdH`6$(e04_V7(^mgYo8tQkutUrc{`v@A=gD?SG?wb?&T z6M`aq-rR6pgy+gGcpEU^#+io1lUeq8ko&(uZ6H?Ot2StJ8{^D69H?fH^ujN~gR3DX z4rlhhWQe=fWz`k=`v+hr`d!bQ!&Ou~w~OoO>_*IQ0b8u2+sL0EW*2w7y|tSWSpLLZ zBb6-hh4dbyFK5)nGOtZgd3z(qaXVmOil#kHag$^Q8Cx7at z@yQV*ll@?AsdKZ%q|J$l$=@Eay;Ied^?TDyR}w3wqn$hB zqZMJJk=-+wtsiL`z+kYEp{=pu@|BM2on-zF|ufVX(R?B@e+3ctOl(Q#-${y%pRn~sn zCXzq5hJzPUmN*VHEYvt}F#ov-KXR1!_6HpBFQ??Hd=(cuEUuldt{e&DFn?9OCkh<-W)C)JOv$#)); ztCVsh{ItGt1jVO)UM9GZ2x6x|Hp*akqhF%{hCvRY;Jy>OYhNPHTyDJje3t#n{S+*a z#$JkttYvGZ=UcW!ApxL|sSQZ{a}oacQTFwqqWayDk#dLFHxB(-8qNb3Z#PHn?6IxS zbd~~!xovaQtve$Z5VjK2O|m;H@A9jP`8~!voOZf>RoIN%nZlWbkSt`aqgFdxtD8Wt zw81J*py05rl}W=z2H#l#Nvr{-8S{RY{jzn03{YW)N~}5t`gvb_0sbIk*^>JwHl}BQ z%DOAj?g-lh&7qD_SjAY$vd~OdUJ**t#w{AUTiF?R*(T11Fz#D~$4x+w{c2!WRs1B5 zax3TvdvR&EEGCqe^)!V|cQQ;v2LD2aywRU|DC0?^SItsMc;wluedjfj&m!leXOn%q zL=i3{m+YMz9~~73OdqS1hvV2KpcNI+vPljcrei;g3|M{7>R2?pl@4-N}EEfiHQkj%B`Bo~xvkwE%?od>__9V$;#LfaCAM7LGSCGeJ>k0mK zyx@5(vzmtAWC+s;(3h?E#Sm(e-M~8_kPHL%{tP<<%%EoY3BuSNz$7)y-EaYU_mI;Z zK66|NS6tG7h%B>VitZ^emLN?(*C-Kdll{&!{+9xq(nd4t(!!Q`Hn!0tjKjZ@OvVM! zq4N_5f#kx3+}oLjR`jMTg<)-yy)$LjTDdnteS-leVauW>T(Ge`hS@IzJY+~qr51Ot z(w;<@+GzA|yleM}WWLlWT3+;IBqt~54S1rSE+8V(GY@#XM{Jhc^`BodKrMm0MP5ku z`sLpH!s>D8)V|j`Wt*x5ui*iQG_oqQm!=2tr_=OjEk16?2=r3L~jn{uXhJC*XL4| z`Q>$lwce@b?o8~F&6%IMhOv3v&aGw_9L)HcUW!gVxgbf}vdpJRB6sr&tL@oB6i55z z9NRq#kR$dMzsiMR#m64>SN9~jngORT;ULc`8J3zR&-MkvfOVwp%T|+D)D~-2n_0u$ z_!2UEDa5=It$~Fy9@C8jn)F=yPz-UwJJqHed+^?Z#yQa-=>? z=6fezjvURJ5{pl9s$}m18#q#I@lN%fbn{E)Aie_SYd>Qlc@02M=5W!CkA;9=E@9QSch>F_83DB9wQ8P%kj*Cdr59XE;B zH#5#ET5^HMcOfUM(Sw$lx$ZCkrDZ07l0Fnm2i>{<%6nf+qUGMRo3c)TomS-4y`u%o z6A+4dL8HU`M*R4Y_l1r(qqgahj%(9x7O&p#vkCe9kFt76FE&(PI*nC_RD;ZAck?%U z>YHo`EVHxadq#sY(5*7&_uGCYC$gkntZ6>{r16nt7<(Y7#AQA(6&z9Jx{lyCYN=mq z=l1}2X6^6<{sRgB+bMF-PixMp_wOY86X!GY(V)oDm(Wa9oT%Av%K%*z;+>GZw;x|H z^*7unVJi^4pEAFIHW(H~tcrDwy6pBnDUe##uE}xa7r4!KZFZRN_6b%^KND_59625h z#83!U8qqg57OH;xP@c0qXUKip;{?*&GgCVwBrkNk*e|MF)$c*Rs}g)i|~_ z;97y2vpN{bFjww@^WzVMSZI zLfG+C^V}j5%S--AP82<0zx>k^+li$b`b)5W`?f;0w|2f;)wo|s-^G0O##8ULx3nKUOw|? zFzzA@h)n=1*}yZb0&nZ*51a>if_ zw6WXaTIq_zTdPy#b1FacNI{fTYn{(C#H5_ZrCCL7dqiqpn3DivD>!-CjPVr3Qh!!7 z7DZa_I5Z7@2(1BNKlU1)HekEMe7`YfCH71#@Ypf{*Mz5bRi%(0?}k^K0tP-14?JM> zL#0f)l*{L)Y9Bj(%jSqT07s7-Asyq87{K{DpiMnZ7i7hMq;uhqN`3lttAs}8%V$+Z zEgS)(44kT`zo3Z(oVm-HqWE~0yk6nJ7)=nP7!?~>Mef}+^)!Ry^*}k1U+y#}>DX@@ z_^lFgPp7m;MConMrAIq%%=KE3`!%5D2M!*7G*xZ^5=Mn()YbD5;}VV^g?;zP1U2q8 z-lQDM7o)n$mjx-c?RwL0Wc%(nnz&Wi^=B5!;Bla8D-UD$Y2B}B7V_%+$;1%1N3@`G z0`zDtWv54#fCkmjr&B$t3RuM6PL^7{D6j6YV`=BoLps@PUu?^ppTUmw!X)Bik%GVK z!wReT$6`9ks->lTMfg4yp*RsYyQ(4p4mv>x=ZWl?b2`l`RU#ngd+=YH(kmTx?a=#B z>H?CBfS5AkfOT`?vuGug|~P$*#i32B;tZ&LB;t7q&`b=r_su%$4xCG#X$R| zY$?c4#2mw)aVtQhybY$>?39@JxZhYMwU}QjAL!ivf_D?NvGuAPa+MJsGyL`Tx>s~ppBVtSLGcr;PsP(n_ z!%3U8{X~diARmx`OTQKy>G^U=Spwx6L6VGWC-Ucg7*nKHn@(X9-MWtIES&F82Dm?0 zElE=B*ui1&K=g+g6kQ>odgh)%!P;N#AMJr$$Q`S+y(f>?qKl0(m~gILRm6byg~mv< z81GWnr?f1^J>1@|P{gsycgLlg)Iv6F&qcOY>dtxveZxRMU|}zA-N}g2F9Gg6yx*z@ zVot58Yuh;_;2$uDSsgXe+*U2R&%8@2#(g?OpN{S~)$JEZt={L{%vtEW)-Bs#!hk@^ zWtq7N+K=m=9_X)jwT^kVs_j&8zTc#E!SU)7#eNcy%TA#7d*#rbgZaH{`~4=udd@{= zb@GjUT%?V8OV4rdtR&Ey$aOF41XciuY zf5g`xSQAl&pj*$+56|_cV^hP}A;Tp#%heq(&HcR>LD@eRTl(wzJ%H|LL5~?aBkqoj z=jmD05GZwpee*^>f{53c;=;n9{t(3*Hj~jOVkYK)Y5c9LP>2r@)DGFI%`RF2y@JL~ z*x^C_*BDGOM-puxpA6%00?npXrl6N|sLE|t*nI_%GiPTUcHay6PwFj}CrUV*y{aaF z^Nw>f@J*hHRXBdNNfcYe&JaSs-!mr+Ne%vu55k-~5l?4CXWE@C0|PMuY3eb2PE}?J zRtKsOWZU_KEtQwwDvTep5|u-e?h6cuqx(PPgG%=md9$LPc2c^NWzqb2%!lW^zh?h_ zFa}-y<)v|L{QD}Q&YM!8Sy4D4-#TWWnS%jf3o-Y>W;_!yYu^>9xOb=;H}etCK4j^= zL&7mW82Ex65z;Ytbxu_{rkZrxBi zj%55RLDT9-N)Fy=D7hmbi*(N&VphE``vKQBupVMs5~H7+4;mU<}UJd!^1a^mOe* zIr{X2Es#?6G|8d5`oxlxPZSUBv}yRCR6MH6`oIOWJ=4Bbq=nx48A5>G*5p8^5vaaN zp@!!DT_$#Ciqg~$BznAYRQL&60qj@M5+y!~(hC%)`ps)uVD%zg$kf}OU#_;upu z=VxRB1B zXgJtC7l*s5P{*s2$)Jm9?=;{%ZrHp?%b3%I%&pT1*?>+|f62|74f3`i{kr4k ztJh>?3(OQI!p}zk;6iIczwRp*lqKts>H!?X*$}yRhc^6t9;4x`psvV1dw}XtzkG1U zxceak(_r)Gv(lW&$T$4Da_ ze{UusR^z7d@e|F9_F|YtN7A#Fp)9AFyt{P&YngRR0{x+_3=J`UVRNj1oGhnDEgF0O zo|>B47@6l+$GIbnJ`oEoEd!oo-QE9#7YWMzwHP48b{dL(VTxI(erGN6t_+x%JM~C8 z>^K|Cmqf|7hM-3V_V4{tU-_>=C}>8YXa7%WD86!jQJ{YZb`{q(XkZQlS99v}8Rkm_9bKE- zvk1%+{zBWSmgow=1GgLp(@BnDqFxiA32_PhvI2}RO@QgmN>yJ#@5hkW5qo6{bTSX( z;EvyuLP3pGd@{AnM;P4Q+n!pCAib_PX)kFYbI|{tt_W&B(SvKS7c-69` z+U?l~CW;VUbATqrNTAGZytEMBo^M#X1XipZ3B!zl^|P7qTaS!n5o-q$Q&w@nXr+sI zz^)(0YbtWT<}Xc;lH_}|XAJ~Ah0tS1$rC`9kl#Cgb($XI)-7|C;_%ho?^Y~jo<`S4 zt``@0&_B+S%T5pe~H!Y!>W-U={STG1s3Y0-nM+1adNKxS=C^pN$)XcDvLxGiniAxB6 z5iN9ROEfJdibKYyY+q^e7}PTmaLWPHW|u;!#MbVBnF=f*sLPeuy_<-eSOoY{^Ao%4 z?Nngl6|?I`$IZH9SCL+0m?3eQ=gA-S14;lAeuDTF00tk>|;k)KQ-dY0nOF0<*k$JSbKgCDCQ1dC6rtxL~ zWWnxH0}wwj$zGq28oPChVQrwKIMe;{B{zqG`p2g(7y+=21zb@vVE)HMB~!c7dl1A4 z5FO}mpOy8i;k3qmAM0N zRf|-#B!uMnxc+wDci+n0pwU&OSOJnB63qBM%5gcj3DzZa-cu~?>qJrJ+MoxJ+(!E6 zy!hgM2mo7`sY#3tWU9uUMDj{Y*!8`prx+M`8?Sy_hn&e6X_nfMq(~>dJ^lRM##?)( zpljqbTjIt%*Czr9D`D@@hBis`A9Xl}Zd*nQ$0Pi&rMoRi^DTrEU!OQ8R6UngtY#sX zq~y-H)xWwA=h?vck;S36K+$Rk!Qm9;BgrCvuGd0;+jY7Ali9B|@1DIfP_0}m;Gfj( zQxfSHb!({C;cqAY#YU!2Yi<_Yr3^fH_*k(+VO6s~n6wQ|HU}dF77t`14>2>5)0zTO zQ#i+hsc0Pcr{wx6NiV82WVc`A7Y6wcE$=Y#Lq;h>c(cnHX z4m1@Ddvm(Yg8E&exMlM(ojI5CLH*KwA`jHQd#?(Ii}s_}etlOaSW5dH4%<%*L21;w zBV}E2jbW>s)76~MFd}v8dBzMkS^jg1O;M#}JV7xabe0`-8}~DyKqNiRu&*4-Qp{4i*K6LiXmbl;_DV+#VcsC?6|#KnuI3 z5VIEt9U?We8&aMq9v%?l7gbvUgQ#`J-zUbek<_-2@(bkG=Adf_NQ>G%vJ+XTNZ1B2NYVw;7==qHkC^AE)Eg_&)7T3ZF6S;ZpZ<*D@tn5fs(Tz#WEAKWWCH8rW2$*otP&zVCBvevKwmL2Yh-r8Vt<5 zGfV;DfZQK+s+No;X6J-6hAx5EFkr#T`BVQpM{TP}IjNBF!Y)ombPn;Qsp$FQXJx1G z=x_qbr*=T{KW2}V4a)YHyh54Xp?Y7-z}KeL4~~*zbn@_Y{-x|%pOYXx=N@{4?wUNz z%p}zPof7px^%2s8+9OBG$HFOniQ2cM!jLW4`En+zWB7Brk*%{kV8voI3ZyU)y{G!*y(4gX(S8R~6D<|z#5Bia~Omm9O zYU_&Gjj1=WRkO+NI+PbMeY%axHroGQf(N-Cec_-yql#YKTNg99-9yM@Gmnn7!??lv z{r^+o(6pl@j+QMP42*m%8L+#Pp}64CS|t{8Kl52))D;~WmR!jb51j=CzZylShEN|232xgLXThg5ByGX zS)gnlH$`RH(V~C6lITMjrhJJ|A<&2C$a@zyJlAJ)lTlJ z)@WBCyx{~ZBopc2`@2Z=Sq$Hdn2=lP-_zUwr7gK+N@8N}C1_r|IKHGh@!0X1Qh^D{ zT%~_B7Ebd6a3}q|oa-B@`U>sYq-2NI$+O`N+LZ5i!9GR z6=q!Uwu(-_#V@|gs@M#evofZy@P_rj;o10jxpS&@OI!-WpjuL?D##gADLnGGU<|$m z{Zhma*~rw%`pl>%ycjBw{lnduyGXr!+1}%nYI>XkzrGrM=_e3wk9oP|yi?tSpT%nD({n|ZYV zWVj7`hc*nAwG=6K%%A6fg#w}m2lfJGQM+ZZRsSJYCC=S1TA-Z#_3z~*#7@f4U^e9s zTls3OYKAG#_T1uRZEpI{oBdj`J$u3qQ;O{v($_)Fn*3QweY0rYqK3v0nt~zdO7?h1 z%Iv=k{;3-0ZjG#2x5qJ(2u)L}oVkXe`vcd6dUZ@J4k?e}@qpu>{}l_qP?3X9H9==Z zJ)qChxfgbbS1|!Y>V>l-h;ElQadJ5a=HuZ+x_{dOK9&6z#zPeRi!Xav|7D5U4}!L0 zFRuDz5Nepo;Q_rbyQ{n6H0rbmc!#P6C2;1hhHc#G3{(F7hri!|ivpUDp$BS9|KF4wrw zXFFM{AmGc#m$<8!O$|^jS%W$W|8n(q-jsR--!?A$-j*<;I8A!WCvK?DtQHD09@(`g ze#tI;{GEA%xrywJfM`;;At+bshh%tWe2jci0%pyNE{L+E?@k1ZC+c~CoT&SP;*ML` zH1R~M66ba}+Ak{gIOU+AX6q4GGYPbqfl*!A?sTaDLb9Ya%S_gN4aPs=^W{nw@hel3 zt0p%g)Q_^!?{oikFDVxt`9|&Vc5+@{z%`>=gy*R5oV7uCn_tBVQYwZIdb6R^--oKjo^N zqYwDVg5btZh`k>@T@hnvN^UC?LSXS%QL-&f@HUUyO)Zdd<{?`IN^^Z;%=ta|i9Od6 zF4AYq=dc+cHMuu^2o_@4x8T4Dx@U7IC#H#U9RbUHUqrmw1F*9n@E4o#eq*tcL+Ixn zCfE{=@6BA9J7tevZ*fmfsFT;0@!M~BBIV9D^K_oE72{0_kZ9iX_)u0jC4zNm-PA5; zCGg6IITNf3oXpJ%=GGV)LTV{UR{a79Dyy&I-95_IA^nGBIiDxrutWI(=u6!AQdU#WwLN zILeGR{Y`e!Uvdvml$;y+OaqkW+1;JtCwViaor|K@tlP<#br3tm@7{0lLL-nju*0T1 z?oV1NzF1CYr&tPeWO&!YB{RdxJrmxBeP5WM;hU>)4fxILx{7vqB)hOK6k^zY!S=Ro zQsx%CdXx~m{vD|4Nu-Kj?`9@$U&h+>IAS*Pe%itNn7LrUfP*J2;4v00# zg0yMoCN6+?YS~+jDE$dxQXW%_hS8Pa6`ln=T~ry91;L-y!FZA;ri@T+5IA)x3`nN0_rArZeY4aQ4Vu{s zD)$%Opu6>aF<4+0IlUe3k#0vF^hjyN~8_wf5EILlk+d&Am2%K}adLDF-*X z%4E0Jh0bgp30f5~(-`7?FAG)5M3y6h<+SCx!=abk;2C~wM~=8BZ~4k0@h0wWSPchE zh26hNB`KqnyKeSXq_^ArN)6DORV!JUjtZwYi#g8k?JpZh(T*WB%)?!x{JPR5_{`y1kO0N!kv=d8Os={&E|_HQ89)}M0ltVo~D;oS_W z*~MMM;QAQR4$t8cssBQJee3SY^?$Li;lj0c?dmi8~jR76@Gd1>csa$&-K zFkLrF&M%;&H;?$}2)QoKBfpSq9D!8dDn9EO@Ai*v2bv}a;7CK`(erkHuj;`v^C0BJ zLgGIc-#sh%p-gb_O7=^f4_Aw~sQ_=}1^S8gW~oI_8)cd!nIVvVWKPJ|mb|%d!UX;i z^iTKye$P^5DGyOBrZ->ux3v$HI|wM<{J8_~t|Xo=T3oABFT`sv86><2rD-A*PZkUo zQPG`Pur(*#zq2Db;a)z4E1JH9GIjsw5?r??P+YBHvc=9R12C-J|h> z_Rr%KwFho}zHjU*0}D_<=D?w{urnTOO1W}=&f5Gh?xaH}LIek6wq8v;U{##@9fHGx zLZIJ2(6`v4%qd4vd?;J=@xSdDHD}Ki%E6QQYfp`lDR*jx%wq5DNSYDgRY<#l5re7G zjCQGTV^NQ8Y3b~BqK}bnF$1Qyvl+C){F$r1-8L{u$phpg^G5KMDGvIV%erL||8TiL zxpl@KoBM;z4NsJA4~&IBblZ))TN{C{^LLFaP#-~gB@8_;0($X_%DpcOrO@zQ>E@#K zu}sv3MQhAyPy+j#)n$H{%m29efO*OgR5&l-)Y*S}NN}2(Rtk&K?Fa=A)G{K-+BYdL zYPpC5w}^^}9z-e&u8Qd@LoZVu8m#w1gdM&NTPOP`)^A#6V#kyJjN9MRDj<~XPu{M) zpL&VvzS!6$;(w4LJE)ru6j^z^v{M+Fd%U20wjuoC!&M*4A^ua{+F5F5ByB0y)exu> zY9I%0+Og#hN3Y8i8Z}lN_=i1Aaip|>d@4ywI`t3s^3__^bW=tsOw7OOT}W?S_QNKZ^ZqfXagiOASvCo}8R zF`4UjzB@6ar|Z>?qbs(rmPLLC5Wd=noEu{{o=z~Hp8_5EPX@F9Gfg(iL3-T?g(th) zzO@ktL58lTX(PYS)%hTGmXPxbEB;!5jxm>(?(j<=V?TDNb8RO&Z?X#Zn@r2!V#*5* zIcWMRii-%S%|DU{`yOL^mokQZQ(NxR@f9m0G|a9QwRPK>A7-2hA2GYBg&FtfY7z4= zrK7S3It2%)JeG=_O`(N-2uw3d<8P%__W7U@`-O|}|1u93M5NKx>EUNr(=s?4wj;e{ z&_LH zdhEjWvhe@3lKT$B@JkOvuvr}CljWzUsUs@BloleN&F0`-#`S9le9pz*lhcH&IjQ)v zIja<200FZNsFN--*md%V`d_&OWv&#&p8e6inBbS<{~S@j2B4HG8eIqKi)VNS;2WA2 zwaXY0fzrWFq|mS#e36K2=o`^Vh%}TwcDvnb)MIDq{bE(&Ri}50=PR}yu+ETcruFLI z^DesSf&SwbhZ^i_-8TN~%Q|^}m*Wb0bJ9IkD_-sJA@{y}w~J9l{UdAPYvMHC*cRLQ zMZ1<362Q}OLnDtz^E)ck>U_k7tYMKGU`_W|!z_?-mc691S1M#rwiuE<#+V9aFWE&KD$7{M&SWiPix>>X60+~R8NX*N)j6N@ z{r%^4PTrpP^W4kz-1l`~?@X}Mon>odXUAKT0$s*_d;^SGJ971zJf)2Se5`7T$9+sD zUS^cMzCsdWwKkJOOHxDI2jt) zr5Wjk9@4oo$v4hx0%#RA=-l~_MmyI^aH$p5p^*JRa`XbL(%&Qzl{sfXTOr@9(ytTD zj=;zxo$2|#UF6mwV8rEtX31AA*+~gmK-q)=((w{tYsOwCk*6K0);^Iye_F$=9Q5$M zkd^6X0OyjF2LqaA?Zs}>Nu=w;@EL^9>bM=&aP#V^59xLz-xcyA#=Q!=W{AEEfc3%H zVw_PVW0D~=)sG8@#iXX+P0${x8;WpXGtxn$H;Rs@YIS*(9PZr66DB3agm&yy*j$8flEkE6Ev8D0-8rc^?-Qtw3 zu7M3xHc*m$`i4NBV+6trpU6YDV8}wY7--H_n4ot#u4W&U8qCs&P#sbUtrWjGXRFCyalokZ%E+$_S zZk7B1Ya+j6OVGHd;T?lcbkb}7$=>Y$Gra=Ohz$WW49qlDY$GvkI$~v-*2SwNA-(iQ zo4T!ZFC$7i7twoLVH)W*&RCp42*0}|%8{=zf^~rxE{!q{IMW><-XUvm%SL@vM@CcFE2Abuf+T0F*RBCNnD!sT<)z z#IPrGZb7_ghoREb^Tb$`3@%QU^1@qmk;Rmd^&PHOuKpy;*NEZ-vVST?JeP-pA zj#d`X18oECS@Xm43)ebGfc2mcgi0!9wr{i&8klk>>9mMpH|IOQn)f^@dli7lub6wKbuI)4elbvo4>7k{qD}95sV3ujRda!WQqMBR+8?$y~`cDhpM8`fn8Uzut24 zO}$~dcjxiVcjXL1Z@_3N`upV*oy%a7OXMyAyn}k=xH11p^1j z4D2*8+2z`Y+}{Q4*zPz>CI}(@&7ebxAK)}YFLi8#ccs(NSlhmNrze=z#ZP@jE2KLsPs8#ZPvgBJ0|fJy7>{n%bWjP|`wOxAjR|G-u2lS;uDU6tLA&Al7%*kO-aHHhqbI?% zTYjK$SwVbMec^T{vyOxuV3mw&enEOYwAp??$l_DucL9zcyiAh`MI`sq(~qjAv3BD{ zu82*`JHtBs9(G){_dbuq?koPI&b6DQ{^SJYFgH1JpKWGh@?97Zv#5JTx6H2_SuK}wbUC9ikA+Z~{9tl+y%86#Ze;b)V9o_C6PzbJDc99*c;!V~-tS7% zpN0MX6N&%Y3muA0l&3oC*qpjh@DeV|{7Rg-m6r8M1vDu8Z&A=EI&-2FDF5nMMbck8 zRrlmNb|US~!L`F2XB?vi@sBc!$_P<5l$QxnhQpsz*IuBp`xaOh8)g&hk`Co zo2-fOJr+>551&k$#9A3_h~HV=J4#7*#{aSDo&QGe;~D8yg94bHf+kOVIvoj{T=~0T z{bmjNSy1BxB4|A-SN>xpS_6N(-uxx;S<}dP+wvs|6npLRX=4Q0BCD@!Dcb89u z`?fDjiL>d#2v*MWf56(`d$PUr)FF6C+b5%4(}dW)>fak*ps=X2q`yGL1|V?67N7;mD8%*Am>UMMlw6CSH|)0^{9d z1`Dv(Kmv5qL1vZZXC*7#n)oioG zV}1fi_hgf$%|ESyAydhQt4n~M?x$~q13aaqrKs_0z`mfOh@?rJ|LcNtO%vkC{HdF` zBw~tAT8oAuSkGZ^3fsT$#?`N$)#mi}JA-m7^gLRg;^e!()ZS1S{|F5qi^*p~)#4v6 zCmjDjPb@-6giP%M`VbQ!{FIW}C%ltWWKJ9y@yG?Ttc_%xMW%^^Szx2&=3cU_I$*1R zdqe^gzMWur;nT&-b0@ z|J!=N&ED_=7ZD|NnCA0xS%M*>lL;~690MRCle#TYx5`Q#pIs}R!Oa#qU~>gN*mqxs z<5(=N5By(d&?k!o#Pu#T-rkn1u4qK@08A!Jxs2V618Vflnke-sA+i>?Km9tHt0W^6 zoGtA-p|=jCp{-xbxE#Pssu93gcEvah-O0x~y+7Bkvjreat97t5ume@E9cx&mteZJy z!6@zopidogk^D>`3seEA>+GqXVmukDkf+|rbLahT3cc-s7-tD6a-#5IFgr%*0=sxT zQpW!>6B+xGC%iI25Num)2o7gk6-KiyTnd*u`vBNhW+e^?^mlYig{0tAM8nx zGe(`JdZnz?lp7$7{qFAvG6SDk!ub_~f)WD$PkhAB|ISjjdALTecA2kU` z=B_{kAQs{w;|k{A@=b_ix?CO7UMmk|hhFJKjs+;}c`ai0`1GldHi)(&F-Kxd$VOnxMbPEUSL&Bz8| zCi*h%2KawxxG6q1eYZ_rtXz=s_~l23rnA71Aevlb(|n&f@d$sk)0RFbYWKu8+8 z1Cs7^BF4ix8Me>MIe2kSlc7y7ZtLRfj^(x&{%;|6E3RFWLjn3)QySXSjKHo|exshn;))EyEdEiI}P=F-g<6;_fz@S|xS*iuNj_t>J zFlN?4rg4Hx6BQ5i3|$A%U!Ljwcv)~E8>nhOC4zvmsA*v^Hvg;ZALXOX&{8+bSiZBr zie;-S%cjdb`?HIEPCaIFe$E+?tS3l+L4;m{c@OvVXRlURc#q!EJLAiJJMD(hyBh;3 z8Q(Ppu9*i#QeWb}ltJm7C-CLQXP;{U7|6p;xI^(-Z;ooVYknX zPFjiUtBZ22BXfzZ3}HUdlbp!TBd1gpbflyppIeehbB|7Kz9MD|QQO5)Y)VHL;O(__ zi9>GMt=b8e%sA@MVlBNyxdX>+e~?~~f0>f{suE=TIXgcp@goT!VD_-RLdG~R zFl;KO$;6-4@JhymsVI3XIWg%0~V%! z_4CJM5x>+F`AM6$P+~{uEt6#z6SoV*c;;u#bZ-fWjnK`kwQq?V-^R{2c{J{|`8WcU z5HheOfT?t5tgot^85&Ts#8jCYKvsCi3E`HD3B(ZGtxx7!wrK|_3YCxld*RW?kT<%; zT`QwMR4W$lArkM+^Q*)kaqYi;4N81^3w@t)f-&J{L z{a?G_15M_e@kgxqw*h7L;Wen@*zmow{IbRL#XDYw8Ep$rV+1RByVWhkEED3od>`fE zkH)|`aTeXqt(tA%gBnW*%((bzNX>_d%_Bv-`0lc?sN8<# z@?#W@{aPLS{{8+P$0_-0_3tsir-xs9470ITzjQ6I`@@O%laZ$>sRj5jn|ypz13&P= z4Q@T3h0+_{?#1C5Bjp;-?R}0X(&`oemn}p;#(4J5C#>C6ggVpzlm{x$874{VC{kN_ z(Hm-gUGv}#Ew>^a!7lQhU>rBcy{}|Mep?4_MUz5;~ffj0Q1w_bBb-a3 zgoN~JxUtzltn_!gfB46a*SztisE{|FUkBgQ*gUsXcNUeV;tXy!sYb>qtSwxXoaG7t z`THhp_DD@C5}O;}?avvCn@^DQ$d>V3vLoKH!4twu92$AZ7as z%Xu5kI94YvY?Nz1(p%&(Ud|Bv7{y`61OM8!VbD7l>AMOBT#IDbzYNmT7zqs9{)f$X z7(wWBH9DWn{k%9f$w1@k>o#;6>(X$yPWHYLLv)A|DSuE)*U3$DXxVzBAlfj0^$25r zeaxMbBDgn(c{|+QzqcUlrbl=R>odCI&!z4igUbK{W-YL1_F5Q5c=(i>rQSP$n z!|-3qdFL`{)eYx<7CG}=3Q}i9?H|sopxOiBt34;{=p_@7o1sC3$TR4vEe9jd6AfBL9hjZ+VQL(!yIBH7@CAYPlM z1FlJN%I>Q~qk#SDij#K9aoh?n^~iA8R&4*L`I-V`&oX?k0)2b7uID^U`zH_zMngIn z|6}QVyw?J>YCfK;_@Z$4i%Uz$cT4Oma3CT_kzS~Sw|bd}E>YIp z-PIyl*|pZ8(YpNTYxYN{p1HN3C(dXRTr+N%0FmG6=cE5VsJ;hOB8gQYygyBYA;ke0 z5sv|(I?r=%bqQ)p5PvnF>5CfO=O3QJ@eMi@v4W%Bu`ps*y61d#5>TDIPjj~Q2x@g<0ME?`C-En1C>s=y8Ss_} zPR3Uht#zgZTg0{y1agAT5>;`uJcc~~-1#?$DTj}AWI_FM&0Z*lNmAG)O2BN^7rt+; z;m~WP!rnV;ZV~DUM!AaPcasd(!Y%7EMH4?2KWehe|951qkD{>MYd}xY$n>p1&{pV1 zkO6*4z3J{z`>3P{BggS!f_Ia!{e`uMj`blzn0$G_%{+TX(k+U1@G`!utOK3Z~9F3mST?2(F;Z%JwQax zQqQAQZQ~{}C!vxPGm`9glb|vyB3}kxmB)5aUXxv+D{UNpW|(~3Cm}v zW<7v>_!hmup|zvXGuyVfMcvO$cHZq7B~(>38Y2Z zkNQ`J;gln9p(n3RMG?s!yp7n*>6iz9zD2KkpT~tN3XKki#$}4G*~M&o1hSCf>)Ork zLbOwWYsC1~l{g%{D@^XgbSzx}=Vs!w4UKgHC*>zs!D z%0ASmTz>`o4%bka?8~xXH5+n?+BCI)#ufqAFry0blD<38&Lq3UPZOV7VA0j0jVlzY zEZTs#9wK(*v+(9lygShH+%*V`&Ou4WyN41zyt?a&>qW5DLkmkCuxH1>`WZG!`tXXEesZ(5lPA#k*WO~2@jbb2bJOy`C$ox z^~s(*0VzmQbcC5bpO>``m$fk_KYE=0tD8gjqFM}}E~BF3`ijjhLav2J>acpR%_+UW z-w)M$P2sf3HL__#X@1pic&*UBXuK-rkkRjmnt2csWF_L5|J@IU>qHN`oHA?hj>IJ; zdfqgSBB?mSTk|5m*Q+1@vq~zYY|p6H0g64!x)27vP>S#nO=Ecc!N+#AeUaV+QNHQ8 zhfb!@4Ril_0bgO?a?iXB0W7kVJ_|2L-2~V4{z(y&VdzJ*R+@R z5xGXNJNZ)zLyM|8Y8%K3kE{ zeW#dfci90@W{$LC`^=A@gYqBn91y51s)$N5x|!nmk0gWfW-ui+OTf$;B#hO;Dg5Hb zJi4l}VP5hg8a^N_%06%7vA&w%|4tkH$7=z zZZeG>97ShZ#+Knu+X~R{8Tt*-e`#|z&5+KoCDj)spQhDVk+IjmU4A=S?O*FNWB6)# z*eym0<*;G)G8f0d97$n&;)*~3xZEN?GgfG=ZZ>=EwXd1_XN=tNe8Ewu zn77THsSDz1CB{byj8~%nJY##?^q|pKEYh7~3>pguLS={L?9$2%GN6Z^W4?2_w3n%w zYC-jwf8IlLX|Vr*9iF~baW(ZkE2P^&n@KKy9j3jrP5zfS<7Uj43R$fNr?ZFjN(D6y zHliodcDq5E@kmEaGUGdCjT#Ynx5dS3OO`-SSlaedcg|7d3PfM~UcyV4s#`{N&0uZo z+8AAm=}mdG|9&OTz^8Gpd8|qjM3$>`y^@%#v6^@@(klHa#l!GBwcO$mT#5HNQh}G{ z?C(`*#6+U_b_8Qzo2h`i+EOqX7m-d+TdU zly)NkW&xV)zR!wQ94#v^Z~F&MFr1Ts=ue+nE>KsT#b&6!Czpmq_EHY_KPiE%a5xls zCoOsR9SJo$4L7)Z+$46`g(d$pA^SG3Yb{D{eR!h*m}d@SyxW2Qprx zy~Q*nAWl!0;aOpT^Z5j$YhxTXz2C_edqr;fQ`6aJt8#)2RCDLCR`%@jImdqco9Qx@ zNQ!lwZcy8in#ZunHL)6p8aLl?A;*H>XbP>&*))}9+f-m-=QiQ3SN7+uF%r;h*Q%oH zChF|uWQR23!t`rAp`ytD-apJ_KmX_iEL)43^$bz~ExArWBhOuCwvgIuI@A~J>T+;` zp=?6-o_;ZQMM!Z{PdcO|S`0B?1a1T_HEg7C&u_}9*k;0wlX`1BbPEz+gdX{kQ@~bi zMvhX8c%N{puUje04LVH@L8meVdlqujuJ8VXmr4mu79Gr_hb0V%(HmAC!R3N>Z8NW~ zGYxa>nlV@{4b?q1{&j!2VNK}PyRK_c2v#*7pjck5>5!(LIF9_p*dsMmarcCYSBI3c zV$r_f=e9M|T|S~TA)}k`#>Fxg!t+a}1|5b~i48Lu*?wwQRu?O$fBjm?d2n;(`S^Wh z7GtzY`Cq%WK_%jP!geEaRXZ+j^`e2Hcv?)P?QbwM&lP61jK1?~mR8X@7{Qi*b`!m3 zQTqF!IH7tp-`1!c4jK$=d(Dk+#Y_wvbj4flE`x0QyMi}lbDvs8${m(gd3;dlR9s--|JG$ma2ElwI~rZ01}{SFLuL!e~*8U2M64TU&Nr z3HTBP_Dm%eZY!pcb0N}*57S{;KdU)RqbcnaV(E@^(OomLbin_K7)|a8qvkO}j;1v- zLj_iLn|knE!`+=u=B4A}Fpq8_xHh=&TqfdEUTUcLSdk1hYLu{sUBv>L zEQAPeAtIV>$GQrSn39^J-yWTJSsjr;N6YHs`N`SzDx@9>AW+yPL3LneO4<-FN)!)+ zd0&u4rqQ8{BgRD^zSC-Ro_L)}7CALI`<2@~>gofw@7A^|vd5YIWV<#ZVjHX|TWcr7 zlO6Q2E$MfdhHQ2h-EkNav`37Q`TZ4^ahJlFrjy>ClA~9*0s|QonwC@FvlFop78eRo z?}3qmUUv=D$pyAZ4#whimheS8PpQwzYACIllkraYzU{ivij;eO=F6|3uYm<)M3Fbn zxsJ#ZR?n3*ga>ES4v2?oyYHKND(mjRQ{c@kSLyNMYS?<`QEI-l65M!xRXmTg(iX!Re~bg)>V9`;1?uln1)x7Y=s znFi2dDfX}D&In21H=af~!f@k$ZF|>uuAeWJ3a4>HO$l60O51KQahS3UcZ_1HiI~ez z{sTgs7;fz<3#LoXh9WF|U#@8#5Tk#l&RX;Q!v3mbk1J&~@Y#{eBFkbKkS|hq{eo3| zn#P4T2=fS}jnG5>w6k9N8+5B5e_z@$YfyLgk(l~u_fz4HjU7|hCi7T#t&|Msc<>Wu zH3mZ+siP?X@re<#S2SDg!$V!jktg~kMU(Htri_Ig2V6-)Hmsc&c$*?}*gcdzKX~vq zwS&5WU%!L+r+mSXOp96NB$m$!8QylsGbkSDJSL<5lbQU`?B}k|xf|axAKt~gzrw{l zs5kp^ALnq8cC()Y=o-M_W?}RfOj~bpc3iH{g!5zG)f#H@?JY208C~YHXo`rNd`b33 zO6nlZH(uT$`ntncf^-K$i&Zo+^*{tK2}*mDxLNt_(Tn5pO31ODx4n}=yHCQUsy$OX`@bx z@FBgLs@ZZfT(I^+@m6P1fucOIygV&i56pQ+@=DM}(Df*l37d zZ>c7mKk{4)a4i4}`U*R8b3d>|j=vp1w;I^WpM+fTvXU($DqFk#9L-w(jLh$_EC{YHT)0l`W)0P>g z8}gFtTyfpi0uY$bY~d{{v7b)~QgHbT=7GjY(2L<->6%y31xe+gVo@e&M96%L9XD`@ z36Zis+3c@0e4c)>nzv+O`7{>nGbo*(&-9tLo%oFV#uU}1wmHxh9D$jl?>nEnAs=x_Dlb@}|n)z=a z9)3)uS?>u|(uuEi(-*gE8~2us18|y(l2m(?!p|>b>)G`Q6o-W-#41k;9Ldq>$X0#z z`~)tx8roNIl~eBkMBR+De=DLDU-p?Ufe8*|akk{n*!nq^s<|iTx*&IZYe82zn{q@y zf1Q^VnlD*V&p#V4VdQ9@ixc|`2o7KI5#{2W+PHHL9vExBlBW9or2?sJw5YC20-^>2)l4~+%$QzV@@D#j zk`#;U2|k*0=EeA5aW2~lopj{uylqf|EA<{-YT?#_KS`LJ2AHVqoYL>#QV7b)Vzq4U zp6`ETU~50R-zl;M6zvivhFmy+YgrKC^RE_#>vB$I=wI#(5zD|51d8hsU%lPmPCr*d zh2k}wzHUquLziMO*H?Xun%MFjb{v8a`RvD(*NCk2`w&!&geByB?YBWkWvicN7kgYD zat?+(gTy@Gq^%!CZikVprgtDZLcc8EKDYYvnehyKUPHaPYs*qhO{jw;75h7}tl~|5 zEif7fnzQpCO5ml3m&mDe$DH~eP>Zy~rm!Lkd;%_*yGOw1)0tm%{4-Hvj+mFZC)0@uZYn+T$6|m>a)Z9E89$pykd%?t! zvpy?E1GHDs+FLuB%!_eyJriCnH)d{_83Mz`J=E#FNz7v3dJ1SU>Bx&m7pLF#5JJ67 z$o&oI!P0y9G2T}TnO+iKJe8fo6Fo~(F23tYpo^&P@mot{G}CmavCExA+6}3+Yeo zg?C~x7J`a{d4+kMOA}0#!+g;bmDRj*qymc$@9UY;WP_SI7vJ|x_)ql`&l^Vl_4TyZ zNV}f|f0Lc@xff-7j?qw)drGjlndoK}I;z2yWBa94Jp{J`n-jlCb;bE6wS?DM`oh+J z{OtK>GiVcK4YrV<5@eny%&MVT&^BPNM_Hs)cLg1mP7;Ez&sJ z6~$S?dIVOS-d@fD729vB`6*W)ao>;EOYxC|F!|rwff#m4o1#1DZQ)K>3a3-b^fTh4 z6VN^wv>C=fuu!Vb`t=@t@-v7!lj4(;JNJXWW}pqhjGgHoT8*vlT!|AJ6=|43k6RYW zyhrS;frjem`5ew~UDsngYm?rr#7z}BzB=+^pzEOI``rl6uyT}giH_Oep77fR@u}vk zrbZC`TC)wKI~7>l4eH7^vNMEX9zp3wI964P*yXm1r#{f(OBCOaqt!X1pUWNIOlEV+ zL1+cV!rl_}ppP{xKZsW-RU&;}${FDTH z^xjgG3zf3`+djrSM_mJfW3R zwO!g*zHZ$e0%g*Jbu$?SJ^Zl4+fO!t&auY6BPEMG_%)uDfY-^)IBjCO4(qg%7L zPxt{fIAJNIrmt@! z=9d(KisrAqj()j3YMJwv7GI@!k_Nj4nUR%zVI$vP9gTf_Va(o7<;Qo%UjCpFue#`L z2QrKYhn{+*Pqx^QKa?;%*QP(E9ulNYHy3fYEIHA6Cuk}$Q6?Oj5_9`5x?4p>22=gm z6Rcgzkgrci)2iRCME4c4f9mbCJ*}HkW}HVnpZXLW*b|q?irPgV(MOQa7;5Zvq?Fa% zs9u=WG3QK1ZigQRkn;~PcnB>!@<@!Tse2y5ErjcCr=K3^3Q6u~-n6}#KU3v79~V+` zbm*!zi>v#+iL9;jqt*1W3V~qWe!4@hz;C^bI`xkyVTT+!X}$@B-FxxDSzoTxN_ms} zsla-S@8fthbCe3?&$e|~L6TNXgS^EWD=yPTkE1gcY+fRQ(06u4Z-;sx6qmdGq@6KTzCUJD68n2_|AL0gB@AVq)yL48eUwwW8JXj%@3@h%u%eO zHUnHqyKbihapJ_(oQZ6mj^!IrV!RPcBM@Y>)YhD@Qh}NKfQe@>KnYAl?Y;*%Ql%=W zav)Xg-7mu~myW~wVP>j|_Xt}TaA3=sr;+F766XY(imrK-e$^&mSLTWqq_LXa&F1^- z$bn*>=UT=UD@_^haxcYtR2$mb2;z~E_k1-v#q2rHZW&L6A0qcY#7n7RH1bbxpOOCS zNza&bB7vgx)KlYkX^CSD`yx}ImpQ)wZhCE20dSjP2Dt}DBC)Hr-se$Gea@1 z2~hXjK*))zskc(jz#)}j9&U!uGxZsKCEh*)<8@S3$xV_m*WmBzz(RU z>>TNYEK-hXq`4{0sXsCj&1Iu-_et9xnD+=@F(Ppiv=2l$AWiNy;$zd@CL7fzh8-Wtt8W_zA0--KK*G3sPePe^l#+%|nM&?{0hOpy(2(lzBGoY`ifY z?wE;Rev+nMRNO|D?Xap(7OdS}{R<)5yw~GpN8;_-*O8Lm-BcUdev>jwq;6YJZgAL% z7(jS=F^1Y)Lyc!K^3Mapl3VKDCB#N8!mwwMefi@cnO8Ih=36q1m$IX7DX!-z8o-yV z5Uz_6CaV#P^Q%pyiKfqF)$p|d)Jg#OxSCEIklz0?24vx<$3-f(7-2)-9;t!`F&0ydWE17q8pPz3;`U%C9oPh3|<%W}JKU8E*{y z^ZxTj7?~w?M;{2P8@;Q@cu;3uvp5;|o9F_1FijR3-Dc_>Ww~keA^joO`JhF0Yzd7k zykd>@YcMDx%`rK2FduW=k27r6y?vA=PMTOe_gVycc&pAJ;s!y?>_=RaA3Cl5soHIG z96>~lJ=gZSTRr{|FalFo_kc}l z6Mtym+MFyW=C9oO|I+l%rvjC zhw38P#$8UZZ?4qp5@rq(uF(*UNpn%SfvRa@?{vDa_cI7RV@tkg&$#hos-Fkn^^s+- zrsW@t-1ke!WWc5GDgKHZ{cV7>Ikiefp>F!O^D1akWEsTu3DT$+tP9cp#0hjTGa9H} z;H*$$yl427n(I&ZofCPQDl62Q5&Nx@+OtG!8Sr17 z7Xi-^1pm&I=lV+;F)uxSWZhj-`0khGvgJ}2HrSB-uM3Ai8?UuQwZm4I95$Ek z=&j@<*7Mt>;jPlQ6z6ypZzv+py_ngheA=a;zI9@($3$Fig`v+-2HPpz`-xPj5e^G)+Dd~)%PKE z#A})1Uk?%76ShwItA(*V!uXTq`hYHV37{*ckv~sAhA*4Mj|puInUE|N0hKsa;rle8 z^EJ&rM% zI>LX`;-Wss~|SP;aekFU;dG%%LdPnoa;yq7qS8fHZu|LtUc zn}0@;RJ-{RWC*5POAl0(necWVeiTy$L31Kobh@WuATod_o<(f)thT3o8tkx8>BVV z=A*@qS_QPi_t4h;9tUTS=^9>7gsx!AH?VD}jXKm1MLDJVuiw0*$-Lx8i>W3?@+0S) zK550DyAH_7sL$G*CqpFSjnD1hQ^~`g+u%RPdfe*74)6n)hBCTyH$m`R95#LJ$iK%v zx~dxW2YitovYiLBW-p}^H#ohhR>GsJbLy+4pD5f;T(e@m!0pF?$*w^zMNkZe-}uG! z#YAd#Igenz5SVq0Z;IJPB*XkQ`sk|4gukqG_^j~c*cQ)8$8Wj8BpDBCXzkpa?+UUH zCMC*|HZNQNAT~|eXF5Epd}CAmVV`54%B@M`cvIz&zs2O8oK-=amX@w|Z~s8r_l~77hJ`u*2B+<_^-7_0fzK zxwi=q0rk0s7WGsxa&$}EqfX~@6re>f+8_;emTDE2vJSlMm&6(B$gg)cWdKt1l{N?l zkebVCH9hC!2!`^+QEkPt-sBWx|J9$GYnxKAD#c!BqMC}=Rui`L`L4-^VYukO+B_h0KspKU{A|IxE-FUf3mw13s!X9;GmJUDr$fO(4q=_#3 zj7I}aL!EU^;4Us;e53ubj8;F-VRLALY{~6qfRa#^GbtftoK<^_KM1#1x|R_(#8u}> z29#t-ps1CIBeh&!GU)XGxmYDl@NzP0GMavb8v~Pzz$rdOhaWtjK0~eTe$geWxO-@2 zu}V&yo_g-g&lC0qZy)v3n7D5g_5G}d*GQ9;7G)7@bAThAsC#Vwv-w)E2iBlBTZ90n zu5@c#`9cl!pVeL6wY-xe@<@Rq30omw;2TI)gr9lxJ+B)98#7kv zEQ+(osdEA4-~_0|rQ|9_?4J#Qk_?9FN<$}YAmCDpBifiB7$Y3=% zz`VY4{4u>aI!1Wyo$%*cmGcL)1RGwNJplC?UdQpBo)tJO9=dZU%{a@&?Ad1o6P=F$>R=zZ-|>>f?@Nz z5Q;7#&e6)*-mPkB$AUmn1W9%8#YcgVZ~JN7&l?E~?dddn9o=m2ZKgx2;|sh;53Lc@ z|Fu;<`LpQ&FUC{`w>93NW6t7o>F1~mX>;AK5Fy*BGD%T7#DQ$Bf>hhbsf*s;pFH;E zcpDs8&rVQ}C#M|DE9jiSsh$>Xg94#SwN07X)Ss$GKgoPprXwF~k8QjcdGUAYe?pP6 z2Cjegj3+kwLE=;70zXa6C_milf*^2+5MSl1f7uMViXs7db|G0LsX1&d+CSJV<<~ZZ z>(~of{qE3S`#`?UVNEt_@oQRbWCnNn=(NA2nkkMdv+4Sst|ph78H?^@l99Q&Coqjb zYSzMj57DE_e-g}FdQ*+)y&touHIB!jC_CFs_N53f0IvEwC8j>Li+=-8^6%JP=8St4 z7ttc@c3O&6`QEPtweoe-GXSc{K$5;5RYB zSWh|BQLp7IKXA@Q(($;`Y)J+x4J0}!&1&Q$0##RDO?O4mPz z#KsM-k3e&qJ^wg8TVTl0#gj*djU;l{f%bK+rWJjt$?Rp?Xxx4d{^sra&ThH@PuQ<) z8qh#U{z?aG@Kjn|6}Vv8=s4>uydqAJv>cV6O!=#O+GE_ztm1GkFFD8!F;h7Ri^(E= z|DvY2v8|v!*{n*aX4oE8Me(TZ^wW11CdGx}W?ll9vf65qcDn$Gn`72Xq&N{%lNs}} zm3~kr;}F|l*MMys?|7;t^+o!$`Rfww(9l--v<>R^ltx4+I}Vc0{pIRR$nell%JDU5jT&3iAN)GW-c zsI}z8ma!-Li!9()U4eSg4@Cefu-9{ZOk>xztwh-I4DBEM;@EPKw#;8It8oST&(3Q}=p3d| z&SmkO>Mksbp#719@%Y9Y^~1Y%QoIx?^;aizru}#d&PMuMm)bTFC7%YW247Sor-med)V8b=iD|dGV&(XV&#f^JPkl% zr&Z-NfBum6+S;eHHh|$F(39<6_vK^{n=X^dvTAEe&?{6iJCd;48ik5-Y%uKB(WWpw zlF<1(NQ0wL_EEX;^dY$F-vK=*8_2OFW9F*k|V=iEvov| zYc=-%L%aQfzg7;YYWx@(b*;rf*`I6I^ly(Qp2#0oaLkZ))JN6D8eT#en%@wL`_W-| zc&ITLH95+cd9?s{3}+N83K(4+ASH!N_ar9;J+0=*-$)Q5Dp%?Ffo$CoL@_A5*C$R? z;&%}hgxU~#>mzj$ca^ky@V?;?eYxbG4rabyD#)n$Q)EmKfAIgt2y$vZ3DH-6-W;08 z0lgd>XtG=qp!Y(O|0!JGwz6s1sgN-uk9>#G^ov_-^}A(syt7`%FM{;pgt%QqwrcUg zN~>h`OdMmv0^T~+Ma?t>x*`-)(U>)jR)sJyhfp-(cu24Iyy_3gKl%5uPV7Sp+)U03 zdckwxo};@iqo2((P-wb?C(jZ0d^?S#-#f6~uiCNW-TlOrF?@|ZG?lT@?9{mY82&SU zOt)xd#9EJ|lKt~HYLf-U<30rzdZXHMkL5e6gB5W}Z%~0wb!V_9KTVF}QUvCrlC=}W zG-Gtjp=yu~;aPos%WpXX4ewU6wTNUhli~t>sJLcvkbi&wMPmljiwU|BAD#$27>XKj zbTbou(MpRo(QlQ$F@rwy-!;hdkVdkqv=rpq`!Z*pn*@-nkU0#%rp zWg)ZCklV0QX7duI?UDMbGrxojYj}Gkc^2L!bBV2xzlfp34YE;wilTeD*lzRp?)gA3 z^Mt9ikYhKF-qFbL64FUpfc7C^j32ny|5U}4QCE)l!IWFm(qC`C6TX+iDYzBeb-9=D zz8uDlvs(9F`@5t)BW=u%EKMK1_~F^|Lot)ny@85}p4Ha)ax4E12$Waz#CIccIUIHX zH^e;4zIoy}wcSOZrZa#Yp})nB$RT38dNc^GFczj8`u?>~$~QzoZFAO-hUWNefYccZ zR{Bhpzr3H7os#iJmZ-qhdv9glVG4yiz7LZ-BU2agr;+*sS{(mrFc?0lUf|av)$T%6 zqFoC?bS`@q&$q;*A%2ijA%}iqHYOxfxc_PO+G@gZ0_#ITFaD~_ul(btpNa_(I8+nI zm4~ee%TY>Yh`}ib?a59qdeeHxD*ssF2mid$h{%DZol`k{;t8in`Q0Yr&Jl&qdyd&2 zYMk(kw9z_{R>l9pUF4LV`Alim{2RG{&8Q-%=Fsg6?Z69aFPOvmP~S>Kl62!3OdEvW z$s4M`;}|3n9AtO{^)TwEpXXKJE?01g*{*82CQ=aAugF1vlQy=F4C4&#GG;Rr@&e&N z)UnZTZxblL;v!weP|rAL07P8mUWms#;Qz9X-5;ahc3F6l807<;QSws-(dn7 zGB~VsVX-anqds*^JP|l4Nx+9n6*E z8ZF5|SzXqJ+iO5dLCPQn!2xD}FR4R}k}-g;(ac=$L%r~jqyDwT@yEQAcVBP>zEyn2 z+RD$EWkid8-S&qaxBJTze3ZkR#S43C3a?peLbI-2@^<7o8LKY)|JeHOc&gw3eWapf zq(tNxMY0ZxWE_Q5R;7$=Lb9FgagdCx%FZ~cP}zHr5ZPsO=-A`fn{%AwcOOx`Ki}WK zJUFj&zn=GVUeD{A>IkL4+H#I6GWKdp2f=l@q~EUy6WzYD)=Y+Vp$l1YDXMWA zw>!D52Gaj*h5>}cH?a-Ss%bbq^{l#>&$;a4LS}Vh`SSiASJP zb!EH0!LAfu)&=Wb6ITeP;;q?rsWv4Gms0u9xt!FCQq|PbLq7|`2sATu=mI30zW{8e zH&9zlH8#sK?RNvd4@Rb&Mcrm3DRBG~TtqUhqdw29y{Fxip_cB@%~RiN4o18O@rC&P zyyxi*7D9gr!nedTg)GBV+*;{yIi~bq(eI^RZrpd2xSYBw-{l_@R`BkA-;|ko(c*Gx zM*Hp!o<87?O|F{5E37>x_c12`2SQER{}RB=h6fZVEuWKwbHC8<3^yZtLwUTwX&YC6 z;+|f=U#^}sB#;;XkteGRq)j?vCnmxn;iYfQ?Gv8qIBf&;gbOx>cQt|l@VVx<_nNg} zea3xrQm!ID>{~Zq*I3(f_gDA!4gpIgauBQSGydDp-FssOfI}|#txuKR(*fjtI_?~6 zyIV$8@huuJaZ6KYd)L?S+rfeu%GpICuD#da1$anObsM`l2<#V~Q-+Gg-%Zn-q1lOM*sAJK*SM_s?0+0^6rP%)GH4 z${x4iH6AK1%q^|iTO25p_40-l&Ub(1n?t+KW;CD`;F#0Ln~0PR{PUx+batFo@67CU z|4$7J@+w#IEjm4NL_p!y8P!f@FNeT%1-UhiOdF@E!4SrtNX~^I=6m{?x79LXi#OXX zO(0}*CI;I|(T?{R?fwAV3RS&?P;}2PL)uWFWGCh|)k!CZ4WCi?nOP+k$(Hgs1qeq+ zzQ#QeSgM0WUq0B-_MFaE#-#F}rJOtDJ(>2rSsqkWQ(JhYxHcqAUweTx=(YGc=z?Ln^Q_=2Q;m6Pyi=*$>R62x6r>GekF5ZFu}d}YX!mi}RF^RrEQ zS*+VtKZ%>Mwila25PZ^6WhinEigM5ZpAD6xjZr3L@S8Gy>myUi3&woM*Z!Q!fgdLx zhJi}^=iOq>UcUhtWjeRqcr-nM_V)bCk;+0!tfp9CU%OisuTx%Gg1d_1h+Z`ES=rk5 z{6Y>$lYje&wF}j&XeZP4WRq*ZAHUk0{yWFL!ICk9cl3(dV1)!9=gZIQFCP;DDT+;1 zfi5ElYo)EV-wLNUt40URzFa)Ujb>3z{GjVkeA0#E+=p(`2FTc2`?a5DX4a(Iwb81E z*^SE0BemnK{_N-;eEKbei)68RW`xA`=%gmJzr7}}K`ZC%VXfU5I9l_$p)QwCW${bg zqqS;ATZ!Ay-J4RL&u$>s9FE6va38$4#~}mZ zqKSadNt5kC?+RC@)<$8bx$3}8X3g?s z8and-pA)nU0>^8;lHSNx{PG8l69%%86Xj#|a>rh($l;asTL7P$@^?T|Y;jBnDLP9V z`ztj3MVMKTGuE;QZMD=-WI9xD>^IMe*JVzOwEhc9lh4FGUdVpON?;9D)7vUBKE#KD zJ}!xih@>ZL2d?4x;((lg3A$uc*LvXRX{2pOBw^eFZr%=9mY*t022}u5tP*PFEZ6=S z?}u67mii)y&+KFQ^yXXx%_c1_L`v00A}7o(!nTL`@+wEQ8OQs|Vt2LR!EvNt*Ga3f@k1LHkY-1 zY5$qDgB z0Ti$3$#N`j@(Gs|+1+MF%w(RdrNn#JE9 zzbmzdsu( zcTp~{shl5Vwm3*I4R$G@3sOcTKD*i=#H{r$bnyDTM(VnF&CUai?W4@=H%OGd20S$w*76M zEw5$B5jz1PkYI;at*O3^oPj#`t8i@mN^ge8OSJxWFylKGjle(^)O9T|q;jD?h%-3BRFI zH}#+@fDnae%RII5HQa^KaDrvG=27|CAd*c%GHG8Aow52!*K{oZKTT2tDYEse@v#35 zl3>-Ogo^o?CKX4f!1ZBGj}L8K6`!b8js}0`HbM^423~v>@>)Rlk!7@uZ8(^WIbD$S zK)$a~_GWVst;OnXHcH3?UcmpndCdeFo~^NlC;o&F?2}(y5^Ts3%3oR{@4VKp^C({8 z)Dr>L$uHNUaST7o9rBMA8i-M{%$w#}75q2j@IFqNeRt>-`Q$Sh9nq(g27QAvp8|;T z)ci}$Hz`xe?H}Dxk~1;3yd*Ev%Ox;)p3e%+EtZ~wQCo1EF*Q!7(jM23@ciPb`JdqG zm_RayY2kHAUSx#l=^ECRQ!HyvW>M@G*F(jw|2X;~e$+pxwcJDlTM{%s{!6Z|LFy~o zu_DML?>FM}w~aSbLMNq-a%jk&O=$t7_g2|p=4DFA3jDHq)xYmi zh72rov$nAh)wKq(uxm)c9FXWk#GaHO?19dQ%*N>z3-;nOwE9m}L4;L8OQ~ zaO7;>EtGb#oUC6A7XpW~;MWu$ZhS(I^%JHwr5*QQx{f{A$?N1YJXV4vJ(??F))Xzwwv!c>gRcARhd z5rh}ML9H8vVf7OYPu`ADJZ_^zLrqm6u*`pqR)!3;C_>@mT;Ea{TmL;z5K<43GK}O1 zQIPdhHSy5G`%k^s?IX4#Q%T1iz7`t?X4X7L>n!llkFit1XEhgUQwZz>`C%r$v!^`9 zR)3l(CJmRGr%XTpgGvKmeQ??Mb;~?$vHVV9yQOfiVI_UD<~1XjO7zrO*+-KxZ@Ve0 zp`(@eyBY?_(i1*pwRQasO+@Io0UsFSIap8bhmieOL+vG_MypxLa10iA9wnNe~ zGUfEwI0tE(@j_J|{fu;b-CjrgzQ(d`#2Uk0W$Hj?SmoO0DZUar!^x$?ZS^1Ch&gWW z_=SeS6L#A57+eo305@&CQ@j?v&AxhrFBL5zsEJ=tfNH(O-T8`C(lC3vkLKSiSVt(k z#t?+7GUr$`FZ`35fXBqivoUk8y>xlNLlNKXcXGZuQOd0_acbfjXr@@+f8yv*g3Mv} z^_>x7)@Z>40+|Vij)+Vo@)b+l3zuRV4a~U<%iCIy6@MDeaI2!O82%fX1q6Ch%&=>o(%A^xOmU7&b$Azg`Pqm zgV=I$GUCxOe>Lre<{wgRTyP!%CC$M){xX%)B-9g^gNw^@gtY!_d5;(zn(5RJ6@B?# zkgO_TX8>F8EXch>oaf>pY4a^o92g6D!@#IG-;99%Gy#>uq<3XXyh(N03=xq{rkJ zLk=W$N1TxH^oUhWrElYt1l7+j|AbP<5Rx&y!(L^My_WxU%Ykt~-N*tSt0YQlX`gLQ zQE|C;I8Fm~xl7e8n^9DO!=UA}AIcKx0HG!oy%ufdn0e#FDRNOfpAT)}cZcV;rw3|3 zzS!;?R1{)E7Ok8`h@`s=MfpMh^e@Lw7K26hu&H|i#LY>7<@QRKZ5o2e~h^(0qjsli}6iLLfp#y{a4eDqaDGD~&n>$bo{@Wv`to3sSu~)34L~1H013hOYdzrGJD%Y3J{6r@O^DrrNZU$iA5?UAD9ISN_ z&1ryw&DyP&`PW@M5gw_%Hf{(ZEcvHFc9$Vpp12}Jnd>M$Qge^OlL%Jdv~F13XKYYx_EK%v~-z?Ig?O1GC=0q-V)_Ltm)?`BX!ahz{! z`1_1~fX{=6o-E&X4zUjnQnjNg@V1;A&2X3S4G!n#!B-H2`Pt$yPoesdY?`8c+`yu; zH1p&^UMoW;d(vPIn<7^684LIx@9P33@@}a`mv+}yn9U)hcpTUVGds;up0)ip>$5<` zS_gJbLS<}BuA+)CTb0?65io6CI+KD?C5)?n!3`~XuA6&|x)=@{0{Vuhh`jVf$+ zHz(_IfNQzN#>jKXj=?&R(b#l27kt9}sMY8XU%}lK!K?*`EVqa49t-eUJI=Ake12HM zn(A(T=`MazAGHYp$fr}6bb;=M!%m3UFHMiODR&kAv*pdY3-|p!cE%uCzjh|YdQ(^L zQRA3nfRc`ejmNC^Y&aQY<&}TeQI7I?E1R^+72c}c&{WGw?ymKqs;!{C<%Tgl^GZ%w z?N-9dCx}2kO(IGZDrIj_F^@9Xo00wT0oCmE21e?Y0#_9l^GNF-(~mTWo(SJ9>?GA< z`F`F#Za&YNvL%w_xJ4g36`EeN$2QmdEW$-MB?z|d_r%{MSe0!v5-0jGFd)0N)$EFZ z`5i>8*{AlkS04KSOo>4g%HnM%HLJDWhxZ4al6G#T@}#ue$mG! z>wifExci=oO7L>hE*`o*sHnH{27}yPMd(c1}VEvc2)Vd<5NxXLV7uF07CZ~;s=@cJWnccwImpwy>AqeQT8m`N-%0Pnl2IskQu2YcX5 z{Cckj^cs-+SO;+Pa#ujUEOnfe%>!&p+$FLn{d|2ns)|9&e zHsNm7ojTPAu&w7XP4=3xE=fJe4s+EqbAdAr>2R9f`j=zZS;s6^R{Ypoj9FJ>T&V(a zoB+bVK7$}$c)cpH7D3Bd@7DE!)#0aeY%fXYhQHLDVRzN;=58hSOBVJEMxt^|0)r2% z!lSA(2$MF1lpAv28xg^ZOm0;h;XXAsxI&vW&uM?pPnkF(g7|6lk_|S+=F6;^jtUO8 zv(lYOJ zbBf^c$5uvPDO%p7lnc~O%GO0$-Y`%iJ%sE`d6!+jKx~#Sav|}`S7V8hj@Zg}b3&~$ zOkV1vz>luzfURKn#{#E*q6J=#G8PF%f0*yq2V8q;{6qc$9MSPSD()Ts zdK*~rE&=^$P=SRTy*!{VXM@IOpdTGy+}onycHaF8*}4A0;8pLAu&A7z)AmHP#@=F9 z(Bxj+l2#70UjQ$&$L%N&eY8CQgNBV}$h&PZA%v}ML@0KgD;9{rYuETz9#6Bj_|K|T za+U#_%jlgrgj&^Yo^6+IXx4cZsr*CF`xZTs?pAg+Rm3d>>4_!c#tr}D^CygCsErJ$ zf`dj1zLlS3JR{4L&vfYd)~pvl;d8?>+`NTBT=Rb`;smy8-<1wU!3_Qrru4kb7AQ|$ zD6_fJOA(=zjPil2WH+Z0C6?*(UZ0#k|D%AmH`egF97DtC3f^ntnqjj}l^O%dujPvi zc*c}dbWsF>5d);{ixC|!jU^`xie-bCSJ5lujEMs^)O&V0) zT~qDNBAtvZtJ^I@Y%XHLmz|XbxY2e-tT@u$4KxAuNUovLt6V4`N1Pk`Egpub+%OGJ zbYLP=M07MHOV;@X?0lMq!#3fw5S06vn5XgHdcpfa;+n_jp>dZV&siTJ@7$6!suru5 zQFiExc5jS+5P$8~)zO=JxMto^I!O-d@zb?cAUxgQ4TvCg!di%%TSX&)DPCnr%-6ph z%P4zRg)RQMA&_xtr*QgN@ z4HOa!*`bhg~mak*>C|qAOVbf`)8)Lq2`84wwYUz}^S# zu044eHBz9thQ-QBlKaFE7pm4jyMrCK)lk*3v5%GVTo#vuRhd!?nJ~D$KtK#;I?*`d z>ehPYQ7fW@*g3XrM1B62r?3Am9ylayWaXNFC#6uB`#7fsJDQk;lv33p>W~snnKyoX zpY@|-v{&AazxF-wd9My!xz*pkAN4)ZCt`b23@k{<3lprECvWeLhll5en3?Q4cdqlU zPaHxz&;9?X*3;-@G}Vrg9gV?e^mrZq9NU$*RP7tvDniWXy?Fqv`upbHIAt)!1>= zeO$8D4cI{DzY|Cj@=Ty?hmDXVx7mS`+c8_{rnj#?Me$~puV_+5Lszn^WFG0&20=q1 zJ77D_2_|s-H5v#|q{Ug6es(1WQ2a7v*ACB_B#J4&bkzj@j?`)!L_r{HIl* z?S)w1lR7BVYKnA}O(d{1-90u@1NMf`$-c#Ce|^LNPdjcE6eNlJ{i z9)JhES9zkl?&HMy>KNiGj{{*tY5b|8DhK~s$mqmL)8!pCZ z6f%3Wn`Pl8kyAX1F2MQKV)}iS@9UQgn=wLqWU@d$!Z*2!hs{ziypW%7E?p1ufO?qtg#*Ze&eLpA<xS zxDcM%s>Dws7wl5UaU%R5V%_~-$ebXlXzmK{l}T?3(*Bm(P$8a0#W#gZ77~`CgB(Ym z$)*X)cY-;x8vU>vJYZ3Ufm89TL0pwm=OnUpv8M<{z7=fP0F~@a%{I;-0yI3{s`TG! z{A0M;T2Ng}Z(* zy0rNdm)KAZOyypBK{kH5T1ZJtAHCAss20^)9?*NlsiJAio5tOs)!QRVoNM<=MF(D;^^e1pt6=yhEA z0PQ`4IH>KttL@$$p=dRWtK?+cN+JEdN#uSziw==jOmY0$WbpIVyS22bJNKwpU&ha! zNzzYrfX_&@Hw55gES83@BqtH>51#pogn7PQ5zzbx^H_bCoE9H)VTbuk$ZSNdWQyXB zf(y$ND0gl7x%~g8e|y=sl((e}@p@#k(CMih9h8g6HCO$F1G-oRCLj3T%ZtvX z{Q88-7u;hVgEEjZoEJN-v6RrX$!8s1#V>7Whqq>(70|Y(7#DdvD>NuzXdn{5TSkON zEAMJFOlO&}a*_JFT51X{Og#>INQGPt*tzo+tph{5+d$EcJcz9Z(XPOEf}@o!MHH)A z;-VhS|87Q98=hoDq`+fz=KoK4zVNCwhc~nMz4SLuZl7%N?|x`eD(07an1DFbd?6nh zU+&*6C{3hH9m{OAt%6D=GKY>lbJlu*dR5VYo@|@N=6?PA@@6?76L<25CYo_bY}3-X zb{F*+b(oCj`17T#MYE5mu<~<7tq_X!3JZ#P;nb1yc7!Zogj09y^vMy4lVV4#_Q?5v zw@itLTS}45^^PPvD0Jk1kJ`38$)VpQx|_MV|Ek=nKBqC;C&5al@Eh!LE|&T7Q~~sTGEb`c7M?k18vPZzt-X$UshDo(+u2AulA`2$e%l^HBf>;k?Pyb@;E~KB1$Sz z5m78S(U*YhHMtd3Ey(a1;&?AHGq;egZTHj-*d%$e?6IYbaVs1PwIPTvyAxK>0+oY4 z(q2huP()EIlB>Ph-18^x`0bE@EQx3_NB##oOImv$3M0uK6x8yLy)gR*P^Vj;)EW5x z8S?vx`=rR|=&kIVPQ9-NBqEX7Lmy0oF4zi@)?2F`izp@c$q(M0t%HZac4sLU$GB)r zo(OPN$4J5=zS4AdREODC>MIt-X10hZz>u?(Jczi$a5Ficg`n>?w}eJgtS;WWKVI*-zKqA$}s$c1Go=|GAr3L-#))pd`C#uA{&a1f1 z!)Z|jJRf*v>55)c9b7k(?n@SbK~J@oI<>l&sd%*$9v~6*Z`JyAi@A4bxnNB^FFNVA z+1G-G#@*~td|LvON2TMJ{F~Rm__cFygJW$C^j7n5)Ue8%`L`tq{{I@g?fAzO8m|BJ zH23;h9AuUsiP};H7j%8S0?3*}zRR0Xooh&e#wBS^R=V&r8p)`xFupC^Lx}bXlF>D$ zy{GtTfv}_ppdc!GpA=-!T(^WwAOSIL>YhLPTd*lNTk>qT^W)u z)fFAoCwC3NXK&%ouB=orCqY)dwo|04h=7WTC~eFM8DB4M)kGb0k!3dP#Du+&<{d)G_B}Ab zHlcaF1$WIK=$d{y05hgkVOCjf=V5KZZJZ}8_Fx}6KKjMR$l9K{+t zA-mZifI6`7fH$FqH|i9o{i_9|?4u?U3^%Rb{P)qk3n;o)-xt`w1*qly?{QG%{s<>w zwoO$@_`)9-Ix*uF7r^PNj(*D*mrSf7#7?Y!G4KttW+>zkAu?Kek#n7|UbP?UJld(0 zXqMc{cCMS}y2Yl~gb%~*bq=|xMVfOGJTF$?f zd9boqYZ*vZfny--5hav()56H^{6bUwGl9{dvpl=6kFxn_= zMK5)o@rDSXndtD5GwRFX4B?U`DjwiV|9;Hbw{n=Y|KSS; z&uH66j;Wyfi*ow=+$gSU(_td8Mp3)(6u;`x_)-tcNt5s>jV!xklyTAdua)oo`9iaF z!%Px&Gp2MWN9wDTZpA#a>z(y3IiDRvf?STbO;~=cQr1S<7eH8AjQcE>DDBV$8Ji>y z#u%`X9929oJayg8ryk&p-C3c<%DWGGWdn-JG|vkBP3pi4w?tO|+=w$c{y%mA4)_ec zK{)h=gdCYTlZkffhaY}>o~pGbo+?11dvfGk1l4|m%p%}T@u=rC{Y{cnLTq5aC%@Gh zx$BSX=XFc$+@ZyOL+>a}Mp$?bzG=78s?Wr2C$)T5OT~m&2m&ciS1M#D-m$_)HXnvQ zZRdXxJ|x^j$KptNrmUv3FgH`)5Vr+;P54`Of@e24MiyKV#;BQRoAhsn{@XBGB4tER z_iH{D*0z3cYHZ$AsK(7)tSP!ed&$0r(nw~iPwf}w!Lnw$s@IgbBmMI2ID=O?*O@Bk zT=A?i@TX+rty#}d_3wR`$0GX*FRK-K1YlRRCLJn+Jp`+ER*nIy-7$B3PO?c$?-ZKR zwtL)qCN(3?w;I9FQeT=sZb!K7=x- zw|1(l^EeIVTXL}9s_UVjSFK#(Gm`NQ`W?%F2?6%jy617B*BM2ip-bwpdg_A21urlpGmfAh801jwI?jx=}esktXm&K{+#uB=y~t>n^zyl@&tL1TLQAwY1ey$g&P+PPQBLG zsehWr*ujaF0Q)*pZQ2(W3p?#qo7C{ajw)rvf-V(;?_sKW)J(dRb2Q%W$VJz%J3s> zd@HJyf%+o}U;tW}AENw_+&eKvctSV(4_prH74=a{L_pceB8r3Rd_2b9;*I7`gD0-( z5egvC?ilv+8z;E-wjR=`^H9!#j40RMyr+_0Z>8ATGDgf~y{+gkR;At9Eks>;hSw^4 zQrgu?HCwt6UZ|A!88N*h8zS{2dVc_(okh-glx5irdGsdEb@d-Ut9@U$#$8=5mLtR0K0)PZXs6j*f+pUs#Eo9B*8&1D zo=QyqPoVFstb)(rY?CV1Z_^;J9hTt6@=KJ^7#h^q?{jHD0ZVne@n~CKB+C4uc zXKrS(^oYl5^h}d|^O8K!R55W$Mhr>9&vpKQ{PwPvH>>GM#Kz35|C4coGw-`j7GLVW z2ZPWk$|-4X^s;17wbHDX0*z}ctu6%$##xTtE~i@Tfmy`>CT>N#D?SPCk5Lbe0yCSgUcY6};kpxvZ_ySdcIoppfc0>u|Ci)8?zR zHL)yQedG*~T3IQK3>7teyN;%FT}D^V)L(=rlxj4>?QBkfD**<4WQ!kk&C}=qr%DG7 zJ@~p;_b(@{aiyv1erlZ8VK^2mAD0>z%E#mRjp2%xNzgMKi5Lr0&=$@6u8N%)ka&5}RG} z@T=Lr!X*<2=6~=S*$AN2A|T??!N>c7Khe%sDGp}X1%b@~D0{J$4$VC?Z6+X5)|kZ9 zu%5dmZH|rl$*rr+d_p^fWcTQYiYJVwiIs2u((1czdy|1we$0>U z5;-Yd;KhrVeAr%+%8{QRtug8%?+>}jK_ORn_~*qZ)g(0+gM4xWULLDI{VKuGbfH2} z6%MZz%WiinMK3)Z z6t#b?0>b1+i%Xu5VEzdqtUQ0*u{Oc66!cvXwaMW*Tl2~W zc;-cRs=1o1FekLbi|5UWV&t5Eh_wye$(x*FTs)s9Iwc4df3DpMcm#lj2I=GtH& zv7Ut-y+7Y-KYBye$tdQDTW{LZJ{(8J53P!-M}}*rV(V2X zxy0U(*CD{H!nz-Gs-;hU&y#FlpA=F$sezqJjyqGN#QJehJ*@8xnZSu`K`ASV1E+bA zP`*1u=DsH7x#MirJU_9u6d%~6cg(smB5Z+zl=H~A>@txR32XdJ(f;{?zj{?YK+dtz zjPaBW!r!-R3YiYOwr+wsefrpI$PLL~hj7qoU)H%t!;R-sd#{%=LnNztwutb~F zJI)U>Kqkt^Bga-+eEGxuR!x%?rB?x8_LflN;@r>vs%GETJDm)AlNtJdb3X9Fj5MU4 z3gd?sT&5B^_p7-G?YufUOmLuvu!AzxqIf< zhf)T-l`QaS6UfW%g2(BG9#T-@<9te&nN!U*PPA`EnM#XWH;3&Vwzmnz691K`wPnHC zOna4%KO3&7^VMZ*>V+heq-Fe_CjFBI#b?SM=NISOC0|6OrxGNQY9vcx3zKa3aI0VHV+2C9jY;%9}} z76hMz_f(Xb>tR#g`}SW^7WFb#$<$t>_jv^=oYjr!^+;VdJJE z?3TGtGB=2NbFZBo}gvx`mRnS$54<1SU~Gh8);YG zBbIMgH22Nd`ys`W-q$albhu?}p&plGnD0H6eGk#zG0ac3`)zvSJ1t3)bO(M9~VWveM#^|ytt=_geOgvN6|Q5%1rDtFyM9sPA@ z&A`u$KuWV^GJHav_xNv1xK6nQ?YmM7=Wj1G8PqZt7MI{nTq-OkswGIl1KY#qT^7o( zVr>$(9NPK|EWT~*S7r4#NrZ-h2J+K5nvnp~Wg2s5)r%TO%73kAY8ZY-9zZ+2m+1^V zkLKvBZ#Fwtj-}09xzbeDcIiEA@*XWNG7TFtf%h#nxnfd>h@~%0`L>q{K2tPMg5#K0 zZKt-UzFp@A?>AF>G!H%|_N=o5*6jN_h2N?zLHEnvYLO|u<|H$>nOQkM-pbI#ea1!+ z#h{+31~#;H#S(=tpX}o%8{|DYaYDqee%0_$EzLwsvtU*EwcVXncwd|wNRC`b-|@xO z2oUS%3w{7q`XE}*)!bqE+Zm17t{EciDh)NX;J8jM-yh(xGP*;gGIAJN0F+iUGZ0q@O6#jE>T%u{A_JD?6qg z(f-~Ub0y_=*4c;yE^cPpVPnyat>v-xJ6Ct~CJ%{2*8DF^J`W2GecF7~$=7Sgy8aI_ zn}GzmGbBKMO)L^!+oHXyLZ37$>Dr<@?(yl_RD#GemPOh0m4-69_i}r)JdX=M;>*s_ z``ahib%A+caVv7_sYvLUspd+I*p$j!Vd)0SF->r@fEdlDL9YP6d9#RN-e?&GHx2l6 zVm_yTb2AIKN4FS8A?QAGe3tck#3tZSeOC%7t6Ue8xTR)p7yJMKEVo}Jxzb>=uMP+s z^>r&Y3bHc3IU=^B5nA^e958Fg!N|T_Byl+y6IE0`QL?i<2b|6C~>y&$l8NmwD=v zXuCP9TDWORZ`VR;=1_OCuEJFdZgs9aVb_#2USk%vlEtC@@?1(CM8-7>&CA`a}JPL*raxp8>RR^FdMZH~ufo=&kzo67bQ8M?>I; z>7Faxh}_aGO!uMKrL7q!SdqY#^+&ZePlejd5@I)h3ton>QX?Bt}q>KHvg)9(_wZ< z(k5{_4Bc7Y4u!N!$XSYn<*=xs%V*hkc&knXCx`YBw>PR8DIZgZqcdVCmMOwucCFd* z%ud1cv!yEer|FAkcl?{erc9M~l%uyyHA7LOP0$@7@zj}?a0fiq1h?qO`zuLo#5@{tj3my?9 z05f7Y1pgEAaI>_x;tLTSn*Pnoea*R!ll2}^$UZ9(6mkz}JeQuQ?Tywp0gP0Cod{{{ zCb~O~cF&Exv^|U)kR>mAFqPn>x4tI;-&f1(n6V>nU;=B}A^>(^Y!w|5iL7j+aq64z;2$Rqp@8vNaLx2jFI^<|o!YMKZ7gb97UtgFtMB>Qai-sjFkQ*OUo zQGoBT0V~lsF*R5SKdG*;v6lLn1twrRJ(l?PNg?t2aVdJ;9=uN#)9tY3UliQ!Ar@-S zaaV@wd&5rp%dsXeKog7DODtlpk*e1Qi*m-)=a}wNMBw$XP6=|8i{@rCVOs(McB_^hCeknRDHSVC zfPl7pbGEHud%~HXbVQqlpIBJ)vqt^juw}msL<8W}-)_Qu6ZEvh-w24w;33)RN8%KQ9B&v#Sn1C&Hdzsz&_sA+5AWhVh&qMb$E~$gScSl$B(@s4 zmGKwRrMA*(RQ9ZCbRG3&fPmV85I~8T(C|#g`q^yxVH7;a6)vd@$OU{q%A!sM_&i~n zL%`){w!wCHFaQ+@O9L>N_N;c<>4q=Q)89FNJ$$CiN{+;Nv@Y;buqOA)pM9eWdV?Fr z+Tmxn?I6+h_4=iGHMct%K1Iz_V{>Z!P* zYAhQ>%0t5}WT1YeCC_w@C1am;;GG~Dm`Pp^3)&K_vcg`DC43Zy`#n|IJU@9<>C_I5EE^!H<;_f7Z;kl_pk#LOE|zv zeiJTZt~lt#$h)3X!OtN@#XFdjvCwMcxr&XUq?2Zq1PHJUZuuBVgEVcfGnL@(6ktn` zxRPU}Bg8R4h8+T+xjrN+jXYK-8y~B6^5}I+KvAEuV>48S15A0gp7`Rt4Z^LWW|(;5 z8iKArgWfh`j4^fUwdIWZ0Cl?_RksH>A8QEH(BCl8SFZqMpBbLJn}{mC6WJ|Zmo2-S z#laL%^303sFn(W3v-cxJR?t-<_V4SXTNGh?H`EnUHYrHim={Z5ahZ% z@<$dNa8>k8o+nv+6Q}8F*1cNP{b6LVs4Qbc3$(DZCCC{yILnP&Km}XOLSuN58x#_0 zs}j-ImaivZ4|;uNN?ub2>YG8+VcP?`(ER64FDV%5c9z?xTeMW?nN&SI6#8GJb%@?F zaH4?LuTTX}Ts5={461sTBm=kiR>s!Z~@S)Zntv`5_INf(# z@GcyCI6Ba#syQRN$~U$o_)?#%^}S?0E9`|R^}Pm@Cbke$IN8x4>zA!Tngag4<&#$m ziWir|5bho|s{p!-Dai7$s>t9crRd)SaQ_nCbSx^NCSjf$9P<2Jz);SA~0-yJKmw?_pDqr%un zo|tDr+Jo=0#DI;Vb}E)#uNSM#(L4N#hQ)Fbm!B$?VRT$y+7{YKFDoHXGXBOjNWbuW zC_BH)T_^RmvhED)hr2~U)J!fo;2d}$M2@3i<<{u2*BpLUf&IT?=AYBCRF0m``c$h$ zxAZwoGe&3nei1mUF);Ioe_nsI34?aq$OR;QK!&-WU%*V`imNyV_MD2cia2f`V6Z0O{0?A7-B zk4AxV@W5c&7O( ze2DwCmLm4hsd6=8_VilrY3Oo#ih@?atTdnr?s17mI13r_qKQG3;go$FU--405HKPo z_p=;Ryo#cRjnajEDmo4i_WU3wvi*lxD?%Eo)P z<97N<=Qz}03InIGi|=)EfN_`3!{$J6IWMik8pASPYzsCY7c9?KS=}Qo*IlrW9$5S< zRGNJppp5RJL=7MMv0vf~>s@ZggpXh0txQ{53XY@1m=XdPo2(`m0WT)dz^jjTpG)Zq zSfM4i4+Hnje&La zd8+0oi`1`#Z_!Z5b{0ueD*2O)xyiXC!wl_EqZGAI&;Jl3O*Cg5EZlCCObZ0~L4KDA zLzX}L!!cam*jUf?=IMHy54($Ff_j0~4TME}W@O^aL2%v~((9j96Ht{=G$$>?6&V+Z zM6!<_I}x_0dPiY$cImeMjISC4hzRV0r4ytqo^46y<$X&x2*iY0U{$N#Rh>U^4^(c^ z?fsmXu@)?ah0%(Y^|wB~&W>98HGNa;%M0JGa94Kf$7#d4Dgl&%I1t}-RbN8Ieb`>* zTj~b0tB!uAS(nwhg};q^9p?U#VKL{0AjE+76tDq@uXn}jyyC->(urADMiBx3MsWL$ z>kHY!2c^S4$1%?+c!;LfLsmyYHcOBHjsN-Gj6Fs+K|XkmVDDqyv;E=8bnu;}%*u3r z?!LxM$*f?`awh$wTi_~bz9iu|;u*Gr6W#2*&+Qlyki*A&dc{z0|35JvCndMltn(}@ zEQPe|PxdM{<49IJw$0XZ)@r}3e=j=>pV1`NaWL{2-fE-KusHD-EDaatMLmz+%QO`O znj1ChxP ziARXrEoi|x#7>j6HI`nvwCmD$R*SqtKyT1^k2Jd>J-gxj&FB=Z9$}aErwzF-KfNiA zYMrB1OZ3-V0*g0J4gfGYA0J>Fdu^Y9YOk>huu4mzF_d*FCW<4cl7SFLVA zpQ`es)DQhDn?|;qU_YXbV88I@5Fc0~uKbWvNk#ro#Kq~^W5W8x`gc^a{(@!bDs#m% zdIkPe$`(ScS|7JanaK?uk1Taau!+@|mTR$}dG0W3uc8&@Nl7K|b$Tm1y)tae8=XLL zs93-s8c_?YFl6H1dKJb-`_pH?`|<;9EI#4fLM3zR~z~T63cPp|7mG2H6-#_2TG`sK8K4m1N>d@nr#QnHF;*JjPz{^O~9TCn{dN z$&U5O+&^=RIeOAxR$M$JIz{2qgypa-H_XL$V;@r@uSrY!ld(T-a?o+h7A55x#l-|q zDKw>1_A9m3TcudAa%UAVmy8E~ZhzCLafhMJ%33-&{~KN5#YBKD=96Ws@m_6{$YP*) z{Ni0wCVS>Y04bfY+#d|xNzF)^I!~>ku&EXS#$PX1TW*s3_l0eCr5LKjnr*+TA();k zy_!;?H529tT#dIO1=Mu1$g~rQPNGyTP4X0@lC8ck3VQGNC+`V|(Tj}`sR;!;E$^Rc z|BE3lyIGA$Ko_tU{8h68h7|)Pm8NQW#N#$(8W~>OdV9QZ%%rv_qy5rU@h@B5xX=-7u;#jCMXsrhx2{u9JcPDhX|Vvd%VPaRJ09Blm` zXk9f0#oBt(dn^00_%6Lu`YA(4xSARqt!gA7Z|s9nR2Hm7$&K5(y`4LvoYe*N%jZ1m z{BQUkPQO&QOnXG=P*lcoMIi%7?>g;#4%OZj?29KMtCEB`b2;F{F2wcPRxaXCk z-^Uwx4~I>OEb6FT(ZWY>hT0z=`&`)Is`w7*8ar$c)!`G-S|DYEh|Sct7XDwc8f8{no7^Ky%lt-^nk8D(`&0n9D0)!VA zMXNaN2PfZ{V#WsesBvc#(K*}m|BtTgj;H$j|LB&h!M(PO%k_z5OZLi0g*1(jaTDU& zGp?0!ja-!#MJXy|WL+b!T}Fz`YtN9q_wQWQ=hOK9{<{zE<8jaXykF;aUgP#dr=E4^ zdtvM4JX=3fbwgohbYEZEpmbLH9W*~8lSyF+Wh#;uW2}*S63kY!o)pNCXE5=a1gCtoFsq28oEsfsWf7;z8=u zJz^w>mvP2-oK3j^x|1q!Tcg3F&h!+(1lUV%*WJy?2s@O^cN9O@Ob$$@)3Zd^V&-|N zwR&EiL*)=owr$gP^^R>mqv(vunief_i~V{K@6SHq-gN_Nf=a&qCn&c*G|bYUK(qhZ zzci77tTlyR zW7RxATn9>FIy%D2Fsu&<(@D5V!OCzfY>OZX!R*C=_Az8AQ6!*Aks9kdXNemLFUgSML#ATY>yKW!K z>Sr=BcGr5SL2Bcn%o8bxn7KV7q?1O84s+( z3Nz=ph6qbELI$jF&4u|nxctwu(XlZ1czqS@b?JJbNW5SMhn&#Ui+wRp@&!^QlQy+i zb+Q_y6wD>$I&zBv5Z$OS*o`jH(__Wu$u4Yt{sQ5oxWp_}B?`^bii~tC!owHlFEIM_ z%|k&ylx$UMcsQe9A$O>*>!|gwIwO7)v;(~cFyvA;CErayHyXDoO;L`|KE7S@v3BKG&uDTGGFcSPclC?RpE$Zb91lW%eXS@$ z$se(>4r7u;d|lwOy&>gr<)pqE_73+0EOYksu;{eLrlgfc!bOz+s`@*IExoxqm>e54tW%2!#4s^Ba0C*bv2ZNC~bgtx^FVz8`r`pe}GE%@MeYd1N zPXsOkNpkNhb;|;w5U+1lO_Y`9TQ7Q7woV5e5K)q<02_PqgG1~A4s=imeZA@uARWW7YCNK85G&kK(dlA5BDs6QSG4jReUEx2_9oYVQ@) zg0fNO)pB7L^86(?Z{662VV?@yCS>D88tJpzykDVtJ4$_=%A;fE7Vt}UFH>Vq{I*D& zhh*hc)N9Y4{x9^2B6I{bTl}VpvbKN*w^&*H*oQL}k6=QpK74CLe$f*gx3c1=tUF&}J#VgaS|Jiw?E<*%p%P_~cKOxXbCW^(zMUatj_ z*(cX=`fA;fV~S!Qx%luu7t1K^5l-($jX3Ly4!rcz#y#9w$T!=n7{kP`Rxd~mns+V~ zCH&5!1@r?bH+$TPzaw_{+$!NVWI@y;Klfv*?dE%Z?}3r$79MPhGXho8bvQ&8I)E{w zCD<~>u8PvRz^?j+_xk8^+F2~QKKpCyeRt>dcxR#ehFWl8TjpCb=651MN)3ERH2f5I za)grQ)+xIzuI5M3ZCGZGS?{NyoRi6lofkew@o&2vyP1Rx7X9*QDr8g4w%SXYN2qy8 z^ov{nWTaazbhX}1f96R5FZ=HdIV8$)NDl>`CT;AmI~VMq3)BWF@PYEnLe*`B)EHAN zqY!?;e*-0LAc*`pT;<-+<4>cC_`;h`fmCI=>(|DEr0%Ay!yX=@z0qATcZ<0WU}r#M z5w;~NbSqkMrvLTP+EcY&0p9R!(H+|c!l~vbe2$|m_op$1T#tKn%F*eDiGeBiB5~O* ziZ#|&2AYzBl0&7J@w#QGA-is^h9$4P2=IrEgPk^ly&6Be@H{o_aMSNRS3kb8bea?K zq2mMVvqQ<@Ht@%|!p0EsZTN>{UV!Rg3V?-fhhZs12L=ZR&rp=QOnx4f&i2<@vx#?8 zy_;RxKUNSJZN;Ew(|sthdTv#tNHMF%IFPTyEtE~dvX%+udq>t-zV#kQ66v6XgT$h3jn)=6hDeGihWHtY^$?EYWzs7BF>5hD{D5b0 zDr!*rkJYB0@s+mR?|WtH)YMP*Uy|qk@wfof6u~a5W4eA8=etkV?wWwYH`E%wJc2LK zcHOc`-70{Igm+N?)^hC4Ed{>^(_2xCDDEMwl1LqXu@c*Q)+7NE*V9z*4c%>Le_7BK z6xFnCXgeM@XQw$l@U}TT&%#DXz++|pTSeRBV;3Xl0o5sOpInT-4+O8UNL?*^nQutL zy+`!kp>4RR#cwYFFfYLe3Zp0pZE=sT{}+n>d;@;$oCL~|jN0H&{OQtMuO?bJkyygh z##KPdg^cUG z&aes@a}MQ@Z`qMWF&plX6b|Sq7!NWi#Kg2qCJ%F!x7o`CC8Z&dr^-t1&il}Z{4$!?UzUle}~QUVAP!EKpFbm1&o%+%p&G* zBtFEm2oEPMY`3@?MQ{ns(s}d`yi3jt81Iq_+PF^n$Nq0B`}UhrmdH?L3qP^^3qJ>5 z#McP2$`U%>3tt#NYk?x7@>ySZ^3PN;cQ_%c)fIKi=9&SpFP;n!ajWau%?=4ZI-MuX zAFUKa28u(lk=7?GWA(j<8t^a5ZPACj7|A#O@h`U-m#xOPs(TitvQ)v6xi+e3C2sAr z4(JSDPmg$pvef02|eJ{_bSIK?i2V7#x; zJ{)h>+b2{*p(znvm%H_%)ftp4!d+AY3>zkPTeLxQ*RT>cY}w9BrbzOjT90@;NDYe; zg-52%Y;)H<`xTORyK|W-Zr7_>HDHmxf-B&8^ z^q&v?ZZUqMFezr0V;O&psD3}c%qRU&brIL}s;Rn=%!3#D&Gs*-KlL9)VX8T>HnnC* zPJ?^}DD5pONWvgoEaQ66#5f(6>?Ym(CuEMvP*KdVu+i4032ZsWznrfyVG4jQRa>B+ zQ*V9^%UF@j_x8mk?n4bxtJ_q2q3bY;!Y`eVWEynH9pWD1Pl|4`m@GDT|5sJ|r+I6F zmV435Q?e)JYukUKqkeI%e(Xp>om;_J4bmQqg`7Uu9~8n?U$yWV0f#O#KZM?7H~}I2R)YAI2vI-n?YyQLz>v+i%@$k0 zQkQd!nIfBd*F{5hApfi!^RvqdC6g}K+8;V{FQK9pYxh;AT7C_-@%fncGFC-kio!k%4yPO@goUBD$3!97&<#6@n@(?%fUP4IvIde4|KJOs9J z@`smp0QX=6q2SNR9(*U8k6cv@^zJeQm7iuDmI{DEb3TKvVHzMG>QKIKLsMjx;5zEH z!8sIj4G?$gCs>LMrnqN6vW<4sBpxB(IEKl!jWFbgJ{^kL5euJsO&`M*6Ka3E_idpI zM+dl`s(Nb=IvEvWEKj@YE{|`hd4+tXnMc$m#D!xF6Ik{oc(+*Ua9@l#1z#QYAUNs+AjRo(T?_5x;5_2s8Z7eA(}DT87lbXhQu(?|7OC+N8h z^)Oa^QO}j}!&>Ml=H6zYKErvrIK}R2C268zQoX(QwGd8JVvAQ*N(zhq8eq=@s3)K@}}G~cTZP3 zNVf}Ub5@pxH{e<5Xk+F**A2D)3dM*>f+AE5=j86LX9U8!cd>as3(S$m!quY)G94M2+aT^ftYOfD+|j{+x#X+9w4C?z^u<;4|Op++p(ifw0JP{7pax z?NxrJ9{*59s&BSOxO?g&*ZAs>w&e-Y+e6e2v&$7>Isf>J8S*?Ukw8;aT&p@;{`wg% zHM>R_J?dS3?~nrki3)NpiaBid$8RHh~L3t~qOu^ka{jWlwDv z2cS%8o4|D1?s6tEHy~M@4s@P<(rY=%UgS{jtQuG+u^#{3oWJ{=JG=y+r((|n)ad;Q`wQ((U#=F9 zs_TJ)l(yl`aZkI_{Wed?kSQ)T`kMYR5@t^VPT}YjoSzr97rjKTsE`p?x7&1Cixj!n zRPbTYG&_}Y7gLju`2}ej<@5ckDj_e%^P@M^dpevGR%tP62+>T}8v7f)sq}T7zW1-pJqpTs~{+u+uYD7U(!ftDjnFr{J64;VQ|dxYfH>E17v?K2-OM?5pet z0|3A^wSm|qF|M`w6G%*WF6$XG%ucNdeR;@4hJA(TwIgrH?1}G8*8j&@Jv(rl0;`q{ zy%}UU5xM|0^=^lKMVcw8@!yD9go^LiUToos_;5BeU4j*U@ubkW(9~sNRTO;NoT0$H z8KEJgXD-{1-AVP*ZsFK)XSs`~u0Raaj;sCCq{l$cwLWrGQ;)?s=)RBxdXyKfiuq9J z(c@~%a%NW~DEI9N1ajr`2*=B;oQc)3d4;z*zw!(3tu&j%FD$FCZ?!dP#~HKm#?~dh zyZPe@CPVwvd(g@~CP9GoEle+j@W3a!#tOB_)&PPobA!Y!^KSm0UB>uJ7`d5V@9e{* zt((*1lJZU(%=Pa8QJ-g~*TPt#P>_c&iEV1PtN9UDP^HlQkJx!{5h7t^bn5UyHRxR+ zzSkclriY#a@+$dj*gKq-EUhUOpu3OY-yD3f zN}r`W$@a{+hpzzjFr7D9k*8~-Bh`;G#@o1kBwpo9UH*ORe}o3t&c%u~h~Za|P|c$e z4S7qtke-Hi>)R9OSKy1I3XR(#8ZzG}_kOjmry3M{lIFX);=9M!gCt9z-B_OmkR$_ z6!KtK=MZo7AQu2!cj#f6NTec?O~9X8Cr|nCd)0(9fC>`3cETLlK6HZs5}tW9&(MpO zV@46&3!&Ms$xbU-;Zt}hOS5DOmhPTqwB+23DxiR2py640K31Cp=GMh(hRcHZm)ple=n z6WaC)7g@OJxUbD5h#a|7J3(L2s+fbnY?o>xy`m$)Ct1V(9S2ee* zurn0hJ)g7dSsvYz@IOLou$KczDWFNPM$o04ijV5uh*NEt3^=G9M%uoJjP!$I5WkVGiNFzLyTz63e)799KNHcHc)tc_S(=y zzd~I(^uS3YTtYTJAit|%&9KhU^NYM~tKw|2Q6SFh&z1zJ`p_xeTrDVk_Xf~qCA!$E zQx$BFCZ(*`mD(Kf>hl4T?!-iM=hNM^#b(9b9!0g2NmY-(m4I=BNpoRO{pH8waxW9J>OCCF*Pdyo3zdwd zNE>!<1L1TXBlp3;amEGDI@vza)gB-`E#x9M?XaIv^cOSJ_q%hLJ2Ltu%wJ@p`k!Vo z<#;Uv|MvkQIgVi_I+Qofw7C6*psHV)Vy(R_>HHWvHq&@0POEzNuV&IU1|BmG zu#vuzAO1SkLYaLKF?_8nYYy8~kJ0R`D{oq9Nt#gK$kWb^fZ=Fppqu9tL>Bn7M%Nq$ zp9}*Lk(jcH-086&5+1)~eU{vj{Uu3&+VW%5>&O{WgQH0ryA_|RN#}rYI8vI54K{D{KiDZj1q{fzc zGNmXdW5t>N1AdGvUtV9yC-5o1ENeKm+lwTmdFv12rl`nUz8qfVqHBRxXTvu?Q_s}& z#G_6yWm9mi_mUmDVIV_>Ew;=4EqpLfL-Ck>=1FdJujF9_W|G$0>o_P=rGWB4KdjLw=iDFMdbE z;2=ap(Q1r6*1_mYgWPY=NxInCAaMqrL4$5%0`rJv7NQqrc>>K^>vDCo#DKy7OoSo{ z`V?^9C!jvTXdud0w0M-LHxq85EnDwv)lt97No$(Vt;Fwi2b<%&q1qUVid$Qy_{Zh5+c!7=Jn<%55YW@!!rVOa=YRHG=XD57 zhtswK>8J#KU+)Wb+mML5|3j`?pCrtJE1sqCjYcj^nupzl*HVTaV_ z>RgI6{L2A}JVrYS5R_fp;7?9j+I-6&7L%-0&%ZxF%SZJt2Kx2uIH;^>bDrr!8!R|d z7vPHDw9uro$0%j?XKFNUe{n7JFMgB$gznL?pi ztNPPR!6}c;wI*}wr9ZId7(w*lltNC1iJ-EgT37vfyz}x`CdGAY&~)-MtiwcI>2sQI zb}Et_uaO+yc$Mq%XM2vY0a?5En#>;Ko%mO_OnMR?Z89$U7@94pCu|>bZj#8!BZJv+ zb^(dry~5#dC7GM?ikPdm6AklU4%0n_zH@xSO_5N4&^9HGW+mmGi3N5C`xVCkVc`uD zPY*pXVZ9Oi27^N=-XJxHXFRJhdK-#XLyq_CsTo0ba)p=R@_4EkS8#L7>fpYFx`x!@ z(ihXu%l>Ssm~>EySU3}K{Javb@6ypf@Z#TdgP#XD4vhP7>5No~Sw#u@&#zj=C73ZN z3%FQjwzpYyjbYFfOQ-x518!i;HfA=hWBYpH$Sg1#&|ahs5!RE6B`yy2OKckMGo?oelZDS7x_^inJg;dODaY_OQV_K@L`?Y<|Z5tF38luV^~R zR}n5fL8g`Qo9qVKH;e@*HJEK%KkY8bRw}9{8O8NbPKAoRwy=7Q@(3>%%m^z^2!^Cy z@b%nBj5U31P$$GM!$!pSF{v;9hqodbL#;YN(6_FGxBcv&=ajVV;Xz8(*GOfoKwn_h z_m;Qsv(6tO?nA*>I1f~>@EvA8vXpqj{}q4wQ!V{D0DrO9Uq%}QdHPP}(Jv439&X3< za~tK$B8@Xc6s8PA2at2t#SH}^4tx}YIrpdx_jO)7GJTxo)OOPv8}7|^dI52XmiYnM zsz(3n;o+}oF`}qwr=Xj+^!JY6zy3iSir+sczVYGuAFty3A$&uJh2m)1K-mTxUHMjR z4iw!!QjwQjpSGU5=$S!=xso~8@e-e&b&JzKIScM}L~!kHhY%D<67Z~wtcs$)RYviz zezwzc=ub^%T?qebmr&U(q?bQ>e+vC%2df7ILS8e;*##TSfN>^6-FMTjb>8;U2S8Zz z*f_Oo3IC`gd4uJ5r*>Gwx1ZCNTu#7A;}7t~_a}PD>l-&RH!@o4Y1;TQB%dxjvki+z!+58HzDoa*6-|ZhL?UC)h*q zzja(tP#8Cy#BgCN1Ftr?9P%rC4eV*Anr}ZPk;pxK{o4HR=W%btNf&A;1V)}CFCsCN zTvkXMNM{xckmr;3(0~DLDpR9cP>jdIYUQoOxkQ*TeB!?!cV|* zQ^A^xA`^4`K<6h@;}p|y+V^M z)tsq>0ff6+Mn>j3enXGx)IgP`69}2%3CVHT0?r;Yb47vV0%PP-)v#w`pVdzVedq^0q{qaT{H(WF-rn#xG9`LVM-YN_;VgTjKC&Q+JUue?#A7Sx41xS3s}A{vKdb=X+^Em-*wZO~KO z#vmDse8tK1`#>j;nP2Qi2LC_dcwT=3kd-}|G40-3;7(#Y^D_q(wCfVvu!;J4sh#j< z=~NE%j!;YfsHkd*;D6n60K%9Jq}3DB4gA*M_8F@pZ^ zR4%z>A)+{z`09$P4^(HN!ZBg_dWy{$!yt2oG4Xs@UM}Bc1LM1cTR@*pBR~~92&7Lo z5*G1>ie0xo4|Y&5BO12dj(A{b@j+y(@R%Gcx~lv{&pRw`I`LYQ14SfHSfIv<8Yrl@ z`wJ*%f8kc>oU-zl_Wj#3%6M5-J<>39e)3A_OA~?cnXP&s-HB*Rw2!< zyS1g7^EXo#W+-7hLuC%yn0b)QtZ#ESv|4%fM{&dxtjO$pk zBLD-O`FUB^<)9Zyaz+wu6UY7bF@=-DTlEoQf!Ii@oSZSmrjoDc69?BTbb}hT4iUsH zv}se`kMW782nrU|@XD3*6P=!)Gm7ES=z!IPu#+m#viCaC@VsI zpu4qO75hF@E(aQ0y*`v_MJ8D=4dnnm6pAhCVCsQ8YvRiYpwBgZfHpxaP3qhs&qzYTJI!@y^`xlv5M(&lnM4{$=EfUpi zdEnW4ppqf1&rS2opFe-TamRY9E~kwp6}tlvYcdkI+Cj#~97Kk)to^K@8tCzHpjL<$ zE-~!XC!#8#_HU4wae;I8inP;b(Fw*BzF`d?ViDRj?4v%tcA{ug9u=d?PmMF#Nyx|O0ic{reDa0n0#;$O^ zIIf0SL%sax%yYZkH>b8`T_0}VvwmW&L5dAbjG=R0ww{R27u(^lQ_!K1ggTF%CfVSh z@<@PCWyFJ>i4myeR-NC?SQFC%ydX0&IP*o}h9_?Z}*i6_KSM?BE&$S~S*^aC+ZDhG(0~xqE=q8XU*yw=!U7|J< zsIN6oDdA)7uDWT?_tiRcOmcL^K|UsZTkDyip)O#;EB;uV`SKI>6zOQ1XhPgM8a>tZ z3p2F_O%~%EO0D?PS=!|;C``3C0D=?JSC}b0fQa{bgp=N+PJ8ZBVGW&5(8E1+wn;yg zh!BTJ#rs1)PW|66;tG$i2^FyiDWzQQ7+5Yu42g@3CKqkY<*Ki`_bDrkhJGY=W$=R5 z9MfrXigKF<;*whzFX{t-`Zky5lS=gy8&sbWuD3fn^)`C$knF$nfS=3GBa`SmW72(r z4YFDW3h7_!@<`S_27m9lS}f!t*^EA<5nq4@6CNL)&x$4$f1g`3xdWkg|L`lG|L1Yq zdq!R8%@lbBk;T)2(H*d_@UqN7?n4qDbz1K0gF#}jLN3V7W%0?Jw86)YF6DgKoH&cd zgFy6sB|&*xclx>hL=od#!BfetIjkE;E?I%#(-ZS!VEM4NuZ6*)(aWjpc_9=fhYiLt z8B-GRNdOtnxq2so$ap59oyvIpv4=TdXhLP1`FD&s7;-L^tEBjt+ zhplrJn4K!~(mX#1{5M04dw~86D{syrHdPFKrRK@8IgP$t!;j6b)!2R*1@f(#a%K}} z9KY_`u)7TpV$ez(CNEf*KbzTea!RAMQ_-lN_O6SpWj~6@O8@>lQrx(HYP7H zE8c+rV*1FrNnAQ>N+Pb`7WO{>#*Njfjgm%d$*P^lpOG-OnSj=m(5?Xx>Vn~rdaJuLV1HK$N1%!Ws}+G%S1v6 zxDAQz^pSiTtBHO?4a^O7qnt*0U(T4R3qRs?KpNv=5iYArOsFKN)qSzL6ko9RSW-11 zRp-DqyGF0>{P=bK(Z_GY$JV72kE;BOp(Eg_<{}&jBjaVeZZr6j58c0mzrMRnzwKiL z2qXWBDf~0n+uNJ=G0_|L=k6M0e(i9jR+dw}sLv3&gke@_xs@WPvyKvq*ieN@jr6dY zn=h9E4?UTtujy+o$}d-7WDw@WEU|w(p=~Fqv@ZyqlJeu%YSn#L1GxGb#zOY#hTOer zuu;k{F|1r>W4C4H6LF3>Hj6Ye;CP+zAGD?O8bxUWyVPKAdarfK-v^F#|3Nu>!T{s( zB@V>Gc|Vf=wE3FM8*|$Xa0~&7&}?zgxkBg;jQMTo4TTsU?MZJE3n+uKs-Hd>k)0}D z4(Q*t{j5PWtgthB&>@eTf`K!5B7NJzh-+*3=)b(#*+j^~$MCS4YU5(`pvw>v^C)#m za#720Ju|F4?*$)jn3wJ0FrXgGqAPr%jsLNJ6BY`OK7sQZ#U8g>-4|w%xLfQ@qg|T) zt~MMUsV)0)*5jSqa2MzOgqfoF-;{*^eNe&!2<&oB{LxBTouR7!6oTD{+h0Dw&Y7-f zpOb&poWVfZ;5BuwAYIK3JN-bNMX_01Uw<&t3ua^bA-yS*XGj_1tzRT8G#_tErHRg0 z%zTouGQRw2I^V?h$NM&?it;bPj!6}=^tnD3>mG_XHSb<49K4(bZrtp$-qOW^m>9=~ z;k7GCY{M~DKxJLaJjX^60X#xeV3v8^2PODcAto0JC28HLl1gO#dC-1oro>}+|UIh zE4W5fSUjKLTd&UUmOGtX_Q zayA-`S#+>HLBAI10Ei--3|^%gO%>cN!aI&CY(Wd`Q5q7^Pb3&?SP8vr^v#X=+-^i; zk!{EHu=nluME;?R0X`W=C*Z>h<;uapGA-$I@D zsVtZnGlCaR2;*K4XAFNS7c3LB=`F2aygb%v)ZKscVgc$AzF?^#=-94#Yhj9k1a86b z2jya?0(qY-_1YEcF=AHCso$fCp9aXxN=N27)TH=t$lF4vD~x86t~>kvg= z6X^KXd88vQef!?K*bGwP5(EbCWsMB4)9%U@YU*xSJW(MDcYxvV88ywG!SiLd=RWfY z43Dic*gjIIOE3Ns$bbSl2<+F&_MC@jWqt_j%Z?#KXptfZh04c0O zw2ZJp^Ma&{$Gfqu*8!)d8XPA3mPn*9-?O{J=X^!ak`F{y3u^ou5C0O6ES-Hyzs&Vz z_^znw+InYp8efe!^y7QG+d)*>8FNe(ATS?R+0VZVXB_M=rd!QGVgpYZtRThj8YBh1 zMOr>T0?HxfV@D}eLQiJ#xyU0r5FK#h7Kf~>%{xuG??0%6gk3t*=i+EsoC3o&(&+~A z3e23376B9k_7O^bbpV%49D4tPrB4pGB z>$-uiI37*VbmvM8g;A3;m`SO6Cmh_0L9@Cjl?DcKQ^-U75>`GZFBCC;d@SwsM$q5N zXK|R2?B#llqYtiLdFxxy)X9!}6d!X|W0paEwBQ`)P_aa8PPw2QAec*R02>bIn(J3p zx1?F&rJ=sMD1d5LzNWZ`Te})Y5#|Ugk9-i_FuT_&`RID@7KKjrY zRHJq-*V4~%M}@}Gm?F$boI|NaLX&RbRCQD5A2iyCSulZLVsYuBJV`$;q}QT|6`$Y^ zNZ-o|OV0X^Uk-4~%aW@#=AZ*C^HLE&j$z;KS(9LB)NJ+|Q8%j;~SG(iR!Ib z4nJd6?~I_J(Kxghd=sF3_(Hm2?xqt{8JfYLa(~H} z%qieeNjnqD!fqRw%sRx~wlP=B;l?9Zot`b)d(i}Usu9-P)s7CY9e_N;d3Vq#`>mO* z7fkTpBF@U(l>4jiM>3-fC0|^hUbT8t+a`2)-+E+_3^?y8wkT0(q8axLkXjjh%|DnC z$ykA?yCUgzTtQAQLvUOC3u8qT2?%~C!6X0+{bV9*K78Bf%8dLJm@jfZ_f*SEHz;}s zyF$wtyK#8CtmW9g#cmLTYnz@Y?-TH8nNKV_uEk=^H1Ho~2Uv-wFW`FBe)=#Y`G1th zh+w34Ua#6qFG6EiLNV@&>y^2}rVZ8VU{+H^dUNz3TE3oAw}op(-R= zgj^TpOfFE^#Fg?LI^+Z>@2)u<%qUt-E$Bx0YfDv;Luc0q96 z?vdC00Ac=qk=pHkN?BM-UJdTqllI>f+)$i^&9??*xM#fo6N7HRO{iOxI8BnNN1eoy zy?u)ia(Th7^62t=pz%>~*f>>4^MbiK7>k7xb*pZB>20h6-u?_VTd3@&xS=FyGkB{) zv1)Nr5_d%&6t4(IeoO8(tg{ECwWmlVXs*FZs&3(J%n{rl!xF^J4O^RY&QK6J2`5j? z8h`?r!u)uWh@)JyV6#RsIxS3yckk5DwS@{{%4E6Zc}_3z&^pLhQd)Hk48abK|r$sIm_;LS7U zg6E!U(7}z19F0L9=Jn4^<;4r2dyX=^mKo-{@`$oAs1F&pVt<4*+)nDPzB|FCth8Z& z{A5kr4h((g9*s+&-p4ZdfQNDDoa5tcIbMf`Z0tH{%Y5}$>bsA8#or5^i?^zu6T`Ij zt8F*_SCZaKq7^9`$4`*;lzSJ}{Jq)_uz|Z6FfK47Anv86Oy!a&bR*~ z<(B;lrAAU*(=QY4KkIJ*cM=!-{yb*%p!fXj5T#EyC^Ab7IhJ0KT|dr#-*^HyV8^(a zS@}r&_Q9QnXQk<4jW3^&Jub{q6L}cYg>uKXwqbT)` z`9?0TjmlerisRA6sik+kv7ZWZ)a)}Y3aD=Fms52xWNkP)bC>VUN=M3ogZ`P+hf1*! z_dYZ3dj8%qC#p-fEdA#4jn;+GD5P8;ReW7Q4Udq}KlS)tBpC575VRher9nfQB zv(xxHmj6cYm->_@M+qd7o18P&uE)_4A_tk~jaexbyixGwQ-PQe>oNHyQ2>?Ps9`Ur zjww7!vJ5sZM2925fD^;hxAQLnRj6F5xx*D_UNHHvaQIBbP?SBEEulxP_$gJpI&fDH zGb`TGVH=EFcq~KsRnB71Kt&Ex93)f!d_+h2w_`_Iya)#cbU%{qsZLi!A^j>Jj}+0^ zUg=ZRwH>6_$UbR`;{i@71mPX8vk2|1r0rKYUxD*>kpxhgF9VU#N0;;?e{W|a56tv5-x$iDvkuPH_|dc+1=}zkr01eT z^uimW&C0#95&VLaOnzn%A2w@O96_Q0dQ0;(vnRN}x<~J@Zk~AX1zv&FnV*{JrA){> z-mJBNRcx-u8!pb*9Xp)frA-memx4aH6myUJ$zR~^f3mT@6a=+^@W0F0ngL|~;sKOs zA=6MequW1-?)9@t(HjFS6$tOjm&d7-jlQtvAe03%lKpEssxniK7eZ&O)f8%37Rnat z`I|#-4!%$G?NJ1Xl~D!OyMu`)`KpsIx{N5o`}H!LQx%(Q^#@BdYGO`NjTFVXk@uvQ zd$5@rxA2rbDy;k42bDqRh)cdy)ZS4)%l5y1hnd&!a<*?|mev3hd6krbNW}HgVL9Xr zpsrOH$Yld4Q+G>lFwh zr{zL`a)ls@U!O20ZL zwf406yqlIEVlW^&D0&;lpL3Z-NcsNjqusI5+j^gELeuFlL~nnYYFc(A5ovYorlnJe z={W4#a}2k=4O+yHXB(Da$>94|FujgA&Ml|>a&ynLBJw&h%Lgo z-Nr-@v}t$h|Pf4_yTp{k!?q6neNAU1WF zLleseBcSB>fnd-xCx*zttluXZQMzrhYbqu1byMyQgTR`rMD)8tbEIfb0=nh7ic)qOQ| zWr$b7iUgPVx9RfpqfIcGlLV#ax4Gv=zv2ykSPuXUVlS?}>lvd>g_^zN4_prh=u<^g zwk%o2(>|)-NUc=B8XyHYh={8Nuda;Ve7DI<$j8n_ZH6+4pWInC5?d8s6^)S*C{YtZ zzQA0c#*JT1R8MiwTB1uD7Q^&oEO)Btu$;m|tJDP^kF!%{z3vHK`pda;`ej1dM9BXK ziNLpgepuc3N(EIoRal%E!ri&+OaKIf-Rzz`Ypg~B@_a3PeH0IRzMUkrnx|voB*xkK zs%cTbh~Z08qg8N~ow(wNSi#JSDp-V>%p578JOg_lM^}ZZVl1!LYEymDLI3TTobAx& zn8LM)E18lV&aG!kgEb5%TH43vqlk9BzXpmwVn#h2+2X!~WF4sS-w{8gYXV6Dy$VxN zt!84XP@Ozykw9focJf!)u$jVcYc}(uqKBJU4!Md-6kEx|${keMG*TH8~8y z*TIN6ZL@NIEdJL7R*_aY;rSTWT7S|c9uab7Cy?oeo{@dU{^zt!l@Qm?HDb>~mpIO) zU$e=_i6TWr*t_swCWQ=k>KDXLT@EJu)h~lf#lp z4mq7ensE{QLz`Q4J@r4TCxFeMF&a-{;wIPhhpHmsBQ5q75~#HtPE~p&tG?ce*lhZ1f^OI{>=n2)3+5x(H3&-Mqr|tOZ<;W-+ zx7kIylK&(h@!RHxT{51CsVoanf05%8P;BP_p@FYDu=()KO_h!8I04T_q7gCF*pBTn zrdm1|;ElacPqSRt{dkkhx09NjT^X;XmAKzSBGP!PdeAQQgM+@rD#P$T7qdp}hfqw9 zlhH@r%fbP;&L}@l*x;koCrXLe9JpJCr3yeU7gF~dr_s+00&tH4T&To;-)*M?$RCCI zyKp5N_#lIAHH!mXJ5N1x#sk^wFf&STOB7tiB%v-~+J8_c^pkL{HelM|fPz*4RU?u{HV`oQvbvFJ2WK8K!3Y8Hh2kaL~@^?IFqSz>6l+KWzt6zNh;MBeq{R$Lf3}s zj{-!APd=4Vr)rIfCWLaTM!+izdGw_)+g=~^n?gf*rn$}{r z02(2BhL$PTd4AZIXkVTqD1sz}KM5DiBPXurhHvrFXskoY>x$JxmK#nU7cw<>-!hl_ zL8Sjz1ndp7-%r^7YGLp@fH*u-N!3#eju;9LP|FsroSapShMiS?&&1@X7T_JyA5tsS z1=ZO;A=oh_#0*h~Z{^e@FhEU-87lJ8A_J#r%Qt5objfW9zGVa^ewUq4ho>K1qpWT9 zP?nw1{zekFfOAeOd-W7jRdE3f7`cRU86IsKGF7tuL%iqb<@l1-qj9DL7#`Ky5_Q@LsL{<$~F!gFZ#xb^63nYc-!(GXPf! zup&|VRNo9=b!f0qs;svlU*__cgqv!`^==^ezb%(4o$KQ8Whx$U< zOuGu7m3kOJEZ7EF{1uV%phoG`kX=SJ+4*y!-`BON=3B(O-*>ZZyg=tSH>Y=3iErR| zAS;!2XQvvowK&H4rK+l5WETC3+9%ym(KEX1g1S)CLlWtmgZ+WlG1mUpR5L?~zG>NI=O4Rg+fH$7jI+&$2{CxCUHy#VB+A&%5=9FEC*w zA8V%GdM|be2XJt|I0$ez&vBJo!56qfEra;KvX;F>5(a|J`2aR#yQvG^Z~rFNVC`*X zKP)-c)<9HNGLuPnHKEBOO_Zf@I0hwUnnR^6(}Ot)&(ku`SJbJjx|Lz4Kv!?E-$AdE zHgU0i+5%Fs$4D>?z~@|@4rl;*?e>M)w@ycNm=AdTwv_)?HP=7(?!a&E%KGVm>L&8}~jk-Ra<;79mEFbo1nk2-M<^x}<>Ae*kWdO-Va>}u&Y)UNkfBt+4 zT}63DpQ@ye&7E&5eG%t5hBORmdzn#@2u=`LfJBoagdcwX7+uIkUf8pj1;ZGj?X6nbGLDxgws#F?92l$QBhc;e4uNF@ zAg=GH7rK}41ygt}2>i%-RYdWwW912TQN<*+DgTN3G%wUX!J$cc9KxFYsOCiKQ@mVA~{~`E0arNZ51CIQ|0}Nv}rd0sEAvo87$0r{LkQkjQd=8$b8u1pO|0 z#8^qngr74D#1w6zHT2-^2HB1yCAwjIjT8P}kpKYwcY2M$w7&8Jn+w!=SkV=H`g(`k2*lAF@wCFCWe*q3%EaHrPLR{JE=*WAgj?L5Q4qAB@mD0c`4_`P>= zJ3QZ%*@Ol(II^olr}cTOs!=4hgte>(T`w`_t?)zrxlP0#ISu_~qyBa5sE;v*#clHpuI-RO}p0-<4$|9u}gEr z?^S}vWkfNSf(2`09Z}qWawm3iVt&+hd(s$p_hb(EEn)mNpqx6oC!*~Ow!z|EGxuJN zVfnnh0HB3!V4vX_oD=<|Q0Gm^qHWg{CErZwR8Ep(*N>+gd`V&ghV|4)c%R?v&rC+qwII2z#e26bvwU>|1T4_Zg#KaqD6z8*gqUh5ZlB z`v}tJF^lf?pM2H0H0aK{6!tq)Jq38EVbtfUlDmivBf3Me&stOrO?XL~xWzy*l3WyZ zAy2tAooO2zdCBG5OF5S*Ypd{PfdDUL2HsRPVuJVDI%dE%xxDT*P}+OTK4|u7iaKQM zVP-2+(J@vjfIEZ~x_(gRb?N|X3ggn=G@D+VQn@)s7-uh_ORUcDJB$gn_mkKhE<9IW zi?)UJ#{WKxIskf>rRxc4UAwESty@QsE9Fhiac8SjQM+Ew5LMlwElHEfr3q)2@@j-I zFCykdQPbUqT<)Z4EO7r=;Jt*A!GH3z(H4Xb7sgml+TZznn#9>awAF`0p@kb)swioU z?bmGsXbzTMOHDd&>{39&5#3zQN7k;8a*2AWMvI@*2~fA>m1+B;#JOw36$574{f)RqEQ*-cdf zJ40y;dCLvA8{x!eU;FCh8$!G7dr$qgm^og?@5#h#vP% zCWx)V+ow)$=&_GCc4h)hJW_*HO58}xjMxVgT!Jq0<}L}|L+(6mT6@kcN!5ozw00fA z6)T=+MCTWIQ?=$F+&?tty8k>&RaYO4-58!3oza-h=Rzms_}7zw0ScXR$FRHGpFL#+ zg8cQE($dgJJ$oRDP{Wq(SPBdyc<~+&q|-^jTE-%;~A4Q)(va<(UBOsiMfzF`)*t~*#gdZj{OS`F>b z&v66U#ond5WSN3GhRv1eu4VxQ|I6n7$2KA01n=KHZHITuKy!zw?zPhx3T?k0qaLsf zxw0I!pprGgT7s)XZ>obI;gkoH@1YvaN9qomyzcjG?9UjE)%={tjAG=Y;*uL_^tV z!lDkt8eE=b03-X`^B!+SJ#M`$EBq#lZ<%$?LStyU7>**^e=h5f#r^B+hd?ybPnBem z`)z}V$**fi{#t+fIfkoOG%fb_rhO^LnW^rF2yY_EY?*gd@aj#}Zy`_U;izP3&=aE{(bm0!Q&cv(`$Mk6O?*g)$p0%`E= zDySx9#Bk^irL$!<6d5p4Y3^R#x~I=3AEN*bGY&C|sZhPS1edxt$c?-`MfJmRlQ;ZC2Ov0%RBG{!Bmejuu3fh%d8G z$9_8USUag37K-wg&c>!KnCMO_ixpmGT;!fN7gzsGd{U3w2}OeC&#_l2?NjSgRWLr; zrHrx)hdiSKD1JsTF+&!oXqT(MCHX%efTG7xA$Yc($Ax~yU-Bguh%EztsBviYq%;(2 z&kCUdsojpE4GM?#LJJi#4{H>S-eK?BGsB?MYR4=?PnkCIl>0Yw)uHv3dm%9z8kF0P zno}bM2+iZdV5@77k=CCRr?AVJ>*Eyxa7NKk7jLU#(GV{)I}&Vt&|Cd-?It=h3TTWk z+2N?v16TAQ-A!IZcC8}CA4Cf3{4d#o0FyLCG|`fi57y%Tz8DwOKF{iJlpbo^STTi} zT@IPn-x|YkyCW0()b(Rr6fL;hp|@+m1T-us7L2gcdOLK>xga#YVHB&dwIBDDko3$+ zeg(CSJRD?_kI=kixtibzkK$1}yPINi7o2Vu`s7)>=DEb)z$GyB?`h2LI2Hd4ZGt(? zKtqGS`OX_m-1c^V+{N#0dg{HbwR$t1f?j_~!@qP3N^E>?Vso_cgn)^Z#dyW0r2bz0 z(%m_KNu2wVGO;2{n#nwR2usR(N8E~BFdj{0u2|-Y+2%-gFyuysQyEx+2>qy1$6xqp zvBHuE=;>tKTPzI0Q5YiAJRS~4s3D`dp5L`DNLjaq4Yz){d^tJ$VLLzIXIF#*nm<(2 zFpE8t`emg>HE&GSv{%)c!YxC-m>mvb7ANb>Y^_?mt;&7+fL9@}LN(Bfjh^$tS8V@L zg8vqsqq_;qKA+ssgUI4up1PaW&O zSNZSHGr-{bMq;!^#}+;HKi9k}FNSlkBIuC|G|4r;0;?O|e*BDtFfYkFMj~6gy*x%{ z>3g{qpo0{Q?qUy`7aD#Lamc-zKs0M53%aUk7OVzR&?m`~BFSPds|u*M*_dUsJwG5- zC;XW75aZZ2)6s~MnU-zB!Qhwru3kCR;^lND}fDjHdUEWMH&M&lP5`20VoAamCON@Q+g;F3IgJH`aGb8UY9zb@}2K+4P3QTP#xWWS<=I=sjyp@MHx z#0MK5%~{2s36$n2X_a)-C{65ca=s_od&Q+>kirNpyQG?y3U%Pv|?b2U%FPKlrALZ8_;xpxmc4RMhdpLOD?1wqJuCTOKn{2`z%r zF2@%-L$bJtE}tj)U-^s3YX3aaF?}U|!pen2#cX8B{=1^T+y2 z160#G%PykW!m=PFwn4scM=N=95KN!S`sfN$e7x|^jJh`=CYkg z+4){=F#^VLVi`F3Ze@EV_w_<%osQ70cnaV=ho6WIU|BJRX7f@a{h3Pk-;>zbm~W%V{4dcxT?-omShswJi$_XuRp8%IP^Cm-n)++52+6JIUcu7Jxj zacE#i>cVrX6)!6&1vF1-bm6yywRrW;CV71tz^lmae&#TXceREILyTz+8ygpj-d8-2 z3BK7+?rhXY-W~RwrMu8 zaof!RBKTx;cKn!V_!tp}=qbEBE3i&P8>0+3L1+51Z!qXrcK5-Hk8d!`Z!?0_fcO8~ z`26RCm?Or5a1cXE(v9tRfFO5Ch($}noiz}aqB<_eWH_f^LIdTl)!uXLRAhg(ls?wB z?nTB6XioF6pBbMx6pC8b?U(6Oqc`u!z%g`#ALJXZQxPfY0h=u)b{Sx`AKRFw&ugG@ zFy>*wXK^4_-}urx>!lZGIBcE&0fjm_hKu6a+>G z)%#;7PxW2o!|SZPA6dw7jPb znS5T;XlxtsVxHhcd!^V1SfA2G2yS53j1*M6+E>q)s6J|=G^ z_l8W*zG#xw^r#33LW-2=$@XijlEmCtm+F_}%?n|0V;$(if;lSK$<{rVn<)K!+`R6X zPhrEfVE?Fg;jm& z6>wLW2>?^XC@1DFghzMO^(w*N)^GAW8Yp(HI9T|sY+uF-8I%I49WP4Y@$rt;BDG@X zI0XBTJ5TBMcW7^{_ZLX2?MKga*3|U_ra0@)?n|$%J;#@XjPl;*96c^%Kg0p+equuS z<_$aQx~V?618ZB3(=7?g)DQT*Oe*mB@Smtj;c9t|>dM0$r*b1wf(HD?XqfH>3KxWG z@s?)pVO5XL|6%xtrPJdx1Brx@6H-G;T&m+V72Ok>KQPVm0;hEvwY1s2hJX`qwTjI# zozU^R->WGM1ROS1ToOW72lSlI;?3VjYpv%xA4qT3d(eKI`?yio*H$v4aDVjSe6{ZHKj2ZfeD<2tX{LhP_hA-4<0Y0%625*(Q`_ub;g9hIbvA&_6~bk;o=Cy+O6m`KjoHkL@X7` z{?nwNgLNULwjWCTR{Zm!nr;)mlMM7w%}Vhv__`;H=~dArskQ#4bxO~5XQ%+V+{ajD zS4B#El-Z$NcC>_G4xrV@AFZ49?K(3pGI%?P0?ku^0I>B_EIN6k^@2eTo+j<5-5)eC zv1?9;O$jc+iHCY46Ook?9YM{;8?%2c&K zDr_9J!OcLatY>PCxGO_WJb<#IMvw7Vx!4#2gmEuqC>jvXIeSpkjdbt8`INWk@6YeK z5VoI>{S}0{V+VVat;0~t{dbRcq8Pm{8hc8y=O12L!NW*qwaC(N_1c^52kpUK8}ZWj zo+9EFFw{CTi<0yz5NbTKTL#NJad-{#0;3|$p(yE*P;2ki6zY>~YTsf&FI zf2QHGM`osO(8~W@!=GPmzpx|^jBNpu>A9QT#oCzHXxLO? zT)qpwkR@haic@r4j?*70wYsP{W4;F5=B2SuX4uOpz-Rq|7__*h!3r!l)vJkE=X;$z ziPQUC{AwHy=`)?2*%b%-pSW4iga~PTPJa5dX{9XFupx|H&g)m?Ll%H0C%||^WdAcA z_s(|5(y-*c)cm6TmC>uj^erw(EZPpc$76bx zpP(NNkwftTV=c$k#XyR?5!UB^L~Wwd)?Ht z&tmn$!FUg8axgbX6x$E^omi}hT8$5A^eOzsZ6ux7;|yQsZj0L&Vh{9Y#izwC(4gE8 zG6wWib`7$aJhx@;lH+gJ=3y~2!ASE(`|0DBB&6&&d`#&3nz6?14ZjU_-IEW6$a&je zPKzq`U@Hp3B1)}}&A-}wEO(yr{nxEv95Hb#mk6yB<;bU$Ok) zQ-DQM^)f#Yi^`Se0vOa7d_&3$Ih@um$5Kqhuv^l6JXtpRvaqv^wRW1TsG^5CYDEHH zgb2m!tXsc-L)hwC%nj0j*b6cA+p3)vy{b%#J-7n8!}@L2OtIZY8n~_HaE7AO?i)RZ z?#FVtI-Sh81OhuJ{NW=ttZmAsku|Qo>)tCrlykz2bp_19ep|R-4w_{Fy6=|R&9{@0 z`Zx9Q%?dMft1(;|BZd|EFfVGyO+vJZ2xCXK<^!p3>snW9ghm%rSO#4K{ldAMR;Ae0 zh&u+nt`6)D`uGCFcqbVXZ}ClY)LgclZ};jyu!CgWFQ%uV2QlXbO%%;7i-~<*#_trZ>^9w-5*E*lmTg(F7aWkYw^TBzw4osK>+*2~VYoVf9u%!V zpO9LNk~WxL707@tXlfm}e&6$GS5o}W?w%a4WZyFq>tfD?H0J&oT0woLTxs`19TQq# zyu{U~L_Zr+`JI#ClUR@ggbV`09Me@CkaPhxYQ$X5~PtD%z(o>NoFZQN2fZfoA3m zB|U+ev`*sh?IJ{u-v)cTOxsH>;7L|m?{Icg z5(3L9{X~34maR6U5-f>MAl0@-rEX+aaR{`x$2Cv3fBNXv_2mhEZnA^ixLLGfJFekr zyWRO#ZLJzQhtO)*Tza+=U-aFHi^r-_$U?U9Q_>)bX@kA}{eEOlcTb>j)TVu^p+Ry& z-CV>_FI$3D{+CoMByd^Hr^K{*n=I~A4*Uvg)7?PA@n)a(w-L_YT$iFq`P`7X+!*{3 zO@>-bZq#jYYBa%~WJC(5Mp1oBHunsVWH0k5x}zDZ;$7BXpBwaJ5zhQlWqBcJo~4@S zS!p(ngY0s5n1pVBJ8o!1Ksdk{by(zqox4=h*FH~-)Gx7MTs9$GG&>5<$yH|QqT9x; ze`~uqd>;Am8^#=2Eol3)RcQRIUrD7<656Wbx2)Z1IS*()0m^E>6YMyqo$DR80(9{X z;5!uB#CB<($KV}c*1hSzx$q*n{gY5|SqDB;m>D0)@v&@X9CnKlvYcL?c0+I5x(z3_ z158UY;-4xF=OekIkGbui>X2zp2_227-(XqK$d(ojyze~zmGj1zQ`1WQJ;9A4!j@fs zc0YZW8K)OcVw%K~J`w1-sA14RmvL7K*M7%TLW`B}y0-VX8#YSvzRJve4y?&QlbvxM zHxA4rz{bC-4mY6|wrd4iy9ZgD$3D?SZ`$jk=s*nwDEpcCs!KR%SlMDWJxFzdMrz?G zB%1X5fU-gHSg^L% ztgVq+MfXaL{^LkW7Z9JUOVs%_a0!H00H;zmD$ST`jnZirZ_qBe7jNJ5waEOc+JlO$ z1|6RhTEux@4f;HxS&U=6!Ik>Hg>+Pdt|wg~Eajy}8$*Ay5g$*zM2-4xyb^}!Q%Bv z#xwbqKjTll9#&jr_8tlHibqXHvwvk+bDi@8+bQdc!n#UT@6vW}q0{(yb7CGC{@^xl z+j?m-Gya`lmv7Q%h>qoc#mjje^AzIROBhT%r9)_^iMvFP40k``&+xG-#yeGa9Kd4| zAdCk0jeA;i^7L2asQ#!LLf5b3QM0EM69swClLOUrV+I7wSR^Y981U-bS zz5a=BG~yqN4p*-HA0n;={shnCeU|!>RI~d0>ZBD~N&92FafhmA7ID4{bh~D)4FX4`C#>Dmv{Ur zM{dP0vB*=QW$&f_wo_>R`|3-;|y8qkBB0 zzsPW^@Ls{Ug`vx;w=h>s0Js_xbU35-CIjE1aj@$40x=}p=X z*Jb(z?m1tF)S62ANUzreX;3KAxDv+pW{9G2gOjnUhq>^RS01;R1eH}sFG-`-193{f zVi$56h$y_`7W)lhMvBY}m-YNywj=fvVl6#RrEHPIL5Xs_oAEoX*H;OoH=f{I#BsxL#D^b=^~hS1{En1t zPdZhyVD%QD`f+p95N`b2b&D@*AmL~LK=vyR5T|4+(_wzBs2Qp$F)oP&C$!WcU^!a4`=REPpxKJjA3S6 zp;U6#J`Iky%;X1M+vdk*&73&?zq|m8H{L{)X6hfND?*U5VQKNU6T&@R%Dtbgf4TA{ zca7ADJ4Gj&MzYS6k1b%_|fEOPJF2%DVs*uu_TmlnT^dKtoR%Vam1W3ROY_- zJlC&lU3EOGr0epub{g|>>`6KX6M!}!9Z zt8ofTA^j7;zG{Q+JxV2V*S@V}{>M48qeG+t-<0eC1)!=#tYMsA2*KxR1NQZ-f zApllwyL!^#NR52*^QAhQ3_PMd&P^T(k=5MOgX%7)^%CC$zTZXo96q!d3nnCZFBGQI zF?BIrRLSag=<&RP7D4U;?ojiHj2IyGveNnGfP(iv{6SW>t;F-ujJ!+N!pm2G`Vv`$ zV&z$B?`Bu&6<*vHu2h$TT2FhMO-}S}{)lj}K60k?6Ro|BAqwHl`Weeg=RVDHZPm1CAG=o2(sF;w$hkmu@x1e1x2RJuwxelMR@664>i7=n+%^^Bu zGB})8&R`v}5!Qr&tBNxZlrL?RZnZb(vm(}#e#JQ|0cbmLRIc^uB{i_ol+;^m_W|t8om4 zbR1CAL0^gx-@PAMYlLeQNooNfc49#_F-dESt2_T}$bZ!6AQAdc4rWy}tAcj6?*A;T zLDF0HW6?W9w`olOF#CfST#pM^b7L=y#JILA7xzO5@}B!dBCvB~{sQigv#?iNN<*~XZCL#& zWB*>&3|(8@r$><6Pj&jz-TR!w{*y~g7X&pu%ZiLD`4P{2rlB=tw(^M&4CMK@<+Toz zHsczdAcJY%#MtKt?*mj)xz}EL>05An3h2IOtDSZ**ebxe;piJASM&Gy0VWKOr&!?Qi}YppcQVlRk-yiN$okgf0@=fg@5WWkcb-* z_eKQ8o^l*!Jtr>vQapTSs#1WLhji5#5$vx!05HG*8~*sLJ!~U;ON^F#Fx@!%@2w*T ztrd9cz*^ZD4a^d`ze|Yn>=RvT-wEbGDa%@Hyh6#h4n_q82_i}zYJ7ViG%SSVyvS1J ztx1uukcfDYjf1GG?2>a`Qp=2?{IzP?_Ti007d-2?cQdEiHQhFTW1ab`(*xh%oKpqqfT?qx7$AcYXWOF`Tyh{6ll?z zJUA@7&F(qB%;x6`vbN1s^kC2K@l-=r4oMRp0yzo=26UoO=`~X|dht|rDLl(WqEvM& z+%L?}MgZ*~paY-?SqrlO7CCmj-hgstAS!Rs1`+`m`DurXUkrNEX8goE0Ip1rL0>Z$ zd11IpB>3*1-~Z1NMSSXv03^Qk`JDZL$ zgN*31?98qV^&g1EW6StNiWcRiqcL`VEJ|8YxEDv&+voT)G7<|QM*&(wniM18`PP7` zj6T0RDqBBLcy{Z>PcmQ=hsC{Z?|fyH zvk@*;!B)71V7z2SWyF|phOU_nrm#DFMsrmzOh=)=w5|dL`j7XVC;)P^W35}KygGm9 zZch}~91zx1o|_?IEH{*jM`xtMH^im${2BiA1rN|Eq(|Px$K9yG?^Hy(Vk}3VI!W3FII#i1CNo^x{dXH zl-V$v#7sA&&$md^xb(_%1#P|J_cxU~?F7giznaouq+YpAB>X z5G^I|q(znmPX<7z0b*bDgQgV`_M-ZqiSzNT7cj~tZLiCtj?0r-Y9Is%C+n&a)H3vR z!kcoQvhJ`&93=6RRifO@B9%|>J?g1_^>%#Q3gZhdD)6S>6YXuP3=d;w`-K{X8mj}^)K%9a%rst?QeHWA?b`z=NX}eKe8-Aw7e)G-e zMe+gHr-zG%nb{v-5R30iRKPjW?SL}f02FGibhC_hp%igcSk_6R>xT;FjRDZ?J^6HQ zNfC!kGqBVnG4~+%SzS{lkzB^!|!J@zF+n zcbTzo4`g~s5u^r*bV##eI;+ReyJ`X|vDNqGcZS#HUBdq`Au#Z+mDwXS-$tfy2bl_R zwVP&-p2*Q<$3qt-o2UK$WI$3~D~$!`AHG++xkQnl-}26(2yeCU+kd>5KYd9wQ1Jxl z^e?P*+%!U@9e`=6l?0Y!$YBD^ZpRbmfw=5V9Gg|cN}RWQY%hc^0nFUdh+-PwWA27A z5d7S>s%#uHaQ6oeZFwGm?QWrf;jj3e&A4oJG#uCmakWkKC7+~Q^3(NP2gR}cJx^x% zJT5m3BgCyr-2r&nA2krocqPTfYHcbSIWV;;8yI9?mf zHkl46YQqQFKniyivN%*(m+2?$B6Xcp4O`2aPBqT+FEV+TYxV1H=L`m>xCj@Iy_{JI z-@vAC1w83X^qWA2`PYxzfEW#cL60{vL)oys#DMYZN<1CHuPnHCFkuSn!5sQ?JA%M% zQGlQ$HM{aTJ=FIU1_8IZdK~x?NKU zt@=6Ds>*AL{4n&FSNdlt^r~scb0?$+!8kl$3j{}!%S|ZY?`P*6E{+}J80W47u;&3N zz~BCHGqI-j1Dmv79QfxCNjCQmUI@*g@ZOTl6MuFO;}3d7=PGCq2VUz^#oPdp#j%6{ z4$Vu6Z#$?-bf-T~T1b0MIzJA0ZaeNJRE_v}3 zs%*~{J<2D$E@|8jX2Ggxn9I*iy+_QT;&w3<-$E37M+e;w1?MbVSC49PhW)xUC=E4@ z;d-ptE6IUlAQg)#9W=(m%I>bVJ}lWBUhc%^mJ@$nSNfhUzSO1rS-JA|VM;}>^=lNh z38mV~-nG`8E}u<0&%^skq8LALICBL@k`9z|4RHM6JEWr3Mfh&A?cVp#MY&E2ROra= zXXG6E;zS3*3Hzoaqt|!efyS8SBa)Zv&}4 zYKDUMJW}N0=tMF5;ezXVcEBz?%@HK}sVy7HgUy2eE>7cGn`yUZ$|j@-?pInf`O+TO zU|}9eHZ^txvy_klv&NwF#42|`-C~4%IlzFm9#Ku*)wbjXY;4_q&O!nz`%iZ`!9Csz z1Ty0<$d;d4`5nfH&)Da}?*v+h&hCY~`03iz)89@kZQO1s-zwrtcPr;`X(fnhC`BxqGW7DW z?iPEn%lVS=U%dd{SPl1z;ZCiaRO+*x(K2FWqJ2)sev`5i?(+p7kWL_F6Hv6?UDMi4 z>8OwZ_L$^U5(66W{7{Iu!EYX^fJvBdyMaHF@he+^ zy21e8dGMiQY8dqnjI7rV=36dB$V z4HoJX>SH+iMKnce<6R8&mXGA7b3!dFIL5-2Rm%?t*m@T+j8x+Si9km!A5cW9N|YB5 zV1;ThF(gq}-dW4s*>KEmpJY8*3n`iUQ~PAr#iPr?XZipO;QzYZs3^_?gY3p-Qn%=_ zbSkXn+AB@B0D=l7au%}EyP*yF;Y6z{bGootjfL-Zry2Z#`_zRSBD-xj?Cp{?w6*~h zwrog_r4y_>&4~F9-L;Ux-R>~?T^)eydz^Rs?ac{WV4K!=CqDBm5Fz(4vLA{a@% zHEcw%9f3H^vVUd9ugdELnZ=^^;J5&O2RIj616r`}zR#C+NQMif)&6`-u^bH7ok24W z-lvE)`y7A4N;M2u_K{=X@g{8Z2zeO4anyYDy~ME%{#zT%5bM%1fC$x;F#d-zra^C6 zJ7bgHQ=Xl^MFM@MdYBQ4&SBu)9sm=@$R_*Tm;_@t?2c#1-V{${xjpg3Hg{!A4!{9 zV%8nTQdyKqTQ#Gh>X2j&_QA$+y&H4!ka0B%|3(S+nzpqq;05Zud@P#t(V))Gn5T|O zVi^s`H%~O?f$YqFhtkwX`$T9D0zSl2T6J5l7Kjc|d@;%=0#Ur`0ewcwSTcx@4YLAK(j(h_Q+EJ4W2g zZ8q;lQM2-*(d0-%$&G6BsW-=7#Biw~CB%KN2DuDCO{RF~sTn6dT*t1u8$*DdTz}Zc z7kc$pskN;V?BDhpuc?1Owzg|t;3-(;#dwPnKs^5c;T06)KpKNS!kcUyE%6Q59DCJWnkQV5!+(OZw!Fs zJ}(>F(%p#%W-|630J{*Znj20&_+ZsclAPwPIGMRRBS4#Pb976c!n1yS^yXq9Rs5wE2dYXl6Q*BWeF=?=SI zwhy`HyQ?Hx=`H9gq!(iBLmsI^na%rXzK zEMIVyIC~3H~ln_(>Y4d$s;wa z{_)EGeI!SOMoQmtypJ56!KiN!-UH=3yjZtU#1ztwdL~d@l=L%BF}jQT@g*ZX$8xzV zZXwm`$~qk^GVw`0@pkHdeqjZX%|ghc>C`?~hOcgqr@_4OIvbd893ueI=9tcuQL1*W z(e(iVTjU#s#1E&T?U3kCEyRYd_J=D=__~x_e#!C(1g#*GOCo5zNKTs z1`5kcj+noC;UN}a`rGV<*kg>-{?z4&t(QiEq<@AAs~~hmzZ^OD{oYFQveM^=+YcJo zFQ($s$c`&CW)0>~PjKm)+lQOxdOzVu=Z$lzZ7yoBq|1=^Ox66_J7AK6D}wjI;RO8Y zCEP$e)v)wbu_~j0%QfQL`jfgwnq=IXCz5LsqmlfmYKLbMVmtLP$Hn^zqI%Bj{RhJ* zYFfyY3SmkwF7R&Cw;pf3F-A}(0sn27SWl-41JPWyAk9hHZ*`uLE6C%OJMP^4nLy5; zrx+UR4N31AH!a?cAt?bPCbCRus39^cOVMAb77eR8bo|8C+=*)UncsD%*ZKW&FOA-T z2ek@XiwaYOm0kNkTRainNDJsqsUGIh7>TcOu@5m!{la)@yzVNh_3=AAp{?p}`s}^q zmTcwqqx*3u=7kXCuBAwXjf*`0e9TgUC(|6z%n+h(r|_Gw zEI|PT)Odb9Sl4|2OZZ-gij{)1o4(LN5`}$6$_e$#8W`n`5)LE32(cyT=X}c^SGD{= z{Gv<5C;Q|nd#k|}e9gU+b-mQJ+~Mk1sx^*q?tT_q)RS~85iK>LTUk_o8I)v_B6SDy8$4xcAyAZ-?dG{a%itngs#G(3OQ}}t?GCj zd68#nBkWH2I9k)Rdj-J~?=^nr@G2yxbm!$6vUkCTlwyPYP9uo^qEEe|- z=XK$cD06NP5LB;YY*g;~A5gBs-bM$&B8EApg|iM8BnOhIYq&>VdS(Anr6DKgi8G51 z^LMG+bb);@9kZXPp!7mw?|yx92gsG??>DzlCuX(_p}hcD9eV;rZhN8Czj3m?!G{o) zAkZ8-7Uf{0Uoyd!?6~~#lx-_N=hYWw1lLm(1nj!ZVo2U{KC^Qyr?{`mf{%w{y=ep7JCkRVP5|k>o7WZ~9+i8-x8B_a1tkcJ z`|M+~yNVZ)TAOHe7S~KN@7Em;$(!oVeJ$``!b(CR)$>y9ZmN&s)6%XlR?0`841s0TE)n5j~vwEum4k!Da zdOo!+?nzKb3$*^AK1bQfN)tBbs`YDqs%s;@7H zvAA%|Z9Nt#7SY;kTQ9fD05qD_=O;ihzD;W2YK(#l+5tlH9Z|rxOJmmXwVw6O157+k zydovVJNAQj0OD~wfLq@C z0RX0f8d~~~ZOq*#Epz<=#GE(o97TOL*eoord471_#n*D@fF=!93$T|~EB|b_{Dvk0 zfUpa7pmIQMoxQTH3&AWP9B;g5ln$RC%mbXwTa0~ot{P`A*^=G2>*oRPQwTgOv=+bv z15aA7FZW)Izeut+xi@aN_;G2O-3wj^a56ijNcD-rg|_Raotu~F*@H9|y98I$eKXY_ z_C!>@7pI89jS%>mup3+IL0_`l{RtBm#Somph?a?6H)9$cG;;nIo00jl%;6g{HKI8coSsfH;2=!|Jpi?n`B0zUA0`Vs4kvyCw zY_lr40%7S98-@JBD#R+k_ZS^QFB*O@eTvFP0CTj?`J3aD@`6$sV(4AImk27H83P6K z&w+fbx9kO5bMBr^TP^nsGFT$*ik7ERR9(j>)!D?!Dyi;#idIB74#fv56nHp@08K9@ za6O^!;&US(&RG#7Cy6H0^71x!2!JNbliS6_x(CjGdMq(PmZ}lzBu`$p`B+Hd><`%S z-?~Zvp*PnL2GPBsJZY&*ul?vWzX8N?}Vy(`H-l^#7xwQp18i)zOpNa9pSUO(Aebj;U}@2P*HtgX|2`D}>~aRSFgyPQ_!-a; zN&1bsvwlZEbn|~d4OM)FEH*7~+`dk%O#oea@lx@JVP*rGscJh22O7+G(DD-ss3_3J zNx%Pi!;A5`*}a3Ia1ZXH5`U1fw71wb_$eBg4ZTKJ?9{=MhLx%&Bn(hh4d-z63xM|V zqMWshGI@6mkSG|600>IChUblEicPe-y7h_}Mtdu01VAP3$vj#IwErFnP&lbJi5Pk00cv)#NFVQUdBLBvUax zOgRS4+~dI9891b*NJ^_CIh4zNymqLu!*wYEyAN1+fC(3->!1`oo=EIY zQjJ2r{LAo!HU5!ocd6*3LDyKMlEnp*BHN&GMsI zo|Stbl7<$$#6YWsS~VWG#(kdvxPyDO1MY0v$3OA%+$Ku%sOqcA(iT<|two;|iPf%4 ziyW0wsA$^D9W_8 zFPxJd%1+6UDx)B4^~tL3IB6lRgNHwY9Fl6>k0%*E83*ggld_CG7F#t!6QlRB7GAtC z2>tGx>^LVt4jxrf`ihq+{DW+6&(Fy=HKVinKA6+?yG@#=TzG}&1&j*T(APo<8e5eJzDqZlp(yJ0FVvsV;LVkm*?ffOM8w#d0I+NuXWw}1nq`PG$(z7k#d`N_%}YnU>(^d*Wy5rYT6cB3VTk6G3aa;`dxUCK^*I zW_d|s|4ZPb&vzaP0I*_P4+Dgp-EDXil5FD}`bCSNOe%<-95S9f86XRl$^l8E6cr^tm6=TjXhg2FaumENss3d>% zJ|k??5ITzBN~JGJ$P0%4m#6B@d!K*5ov_Vy{(LUyN*s0A*D0U1jx9AfE4WqChE-pf zHsc78mv1;2q5!CgV%wf?qG9&s1dw;_6*)GtpgZ+SG0%8|YfGmVPMS^{*SqjCw=kef zW{eA^&z>G-%v{6cr-pwim0t8X-zL@tQmn4)y&63PbmBzTw4lR;vijUNo(|S4t^_Fh zDBSWDw=F}eVB^$v|Ne);jivmg&N2fb`#udi#*YrHM#6HFpqg$`;an|Tb}^FnB?(l$ z?gfMXB`P%eyQTN3TU&$@QnW)DOVKdTvxMvdtk-VxY+5U6vfVLF1s{O;Xh2$$-FPds z3tvK{Y8KUUQ#~LB7F8lh?9s-oCEs!1km2BEPk_^Ic!qRVU33NET1>8 z54lQ#>h*r63I}Dd-e+LF35cwxUU+S)PuL`b-%zRe%Qg0>&r5)-+MMy}TbP9tF4h|C zC7k#c6`Vsr3sn_9V#z0!NRY)G3zC93R9qO%Z!5>H+4b9~-q155p^6rASO3l~5~Fq5 zS5<+^9E#=I3Gt4e!-)%7tdUww;=&7CB$bNBJYtD^twg)hsPTMzXLud(58|b19)b87X&fe)9?E0qV9#Ev~&g(qAee#L8 zf!ZVe+ll?8VLk?0>!UOu%w5pgwPecp^-wZ|!+Lo#P6` z!-sU@_T8q$EZ57#Evk}ScN_V3@`AXQfXa=^F_3PT3aDc-=_DWA91856R*k8vut&eTkeyTSm6KQt-z(mh(o0+Arb-PYP4qAkkVHq zCi@9hRpyG`x#|1&5*jwNB9Vb@h=7J z4P0Fe?9;r!Ay|=6yAhRY@k)pnptm>~e3;hp1%_`0P&m$Te&{ZX<%X)a(vZE{rV+`1 zlJ>)K(U-WhPS0(>qf~d)$NuMx#8;HurmD)f2IOe~G`7R{gR%A8 zt5biuf9jA&vstNkOZu&dp>uyL2AvoaHZQ73%sFJ*Fozhn`sUVXjH=Ee{C~(*0Q+vR z92tIf8eRf8AxOGsHn=Gsix1MiDS671$*3ILCkj2ShX80@^0_dp;cm>jSaIZXO}RcC|-{D z$?AAQZDk*+kviQ#Zw49Xhj<-)IHelH5ei$RtZv`$>=N~OHID1tXRfQ9$BM4wVgnQ; zNK9YVD-+*NUH0!=z+kvBsIADjbB_u%d=27Mv%l5~TQ0X`_sC#Kx`>Zc)J05)v2WN! z`Xj_kzp&y4$nZN8q})DgwYm4WJsmgfk!-4LYwjIlVrk#p^;YG%45`^9`XvtiPjaHn z_<)5*IK)Zm1|OZr@+%Y=^9|KfZc~u>Z2KoJjTe!sw}BLsY+6i2QTvI=+v1rh)7glu zADJrLR$rY|I~LtkzN$ZTUZ2WgF6KIW5SWjg)J?!Rp}|#-ME1x@*BVDk?qu3k0iw58 zUYC$H38fM3K5c+}*}{)A$k(liUt2l?quEI_8+LL&>->Z=#4@D+(sYt(l^0D(433oF z{Vd&POXj;P*2FZj+s!-vtXSR+|Df;3Nd+>Y!Rh#!H}Ac{IJ^l9_-aioP(5UtE0gIE z6UGr?2~Qv34tO-)WE9@y!wDhjdS8%ma%)OJ-T+sx*DGY>S1a)^Fa~&|KYWWL>~k5p zv`xs((nhvBUm5|u51xfjJy2J5Q9Pdvjk?~WSSJJvdA9Vz3N7|jX*aBXGmP47zBI2{ z_;jZgs(V}(=eKP;DX3l_SQbZe=gT(WMYR|q!xsNMS1`4!!;xA#po`Mgf?WKnmAHvO z{n5+kUtLOXf9o|i;8sJ{1G^B5g(T~&%;%|4Lqyni(8#Px?zQ~OZ@UuYL?9@Of5#$_ zZ`bpWAIA^4(SKZWjXr!@*7=ifvEY7pHSOi{E0Hju9;+dlwo5rkig)PU4-}uL6t(a4 z6?|%87xvScNcVN!2+#w!d6~0T&~VEb<)!A!fdP4VhWX|bf+~$;#;itk)SE_$|BIqS zB*)M2y>IN_lPS|L{1~lx@iW`{nU9&|-cDGM59jtr<=5mU^5<0}+VMMHJ|R;#W=!IL z=LL8QEuOMaSYdDLu|3ep(>;x(#Ct^n9(`kWs~RBx-gx{6xTE$k+>L`&c8s&?fu?P~ zmCq7W<5I_GcaJDOCGN>j$?t(_~c&YX?&b*rZ3yf%X`1$lffre{Hx912-`-ogIH6Y;)` zvW*JHzY{d5x7Qae)qGa}#&co>yT)_#{to_;$xoN2_oxy6pjbeMF`;U*W{#yp-_n?Q zUSJ{QbPRXaZSN0OuKJM&dqxyykmh@WGshCAKSVW>&sMDG5!LgV!0$4c(;n_8vLygl zPiX^!=D%M0N!S`&cYG^7&tg3Tr>EajElR7>!t$%BU%Zo#F-eT+I|u&q^yO`miQeC^WoYy+QXME3-KQEezzAd(hnpk=g=EAzkOm`CmUf^%2%#R{(O4#;T)39)=I2dbSsi z^WaH~YltioFO=#U71w72f{w2KEWlf1$vm*ry$!O?frRO>UKapd@Vrxi=bJT9GCIp1 z;^fonN(2Xp*xpUp!)sM3^Ue+_kl|xi{cbV(1fT|TO?pAq4p~sbZePbgf)zwU|BEAg z8vxdZWoRuw(CM(2bhZYJBy4AVEp3!q4D+=f%xjb;KRg0>ozeU$Z9(p6n(W00QIIoZ z=l*OKcSt>#LSHq_V#woiWu}eE1oeo*YqpvJAs;oPLU$F2=xBEVc@n>ej}lwgQwvEE zMFQCa34!o3#l3WqQEzJj1dS+I3y@hF&MK|a%2$!YyZX!8jgLfn&~2MI_WUPhuak*n zVbfoDb%Ac=BATa*?mro2*_U;dVlB)I-_V!pyF1qT3$i(+7OVbBdiO?_hx5+!eKhr! z8?cc)cG?(PR!bpkR;lTFA~Dm>dcRmUsi6jZn(3U^_X`Vd!44uEeD*}O=1ZdXa%VrK z`XceOBLxN=j+$04{A-NqP5IgLSS<@@VN$4-Y!lCB7`N=cayM=_M?iZ^Xc3Hi#tK;c z9m3ZTyObGN}5XJNvW>etl+$J6HXfw%{_2}ohY;Q(GtLM=j*y@-4!L9&3)(^;|89ZoL zefbJ8x-~AJ$`cTv!RA@&7X4UUo)|3$z{^0kp+L`t{Mz}2snobAP}E{6zY-&q5mqRt zSLg}n&nU%xMZMh>B|huX3qdRcrXbdFr)H#;W4~52if(W>y}?4at#hJWVE`l|WE@#M z18#p*H8&C$cH(~dz1!<~^aS37*lepSG2sDPY_tnZB|U!xPq&$3b_CT+(}NFe09TX^ zZ2RKzF?o@CqwE2u8RLtG)kSrJdW0l5F?X(|G^PinG&?)M{kc-rXvT%4XWhlBbVmI_ z68fg~4=_t>yZ)z@$g2O#ZVSD>a)?5&ex0&St41Sa`a9aKs`q2VQsAdCU=5c>Be#>?}&PPw?- zN@SRg_1#5w<--n=ur!d>@LsJJE# z&rZsq?F?4xpP%1mLUAe%(k(VfZxjP^Ea#o0XWF}{)`u~n6Mp9zGHY{S9&f-uE~TJO zFJpe#jvfrf7aC}W=*>P&YQ`Cr@Y*D0N`KvL7X^B13KpX`wr_?{CRC1?Sd&UlevW=P zoEi|Ym~y1{-55u(>A$r=MIy$0m<9Wc%qgWs zF|(uY+^>cp;6_VGpFr|GX6T7@Gwir}kSECjOS>y#l(m^{dc!Ws;ZSIYn3u7z)xU?9 zJ@le`0>H-x0VJV>kJW`i2@Q8}3Azt0xT+QaycW{?#xZT?$n>UmB_bVul)KpQwGO zW+oOF2l$*UB>AIB+p@yjnFbl9W3)*t{{!&IZExdE*y%s}h=06SjKP@CEgC#gXcM0Y*jCIc>q%< zGv#KOhrwip#1c_p23<2vTcYdUJbMCjhx>F+=im z0Y85;_3#93Nt3B|`$I`k^aHy7Xrq`&oPnFNYR=CuCrfC$1Uh~~0mjL5tUAwpkDYZP zZPXj2b_+XMk(0#sjeE5@IP@hyxS-fS8k`TXcKKW;415iDLZ7E?s!p7wo;J3YpFBRH zmG;h~y6ycMsl9#lbVt<>h32sQ=aowAEv<$zvogE86YW`awT1_B>um0R<@A-`NW-|H z%=uv`Xv9Vc{jc;}aX4)9dN69syAn7QYr+`FLlXgN2FEu*vP3Yc^MQW1%mqTRTqH=D zq>{D8R%)jsmnw&i*m+DJ!*fscM;^b=)GG*sxOVNqR>Ph!Xt)-k!xK85Yx8Va4SZ<3 z64DeVmz(eE7wU>lbn2Z4)aSly&=DAmsoh*|yy2L#u9>rzZgH`)W1MYk^FL{63wD+d z#(ZFTo+*49PpCkJ*X_^0nUbTXsA=!R$dYX~^sTH17XbhpfJ%!2$R%KKWK~UHP%^#X zcautyh!Efl>^zcldbU{LniE!+ePXJ5Hp1|2s|#Cz@GO{^JMl*kj=av4Q#aD_YwR)*RvksY&%jztNh zpgN&{_ILKchGoIlDX?g|6Rv>aur#HqMM@tH&WrJr>P0 zA}!5bHJ9F~i-1Kk9r8%LJ&*YT+D>*65{TX$-ixK{n)Y3C<1%Xoo^ z^<(rr&T2>@QJ9;Te^%Ndyx~w1HcQzH>DYhBDVxb>B2@Q<4~v7CA67L6r~m zaO=V;D@Eo1*RmK*@Wgy|72!q$s{g&+{isthYwa;=LWDwPbbraTe{o5XIO}IRo>D$C zgBfI;EArdISIif2E#)B)uc*rsfn6QKIpfSj^$sIPitP!M%B#6a;j!ummF4qT62r#O zq*cTHyoBd@3#pqXD$RMD>leUhd z-rOLcIDatl6^`K!XD}ACDDt)qPKnnBsRCz|d`w5eJ<0#S0opzu!;>qqqsy2~l)4^w z7!0VO<3zD0zfaijRTNCS?;YxZtbS{I06vM7Sfs7$swO>q6Vp98FDT`B@oT{jgEAXZ zy4hCwi~4qN^OznBpx;GD2_?iHK3~I`hifB`6`Q|KMMY3TXQ7ZiT?6M!PDIp&N+$W& zrQyo2mCBhotz}(}7sGDDAGd%k(3A>uNBst3Gs2!cxJk!7F6El)Pw-e{kek^q+M&~% zN7@VNBau9FK9&8|fN|$Z5Xh`y2C((}liElUWPbuq78 zbog|D2!RW@%fSDo8~XwM7d+E7NwPBGk1p){G**@}hhZwSk!24g4Gd6v8LqDVi1 z(mbqGgx>Qm;~S?N`D+5rC9LUd=D@5|7y(j?i4G0f*b3>*Wo8gChQV^Qw0cK*(#_3A zT{a%Yik`pWTimc^uU-m$OO^dUznTWS;kwN|=`f%hqbPuR_lMDT!}k`8q{ABZi(Zy3 z+0Efs-xaW=zfFSj~5FOO85N0s3NSre|fMezZG44f6?nz!1)|Arj7r6rZzzlz zs#u@@L)kgg{ViNR`pFy58HqL(KY&8+-9w$)l-fT&IA1FGtk>&8%IAdd$Xhef#R!E6 zJ6O8=;hLqyqkHB0-#_L200g%dp?!G0U5iKe2hWP&Zp$LDVG~{U>Yck5FS~NJ3^+gi zsd4%9MimtI+QqZRoQWS<36dAYw0iyW8dh1k2Y~zAr?)>Y{10s_RRBwNSZX62XT~GK zgW)h(-8hc^rg3C7{mop!Vz?iGw6}hIP8-hP-H&EFiiIEmTCpYl;ONl@iMFodPlD%7 z7N96V1bY@ERC#_@eTD?j0?$v9Rt(evAwPU|GYZq$Ki>u1(qKEU>w(BNhCg#=fGIp8 znL!LpnI&*Th8P9FI@tWLu6D0b&2?qrXITSv1~ZK*;byv(ye;wnk{P!|-CGd)u7#m6 zDdlJv`y|zK)@;y6@$N?>6xzOSWhIi>q$lC5``@_*z$N22i)KdI&ZtQ>n+*#t4H%WF zF^xLPlbVn?3UV^?oppLCmH zEhvpgR^&t@UI8BDe2ntVHmy(54yx&lIZ>A4_l(66DtgrQdUf0d7Vhw%Vi)yf X| z$=OfifCfn0Y zJ3Co)`7hQ;cW<@F1_&bpMFlP{?rFp#Chu3iOI8vpB{ zh2CEvE3Ullh3t-)4VV4OpTT~=_;pg6^GXMF6+$_&-9MfkhA1<(vG^Z^%8U+>rxF$x z9Q%isNf7(?XjFxZ?}w`z(#xHS$~S*aG}Uv!;&e@7UNve%gq80nn)*OFfPTT~EI!<$ zQ~u$#;L{TsaSgJU+c|Y?aP=~Gsbh`(peZn9H&1{cM3|G{rFJt35Zk%o%c|d87A1|n z*uA>6)@X3D1AuI?PM}IfMr2PX(4?~_#5ZA}bPdu_(|8e>=vQf0CaU%Wq%0>k!9}2GlPf*!sd05w~m$)Vl3*H(y2O6eQwgSX!$r5q_St_djWv;p}Qdsd}V?QHJ@E01@#HW zly7?<;*T)G*0n~7p?>u#RwGu^KJDjVv?x)m-zFZ|^A!nj`L znxC<>G5yY}(1tsEg*{bGTso-Z(a-GD=WjQPrzhBU-JEyb!Bs(E)>eub*%{>s{etqdh-%4G?r9qVN^d#+I;`(L;xYIUSeWzboze_G?z~ zwoP)Ef|ogydu(pGB=Yi$j(?d#@{!{lFh#&Cs{ckuJ}vo*_{$Uf4<@K{Cg!XS#7yaz z+ny>nG|Pf4$I>pd0z@nzD&XW~J;z%%?E7qf&xZ>?+xBh*7jD6j`pl zo3Jg8E=-51%Xry^brg8lA3gOX|a)U4m}My;&Q z^7zs8W{utqcR`42(yrC$$UhjXY8Zz+f8Ey{4>0eh;zs`rBn7Gu~7%oEGTYP-A(VZf!KIM%XvmRMrP}oR!7E z(k=(a2*^e`-*g?=bKF28Lb+NIBLKOZa>J&OVhXkewXCBEY18=Mn+9Es*y;8k!)j@;;KDckv}#p8gXkmXae8%x=7px z2w}I)3rZy~Y{;u70qU7`8=z69Qq#8WXwpOzTKDJ8(e0^ykCXa&Uo`X90SBFM(*=V_czQIFsK zRMm@6J;NMB><1~D)%0kJ{s&jDS&w{Qu0klAb;qxt@QCdd;iKUWDS z0z<@<2|v3O=eE9o!PeJBXg}MGmz{ZCPB(vM+Bo{gdFCQ{Zu1urxN|-;gy!h=d5GYw z1bldfKdBsmh1f{dZob6Cmk?vR-s+Il`sq$wM=){Mrs{3)(Rjxj)&VgDA7#ZD@ScCeE%mVKaS6dp?iJniQT1^$A7mX2bW~w;Be|S#Q z2V?AtmOHIiEEeNxNUkr z9)~iOyS4FlnSt~E!G(&vt_Fq2vH(--31@z6>gmMFwz~ioBeJxSouz|gAKfYCz)Zp4 zc$S7obC@n6rxwV;P63uhWp5;>g&n|z6X{wir`zEcZ@4nRbpx)GK1&2CO#{pj3&Aj` zbb|V1h4E6hyClU<)sc9^!G2RVMOcL@2XBxQ?7?ZdG3PH9IMRsCPvFr5EL50ae>tp7 zP9K3&QP^x1QIoO~)pNjC>00}+SlwwH1^ZefaD~4Ipaj>}m3@wj9TT`ek7^eA#e`gTckXlSIFAAjcCf0Ma{b^C& zmxi?f?B9Fp3qnf)w5|DiDEMBUdLuLQg&D{7>(7?~6e!3{LvF8MXg9GHiEsLI0l-~W=5PDG5rMrT=Y zJyTZ9X3RXQ+S-OauwV_6nQl4fy6$3gwkcHhH#4Q$R(2fKg4#D?n@lyP6hh$r$3nfu zraHq;Qy#=1#2DPm^3?9m_OP_?Q+P}5aIe?s^lk5r_=;(CcJIf)(eLQ9Xg(VjHF+wP&j$^$-K@ZD7YHzC0lel5Rd5(OHES46UX4Va z*sTHlg~wsPs-XlLuYPUw7|a^rB(?&4SpI|J*P_^HNc@VT(;uR?0i&`cKcF@oYqskb zU|_~Ya2@ojH*t|WEqHsYUPk#vWAOn&cn`3T5GnvT-%34S!lA-vfX4-4%EdB67xnOr zP1{}Dj_+uun%l1HeDG~Av_t~Ukv)i*{_XzzXY~gBysQ|p*c)+=zKk$?y19PvlJ|iC zs2CN4&{3=wv3FRhvLmm~v4-sdqNj6ZT0`1XHWkg`kku!>m4W)8yuG6Nxb2z3Zxdla zJH~dOWa1ox^s3wTSbJ%F68!8#9MVr0g?ko@18XFf5S#M>hV@Rr%57p+?*dOoiR*CO z+O2FDkPhU`|DmWZl2rUhBPhTtC10>gM9_Onh!AMBhXR&mP!U;S1Uy!cS0U`+Ngpk{ z@R1BGYga87Kof`3-)!L(rhN@Q3&9+Ka|<}RfCwS{=!WJM zO(X+VI3qEo+urv@0EG`;ybYaVy!(Of`(|;o=oX+}R&usk0bE;X6u{h*hob)l4uK95 z1JIcq!BKEG%&LqAwn%}TRg+Xaxx_wx;;z1265N$%p#LCU2L+7AJCfGol}r3t$A-Qp z1_6WRL?qv3s&?gGVUd8*pI!i48zzj!dNnCJ#!D)*HvqYy+`U1tuA_6^E_fGcFJ0sz zvMtSYRjVKRHGmKvAwvxTz4r_!Phl~vSHeP)$Yyl6{4!(FJvq9T^Ok$z11wGI*4y6q zg8!`VQP+?%EFhU#GB!?9l2>z`Cb=G%jM^Y5c*8A{dnxnp8job|zv@eW zjns7;cESBvO&9+Bh@6K!rTjGKL!18yU^>54BeIJ&m_#eMb8rFJ83^N+XycPM@PrOA znu1_+wi0_{!FSsY$*=N6^xX*{XxZCP@D%)!A|k|$_%`#Z3GHT&=8&7Hp#bN}qgH^< z{xDM4w&NjZ(!=WjN1eSh`UXx7+KvKkcwlV(>)`&C|DfNbW?~01qADxkGOEGex7EbI z4(mBAe5!KL zQGj0YA*Rl>>I$E%=!Q@PGGLT#DNB24rDF^tP3e9(lmiSK^De++e0&%DdW2=`%4E0C z$Fa88WxlDSxO+~Dq?Lf60^7C)%lnNtHO*VEx7{6>*v}bXE??jg2;O`$gL(0e?yU96blbU(D#~?y zuV7#3 zVBiIRgdcKF4UQhWx4()JhfnideG&9u-KZn-ixF~YO3xH)1iY&?QODL-4q(8JsCkEW zyLAsQy%EjV9Gl(sIDw-rgTd$1^G$W*l||7jHh_0gHZP6ajvd66Ia&DXP~l1ibun<| zhN3cZtQ$$rL)&zZp}mLego(=0>~V9H^?8+behT0vq8{!#1A_~mNiCxNZjT0VAV4In z#Ow5NtZAc=kGB3xng6$9^W!axzV-0fL=u$kQ)859$E0pOCao+#XrD1KSa7KL<1oMP z`D^=a^iH6DMIFXddsqrf^S%r7z=eXIcx*Y;2@`7E_ZC=v?dFUxE`~C)L4w6P@sc{f zsAQR$)v$u<_&2&w6>S>^7O9h+xjMa&()!7^86eWgFEu2T@bHzF&Aqh!|1&Pkk058l zSp6v*(v$dE?sjqOz#hHSm-gnVu%clxe?7%Pru(R9LB`~dg3@|g-xN;J#X_D{X!-xYn! zA+{4|9)tttyKoCSvq3idMw}ZtqOSLLG}#DC0^6Q-!xvc;A??WOb5R;X@2PaN&Cr+9>Iv6oKXW+qsMgc|2}bS#0v4{p=Jr&ZiForff0m-NY5r`ozpTfi>MVrX&Cb+toI zA$VT4B{g|SKkX?$wiQNYB&ysb4YeYQ58{a-A&y6UE7=~y_U$(Clhd2`g#MLFnbNm3 zAOix+iE#Sz&gCX2j6~J(_0D1DJ{qr<()vTK?9-8JUYxiU|8jHu)fxH8j;YIm2-hM% zJ>oKJE+i@DB}0XF4bxdd#AYCx%GQ&1GJ5SJmos;qJ38#o(#hfKXSo@|z^8l90=A!1 zg;8ZQTru&|Mc@Y3bdy5PTkY&ywvGBu*|ezI1x`5Gn;R$oq6jgMlr9o%MCLXVE*2Ps zXSPZ1{P7@?KWwq<=xc7#oKV)jd+Ts$h<<4BkHn>SZe}S-bAullq5nQs$IrcL55Ceb zp6A%y1(zsVer(WBfPtnGreIG0S_`0Q`BM;cT>rv{NYYcr`CH8$J6%BpV*h8UVWelE zQLKX?kZiQ@V~@XkO)HMUduQa@eHYB)3U0Q$uTb1_i@eF)p~4Qp4Xl1Y;AAmM_6;my z*oHgGtas#)PZ5C?)fjzhNu?O&bPc= z;6e6_+^3;r3GC`kz0xD_&a-fwf;fzcilWWhy2rHfOmyz$-=Iz=akV@9u&zy?H)kf4 zkFL?gwkE9UB7Mbhv5tIO#GlaU{r?>8mvSPf4G70zjnhV}K1!gcUB zw@we5T<$${5Mle3sNw};4ER-wP;pCgozeMsGI=ul$Gj5lbnFd|a1zL53JIy>Om@ngNGaavtWn8k66PLID`wrV`0FVE zhYV!c_YNbcM{|n1!@M;zi{+_^uU*+of46n;7(=7B2xn1+JR)lJ5p-^>c^4To+9iN& z+C2N{@vr>}Y);-R1i&?YGZlH&Q|U$qduHd=7u+5KSeY)F6qT} ztDRrbvVhR!LR@Hx7^_zDF!K9rGVFe*kJ?A?aN0S|N<(MF+qfZVdG?f}3|1i@`57u6 z_XddV%h_^@0G4x$yv@Cmuh@`*!HZ3o-_8t0KKylU6vSf;RMB3VVwDy5bBm}XE{!-u z!!yrT@q+E-SC>q!wcNC(RGin|uOnj>QU_3)c|x(26*6$PAtqLH`NrY8L}X&H0&Lhu zIay`|TIpXk{aXMJ+(w7Asq7wm5!O=dr^urvX3KG-{WW*k4V3EJ<3IkUMex>E+o=ie zeV4h;#-aNlaQZ}sRsx&I6Sf%Qwp>U_GEN5e5>I?q-=p{JPen`>NMrr#?T77mOmN{U zczV~Po>PMiBrfigZx4dlhba3K?_$N;yq~#2s6pvn`6GLkIvdainO|l&wA;i&4s#-7 zh)1L^IGa<1mLnnd7o|@#D>tP4FLHxdw)3(t%(SF8RXjGjY$B;GV}9QrEgm>O z5Z7B!4z>c78{ZjflVXm{1L;i@)+>7A{6IiB7^2y5%seq~tjwlQJy3n4Ks6hIC@{pb zyVo7MNoPwZc*ry+((UHUt0uPb=PI=PAX5WjC|*Qz(fGf!`> zgh*8XCQIt~VUJ(D6PZ{giMjq#Ym8_X!&mogH~@0jVZRcY3_FLe`cTES(CWXDW^ZdZve zmas|q__SNK8yl8PV(LEiNSo}XHb3#rwo=3ou&6dV5?^3*H?;!-sCRX!P5M+{!bZmZ zSon5&LG9H2b4fDvomwU51vy;tC1Kx+sq!Fsk+LSJcghd&r}GTE!jj*5;xa*<(fQr4 z7hY@&(W-=;2|JI~XbKB*iqn21A{4&ha*`eF%b6T5#A6CO>zAP-H%bZ8B-eS%fBq0(D?2buN0GO<;7fDJC6h% zw3*^`;F)0&7q&%XV{E;VO-2c?z&i=@wY6L-mxG5Djh2g#kCaoU;-FL``)l@5%}*#O zgb2IuElibR>W78E)FiwxIq`z*cv8qdP<3ge>>7VZ0QGwTgoo?R(rr!mvG&g>P|Kl< zOc&=lh6N5IqE_n zLdbZCL%5IzvizMAS6}Q`0=bYQ2t(4#LV^CaUy;Abz$lyN)|D4`^mBdTV)yuA`bt7t zF_iMXH*3Y#@jA%2!H>aT7?3i(0V|AGMP=6UT9oQ_{oW&ge2zX2_L7^#zaeDDLA%IfCK zPKwR8(NL<^`uZyjpw{OO_%;!AgUCv61g-^L;)37N2~;G}t@@;oCmZ&1LO*G}or@^y zpi5B)p#8r#o2(+O|T>WFM1go6b zEfH3c7av6}S)}#{;)tVR+5saeI z$Xxv|m+*kx)vqR|s_l2*9ETilRC_qf<4p5eN*mW)OYI)f8xOnPnTK;#)+RPob`hLz7xs3M}L0h3BK&{ zm(<7ZCdNvCzA#71mr7GutKn!s&gj!JoYu`wJRyXTNuzBx3Nb$btcf($3hFCD*SFhp zEhElz?DW7}WqcfxbJ~v%b@n4_Yqa0Wvxf$sy)OSd>Gc-CIc4x#%h;`xTkyJ7LaY=ry~ zsAL{B*%Qj;E%rEhxB8{n#sfkqqQlS!CqFvy+#UI_LD+!Mx zg4x+-tS!mF-kljDVR59W(R*#~sQT6Lquge@_U-(BZ4r3&A9rmX&Rzl43WFEqDzSMo zxE=o@{Awsq53$EI^euw($bnVntUiO7xG>hAP)?s4YbLnfEn4x0PXunB0La#C$~4*L z`QPUXh)sG8gT!=u)*YNDtqZ$AeSJiHl?9LG9iYq3GhteBC^R%PiIrD&N#KLuJ#y7) zjAH59{)?2w;k^QwUlkYoj=Wt>E%<6g5TKoq9?JE$0dUyZTh8y0rEjUs z1JBM_+M-amICxi&g|ouQ-d+y6rKMd&1cjHeTHbM~Q#S-XBt9pGWst!>)(z9Lj)fX* zbnr(L!#c;IV?%s+sGprpXD88o>fuu#e&4bo!RZ&+dwQt;-GbQG+i#1WlRQFkH!9SG z7xhv`F!ag%?8mNp?G+m&uKSeh^kWq19clbroNXfKI)%ShVC25ORWamPVh0gUFj7_F zx$z^CI;MmQ*}^6`Q+i;1(ytp1{c8s2Vmh(Bc3O9c(`IP@>z={EVtxuGv1 z!(46HzvCU%sg9}ns)s|-u6p(ZDIcFDjNf+YrOqBtk!@%l6&LOna&o~|3pNad({Xt( zTdP3B*{NC}37}1FI5%B}E8VMgVqoShv=vs1Ps0ZM|qoudiNM>~!lB0M-5PxN`^EnIin3|CE)Z$McIeflp+GF-1zDi}bTIu*`l)@A?AUC!R_!47WMStrQR=oNk~T@Cw-R*(IQp;SCHBm_CU&@BHYBxR7D=yW-)cIS@tLHCqJYug=JXM;yMPhz{3YO1q+FtGny?WMj^5iEG z--$^5qVjstpK0?l_9?S35RzqbSp^=U_A5cU+CY!yF~Q)B_QybX`3Y!+l6h=WGdEjI zB6K8LK1I}=;qX_->~~U?`dO@>T2XJIjl(8TPupZC<0uW;O|IPORZt2i_E#slG4)pt z|GTMF;E&;&`bzdRwjr>AGRbNq0%x5xcE&`jwyO@$s*^Xl>d^hT16&x1F+oHo@6MZC zqdtX;ZD(dTqqiwcYgPFN6c8fgRZF9HIucCYs`arUWH|P64_z!g4fh(yih!CgMzaFd z1c65ny(gP{FMnmaRovBIJ*|QuFI=oM6FTxuE|hi&E>`SmLAG2)xHyR7M-rO{(-$zw z`FC`U|IZinNelU<=|*BX^yQjbyqEyyPASvwGop2JOUX}k8v@A)-M4n;*fo?6oTSSj ziS)1fYz;i`cm8ZiZbf3SM;IOYLrhN8$n8vYd3)Bwoj?D2q#@7s;YVjAmMV`zHc6+` zLpn_Yh_p_m@lR%nWC=cA+Nx+%Fs)L)7#39Dbesz1bk z|A7kiRVj5jYF9+1_&=>GF+R-I)yptPJ_hr+mNcC8>zxm-yt6|+dQ6St4){9Gj9b#! zzYURgfoam8^Oo0Xa}%^fp4|@-)L9YuO_m07Y*{gOeWSyPHE5Ih0x%mc){qN%5S+o2 z#8VBpvFl4?z1>vM?0R5)(9kOJ2<1MUG0o?h+jN-I^=FeNi@v3tqD=s=)|8;;&~xiu zIOHgY9A!YkU??V)sr4V9{jFH9yBH_X>B{()yjvlt)^9yxU*sI|GG%x>oFhF{W0edI z1qR6JNX=#bqUak#fzE_jK7q0-~HU2KbeB<7L zhKPw>8F$zvZ?x)LN%`=PG7|&qe{9kM9}G|-)E3a!kKJ9(_bNbf+*rQj=0g9pl&>O` zXR{}gL4hZH4FvL)g`^2*>i_w8|JYeAPq5=4;qNd`GpMezbnjlQTGtk zU6EOx`L&s~k<~J`K{>i!Cb1j1n$PLI_Qy%Wz6Ym66%{ukIK!7ZO~avaDHv<=dc;q2 zLv5@3#J+H6I))C>9cljAqwl0?fl>k|6=7l?M*N%$*=GG`ERvyask4zY7a#BDH#*#< zEy@r*)<%~|zza#(Vj3nM#`CxnsQC1_54pmjEbG!N)9?E!PtwEzaP%i z6b!q&O9U)P66#w1|2K* zhwqvT-59qbnq=m5HrWJx?5eF_qn0^y52C zggGRz0sSMV=~Ek}#^3UHVm_Ma;jp=jO;UY&lRr#`y|KcI%RN}qn*_v$IxD>~ZUu~e z`5OdTB;ZG2BW>iXTmd&>8$nFVn3R-{M}5=UbxTe?p_>O1iKf&6dW-AXmYh#1#>{JF zv4qu)Z*}uL265eGFTc56$Q;iXwXO&yI;FCFN3%qfu3R27aZzfIHfOh@k&cKxQF z<pp=S>HL0xYW0*`n(R(Nd{FocP{5E&pZOQ(sSxqbghsAeM zRY)28DTGiT@ZZH_lVm4oYmqv~B#Ej{A3nU%=w$Y8;$gGQxkHcE$2q5-_JMPRoeBxY zlG6?St$@V3%gk6k@x6bL6@JoUuD;T-%MjtsE@fT~nG)*m;ST$>8X&XzB&oCXXN>{L{At)17@fE_q%I}OYntj>HH=Y8g^(d5x{dx0W^?l_ zH8PBy;Hph@V($I*7KhUb4|S9hH5mf}2#2Ag+QcJb7&}}KEEIG3w$s0+UML+m5+%St zHlTNa{XPAe4<9Yf*<}gFgEI<~2bj*&V-XQ@U{-;~ zLQ6}t!%84V6|(#vS>FLo_5c5md_+;AvR5{l*_j_@U87LMH8166?|Cm8Bs1&EH7YL2 zmMx>~J+67JxYjkZxz_(pD}U6CeKCVTTbmwdiP()1n{#?YQzcrEEWVsj@ZBwU>q3E(#Z_ib)d52T(f4F zdW*8(|CMM&j5e+MFyvrwh*C3{o@|l-S&M+fr%0Vq&QtfHs6YgCvxVREvS7EbGj&s% zUd7q^!!wrQ@#0US$Exoj%!!t`>dO1XOP8bNA556E&4!iCI)X;U*P^mfxpMYU6Z_=b zuugjx7v^NZ(3<-Yh4Gs}+P8q72T7-ao;n2-OwSBbosEw*_{6Zyv)jWs#B(y-$kuGI zV`bP(@2Rz9+?6>vw|M#BdoK^=N&KqTHoEVxT0kA~jleW%uIhg8h0q)?s^!z@FOZXJ z!B(uLTo9s2N7D96IBo$iE!_1eI4zvG*#f6~dfeZI5(wShdRZ$D<9wfO29;^T(jHpu zHHT82xx9qvfFTmSKKGGewE);q-q8eSrci}FSSNU_reybsviZ5mL5{0ng{?QJD|J;b z!vd4iL_xtn*)Ll9$i1niQ9pQ|e$HoSf3S8egT29d1(u-ezcH+{3ZX?0-QdWjS|xPx1-oldY> z$Q#!ANadzfk+2g=3BOTwnE_y0dk^UkyFLW5m$RZ*Xr3vp>H6}Z&Oy9vb6Lw*7%+qQ zG;InLp!`AabMZ+pj25Xl3Rf|vUomvSM~?y?L9Z;*H6|sM76`MWD=W|gDdpVl6E~J1 z34=3{KKHqsu#tj`t&#c%79H>>hq|m5(z9%1Gu^1WCUMsP z`)~)WJ@)SlBOjJu`uP=}v$L{)js{@F(YGkWobCQ~x0i|QcmF6OJg49zotPxiBuqYa zSjc<*$OIXqH7hf5y4&%00uY1<6OTtn|EAe%f_0I>#VtjpR-gUBO z3fPC<9OI_uVY^q<%nhm7Ln$6EcdHoaUdOvG6sBq-XKl{;CNtvsZs@||Ok;rA!J|^t z=no3H>=`Kd00Y)_-_-}o{EgG6@X%0Y(csZSCbTeYL1)`iDxPLD{a9pMqg`XP=c=H9 zqhNK0$L+K4cxFaNh!buxiO}t~r&y=*l%0LEOUVSGCK5&(lExn_SmAW%`AnzEy`2`q zK={k~66{J{)C^Wh+zh1glqG=De;pO6a2{^ypB4cH23n-`kKlDP+~WB{hGYmZU}^jx;K1jP*trFlQVeO z@myh%$3H9vZm=)nC|KL~z?9EZOQ#)FUlm&mGQ|{b9r9+2$$B23ccji{5xQP6mz6DA zp9Jrj0F`FIhScd3dzHq?P8u$>^4tr0Mth!0h+XMr=F_G4wwK7`CCMS7E;1xUlrpU8 z+1^Vbo}Gq|Ge3xih2*WdwIOB9St)a zoU?UaW^lTLt;7FgyQljOKW_c1C|*AQeexUkt^D^Z4%kZ4*a!nxTwKphsBy?8ca#rQ zsuVac4%Hc@;*HHa(~f<5Uz#}6r!DDT$Cw(w6)^)E4}3z4RGDs#)HLpBH=KAQKhp^6 zv#n1~cI)H=RfF4}os$pX>OLXb2DhvFveG)^X4Xz~=3}blQV*HjWCFjVf9gC`w8hWv z;IJ?u^9xb6rA02Jtmt)M3a(Q@c9OC-p#6xZpZ;3Wvu|F1lm23S3j42n#7k%@T5!KR zP{F|^z0&sbQ0N!_ru#5lH_4!_LYhPEUlGY!Xp(~r2*dVY9Gnp)iM5(^(u$BzKOyb( zNs`k#vMYeABvfmiwvhph`GlY|%7@b$2!G@MeP_i|4SF)PC3xLr)^m39Uwj_9N2h|g za??<>NGT|jzdD`8#M%vo=oisHfABI*nnAdDs_S=je86?@>5HcNxDYDQCp;~Wuj4?e znk&fsgu2YgKy$I{W1dKpRNxS}kO{||FE=BOql{Txy}t&0Wl^Ylu|@A#F_dBa6C?lE z2@lo|5a(}lW^~EHT9flsl$OY@45Sp`U+Vi?lk(QreT4je?wfBETjgCtx~LB3B;M9t zC7w&FwQM&(SI67+sr_dUQOb}AAK8^BBzRX@Nuu1+FzC4^)m#$KO&rJTyU0NE!>6q9 zF2Z@`bI;{o2e-E#H3hWv`70XG#s4{_r*Bw^I;11_AlYEH!e0%}z$3~92oKM> zulj8^PT>q@5WOHmSt%E7!MOo{(3utf=?(y(+zsJ& z8iMVjPqw<+m2T)uu`^nmlo?;5znIAI@f$Qpm1n>mvU1p9&SzXmDK{d_f+aSy)nbba zTbGnttn4W}1fn1wF|6+)+1VIX{Lz!oy6aCyJlsFz-@y)=TLRha_DB@Sn=cRP9N_u= zmDOc{v+yT|{`scvA!!S1gnv{_C`lmgKSk%u2j$LTa;qb|)7 zcfeY3UMlgNGOnSjlP6CgT)qUz0$1A7avklV{i8tsG&nF<`b8P-VK213D695~Ma%x* zd|oZ@(e#asZQjT}vP>BfE&P3X@uJA`)D_b#!5ELZp6lq zXCSiE#PYJVA|wNqISR7hd{2~YY?7G_g8{}$q51s=9ny-DXTnZudiODs=TTx^IdM66 z2`*fg81lI3D&67wXVnVt-tt(J#Q*cm0c58al_-)$!PN5wtS0IHcO&1d#J%&?eNr}{Cs`?LHa|!_14!n-M*us%F^OD$f;>f^Xo*S%yyr?a8v{NOfK%Yt#N+Z z#XiF4)9t1=suq2-29KcSyv)@Z8*$umh06&n)nei0#jyLqTMK=~hVaA;zo)g|vc@=J zh%dzumi?+lHSdlq@xAm&C23U_;nJ^lLCSEtMz*1zZt!H&tF$${A{2vtY$uEjq9|pH zrSM0~-BLkzQN$06 zSD|@`>EE%TymoXI8FVq*2xWx-r?@4%K_?TrWjS&`Q=pr&I)NLHTnv%f05!>0NcC0a zMRI1Kxqyk@dh~lE!jR8owGy52!yXCkD;W8>n3!0Bb6kDWC!(aWZ{8i>mNaE3%rU`~ z0nV);^3cTn&h;HfBmpEyVp3!e!YEDLtRic+k2Pnnf!>rNzP|c*QuzC1FFz-+bv)Wk zUPMnk7P|mGsJXV%eMEih;EgCrm~Wh=#orN9fdf&8ePjYT9)+&dNjE+to4Zm+O%08$ z7#bf*V^!p0IB_wDSskdjrmUPKxE}qw-POC#uiNj?CDwcX0V+bPL%mjIzqhqi`8y@~v{-jkdN(5w(knRAzE{f!E%Lqc z>D2*TF#uDm-D--qq?DY{7ed-&oI-;F8<;+!_w5$_4Se+En{_~nO>pqMU7~C1XHseS z;a>5zZmWKOr7S`=LKIy!`*PF1){kA$@p4<-0bDjFF-~z>w!YV zj-!;BPL1o|FEmw83iwd@cG(pUz|xx^YM?CyL40(vO9T$ z_YC**b7jmQ5g$_5wac(Lg#WtccCP~@zHyAm_L1mf?$ezw3gk+wTQH!5*1%q(DFk00 zwCEey>tK=J3V>(BcY|9Kgz#Ca76VWykshTvHTdLy#QNme?#fkIM_g-lYday`(G__P z1DTrGMJB4-y92}?Y(3&lrNDFbW|;JiZj_?j2dWyrR24(pjy>YEd{`u{6`;+Avz~G2 z%bH(OIbF~*~#1b;xpYF(HGjT}vg+o;& z@E0$#zk6b@_qnnUxgkA4d?{RU=*tA|By&4@_+_`CH?%)`6G+5NmMbw6K1xz&11m4R zQXCy361iZtrrQ)~OlLf@JH;X|{QST zdI3gFLCn*B6TMTqWvIe3o}hEg>5Ne|F3;STc! z=6RbK;2&0$n{oBrd1q^1J+sAGMlk`TmEv2~+2LBdVc5%|1e+u~g6JdA&eE{*8zoj3 z(8--b7EpYrK?-1lYQZYkupEg7y6a?aE8+suuvUAl(a66C3(8`o4c#>t*X@sN zLB>SLVG>N@Xw1x`_x`%^U<{n&sNlb?2rQvA@3KBuSCS-9_Gh5GIX96gr5)d$AZ@_G zy0ntmXdUr}KVcW#WvQH*zxZTb!EJW;VsiGvw|radHs6Z3A6}PfUTZ$Dl|?dTUz!F# zn4Y}{e!C?`lV9$w<~GzO^&v!wJa${?|Gw!z-@9H}Fw*ltbq_cU-4-I;;VCiT zK4Cjjek?}XuORT5yY;BAlbl`ZZy)EfepTbYEIE{JJct@w1M(UKtp@n=*XaEBW@7@!N_mx{-AL5 zQg#QR*X#$kd`{Krvro~rnXrwj5+d$Dh5O|=0tOAOh<}QJBA!=$j{gs+d!c$zUE+Lz!U>{Lc~ir@ei}}`j2T_-c>+I-0f&& zUSaRKLPD|n$?@8W>Cr;uvCVe!E$kya{0F zr@mxmZ}_Lt@l_9l0~wZ>+S6f~1GZ)d7$2&=;b!?#A2f18e;g1W&pIVG4OKe&j*fmp zPHbfiC;;jv2D<>3eR1PdTK=JAMXT0gYdSfB>E1V*hnIbNo-c-Q3sIhqiZ$4p1qKIX zFJ*L8jjuRJ5X__x|LU>THIepHf8wuUZXwZIOrR+cX(VAhZ^7Y=vl8yCY7#yINlq;U z*V0OLkIc|xeXt4*`)f`!*Ym3cEA93~9p^#*IfT>xLAIOa*@?Nu}qPwyX=kDv;%VZ%2{hud^<5gJ8{Eev$ zJ}8^zBA)a#aCyRJXy>*vcwE)SVH;xG_3LM!b*~#1-`{a6sy?91yb*;*?M)u9Og`KN zl!BKKEtg>n%GtpD7kCNb#3{)MlMsn;KOO6lJ-HV?`B8rF>*a^FKQ7kZllx8xjBEzZ z|HpVg`$(aUqmJ%FP5a?pKvFwxqvH6kf2(xuxu{EnoF-!rP{{1=> z{QvbV>paNJf3RcAiV>o+|F-c^wq^_rUbh0QNcXXrg}oQ^w>bv}rB!M2(^(AJDJQzeuxq zw=0`>Sb|Kv5E4Qi}feQ-!aut z8D};4>ja_W>hJ#ueXQNsZLd9ScgK{$m&yjC6;5ZfE+WD-IiTmM@6_n-Z6IpD;DH4< zMfiS3j^{{7%y2*dt6?7yp21bmK}V!guXt!jX$Os|3ks|nKM$r}UxFWq3XkRvVCNsN zGEp46m|+=f2WR8U^OpijsFpKj<3S?=|2f9rnYnPF`<0%dwZ8m+ifXT3o^a3W6&7B7 zJ9j7&AwO-&2wxd0n!gfGOy)YX(f`KmAST`*`uZlhfImXC?@M9l&L38S7zw@;d5BT2De zS&o&*-$CGc`3DwznFq6OPb1z&*U2{}B{{Rf+8WpBB{}&Ke}-oMb&B<15*YD}y3e}iHxfxDnf?bg59+UU;L8X6;k<&G|N9?+$<-4n5lIUp(YcEORGof5pB&jjtvfn(-P|;7 zpB(Ajfhbm|k!<#PSjOXycLP{}se@b+Fjsf0sc1u=avu88WBNwWZ2=nRccyp^80PgAbp(Zhe3F=bZg2>eN+|Hi)MmW~Hk_ zlfG3H_m#sCSC-Pk^=|KsN)0>qGpFz5U$U6IFoChvdh}T}>ehwWKvvve0j)4+ppW3U z7uKsG=k2}LL(RZPSU?GY_xYF9y^z$8@bBm0?~Z-UXkIic9Ip=?0b_L*PbbnGMaQO6 z)l5-18Xk2{9%>!Md;_R%Pd))r#-wyEEu3gi(6a*Mej1~x_#KWp##90J+O&G7BtLhw zVZPAO6*=1f>C3Acbn>cxH6Z_Hp6%2M*KU|AY|d|C39g2B`xvoVFC9wa z50ZTn>;o^wI5vIbZLX(c|4PNQ`cmI9d%of5SDWUy4M+$I7*sGiH~XzWXKTSA$!fB= z25-N8Z;)**wdX`SX@>>vudThyy#3EtuS@ebYx0Zfk&FI$@`LM`%m$&LJ?UQ{g|kqP zoakieT1@oGXIG_@gE!T$_qOa$CbMz3Dk|2)9l@}vi)K>WpKCN3^Zk#OhMZe23YNc1 zVNCX2ttc_`mOsMj3{j(&OxAq1X)J`hu$ciOi}j>YG#~xmt~G>E51a||?s;4$CVVHjk~fv6vv^hq z;4Cgx_6bp@eEM5?UY7rCwkSxNpAtUB0bRvQZr8slR6RKPAs$F~Y$mqtEk)D5&D=N=Wd7 zoeZ6z+hrO=71gRc-(HnvFS=Ll@I68-u_T0G|I1hmxdTtOC;e;i+caM?JhA(AOr>@{ zIyQS4tP6b(L%#y)~(0bb{Kf+mHxTYDn?Z?b*y2!he2;|t=y zvZ(mKICWD&KwIXmi>5jSNrf3&D*S!LUTLv~oQL3FYZF*<<(2E8gu+qZ*h7HKP|P(V zd3m;>$TFem^dJRMn^UyDS(S51EmXa}%Vi@pDjM&e#)|aGdChnTFJN)-2chqHQelXsZW^d-AI zS#oOO;#tQJ>$fD!%(9M)QkrNfBep>&{OO0lQk#jSZ&CX~7ryF?pSLw^i+*P=m;~O6tlBIjaRU z2Cr^0r76I0j-f|xslf3ZR5o7>ukZH~rnn_I7m;KdW_;PAtp8UD9oRj0i0nu4ppv{K zJ_m(rw_obboWxdjO~-m}IVZxG`fHfrJ1aps7ty52AN6{^2K@14KMxvDYd7 zw-!JUOLK1IW|->eH>6hMmTjP2LSwjCAuYKp8{H$do_w1h z^PSZ%lNcrIfplGnIOvNSAJkBc#0nff#pRDQRZkxLWgY?<1Gip6p@z0xxcP4tQkZwL zdH^8r$Svt~Wf^4%SLW5eDZ^W7X^u1y9Vqw4**;HRp*fTt`GhW3ae6t=Z?%n8c%uiN zK`zKSGSr|`IX*?z<3~Y+OLBlZ2m8j4xdMGdWzO%D(=z?H*5Lm+{<;ogEAk=09Gnaf z1bx%*oa59J+kv<`4axB8&3+oUyHoTiL+ciY8M&q;532U6I#SvI4>fb#5t!{LX98js z;~PM`;aqpTM7>g9Fe9AA4)@8`4A{_kMMWXqlH*rXVi7xhunV<|#zZt_Ur#r5=dPl! zXmme+t(9P&fIy}h{ZQFh4Ze_RuWWhPvt6352IaCW{wQx1)jYd%3xceeu)0c%+K~zQ z%zy}(gGa{-`p3ei(;T{3upjGxR=UkcfR1yd*DGJF#|U>@?w?BmJn@aX$5&Llgzq6; z)-z>;>=dKULlokI(>q{Niw~;{<>zJ=t7eiI9^1{ee!rYnD?LgjJ@T`9a8EnnNmp!l znGLwbHEpQ{m+5?Fhc(RwiNcWU@0fryW)EE`LGgF0kce}g0NZFtNjT>!)w)r0vnnS9_+{8 z6hEIHzM=Xyw7c<5rp_0)CC%gC(Vzg(f@k)Il)Q|J0T%Gj>$>tgC`6k<8xsqvrEO_? z?Pj%A)xook@8WD z#0e(fC2~sxKDLN>y1l4P(5+0?P(fxb_VR%|t`GU|rxY%_QJT>k z{y;e&2|%LFVP9(uyF|>)b?*B{Wo`LQYdoFqFFm<1Z^N3fIp0O-9Lo|325YWuNX~M@ zKZy%j|K=~&B@idHA6yK8P`UrlfKgv=)_SQ29dispZU{5DxBJFSt5F+4loo>oY;f8p z8y;_H|Cp+q>n&bD9h{X$4);iSuGJ}iNbC%q%=y{Y2HLwmO3FGIQ~GTjZ?`;??5$aZ zCbm!5r=ncmz95e*=VidEQYR8p&YQ=ZBA>aBbh}o2UE3iOuwHw_Ti^_H z+o(57_DA~`Rqz0+wN8oB{AQ0lIIlj(B$1rB%t0-_YxjFh6UqXomVwKub$fTtpY8%x zLva5;=pV|K|u92+0quSc9#ewLgBt$e{Ej#XI9!Nxw=`7!g^1t8mnD z@8e5hjI+ageO&4365rHf-lx{Ca$Ejt&II346pC56)!kFb*loBoI9+HvW@zs!y~OdZ zA}4PIYzW=D2X8f*)*q5-tr!4qC@1ms<^|ZxszWW@j6tGR*b+AX;>Hv`fd*Nhuc7T{4<*4vR~_J(v_Rk?esT zffwl$4D1itS2&P@GOExl)=qxg9YZPuunqI#<6N*`=SCBcZzD=KW|hg=aA`F^JeP+H zU)NY(Rbzzympfo76D;%I(L9Q-qP?mcv=NEM+9N*DAh&ekosy7pu{<4VJuqlg-gq_n z6wV$-PV4JT-#m+8Ol014kBdWdM*d9d>ST#&5hVtQ(trx?w=Df*Ep6YgvYFDPAQw~8rHT>#AndOyxjBy5OOScJ`Em&=m9OkC^`A?j0TSiY zK{T6#yva3*G)v~s!=HP6)B^cRw}VZaU>LQqP2X~0ctdYtAtywUV?X3S0#!mPLHBCb zoZOcM%M{AX)ub`~7Yp?fKl-&{j?w)r=)QvT5?s1BsgONf6?y7x+DL73{=FSxZ$uq_ zg{D2;Om~~|7rJm*Fi@)2YL&J=tJcwnT0F(sPb+FeYB(oSs!+A1Du5=HUfhi9f-q(7b&M1ss7M@uHNG&qU92st)< zLa)OF8@<*Jl;5rkE>~wfGGM%a6 zg%9J-F`<>y-RD2|+Wbbua0y=y-0O?JzF?g)eJ#@|G36q^;1K(qt={c9ne|sqclMk$ zBB8I9r!9?`rWCjeM2K)lio0$snkz%F4E)szT1(HLm~?X$BIo1RwXKRmO=Y-}PUk1f zb5;*N`wtbHk>0vQN8HvDUQn(xSU_=|{FX?V??GXJKAQM0ZcjDCWu;YV!qgRaH-&AM$@0Me*Klm3!Oj{?K4dWI!jn1)eDAaUAYfXl zwO_fx$I|!P^u^y%bd}mEpHXdc+A2epti%Xpb=%s9hIRDGfwonB#8sjEcJu#v$hxb0q?53?Ie&mctG80iA7sA=$V z!9k5(WFUw)wg*?8x2ys1C23AJ1m;%Nb_3S2Q3`&z`mfNG@uzdQ2iV@WGtf!m!codc zeA!ufAD`E1MY>gdq&djj5)&MX5KNAi?MLu)m+HtoHIC72zh47>R3Wtm@zXnS-4-ip zJ*^=Fugxpwx0F@jYUC3d5>j)v>+q+K(}MEdQ)dw7LrSWHA)37Ctpi|&&2oS(igZHt%iT8PCW zS5gzdHV(=>y;4bb$6+G|bj*XC?uBaDhUVV_^eBF2J2_DSbH&bZhNDEu*{wb?Tn^=P zWD*-|eO+st;LE_@P z*VYR%`eP64E1U(w+h0z-_fjvUzcp5N6LT+`d*J>k;${_QEG`8Sv*Q^i+9#TMVbf0j z!&h9V^n;gzL&SGjOmkNLjz=laPaf`iwN78-<) zX(OypyEsmU#IzT!h?|;-&2}I$-|~_PAO&GF!}{2ZKF+sD;u#C#!1d)I2BF%+4Ll_w zK92sM#R?v#pOxvr70RB?3v@?|7Qk9_$ZV)qiB^z zJ@++3*P8T|=FtoBV28G~JyD`*o-UILF`*~Pl$WQ017W1&7Pqv;PuAd0E}}IhrsV@Y zr21BD@e#dU;h5#A3+c}m4H9*dr#2)aY}!bZ>bd#;zQb8#dyDg(^UYnS3020G&H;28 ziz}`&kcSp#(1ZVz0> zcW$aMa_I05{|r0jHy*Z6a3vLB^b-Q27*yj-4ZkC!e zeeq_a83B8(=YvdVoLzaE`kU%rN1S)#-4j2tzEF5y)wje@vj?AGTm_6o9ap6HROgeh z__?SldeNZu(QqBs2suuaEmLnP*DL5kBu>A!Exf2z=g0(Vg5zUfF}h{OdhaOO`TQ@^&nz+Jh6jh}JX%XZ2edFfG z2%w{~wTQnw5`Ep4dL`^umlUe&h$~G|y$iWoulM@mMvnx>3y4qkaxIssZW)F*F6;lZ zb9zbZm`QDx?=HSG+IPi4jXtrUUH5l;1PlN<%_z3IvAdUQzn#&t-CUgStNOeKEvaMU zYC`)B(ViT%XI3Y|GrKi{>fM(Tk#p(WZ z_e%&^F9i@4LB|GrRb`vntlx0bYz8N~vSn>X2VG;aMs_9GyZuTXmN;0?UFX=`?<*K2 zURo|OwkKL*V@)e`%fGdwsPb&AR+K#Kxcp({e*E_*6R**S;&|kAIy-N5@y*7xU_HZ) zwqe}KQ*ihJpl*PvPcqr`*iQ?SdCl$c3ccOK0d>?ff*wYBNS_&CP4xf~v;v_lr^TP1 z#6|9RjUC^RkNRb~B#bEI)vmuOryBprt+4zdjd5DvR5R53LrPmqE0}S1D;yy*Y*33I z@Np$>B^t_OA?!kuA8{PS30F&ygfQZCNI&ytED>Z;MBt43Yf17PF)eH~Bkeo|IC(v0 zz_0~6lQlfi+0p}%m+eCmja0#5YuKMS)lJHt&$J_MJ))?)wbL%5izxUvU?}T5@e8g) zg-o`!W83keOKI7Mkg}oKpH1B)?ST0gJGW+pC{j%mAwcLkr~<5%{mS4l>p3N>Fm%c} zSE-~q7wIe?OCvqzMo`9Vc8+p#qkEgsaiT-U^23sRzLeY!BYPmc|$uWlA(Jgfh+to;1zShv?!!F~z%p7IXgP zlJHHX)F@kdg+)M$ESe6wv8EgpC=IAn%ttww*4TaBS6pL(&k@aBRTv`Lbm*)eJ-qL^=uzQ! z8hQizQ)@t~Bz4UM@z)FPO!T3}{>bzZ(S}C189zDA0|Tq6 zUv+P1@y*K*g@VR6wTkWtEou%tKK-@@CPZPr(&ywFOHhu8*S?R5Mp=8rYVXNb$NGN^ zv+hs|_dpz`l{Jad5#LBl715cJ#=jZNlFlNh893%av4=Pa$_e=9THrG_2Qx2MmcG5! z{9M#hsw-=DO)t{@tTO4OGtDvT;*L8-r{7Ycvxg0HhF^X*OyETGtn07UqNXwrOqeAl@2Z%b(-2HCql?=Cq0BuZCgj&IZR~CbJ=AMGZPNx`0wp zrCWa&6&cd)oaPXdORGwtQ;z|WQ1w<{WtH*%Ye;Lw=(C8`P%kaCP@}4Uzd9Plowd~` zJ9C=A4RZxm07m8Vr0(um3gCnlki^iNFYp!NO$pcL!(kYrl*8u26Lcv6B>TrLUG;%j@{B zYIG@yV6(8lC$fNMb^oGr=nuce2ZcAww>As#)5+H@rcc38CUNhBBlkbg^=w=wYyPr< zCf~1WTYlYc@;&(dnH8n43@NtHZjb8#A3>vw-|Z-Nw)A z3$y@zpiZ9Abxmqe?+Ll&KH6_#c6z7x{E!6^h@%ew`oJ(aOxwlcEHumDj`Gz;byo-` zXKdk&*XRNKe8=6=eQ~q*H@i41mrI;{wCK0ZXMuYU*EmCXZ=E= z>4Q^kwV|nf2h_TsGfm+bgV+GB!BS z7W|T&h_fmW*8(6$BMQ>{T#6pz6XAj;&V@JWgfh&?`VnksA=Xb3NgZ)Qa(zx&+Jb_M zgKul%^Y4^p`JGm$xtz+0#U3jw9`jriQcp}e>_bFY8CSY-qzM8y|141n1_dB6deU;A zb?MCf_;Pi|^jF*~0K8&=cnbNh25K-?%led)_1%rghfAz=?DLI94^*mY8f1=xl47ay z&;(bt^V~N0IUp;Qxt@CT;83jzPs=soP({3DH}vJ5jt!8pUOm%v zK&&V0P~t5p`_@^FY=O93NOhm=JNHrVlCSuxy8pU@4thw@!{}SxE0K2BN7mf$FK6nG zQ&ktbEzIUwqxR8GdQ4fS^bc#mvcJng?d)|4;elpiTkOk)e!H%X91m6CqIhKAu;cD^ z9b=-Q{;{D4;5}Ek6pN_SfPYzEQ@Kch0`tAs7W;Q2o!km1QSgnl>~OI|+J81U>p_AQ z>DoZIa|fiG--fVA`=%C0hn+Dn$~}{)IffE}jOW1@iKPe71!@AdSH4BCZ zq-@fI(CgwsX)LgBwyxk&Mw63|V-3pd_rF<$-yHN?g@8fV? zPv3S2GhlWFJF_b}u?acl`tb`qI?&sv6-_61qBL59V|?~8b`d25D2Wz$5>QjH>6*vm4|zU(&PbwAOam}Si2aP{DJ{JVKg zKHQt>{wlGYy`%=kONhDqQgQbUprXVL-?f_xHL@?(PV3-c_3nY_AVmYIfjGEeN@=4@ z#e}ffEFM4`dd<%eNHcENM+{QyYpBUauDgTZ-I#&Sln&O%hKrrZu*(5r0{j z+C?pjhb*DJUiQ9Gh%O0E zXa8#SbBc51MAlbd^>YzzYpCM0Vzt9qK37}vsE=%vwd%^X0cjN%QDvU`+7e52u*^*2 zqF&`+vK2Yu0ds45rN*{&ov1_(dm9ZsKSWpT(Sn`?Sg$@fuV6?CMr!PtOY9~DYFf8= z<)6zqw)&XJ`Ot1d6;iaYD~0KBH{%@}UG8*=l}d)F)17bHq)$y%?h>*8HrvG1SfU=T zRY;B-`sbjVkIYGcq9R4(cDwj9bV+m&kXZsHEL2=%mm{J`{0OXz<-KoU^GM$F`S z^e4?Jo>@n|B2y@9|BN@USc}ugDdfC2P{1<#qzbR{ zAd<ny1@zBzU$I=z)yD>%w=?$gRE2R@$f0tM41U1smPiNU;;wI@$4b}7E zbFHi>ip%E>cl+G~_}R`dyhzwtUxjIxC#=Mzv4tsw^Yi9IS9A9=Dm>^@gK)}5LDL*@ z+5jO6C`KgPFv;sL;_W`wuLGHhbCG_07;+TlE+vQFR25}1o-Ws$PJIpb8E=~Z(&yr= zSJFJ2SeHaxDkk5&ODb%g1nBSq`W8=T`HvflGXS|!ez0!0p>>kyg!jPyd}@Bi+FfQ; z_#`MduiSA+Q7cmPlTZt76{X(1Rm429>MuUwFinY)d z+W|T)TZCR?0Ei*-5vsOp=ZM)3@skyU7h8zWFBq7|OE--=%PZCHA=%pSO`6jZnB87fWYn^G#QqM-}cD zt$F3ky6&6w@%HMphj{C!%t{fP=a$&B@Q!<#HbdWl9`Ogdg@s{JhYRVDd830Rk5p|% ztdavb#2Iyrwly>*hp?%dAtr=OEINhGlqt~EKdL3idd(Z zH<*`-D}Dk9k)UK;M4P5q7K=w%*%j!HQ1ef0_5tpx7O`6{F7tWCP{pn_vK37Et%={F%SeK5CP0<~S-EsujfgYm84Bn0AraHhEuUGH z4lps}i$yX&gS_p%k&8EgZWSX{pv@RJdo79%zu;FuYJ&e@0=X!P9|)!mHX{LP-mu;Z zcIZeAisKrI)&7!9M$TC$?%io?`dMC|1adAojEL^moKvxuHso2;`EaNoZ`463FraEo z&8)ongC6s=uD_tRdsbM6tcKYKtFeUXFJGT+-r&lz1I0MRXj7}s+GKExI$iPb&ZZyw4f#ky;XGUBry&Djk z+YXfQ(&_hh%$xxOS(8p1sPOX24@kEJ|10J2M!1JQcr zD2hALcjbm>*5qk@uvF{&tFKQDcB^+!{C%!DpN@#=D=Yd9%BkBSEg);o0LAX1De6|X z{iNmk=G2}pH2dG_o-K|_$%YTKyMMG5nRg5g-RIG%l+N1XSzjB4YEkY3Dy;o^V%k&D zZn$`jwy?_He0-2b6C;f6o{Fc8zj8YxKMPXPk1pz*YOOWpN1=Zr9w6H3A_G=kyCxGN zHuIH{gPcb1ds_efi@hLSa;iM(tE{R`BP$v>nH&sCl^Gt$QOph3OAG-WYpN#Kn|lkl6vf z+2;G?1Z9zJa(}wmwErq11!+g_C@V}?*U|p#=Eckf2juaOr#KyMXI`Ji+|%H!*1=u{ zHmh%{rl&gajzSP1O`c}jhf=1^bkeNPKh&tzJt#O{T#9dwWXZpM=^{BHQ18nhgj{#w z^|LvDqSojP&;$DCBM%bJJy%R0_aJjRb{E*NgpfDWzUy>5j$N~#d3#7;xukw`4_ZWgk~3zj&SopFDS!+jP0?-CC0j;Ll4EoK!CstKWTf2H8|A6(ND6%ohx7DyvIypnL$a3xm9ye)D=#Cp07b z<$ zvyRc6GBa@#hW2jKTm^TD#`smoZt9H<8xzmT|EAQiQJYV**4OlZU+XDvprFPx*J~XE zIv$6y%|RDc7r0R@Buvq@*d_PkbITDt@vdkF2kbigIng21En|Q9)@?X^8=ZA*2*R zX({QJmKZ{M0Fh3qL278EyM~k)y1S*jyT51DbKdh?-&(U+{-HSY%za<6uWRqkD}XU| zhXTSv67eOGIXsdTwccfmkbyK$es6e)V?9eLW52*!)MOd)H=a>ELzgKZc4)Nemx~7* zm701bQ$tquR=)%~os+~5f3H7gh?bw1+py>u%otU2*GN>A>#DaBKHv@tQxL6BFSVK# z(475@ZH>Qk!5s^ z>kO-xZ_D@BXmTZ8*TKrK1^!lsq%$hp4o8!o?}0SkJ(wBvNf1!S6iK_|DhC8nQgqDm zWutZbIR-yGz<|2h0uI9xK7HW7s!3rLngZ319taBV|wgW`qzHVbFf$3gn5 zFYv~+L9C!|>pf%kUjB_-&2J&(-_SOXl!Zwz_0mKEm>gyNP}w`L`cgJa1$0|fg!J~a zGPV4cdRp?7eZ?=ATKfQ|=gk;Vln6vKO|$!}MHxZqWg~JTpc3RN&Vqjsy2Fv2AULbO z)ImCvgE_>s86PL&UkOCylO*E@y^l$EO|=t!T7UE!Qm~;fTgTp6#yhcrT*RJGZmU6h z6)}Rm`AKfYAw`k0)es!VAQIRL0IoFnpg;{a z#W8|ts)|0YK?=2<$)zS#Qc^dvl`53vmekn-(c<_xSEc9(+^Y@m>)ztMN&&XBt1hfa zSwAxa5W6JKDznQdVCl2Y5>yL_`-T^(1#S&+DhlqmdI63n(8Rl6ZTU;RLAS#bL!>yd z3V!2dBGG6kJ*o%H%V>MQ5|wyS_5t+5&KK52!3M*3RD!imTd_lj!LUE`SauLPY2mwl zifHOA_FiBYYgOt=k9my2THmFVUUxn1#RzbRNdiSEZstiCg$+`c2Mvau+LlU#Z?By` z(L%9a05<@~8qHQGveYQDIJ&rwDeeRt2i8<(cg4w-02LQH2h+FhvH5^xOu}(+vZ4Z# zjYV>Q-_utOSo4lZ@R1^euLsHdbWoJAMDh7hRBIVTkAC6&}mj6ezbh^JH#=pKAUL{9o++=iu0 zw^!nIAX+wE;HXG3m)1mP`(U|UP`|%3om{!Km6+xEeC!z~tW@&CnBC!Zv(WG;f8m$ZQ;-24^5 z#UZ5x^kgkK(&uFO-lze_1#QxWbaCEx+3Mni8rj<$DSKEN%=|=i%e>krz8s1%(e+JF zCo=79ysFE#D{y^EPUR9W(5w*ceImXk`S0y&|R5gRvD+h?}IXXjcg&J77J>Y zuI2%85#KgYerldYI41EQyB7SAEF8HUcAMu~Gv(Oqu)cV&73Xc;;AaU{D^!yyJ2OFI z9(y50?>R%f`{fW?!WfVsV8fCB&9z+@085trZ?Q`E4AbZgL-$@MFdO*rmipyq7MK9< zc1j>yc0NFsy4Q09*n3tXdhGLG@Ct+Wc~Oe-iiYe}H@m*Vn$zba6+(|-2l4LLXTvg9 z&V%dX0GZXa9bj7;=b(rx2!L$rpdL#TEg^Cj+-_Jtkf;6voz;3!Xc1hE3<@;lJ~LGW zrv0v_KzhvJmpVwT$Qksx*Nb_N^pr^&F>Qlmji2rV{kl+AHZ@Xh2plid71s$S8b_rK@%&yriCfbOf!O%X>8ez96)pU<mw3SbbF zIVh=zaTt4JIC!{`MWKbqZfc z@q&hFsL$HgK{xv#1}D$xk(ELqI_%@DdiU;YAo3m;7v1L}UX8H4xMGWR(JFgxXo15FJ|S=3PO9{O_JL~9>O68nNz0uI-Mz5|$X?!Ebs zKT>(yGi)fVQ1bhyt*RuguaK`B!_y|C9{4SMMfzkwzBv9Dx$*>bTjCxc9LIv*?(VNH zP~SMVjG|dNb!v2j0~a?9ce{A$+==G_3FwwyY^NACm5 z0n}}3iOnHvo)}Z@&3udq=Y!A?&}!|9;7*q1=aRT#o%z%r_fubY-t@-6@|}yiog5u~ z`4tp-d;M`mWlPp)xAQ@G{(;WT`Dnqm>&nZ~xNee*{t>{+)or-K{i0&@{4nsA@7yfF zY^_&vsNK-YVGurRKf;J%x4+t`zy3oQMl4Y7sBY9<9~BgA19aIT=(X#v7j-Vca=y-E zgB3ef9ShDL2oBF)P0uO|Up5K5O7{NNgJK8dtNQM8Bx*rQ-jctXIW!d_fGp6jP&3Q2 znlQ_mzH{rB6X$v7HVj&aSn};ulECkdO8!|%r@5~yWBWU3^CpUNVlCcgy%FK6F!;3i zi@Mr6DEa*L6NQbEs>$&RDAGCjB;J>UvNzHZzz^Qd)1@muq$+U(N~94Wh4X6)1s3|U zdLRcHkq!Mw{XTQOLM_mmCl_gP!-^E zzs*MJE)?(>op#LL)!zN`o27Q;-3*8XR1I5L+QzNRIv^B}fPX!S<&nua`Mbtgc;H7a zIC1$(e@rBn$K9tSWHZIx(SM0=+%epfnp}1-iD<&>LAR<8mvnh44mdr&c+&+k<4V(xZ$r$-Xh?Rk(dY8$Ot$%=m! z1AVvB>?ZgLL-V5Rtkv3iu3a!h=cM{FTx-ps)Nqs2;4y+d-<_#3FO0hC zPpMZZlAQo$yBE@jD|heK|77Y>6ATdga^+iT8(68|0)vZ_o;Sw~gV3J;IkV$vRgZl( zaFw0xUkU>bPai<;XPq0M+xr;)1&e}9OmbQEoUdjco&Yr7dgn&M#P`#HghD}mN^9={ zibVbKNUHu&>Ztd(UDa2jQ3B%do&e4CaZqwSg3EmnAq0>sW48)Os<#FLv zkh~GN8YB$3gQ>yq^T1LqNiheoCgPw^-y1Y*T1-jSLUI@Rczp&Bq-|-v>Eh-ktfXiF zdCzi54|BMIN~>Tul|p2HK`Fpt;&2-+m;MGXxA}!6D`CtrzceL_i=F45ahm#)KP?1s zmD4YXH8X)I&t2UMuF5xj*f-Xz+A&r@gHH&^dTTssI3;cs4Gs*48o>c6+ntW!yFvKP z<}7=v5Psp-zko!;2#%&*zu2Q=IRk#E|D(*4$8Mb?FK^RF$ZU}52cK#pnh*;C|s^R6O-1y|_itXBo# zo`W_M0JG<#yDvKCR{;n#G%YQ_o5Hxy8+072_LCwS8rDw=lWbo(+oj4wjuja*to!Ghyi) z^8j{i|D)EBP2JSF8rD%BGTld+uUha#(_fCmuhc9UJ_ChQku%E9(w&xV_&ak=B^(W_`R3}T zd=5<*N|Iz_ChAKCrL^R7Zx2VCypLOXG?VkMq$rK{^Z-^!%&l~64R{$g-? z@SsM}{n&W8cH6!n4etaMyw@VC82gixe5052d@0L=iSN4M5unN76OZCsT6@_QQ!(K0 zFX$&;Os#xy#f{l_!l8?Rb+7n_3SXbD13}*7v48f+th!K0GygEFG4pqE^)j zBdC3XadcVk(B+m%85_EVwnP;HQ^j)A@OqKUART&M6(Tn^xVY?TCwKmaD;1B0e4VN+ zUg&mA#v*-QwBt_80HCohPAjC`GT!TTTaHz0VhycXHO86gMEn2B=$AMk>Cc5s)k@a3Ex z?hG(ygwOLHZ68Y}B$t>X30(fK?ZM4*0wYMw4+&@sPop`C zKMKlfmt{U|4O$%;dqra4`=5h!DB&SD37sc||3GOnG2E_Dsa;ndzP{MeK^1he2&v43 z&*}Y!U=??Ug91O)fGUmqy)>U z=P)3?7GLKnqg^Utpe}orDlk7J9*K39#^Lsc{0PuVks@X@RD6sZ5`Wk#;T!+B&AU^I zmkmsW#uR$3D14Os#BtuA;e$qnWw2KHZ&!1yw@+pdna&P49IX?qy&eOO61`Z_kryeA zaGsLK$PEnXBwLK2K~ahvC;A5V5EB-EPJ-SANv9}=ju`)s@bC&Kaj{F`Os$-H*(z;RDrS@93y`02d z&kYil=dJOrP9-RA$m! z4-;2$3GILRU>O`0l)7QYbz&AD#u7}q(b0B=f_8qtNWkck>!^m2;dTQ`!s>T7pVryz z?}i#Rg@+8G6jpG7FNqiBem(fgyhiSHnWvsl89hw#nsEalGQ8VpqXuMB)-Znp3N6C2 z&I-NGBPH3Hf?K9*nDwEqi!^J8KUV}+_#`;2XWdR@cBzgFed)a4bj&LW9~SWXhxJ}2 z+6_pol{C2ysQN_U&N?n-y3_+Yd2|WPwK*AAxwEK>H_dGl)hn&NXj=_SZfjE~PJeS) z1?a$?q;bWoGkWdQ?DX3^`X~BPN-R z7`e+H^+&4J?O|U=Wg6INK7#n__du{xR)UCnHp^)1fzO8PL>Tsxzc^PMMkp?6BCkmU%8`T^dY=q10FpWP__Dg+ zMd+TK`PLNmLW;&3kxwayr}C4AvhJzEA9F%WP(y-z{~*&v?NWS$HaVS1iow+qBtu#* z^7)yRslMiRhyev;qC})ahU`qjpyV905^7&>Lup%Yur$Kkcg$UP{&BiV`Ds`vjP7>5 z^4do3Ro`Xi-ZSc#T!~%k`GfzV^M_$ub5g@+uhv!S)f1rSYinF0_e5O|Q>nG~=mrIk z$7m>%GQcG~p;U@g`KOvTPslRQARW{`5yzh=Eh^GVPvAFa@L^PoX!k71uIjXg%{R9Q zYoUQ_L=y#$WW(w&mmcYOiNS)nnSBUvOE6RB5IHWvQ3^8c{`xxCxvis`=@s5zp1NSx zOo|)d6`k_DLsiMzeXWw{(}uV&s<9}GE_Hd!>n9@`c8+Ngj3akp@mTVyv+ zYE`A3Udwr9Th(eV8n1uK0)VS8(>07W84)bpvrPMR=wj=t-MG-3i4S5rdAdZ^@!I$@ za{3kkcyc_@mDL=nX{(IW7+caKgU9)-81Axn-a5{ZiBlLq!}07~7v4%^Ri*m1Mon60=(NR|`lQBY<~*Sn)8~!H%Cs2%Nha81V=DbrVJyXhRPkD@ zl^e+h6;=b+rN%j4Q57zqDK8`-e=_P3UqwtA%-3{SES(Hnby4^a+qYOzUFCHBbbH}x zib+Dg5W~c?Kq0=9nh{kZxd++Qt?7&T)~UQ+0(eFKJEFA5R3)Ht*(@5yVTgZEY18&} zSoY;sq)?#Cs zowL6Z*KU^j7Mj+`<-dHkr6Y%-us*o#bXdUf;A8?d2=jR>0!o7o7oS>p zFMCApnGisV`wPmSTZRXnMarH@J)|nV)!DvQQ|h02ixV~u6kwV%A}5bdI-dC$<8IgU zlP+*sGFfcvq)`HlfKK{`qj)PW&;UIXE2O{%zMjZ>7Wb7Es0x?8P1_~=3AQ7;QVgcj zin)Cx!artO$0QpG6pYCFWJpMMB<}# znygf!1sL~HqKy&^*Lz)#4d{0mvl-P0gjZh?WTT2irBYvhdQw@sH+o?O&D*snGg-AL zah-4s-A{y>j|O_4l8ju{J1oI(ZnZsIp2>7I$k@EGxe+vy+-d%Et_(5o7OM5DsVgX!sGO2H#V^- z{wfnngC6SEH&ZbJEY#G-5)rtXjnrDUno@arZ3l4QV)cQP5chw{a z-!Q~?p7SRDfC6j=G9hQiu8mG$F;iI|_BuvdQK9uDmO5~AvdQM~!T0If;|b=iVdkxo zJdMFvmA$VzfHbDe*_kwlMVe4lle4PnFx<9RuwnxBgYvNmWiGWc=!#|&ReTKZC{726R2SqF3PV zQ135vZVj&22XJ`INnz7HslC#plYGqL-TDhvbn9nUZ-2M%(wHRC-!Ey$`~0JMFPkga zzxED+8K|hyExw4g2w6a%`d9p;7L$)tX!Py%lQ?ws4LdEXi2d&b<0QW#MK;r_;@>-I#2jpF^e0X}H1SG9>;?G61B?@=qgaN)B(^iq< zLUBv?{p$N^dGZ4Hs&pVM~_m5K*WO2aW`CCpK~gB8S8COa2odP4bUNxfJ@G+}Z|7FDa&zD|u(EW(vWM3+E>CXVHo+fuY zBLCz02eda#{j);PuO8k*skV4N#$m^aW{Kl-a-3qZD8gYgYr!de5Y2qrB2)Ber>^=B zMV}qeiB(o8Os&dwI{dCp3o3~=bSy%;x2E5OUATi_hwZ=2H5bSxnP^L^syO7_=NP4+ zRSfMJYmBLp;M)L6Ar}ojAENyjWOmqvbi6-Z1&q!M4MXVyIjy0D_tEnWS!_+&9wBS{ zhD?MEZc$gHHA?A8ic)ganI_0O969Tg-BmRpm9iymI;E5pH(0I-H)rY1F)vhtH%aTZ z0rU*c50XU$Sw*&O7FHz|WEH8eb?i#&P5(@A|EwLOo{qsJS73C7$+d*roQPL7Fhm$e zNsvi@26SAxw!C+t28_s@mE_BZmcGEFXT)ar*B!Tx)F@{AL_}#za|h^}d?boM&o759 z@(B=%mcM0!Vj9#5b+_k*7Y=73xtWeh%X()3GOXpiSB58&yW+U#6ZQIuq(>=XdV^uT z$AHw1(D<&#QuO&jA^kP|+4*MN2B)VJ_Nw~ZGedboIIF$89PZ!?D_L4bwz6#1Z*C`F-b7X#(g7)>4PrAOD(j(#^;xmnxo2?rB5^P z*)8hKBf8>&yn}k_M&azvzs2>`DE}m}!I^O1@AZ?7B14i9sM(#PoCN-EYf5d^pGm#k zg^{njI!I@r2&s_E}z0>o#s^`!`x=p@6}OkFz4kZwK1Sz-*q{dq_NF0 zH|gTCOQ;z4DkiHLzJZp9 zM0;VK>a9h{n$6P$^4(FPcOlPnQz%-M%Z^W?(?S$t`yu!*RTUz!@cTv94+lOrhsY=+cp5#|zgu~YZI zr8X3Z8G~IYKHDIQd)MjrnNY<-MB z#gGAJm%nSh$3nzi?H-J`_McgjZoFPkqf`~T-QXuOVRE)mC5q^{3mt=>BSlO zOYAf|?(AyJGM%7bn~)R*O|s;WmhELcWTuS{eh7ZYKEeAmp8b+TD~N%D=DWb0oR;G| z{Y)G?Ne`!Rd{8YWbJb*w)XS}}cmG=JdOo7D4mQ_JZPZr5yTy6*PK&_*-nb^Z1~Mhd&!ypI z*+RrYoK)99%;*4s1(|5@{ye_72#X~-p$TB-y>U67n^&xS&s;x$f&V3I9UHt+8DJt0 zJ?JYdZaW}72{?Wc+a`d2cCa#zI3DrdfTn-i*7&roT^fQwP|mQT=ga5!4qRi*<#3+Z z)_yje*L*m=cMa39QAgxYtlO+GB+Xi0KkVSE0?TiAvfePhNtjp(ym>>^YG1{T)hTDe zA$?5}o!Pz6!D`763FW9W+4K*G{(lecj%VpfA(fUrNJf_%yYO2wzPH$P&a1_2?MUzo zNxht*73G~kaK}(D5CNF;XxGLL=hbx^2eDMQKRAEX8Xs(w+KXsh_78N@kK6hefEWY3 zNw0o~7mgat#TIHwPex2O1KD)h36I*vaH)awH<`DdnU)0O;C&Z?;Zh&Qal5qJSVcCj z;E;3bROC@(Xz~aUv+0xc7EMHaQzC&bQ!Z0PUU1!!cqpzOwD@cbG6MztrTd{@Xh$Qa z(u&lAL{x0?>mPvOFJpa{&K7lYD|!S8(2+98!70h_2&LsPC}^UfNEl&Sap@$m_4cQn zflOrO!J(5HM_rCfZu7L>zhCX2KU_W3-KzH#6DGh#ed$|-zv!E6;AL}`_1!jJvSV#M#OD-hma>)3qv%lLD>5K1Oq?M_(b$3&%F#WR*T6wGJ z4rUJZx`V|~N7nh~CM?yaIp+j?N1iX|$7M}YpKkElr8*2LxW zv%E^^FcZ_I*#>WDYw&BWDlOlorHZW+@l2@1Z@HD#M2|=gu+>bwwp}tNqqm6mbF-kG zIy#lnBv`yFXH1nAoJ)K_8wlv0EjZRzClGgAf|ckM;=IX&N~fzpLI@5WMs8>N5CiEW zxe3jRgUN%b$>_QB3^!n>sGC{(I-DgvS?hUD9{5-*gk&A@8`H)^Psj@uBoIpz7)g4+k; z55%$wPS;L@$YLdAuA7bX|EWJ#W6{iAnbAJWj4$y#OY$vZZ2u8FOj7rwTHY(Z4Jf7U zF(!hBkkx76pNm$t`oxtsn3d7&;W;C(mieTKi#Irxud}b{QAXDd>koC=z76?Ns4u$E{t z9WB{Bij+Owa?H0JTy z(|EY7=fIi=<88B=^^%`8ytTh86U$#>DHDE^z|GPX#9^nh7o_J{UW|rt5=6KwP?+kw z64#Rn*tFD5&2pEpotJfS9!-|_V`-IinUgTxSyq5#YlLqws2`JlLfuM+2Eoh=>*J}H z%Y9?aV!~0qr&1Ir?BXXjY|78(Ac<{*OVFhaDTR8XbIIa@&y|UD_xrf{;vYw8q@2N2 zr^4;G67|vSS~t&kA`4$W29XD9Z2VwMje|<>%LQ~EfF&krWmt$S zP)(LTrs~P3iBcvt$$^zG^0-4%-7RJ@tUL>777f~fy6~EsHp4BO@Hu|r$zJh+EMP$| z_P%ac|9Ps!x!NGRQPGE>a-s}Jrl=u_iR{vMae5E>tWv8aak;w6CXB;0t`!LEv?Of= zBd5N$R7zYc2%?&hN)E_dmc&; zBE33OsUVxuZl}HxI^`DqL@&&q4jZXvb*a`(qoQ%4C1z=I&uQo|5|P$eie7H%`Uiix zj~?t5j!W$|L{(m^6EO{AIJ=zps}Fi!GNfb3-X2={0)qrA8bkS? z1bDpMDphoL;`=;6aeOA%9lKhjOO}(Q}+XQda8AQN*#zaHF z(6wxau?czy5qH6x2<)3_8qPAd^}qU5MW4BRzTi9<>k#X%pgs>E`?;80Gm=%Hkk%<; z(H96~Q(}t*BNAbf2ri+eg@;RPa0|x3QZie5i2>PwP>9pSYO93-#)dSo;ub^NT%=$r zajt9wdic0gi7%Xk4I#Bx;?r`deIYXXPbP_$^C(kdW$J69@{yR*5WVvQ1hvj^1Bix^GX=8ZR-XNO}Xsz7UxYs)xN-!D;a!fY6boi>VT z+ehF5>8=-?Hc}7SDn2sh239Hv4mSF=MovWdF6ba@Ue*3nm!Y92u7PtmFq&b%(-+=% zG=mr9>#A1{m$WBg7xU$R`v?(P4<}2%-lv~dO%>&~H3wh<($`M&k#|gASYp(uHe1fr z@M+O7E+pUo(R=%3`elA6upcU*ww__3fW1g0`uo|MM4-h5)$G0&=fR~{FfuEtOO9%Z z3H-L1?WQhl%1YM`^_-H7@O_ZaK9PC&Rm@8#Q(ufnPa(yBZENP62<0TDBKU5N|P7ayjx0}LGz(b zimy;xcI?kF$Zna~*@b2kvhqY1X=U`txN2!?IZit5@9!GuU#WO=5tzrif8#f5*O1H+ z`m+EshLh$-E>~x8Lplvbf#Uad70o`p^nq3d|0#Ru3neif!RjLi56Iv5H*%jZCDPTH zst=5p0(u61iYJArKDmQ^&-UkKq8oWFT1e)t?BGxL#!!g>=-rXIKZY217I3$D-wsF3 z?S95Kxekj@S6ZfL>N0WvvZtfR*sEJcm2!_MaPMVANR{;75UWFE`Ywsjq!Acn{-s9# za+W=T)Vc#q#|D_FOdN`OToC4zCDPVUj^U5i5nZHxWf6ver|@T~%*ezGVlyb6`LAB5e||1&qaLdq>MN?@ z7XKk6Mk7|lA8oW3>)NrHX$}&D`lpdN?7)}l8a^b>Z;Hwhe_y%}(MO0oo!J(?zmL0X zBQ;=DMK_73sMe+o&b`ZHITgclH3VcTh28MShHpHU-&dO+c?L&*C;^tnm-^Oaey))o+POkf#7qBq}NsRv&sDR2!17 zYPuEEsYZm;PCi&rqMsa@Pp2kl(`(j6wPD|OMBK{EL}R{51}HaeA7h)~nHDskNewvq zH@8twTeQeLe|z-Q;?LWkpiA20iB`uA-1{8Y5QbB_SM!Ur{Kk9oSCO0l*4ZV_uVYmN zEpQkfJk;&?F>E5q>81mI)7{z!*L{}%as)+K(er%G2<<46?-+fAMO*&J52@xVa}@fS z)4MxBFGLuWf^S0#Dz{hYLII}}yw3sC_HrPlkqDzR1-)yRJGtUS#_tVo*5BzGmVYIF zK|Ik!GO0R!Vt)se=aA8^HjtfrUEl{98l$7mPj;QId>7L}hM^Wbs<0d3&bTq=L;_(o znHs=>h8vLAFR#@fjJcAHFeVu7 zgf+k|=!L>=pu)oBr*wXMIA+Y<6dUPnq}2;yPXwRi^}~Xmb}OEorIUEN-r=%wIt;BT zLNaE+rEO7LX?gU?5{Il5sfL1mHryZrvZ;^HL?@gEBLeIQ3;_Wn0QBh?KYk zQ!ddgmll&%MBxxGN@4oX>=*bZ33a;{7z!TTcvvEEa-X4N(Srj^Nn@)sTMC{Lp0xjS ztby~LM2V40SJcT>v)i&NW@E=J3v&PFnoVq2)5%uNQcD}M7A0#aCV>Fsfin3>-)G*E zd~m;AeHutdNiG;u8)H9)eBB?(daozWSeuEEibwA$c5v7bWTsoTvtV+a^n@Fal&aT9 z(i%oJ8jO{4r8_o`bo;5rT`iyM<}SneX_qpnh!#k0@sg^KDzHhw3eIeVP0u;42Ttjj z#Dsx#3cNuPOs>Rs;g4B+=8O+N1eP1N)l7r`{hY@*g`W#CbT2fuQgCv)(Xk$b^9yaH zkcF!ud*=AP34Z~NKb3|PMy?x2_#l6c)4{TZgd+emiOAwCdUw#K60^yN3%Q%dw*qX| zZ=u!+vW|N##F?-WSGE{JJh$SzUxB)BkAY)*sUX`b8B=>@VpFg@LLkJN zbb_%(C~fLIII8FOp34VM52SFr@@m9+5+tsQbWkaJ0F19l3AB>PsEz==%72cvZUWBk zWtC@(bo{4LQojpJj15_jJLt$;?fez<->O`uNf9@HBl#aFx$hCWcFi_0ooL@@0U3(O zVzboh8VSxG9SCjpNj*%s=cn-GXc_c9>T~#4EhcS4`<9WVSP>ZJd(1hqWuca3%V|-q z@}Gviav}k43d<2XF3;Zmo{Gi>UaZ58iOW9whlAe;Z%}1Fv6OyH z<|=y+fM2%jf)XWu5Qx2D(PO-&&qo;sYqR)%n)019)jifF`f%eoJS}h5n6h&Y5up$N znE$IHL=!n+jARB-$8ot=DcD}*=0J+=6ZACfsYdjcWxO(@c-t;#@G_hO?kFt{zW{W9 z+XX;#TIA~=H5Xnq$+r_!OemU(Zi+J11s1xpw(f-a&K2BMnNV;g-m?G8Gi{=2peW22 zHzzwU;M2BIR}E@qB|KopQ^U1-k=tE7v~6m*yw|U@(xcXFmp%IXP@L2Ll0!R@n6%Fm zM5l+c1Qvp)%n63j{${0r($hn9?REVMpkdMuKn|i`%%H6-)CF;l9kcI}%)dMVKN1A%ldnaMdj0JXL9je3Z^tx5W^iq&o_0-N3FjknNL@n&~3 z>|;a+1Q;7Mm_5{$uAdKJE`K>3x|irw10oy++pL6_Gv@YmJJQn$=iQ}?BF?&O=O`35 zGbZky=UF=G88tc2dn7w;BNt#ED{E-q&j9HKv0ZAG0(W4w55VrMBwu86N6`>Q3j=98)9EzYT(W&MnPz<$C=% za%C#Cu~&+>m+TT{uMvk9<@QmXufG%cyxS(IdUVe?ZtmePp(P^b$a9hl8^ijip&U)= z>(Sp)&mzcJ@Jpm&mri-V>r5#n*$LON>6EDcYQzY1u;b&@kn`IG!^0KJj?N~@Yrt`P zV$e`viztT#}x?f*q3fcQ^x7oBLU zuzIstO2H-|^Hwp=c;>4tvyG9w_I>G$^0KR}UqTmLR$9*V2|PEYt`oYJ&zN%9v?3~# z>#Q^66c(A8PP2LkRr#S#m}x`416Y=s&s2p8o!*ok4i(l+o+e>s7=o@U)=xwicwW2KDsuU6;Pq8 zfqm>=<4Jnf(C!SrcVAQd2_X8#lSUjE&PF=9k>fT8aY+2PXSPgYc-}mFF!k3$EDU^* zt9kK`Y}WLedm2LM`Td$dLpgBGZ`Uu~2w8gK-~r;nuFw?fgP&taXs`q0MY|aV#km!` z+(k@|@5fT@@H7iL9zA;qhpPfYfbAp`SN)W*(;~UjJD^WJ^!4J1Ab)}TO!4D&zZd`8 zwgKL7Lj%sPz^B^{Si%npq@7KkIT1Q&Axxd5_GFs$6J!p!WQaSFKoj_Xf;JGk*)1^y z2vR=zPTP-aMLdim+ILLOhuOLvh4UAkE3t14;CSwhmlYgWCXVCmzAQV_kSHQRU0h8D zh10=KgLN@~4FCDBjdb$7VdMakQ=8e-|J4H;_+toG9T&O4h5Fy_x4KvPX1oKhsSHr0 zD@rf4Lz~9|<@^z>PV=9~+_r*mq;fC&^ESjGbT>76lNBZ$@QDglrme-0`m!nYyCY{= zRo#^w!5Uld9hm<6=G`9X_F6FEdYq-e@h8IjZ{b4hEnAqh73SVaM_8u^nkp{ezQ#+= zJ-wfrUpmePQ5+xq7=f6S94xi47~#pg3Hfi*M%6Q(Papm=JnVV76VKB7jq#Q?q~%e2 ziuG{jtE!CwDUeL`3O5mYdai1PZ=dmn=IA^S1F|E|*FS2pY7%M|Sj3zv@NB~u=c?d{ zgTio~wF#4rzK&lGVXt*0z~knm>Tr}DmFLn!T_KU5ase~%n5;n+hTUeyQw9kSg(sQQ zg+u+jN%CEu=g~#CzZ!jB+#_@G*gtehmN_T#GnG(L?ZKkeOS#KDDa%H7nRz1#^3I?$ z1;alasJ{<|T^&QZqsci1g-u0O>Zys%&`v-{ zo#Y-mvPQF?KzPo8os2qtiulnrvJYC(k4gWa&yA6fX{&g_?)=489|BWtpfrVCZ0tSO&gSqxWY|f1+UYf7Md`^VhbH96-?L$V^1WUboiZ8mqiMTunCwl6@ zW=a6+DPlZom*Kru(7q|h3A_Q)MVE*Oe6w5bC@$GAoz76;O}ktGw7<4gyhKJ3 z1e;#BcmG7;T|TSt_FEmDfos+$rmBPbDM6MktPe9d!(tTFO1_33smmms00OCh->>|A z-P)0!69w^pEgm3!>rUhBhGpC-vp4P+zhMwOWDap&jArqc31ChKd3UQLbJq+K{w?c( zTn&(Vxh`Ojk7!tzns} zf4~_aMixFY)RlNr}pPO~|ZG3wT zPXjvs8ji!`YB!Y!VKxCo=PA?xLf_Kg?OYfH=_?vSC(c zru}NI@l2tHHu_qU5ZednL~4bYiW<)!qo5xV4b0MIvz(fuXu=huyjHrkh^g?w#*7wN zhv|fmsUZpzzZkxk@3l&=rAgDqO1kife%weiD;KCs5eJ7HtdM^`3gC^}CG`PKa~bpt z@u!>>mk=TN9G<$_Oc6Zd^^z08l-*qj?nq2^>G^ksuMY!#86x|uX&bSTFTo_ja4Usm z45pWWWC$oi?QQe-zlGy2+rgE=j64!W1+?j!WgU1QlC7qs{m2WR&UDPw&j0tJRb;>#YF>}=g5X4QX&q?0JpCXj9=WtsB!IhbiOO~imoo6 z>}@D=RB~ify~1JIIWc*nTw(c0YB}G%_J~pE0N0jo^f~UK!FNRxkgb?7QsJND$nL54 zU>8wVJ4@pp?cdAYFB9A^+)lYLGysIu<+RfD%fUv~z-80N_??r|P8`zE#>Efhi=mK8 zjsTjxDGmX;*f2J^9L|hk%E9)8(I}XfnL(n8p(-$e4~jtRMUqz*ar*_8kX!y&>Ghol zcTXR+ZYh@NNY$F!q`XOvcLf)?n( zSchDvR5bRT9iKj<3(_30tY>fc#~;;E*bc;ncGivfNLg}gJRUvJItnwUMq&%zkpSZ` zr^{cX#^bQJ9%pahTTyYdE~`El*e!LMIDdFfYCj|aX?p}kK;zU=;d{=slIl4&>n85 zq4WzmQjYo#wy4y~3>!v6E(7S4X@PCvs)HfyXmBDj~4NX(0)ZdPr z8B4s(V`(9X`ZoDz^@5!o9cj-r(;{~}70*xMwGLJ~40Yk9;@)E@RlmRYY4ANP@;pUD zk5O+d$k#)5otf$UY%y!^G9+h60m+`<3dZkc>X(!c{+F>De%hpO5t> zWzXVJ-U2@IIGwrO=>WNrthAl7m(F>C8wZon*)mrk=)DlSTaUw(`JZByo4%pZ;a#8( z5$E9(&#q~UQz^q{#K@_Q^(ZO~BoSiNEVk}v>4KTxV>Y8ImD=uCUG~{94)}2rUL`>D z$&%z&m=^^Y2G|v(1n47V3gE1Wg|%P}GmHa5)sb_RUh42ek!=YD+yG#2rvrw*%JeyXRdAZ zDu)KLhFwb)C;;$pcKpvDRD;kiFv!Qkc9F~aPZECfF#dc5Xj`>elu*FU@XcXlc2h{?ODWc@!OI=d?d zt#}mVMIJ1zL~1O9$gN~yB~39OHO4OiQ>n6P9q=CS$=p%7_KX9=#Q)W%Yyg|aYNSY} z&9W);WEIq+IM{{(BlV6{_2y3pSxc#VC zJFBv;0pgDE(hTXUDsJ6_=6R5^ILM-jvlBd2f{2=9WYnOj$W^G8>&%BJ(kjcVP${hN zqHDCGV;`dLI>7b{UW~Hms&39YN)&zIzPuLMO)&(!6eIA@2{~S@u&T7A*96zP7Q<{c z?^Yz?RYjC$hay~6rXKt=MCo^V>hixvRE37jx1f)IU!{N6LIb8c+*6NozgM-E+3Jz9 zjCq_o@_5N_1%xa#4cU@{8ElUBkM7#iW_X5>@n!ZODrFvmDebVR9T{YmfrRitfC-MM zz*9K(et?t)UV-Ymcc>*%eMkoi zBhiTBxtLW>%kc8Y1^*`j^~BnJ;<-3`%ktUg8=L=G6rkMyjIxe>^~;nB=CdDZqMuKp zPSvj~I(WQTv80M{Nup0)9zNW&pIRb-K<)+iUCr7}iIsfdM%+Tg=%VAg)hwOzo;T%= zzg_bq(n{HY0)+@L*akcf^>DO6H_`*zPqV(SWdhxUC&aj4;IiddIh%^UOx0gkdB)mA~>KRw}`Qo{qA*;H=WDdtH|{V0?2VBN{O z_8KUvKPIq6c|{i!9rd6c+LLyE{MY|$|9}T3Mp;L&`gBW*?U`9PwEY9rWLK=cEEfYk z|KM{E(S#M_L06MBJ+akjF|8Y^CU_{xy)eU23>RbujE+`-@k(EL5pm0H#{CrjhZS>5 z`>>YA{x$F(iP_K@52;2Uo4UeY4)-`Wk1(5vtZ7ke9gK2QC-c~q46J4LK{Lz(byT#; z={!JdjQ{AR&h|g>?tgztAfMe_6^zA4QxrMdUA|@WPHc|?_~DTvdsK%@k|I@t$n?lB zC>qV6naIiLG8lXQ5(+;V&pHO61>Sp`St4gh08%+tgEnH(6ZF>s3uf+Z!KLJ7l>fe> zx&IpPZ1w^0NXqXKd2A64g8y@1Iq9Hgv7EdAepZ5=%$}sjuH+zCya(pes;6!AaGlFB z*J-63cw}(}Au zHt*i#s**DMz#3&t90v7ZKX0K&k)?m5Z?0poHS)7odf%|!?OUqR?#b|Gso8PZ&2tvC zZ6fF0d(Ga1!_(re?%ua;^Ps4sFi_d#J1Dw57<2K_7koai`nh~>2WwEp8CFa&TLCP* zuhyeiOX!_{_(vS4)z9hC$ivUp=qIzQOOQPeXKpCtn8D(Z01nR6Dlvy&nR^}u~|1>rA);G7K4*h(+{bJa2 zzUtAgFW_6Ov)!CTHa9A!+@(`O-$PEPaWzoPmB>_MHO|u(zB*-KXPlCi31Q%E{Ky*) zc_fh8@_&~bH%&v3aFW>=uTW5XOw=`5Fvgc~^6lf{6d}fqt;a@N9U*fhj`wa2-|b$| z%(rK}@^I_G(1686G~Xr>=leP4Ij_9hIul~r@bc_gk@2(z0$~Q9S<~CA7LZt<$2(VC z>Xk>kvyQu-_L#@t`6X!K!or!SH(~SLY20eTFvRr#oLNRx4elET1ZZ9@<}Mq66Z8dz zpBV(!@)-W@o_$~erA=J$drI!x1-l%}PUUw$BI=C%pt6 zz5E3~ZG)?^xlZ5A8LnC2k?#GtU&!c@xT2Hgj50+;oip5|l_0n-kccbZgmQ#Re+t9D z#{X&KulF?*b~8OoS4b|WW(1c2R|IfW(knNB|8X!!Z|}>%?Qw>?+m!8$9)KD_zQUMz(Jc~7=vlfRIu*1Tb;Lt- zY_Q9S*ody7lGQ{+`kc|#-*-#+(XsTFjlUH%Ud})nkB>Yqgi{|w!E^^89R}+ z&p{pi3-0r47Q$nMTz^1s|6HCuZ0Jd}yqjhWxN4RLpnOnFj%~;Jp~O7IkZWPYC{#Ys z<!lqqQ`N+12$zAdMaAwu$~Km5>4Z(lKw}B^LIGo1*L4@%GXOM z8`V#WPwTpOHBF={n%=jqR_X+$0qeH^qEGNck$C>`Y7C2Zxf>%|vk?D*VWlthA=U7m zECOpFqjIYv77zP8;urrrh%vx;4CHvm-RfD#)9V(@Q$;+HV#M8P#qgGsHpG1}aHfX` zJ#Vo(Kg;~cd9s(Qo7sK-lRB&~+p=%ir}<5WYAjqbGyHAMe+3Ptk_JljXxmDK8+9k@ zitifnzo2fBX1(oC+RX6!`TlBGl>=~g*hUzScwC&JfM*8r7?kGxUN$h@x6(4UmXkZh z89y#*6wp3t`sXP2Qw})st{_Y5T%X2XkDdY~DrBh$!J0eRDH;Hs7E(TV;s>Zw=F|ap z*+He2KS~hK#wgB~BnR!mwWY>lHDXmU(P!plJQFM^%R)@ay`GK38YfuLI~S+z~>Bb%;E_oM(7(-g?Xyy8;SD zne~A4``y^9>D^js-;jq zBw_i$DqF_1GU&FA~D-yMRcwK}*`P-Kj zh%C~_jbL9k9Y^Zt<@z^xeuEuuUL(di$=Q0li=ASw-Tm-S>%ZlRCy$VX-Tge)3YG2; zJ2Em|JGWUZ{Vlw<3#OI+-2qEUqGRxSJM;sOAoabo0yo3VhrAnQQ@4>vG_1E~B5R3x za}LHV3g*mqFlis3S0@$%jU=1Z>y4sTa&Xk|YHmYQ4 zu9_6N^EtHZAs{er^CtJSV%+r+-A;Wc^(jhG*L9glgiNtCd=ubd?6nb=HIbpmQXN5F zn%7Js$pca;kUQ=@yzLTOVUSIP0gm~R&Q^9-Qd~RbEs>06Q6wI13mDI3^fSv>RpS2~ z^62=lo_U(_Fn4<>x68k~89ER$R*W6cNB1qddsXGex6sGF9f`8X1%;3(Eaj130%Wpo zcCrcE0X zanqjp2h1l2`_Ihmfvei*Wa2|X|9>5_s7503jY8Y*l9b%26FU*)4BZ9sak;gXF`M_I zfixm5HMx<-@PnOxsmgLBja9s_2jm&voenH7H?m{=Xmp~9E6deq`CwwT0Tj$<84cy& z(%2WgbZC_HdaliZeN1ysnykdZnqu^;q zGYYCnGqr8if*Kh0CP*~~#%(I)p0KF9*D;m81+~9Rr>-+LN*IT#98sGn_%v>tpeYnb za8j;AaE67+L^a1i>4(Ps^3-4QMl-^5iQRIj?<1a+)LvdTkTVwHNm}*$X=L(%!|$I? zEKZ21PQ|l>o-u)e{cpl)f_zp5K^Au>xjp(Fpitf}cve$rz`WkQ;&+_G3WWT;pK3sCie? zG1Itr_KjXpR4R_hHmDE<28=nZeEDH+Wd~xBhGjehIg=9MN2lfqWHL4+SMpZ*qV${A z{=GE44DT3(FyK#Mm#ijUcyO~I9)c*3#vH~HP|tDOxMsk_XV$;N+O;~G4`z1YMZL|J>6L3Z`F3;>4I z?i;`DX3l&Y^5~3y2GY>AF2Z|d?xbTGEL~gbgE@kEui{x1^D<#g&Mlms7Y5XnC2cr3!evcv9 zX&e7`_oM88&ftIUU0-p}vw-oa#%=EX&`?qHDL!270o-Wuj_*5;$_u4}O1sr47Hi_$ zG4ny&)(0?mONF*scugxf;Ld(s57a)HEI(htH1wqIs8lx$ zl@+Uuo;Gn`eu(Je!>nm3f4qc~+$7LuenttER#-|eKvC?_t8pD(!LL@!bFBtwPDLQg5~;C}+s9wg!U0LHneg1czeEc8`R<0~B1;83dKl1m~Z3P`9>t5+E+? zN#li9Mk+!OAeT#eo{}y}zJJqLX|8Eg&yy`8nb|L;i{T_iM=GAtk2`j!Tcloaf5a$< zsOHyI>CrG%T=VNzsrlNI+cDV}sI(#Hk_Z|JKb@EKSQxE7E>G~+f3XhjkC?2f&NgDE4!wAW?N8pEj3?A zSI>j7PEDOrcgjRrf3loz?a_S4&q>ob-VPNHTG_R?mrCw`ZqHm!bSt{eo?tRYvNJud zg6-!@>tx;DD&#?dt{H`PShrd5Jee*s0_W?Vj{0^9L5PG_+$5aW`^AILn~3J0@h+r` z#>$T6Sv<9t&d|L?jL?Ccy9JjQk=8iX6Mg2OB3hrW^WQop96!(fTc**=Ufa_J_F$Pf zjVTP5uA1*H%!h&F#aHInWib%s)J0DbYhQ8Q-(12zj{&cD#j5KJsp#?IvcL+4NQLjO zDoe~73xl8;utP5~FFEwWNNMTbb=>n`xGFCQrcqd)WfYzm#fxQFTeMZ7IJicS1l39r zS$gbL=0iu11IKLEsM{QK)7($T8r>hoQEPN9pIOZR;iAd=h0L2p5hilpqL|26% zcyfoR8*E_z=lB9Iau;&`pj|-@5EW z=Z^0#XDt>EEeaj74SdG=UOVPzxZ@x4n7RJ(+7psDlac6%@+FwD1CwPOO(;rt?SdJ* z0Zj!ps8Zfn1u@MsGj*X^$6BK{97#W{{zJG`D2%@5#S<>yr*vH0XEnuj0&a^IxnEW` zF4{hg_u-D~|( z7gBy06CpOPvou>9^%v*J$hy*iNNkR%wk%(cgsRpy?^T~|X(I&;;|-;?i=XvpKX;MS z;9{rA7>3s{N1c6^bxXaqIs2>$UxA;>WVTea-DFC2UiAP4YYM?Ld1a&G68pveQ-s^k z3vvZ`v&Ql{qe zH^Yf;K;?(KN4g44O`F-1AU(8*U;qB~Iz<8m%k?i?2@0$Mv_6oiaZmw+rFSnu*31=i z{4W54^!Wpi(=DW0L%*U(?NbChUvP*}qbtq(HnUAo9!okU}>msp`q7L$_*g>ri9Ce_eEl}bN5Hc`=a|W_BTWu?v z;4>V8Go?AB&|QF!q2~Wo!%RSdstjd8QYhQ9y#D4Vzq<0Hq&g)6_0u%mECWZd_eGKJpQgnzbFk|qi3wkqsxZqcCd-J;jHHs>Hp zKyU)&X++sUE@B5C$G54|w6`KStiyTv3$PsI!7G;H?ax~s=COrcP7C|}ZZ+_GlH{X$ z<=cVKxS8jc1(WdnpZtv(-Vo|>I_vP_0ro3~X+!>$kBYsMDAJPq{#5{G%F>X$Z~N@{ z<#l+vI2bLrm2^CiMJLV^bqE2$6W#n*0C*B~(}TG+gRn?ATKU)>#7=q67bjD2NKgk% z)Y^R~KFioO308a?6dN3#!Xa069#P{ODc%x`7qHy`#dFEAZV9_aDw((GPmb0>_1RDb z+jT{IDvqcm5$>Vjiv^c5)sLuE{#298)h)y|_%+Mjsp9=Mh1)DeE(IX%Js*Gq8WfF) z)!a@;0&nHPqzioud&=!ZUnyZw;#-V(nz5{tDd+M@gO&8$rdPuC!a_eb+rP_YjZf8? z9u+%#L9kwJ_uiyfZUO8P*>fSA#Clet=L;g<^mnq95cbma+w4cOw(?DM(Hu59{Z-Ot zzuN}vkI@-(p0#>*~k5Mt7)L{N05V)NnJmt^m<6@IK8+Cs;Szn4)T=Csqu3MH?-P=2(#r;~Nhx zSuZgC@BW0GtVTwSZAQlQZ!o}wpDAF|G%KYOz z30xp_b46!SFq>sSnUkS+EA-oOZo%;rG6p~QH}6x%6i-3pRCSc$Q6FT#!JNW+%)59N zRgW8!t~O{$y5dF$VY^8yb1WY(ElKWH9Q)?DP1=*wEWuXd^$rRI<#3nEiJd{QaZy(P zSY@w;Nl{ejXfCqCjaAbrJk)7?>iLG-aJ-QcHJ?>-_b&R*QMZTtOGu)#RJ{8}yLiAo zH2N94bX3`OSJ5-zc2IcroW^(!a=BgZ-bpAH!Hrj=cT5~2uU-fl?##8V>@3JkOWKEl zi^~j?xPsV`PK(?iV3>GV%SdN)>Bg%Et==_2fD#%1X0#Xj;}$s%dJFtB&d7o0H=Zt2 zGO|~|biNq+6^&F%%%MQGk}ywk6pA5I~kb?ThZ@BtYYfBAvNp$Dx$D{DE?m1qpO!64!WW7g(<}OWan4N#q}AFks`BxeG(Nw z1#(%SRsV#xvi=Hbu{*o|&{Aq8eBjC7@|G75U5|sFb=>VJsQF4G%%?tRe={`35z(Pr zp7X3utx5bmyVqmqti%p{T70DrLkp5vZ6vCqmz-LZBE9` zX{2W{$vqoUo&vgr^Qy#}Eux^?SmwJ_oHu|Mjjl}alq9Sl-L0EhA?YaZR+9~0EB~ox zK0Jcir>yrQl-1isyxl$GbmyxQPQ5_8yUH#|o|MKG-~NfbMW--JVN3_u zIjU6Eova@JSq-SqGKf_;oT*3%J8p+8_5|NU^V<*lU56`d0i>orA`Cs9DCWJL_cv7o z3vR~8K<+Jx(1qdsa&D9|?e*SN7&8by$Qv)o%!=f&NfK$n#qcL9BEaY;I3QrlUNL11 zfo7yVOy1JKX50DzVcZXwm`+W!K}WSYsOUi_${ zEpUqZ(TOxN^v&M@6v&w>mcEn3bF{v^#UaFQllU{w8}sm`}iIohg=CJR9JDULG4jFyUt$~ezF z5-BE-$oud!wgFh}V4QFnL?}BVqsI?mMWJ*JZYkZ{C<)R}#w+_A2OM>2haWOn7>JR^ zqU-wcTX?Tls(N?ss91-rK~<16eiFDQB86Z zcx+K8{&V^CPux_vj`{9(C;V#b!XjVi&xDJ7`Po@}7l+nu0#@n!)`KZ;+N$39P^SeP zz2sjPQh>HNIK>h3MRCelb)Klm;Gd`jZWITrxD^SzRrAdyu2rL>YLl0HX2=IE2#p1E z?Dzxz&O7V9Vp*Ww-xOj#DCn@6asL?VnZIIEu5(bdj*Lim%Xs6{g#jk50B=i_5aGlZ7l{|1duzSA7L3uq0wAfE=LvWj=MisM;ZKqt; z@3$AsUM}^9?Hg}oxOSo!m=TNNyH9tHww>qVek+-PZl~L-_nX-#K|bWl%*%IVNM99+ ze)nFz;DrmpgW_4E>KW3sfqWIsPKpV@0-#R}!3v&^Gq~u4qwSZy{&T8Q(>1(L3lhJM zd)0X>x_WET5Ndu8^70!ncCLzgu24`T#NL0zaHwBN3Oyf}$KNho3f&T7z;A6rXZtei zT~gI}6Rpu+7IH}=vX{Vbt}9=*7Xa_DA4`hkhKcBcKg#IoR^^E2lZ!0Vi+(R40hbsw zdk+QA13g~FK`_sP5MlZWzNwBgl@V9{T|2z+PxHwC>Nni~~p8 zX`|`5PcS>rCf)d_o!25Ry7G2S+>tgVwbG?O(+3$&3(nDi*fv1k{K_)8%xBgMtMLsrqzE(bC zi<|K-KH=Jz-O)23D5GsKH)Oz&Aqk5_SH3fRzYkiUf%7>MJp)Io8tg(@j2DX8w@<6c z5$6K)p?h-pX=MpXe%!csKPD39pGGdH27g}ii)a76|Gn04%ypL?equvuEeQ)8{xm1O zOfhGwG%OZ%R8+{z`|V`A$$O zMa+q4H>C95)#g|~{@mX}A)xsEb||N5*Bx&<9}`eR>`;(2NJ^~h z#Pl(B<=STKa^O^SZDY;ZGqJQcZ4Ub=2Ih36ZK{2~>@zpS;g8&*ALB8bUjL_*AMEK# zPWqG`VX*6_MRQgSt1r>^EkvKRCePT znPKT|P0Oe04}y9%>6;-fe3t9$r3>T)E--E6B8#Ku?sy#J#;dx4 zaiX(cd2OKRt9mmdYsH>671Ko308J&j@%`1kEN~VK55IAqjd_MK9y*dAoR2QO3|54ipsr&^sf;%> z$zBMDgp4bx$K3Ft_tfPtO-S3IS&Z^flGN{hGEh)5y1z1381oY)SS@XC_teFwa6Do- zvbB;K4PD6!T`9%lS+K}AWIUltE7p(X%*>c3Z|*$7N#}XSS~kScrZQ6cR&Wwr^&^mmr2;GT?sN}DZR{jwt?lwm&en}TJAxI zx}{F_P0~~C4Mmhgw4~!1aAULqy zGXckc-hrnDo-eO>DfD1^$2TJw zIK_nT7`Vrq?l{@MI2Nk&v#!LxtT=n|&^9ELcm)K8-2EF@T-KlG!C)RVmQM#Jr17tv z@d!;E7R|RS=Oahzu8!#ha8^zG))NqZvey6NMpx*HF&?D+Db&R_!b6K?s5aw7`aoy5 zLsv!{=B{;STMHHvV4PXbmegprfb^vjn8RCW6LF-pl`fO*d`tCEwEBvuGlwdQ;R&T+ zh5gxQB8+Fqj6@6z>w4l}_lf!^JJM-MX9wVcI#`2mR^dsRxr4iII;)83k0 zrr}~qs*&%L`$8R>vm%*mHpy)T3$LhU5;>b_NbUVkp{oJ(Mj^lM+*Q(`XSuvY05S!i zV+%K?7ANCUmf%~WX%!YsBf1lj1$b#G^xZ);M!P%??!FAvWQ;XHo%y=(>_?Id;0`9C6T8cfZe zm!q$~KJ!?o3+a=NxR!PuKV2`Zjm3i%)oJ(g!uur{-4%Uwqr2(a(CHICKMr(vt9aMk z2#Pk%duMjA#+}0QpU7>Lm5u${GvHjotU2B8h*Y6GFdg<4Oyx`-n~6lPrW5f1X9Che zghocsO@~STMdw}u(+QNLmB&1&zf>{#me12XdW5<$RwS7~8@#n_s$-9ZBmc-0m{0QA z#1F+0Y7p%sbpow=pEw)^H!<&o?;}a*-JqLBhWz)1)!Y}k~yRmsJ2dM{M*6NypAzu zIqfCM>nrX^Uck@4x7UykWyCFTM%k z7NeCbWH9AX-1&M)z7JhOrNkjnuz08(Hv!dn*+LENg6u-64-^F9~EX5fegDJ?EDh?iD z9N(JVfOO?qG~>t{kYniRTako}eqZ|0x=XiZRn|5i+W_VZU{kzBm1WvJYl1-v>5^{I zmh=YCNy8~@uaLKimfpwdOz6aho8`J3_0BsC zxO=3|v16>n;FcI*&v!s4ecKiZxkIjep8y?}f%84U zGvT&Wybx~Jfh^a46lDIfzWLf>L@@>Z@z!JNGwq7*QQF%89*WjUWAb!eH}8Q|PMNoH za8uri#L*^`j&1ISZHI&ibj}?F*}!1;%|h?sYB8%&W&kpZ9L16DfuMI;#a*$Nb_Kfk z;3nWM7o$oR@Y(yDEzYE9gK@_njOUqhhBbDnq*TazW7FJonH#_&c9k z&fq$Rb5A7A6IUs<%0~b0rn%4v2i{rQ+#6myX0q;_cG-}Goiu4x<66^ zJXYP_g0Nf*lUTn>h0ZuQs`njZ_EGVi#(le(t>;j!IKYWYY%FIs5oM&Xn?go5e}@81 zfjeB~o%AT%N0tmBH=nI)={Cg(o*=NgvC-Fgy4rT7*NOG2D2`6c7eHiWer%d%WJ_QVbfdyR$($oiO(! zxpTo!S*j!x$+bUGa3FJug^IC5hU=ndDX9x3WX|@;N!yQv9(89%Puv_DuFt|KQ-(4ZI z+ByI`bqv$*9^MA*4sN5D)8%muU+#X6zj-g;Ny)tnF4aYw)82Bn3==hHMLx8AQQ%;; zIP8pLc1Cb!x8fcy0_$J?BVBEHbA$alI4St+{j~r3SJP<1LotTZugcZ5BK8mPvf2 zbYB30Ndnm$r%kZb|)8Y$>Oj3RXst}HEiIDAvv80?nmfCAX7fW zIZRX68KpR4RxlW=&4xYv{>o`OTzj@hI$%yZmfTnZR3v`Z(P7YK1UUkDE~%9(G!X@(g;s$2-|Tb!Rq#of^J zxG(oD!K%c-a3&AjFK4fsD%fO3p{c4X#-{Vo8EMjLTus_{?=sT z?udSJy@EgR5Z^E|5H%{`vQi^wAc=8wg_?1@Fv4TV;qlPw9T32z-02ZakY~0so(I}A zMm_iW#33TyG^JmOrtfU?i=V^B0Q>juZx4bIeM9B2gOF99M&86s5E^JYj1dz~$S>Q) znPSQSXUbyAx1q9Wz8}%(0{1{6@@LI@)$;W$(w6?SsxM|kc3R#`_;nFRzXU8q66=I9 zNcCrl^Vp?B+O}oyg3TxPFxf(-cz`z~AQ)C@k_LC}RU|zBvIdX2GyX65^M9#7Ubcec@keh{E72)mV6*qAf~ zvDNV>s`C*5*9b-8GQE#Teg+lsI-^^i2K{N#dZ88t2I&s>5Aphz8Wqi8ZHgD`NN{D| zq9TD)VKp(})Q?Na;mD%1oK8btyFipKOv9Hg`h{2=Cd6YKKHqpnJ|G<5QFW*c?F+H; z@r~ zAB1zPXpIDYq!j|=Y2_nJi}K6)EExdawd!H=3K^rUZpNk%kOIGDW5Ppx)DetmL^e*6@P<%6=Y>)I{gcU9R^0lks%d zA?#CDP|=>zkE~JP62j22Ak}!4IF6uEm<^vvg#%6nsPeY3XbDpSJQ*Yr!0&OcqeG*rh98arqMBjtD-&n%lW<6x&E6Yv?q7 z$}+-{HDswQQIS6j6b@m35d(lZeRq@I^XELWPeRJmr$`RS5u#~>Y=VI15@$(ttDezHa&>sz3-Qpv2$^vnsrX+@~yLS3T@Pv?GGmN`8Z?C3L z+=IMa>T%VNc&00<30d7!Z*}<+QaLGQY+N74_M?9~A>`;pD7S?kIZAa8z8R)>r6ocZ z?7S5dJ%H96*mkrW3gyNVBGm9>;hsrkGeQB6Js_R4(Y|p;V@KLE3A%TCDhEs+6j|ou z2It317n@gUN-*LY@;1>YI~8TGo{T+!e6_7m+VP<|C>uv45DfEP&6mvQ(S^}gG&;z) z2!W9AuO|T~6m2Cy9fB@YlaCMwyIcMS`wat{_&E|8QeopLI9BE@UD9?GKP$rL zjJe%&&g2J8JhjE8S|<8&yH=yoi}4lH?IYzK?pU=e&RQ)el|VMtD9&51?3VtB74pVB~wRyN|3 z;j^5&vjxW%vv-Sf2eob+Ydb3rt_0Xw@yJfrF3EDHcu;NEx z*hS-fj9M(sbZ?#mh<<2yL8;Jd@{WIeiLJfdv{UTqn3C8S7=?1<)lR8G;$U~ieSWE- zwr=y^7Z)9MLyZv9hVjhTQ26@S z(|Nyb&bEBljiK{pxyx5Eh$%taFqT$v^LCZG(d)4HltzGy@rQ%o`O$O1QkW}58$S<^ z;Yz+%30-Xw8#(U^no_+{7m$L9*;w_ubdpjMniJA*9iP-?G(|e6e*did zZ)ipsAnK~FT-`CWB4nvnzb&ya+)^pHC7(f}9$Ga&w+Huyn?=(K{Tj&L<$k0TZI*a_ zU(oz}y+W0s`SJn}v#yqwljeFsdx=6-W?$5x;gC~~g2^7`5Af3s$}u&y$8_{#Z>DDK zZA(@(rsf8+-P9lk*PH}xvMm(}Ujw%ufv-pvG^A0IO1WUy_zBO7SPn z9M}KE)DB<;@qpwT$FI^7Lo6F*i1wSj)6z-!x5YhSwy~>DbiS4_dO5l zx1y2vBHG=BClBYdvl5Zk*tpphKH@j1wWU39kajm}{hq~t@&z93Fe-dL^_Dur5%8rht&7AGo_o0gJXV?l zfx~p@@z}&iX*3Gwv2?ynZa0SYNC%O*?`bL=KcSoaVtol8h4!vS?c3Y{d#Vw8eXvy(f zL>YR9@Z^x6vXCuDVtdv)P(0Iwvc}i2Y`5++%B^kKx#(0JQ8kPN#BAMzulPs8)jL8^ zvM(SPy3!_iW@bT z;tF6{oZtJE)m@XQLrysqtY>T&kM86br&cpR*!mOYtaMlXN(A1mE> zwiS9*YuK{F zkTJxVC#yV@6rHm1dFnGoHZJlL>#?no=~lvZF%m>tuyx-w+fAm9v_}&RPID$-?gO`t z*EycQeIw;TGXIOjZYu9S+(1>mGioF$c->KW0DMbw&+m`8pdt9iH^2SNoQ8z}UhVH- z?Ux-r)s(|;iuUVE|G;yxQsX;AkY({&{c9WQGB9Apde!eb5`Xp6 zW94mw^{6Oq_cjFPyaeh8HxS&|)af)7z;=|%z0Lmoe*0U^zU}NOFyfOR|&<*i@|BV??UT+?5q|Ls3B0*}MMr_iT4^ zy`R4$C!BqAxjh?%?(~d(DdUB`NGm)4Ar^nqcWsC&c!<8O64K1>vr?s1rt0=N*XFrT z*O&aX7&5qUn#+Lem>k8)73(AL(3aI>6rdk%WynA)YkB;lLv#yyrhWuS63JMxV z#VzP~kxl4~b+PV`%c>Uek`XhZ1^Q;7HVpXc6hJ?FL-IB`#la-PV&@Gx!;Q#ZI%%9x z!^$#5Y<0APlh*G!Ox&`bPH$r)C}}whRvOTKM70{{BwCK@N8<$fP(6LN+Bh+Dek8+Dwpj zUdWz3wA~i#TbVzPjQaHU;>vL4JP$PSI4mh~PGc&--G}FS`Hs1-sOwP3x~Uypmo@mo zC(epS8fhj+){dDd$Kas#DyKQ*TtU1CGnZb;E9=UD#PNtYzK$8AgubSHW}RU+QtgUi zutYde>@SXuYVF^0=N>VPW2*iStIN1lyiCSrhz}lW75T_%tGChMxCay@%vebQ(Z78E2Zj3t`x*KmUpOYXuXq>(kSrB8kH^%oBa8NN3-4^Xtgl`Sr&lS_|nsrW6 zucWb}!ktu*(p4hRLqAdA8f5!%~Gb8Oq{5Y*Xe1=@j{ zmKeFCxzbWY*-IL+L%Gd#Jc&bqGG$_@t6Q;~kY-jh;zvMu((Aw=sDU4wfaaz}S9OoK zJ3>HG0u*QFW0-G}5oFcOLZ4f2_qt{Y%}c(7vTPLx2Ay;A8MccZrQ-~rQL)B>!M&(5 zI(XZa(P%r%cL^006Dpn-{VkS)@Ii3Q_d%JSn;hH6fyAZ`>08|bn!HtUZxmlp=wFIb zcO*O!_vT}t*wnwYUTj^Yy!vED>bJ16gr*_o^W-wUiAWOlPxekPhUt{=t>aa1*T!zC zcb5!^?PL?2ReltRJ^pC%9jpGk^ zNV(mYEaoIMT6;Re>And=@Nav2;Sy%)?Ky1C-tL|2zNNffc>WgALnTj&gH~dWvE0Pk zT|Uf%RU_FgfC@w2i|SlYby3pgb~OJXw(OL!LJ+Dw?cLYHpP$Xy`M8Api&?IirA2i7 zE_61^QBH(`x!Xl)tjei8Jpaq2cjNUkd0S|HL1bp|*Uu*3b^&?5249oYvZwPg+8DFG&=XXc6SfRd7HbE@q zBb$Vjee?g1tE&Kt>U-OOD4~D?5>f`;4H65AbT>%1h;%oIfOL0CN_WR12uOEI$5H~) z4c}S()&HAecJ7S9?%s3X^PcB@B8Q${mMuR`_D0`1R^(X(*PVVJ)`hRcV)KW)X!|Rv z8_iuQCjm1hht6crps)NOZ-~On%cRkT^jFBDbKyC|aPWa{Yj38dk-oi<7V69)l>Ywe zxh51}y>qF4LaM6|mQkgna3yNyc^Fo|!=<6P2KItMazq$qFpSzztDvrk9&A{XxsCB; z{G!Jjq4p)k)_fTGDZ|Ry3GAtERDA}+TR*=FrLc;zS;EZy`D(afha+#z@CG8Tk*&;X zwE&|NfQy)jq?|UATL-1dgCWesDMFoMN+XwdKEq+=PASTpFTRWeU54^uDUOZ@^p6S}0QmpLK zvD`GAMKJ=zB+>GL2j> z__SZzr8#!=fcr>^Kw*(3n4N_f9xx>6h`O#WQ_jm_D=nq#hE&L_fR&wj#)cpm?x7wdVZYjiE2`3_SL*W+JsN`E%86GB_4+3726F8nDeV%WgQsc2vAp?eD0%nH$% zJ*~g8N?E_MW$+i_;f~}zH<=}c)5V!dkf=|@hp>EK&hU7R*($n0Xl-S4jkj!9xyB+X zBsrcm%#(aL~)yTQhgmk^>lOy+#`-U+M z^bC3+cL}~R@F4c7{?HVsJ5RV{-a++ng6nE0Q12}YMJJPKy=p%6owO3!Hzr3Zw%KWJ zN3i4TAs_bk&V1B32qmBBA)&BP*WxXt|Es@?7gR<2U0DCgUkQ)n4=cd?9QiQQi>@1< z*E1tvp{WeXV}?^q)liWVZ7H2SmY>Nb#Vbdee9!0Bf7oIvGE*J~g|ma>82Q)4G3ZIu|5;}K2yRvIy5q>~`0f5j1R_`YOO)bV7R?o~)qVJ`ql=A{C_BYWII;dTOnP{I zT;x3|-qmvK#uog+O&14#ojfaq(M+~RF*fO~M|?cgWPl}@f&C*5<521;VHyQ4rMM>h zX|_{N&1FQjq~A-SwBIqw$HcD^z}!v}PXv7Qu}X|fd)fN@2dHW9>d5wc7{qI5#0sVP zkxJW2IYZ6XWuM#(@xM!za~al!zgL()!m@yjxa{`*)p1}tdrh@-`eCL*)G^?lnS_v6b_qcjrO z--%pwMx}tB8bhs9LlQsW9LOu((`;}scM)1je_5Efzs#>u^(8H|N4@bA zLDjt!Fl_D~!)!s>ZNY?8)$jB|w1x3$RzXVAgdd11R|M8h5(y@apiI8z@D z{HEXjOp}*p^|fk_>*IWNVJ7EGoRQT`DFx@&7%qu92|B_9MYR~>=xX}VzX*EZkt9Oo zjAzay4e#{fHb2D%zNe%)Tjw2JN8sC8$3pBAP)6tEyV-ggTE|oqxFs6|gi1Jao=oq_ zXaDvB=$nxAT`MmVQl>c*SHRkweB8z@G9+tx)E2T`u4kd%?B-nldo&4AZ|0r`jgCfq#5uAer5vO}dO{`t zDv0?K-5_}K0af~0aFu+wTb;%h?DtH4)W?gG=Je)>6QauoX8y&G55&Dp)+qSi96TM&$#ZG>if&nMVICbZ;V75jthqx~BHtFR%{op6)O5?y zxn|w_fks>#78Ueku8Bq+WaUFe2Z{?z zrlrOy_n&%7u`yl3rxV^&k90AIy-k!E*rBW4-a`WYm#WICZ5NV)X3^uE;zeddIvMpEHcp`zfw?ecZ;?EB{bE` zP#Dn5^phAkDxROB1 z%fz38Ppr~BpLD07^=ENf&u#KFCF;6=@=7V6Nld5`{M;|9mjea?$Hi)88ESPlM)eqQ*RuQcCxH%IgYvF^ z=w^Wd-3=Q#N!X6Kh=81!{b&))guKc!_t^S(Bg)U;xeq>(uXY#>ctHk+&4@H6&sKB7 zew29R9L6SL>lU*0s|!W!(@B{X!ahmct{bDCqma5PpK{eqPde8rsS>@{OL2-TsflT<2rc48RXEH(R=B^n5dIvpINDqv|vwR z<~tE2kIGwwMo)>KTvBza(yEL4?`w2r(X%mY;b(f;mq!7zs4Z zmY8kl++?nFYOEjn33N#4VAxOSWp)Ud)7yf44dYr*vx#EqG<0Lnq7c{IYki}Za$X=X z`AR9r5n50}K%8kGVjY(9eV~eS_C<^PH;Du+!mDP>v6EnXWwU~Dt&a2=v84!i_D!O< zCoe4(q$GORHi|w-hn`yuZTo>KD`1rDjf%LB(?rc?Q?FxiO zcyRl%_qGrqwhrO~76;iIQxC8A!aiCD$K}5AaiW&(Q;D7mIZx|~ZVsBu5yY(}{azmDRcZ&6VeeF& z!A2G-@$fs{ud3Gw?x{dOgg&m(+;sNGz)cE4YZ2+zP&oF{iF>g1JkZ0XfXw1UTrlRN zqr&hDQz2!|k?r(kaH^;3X?lZ*;M<~MQJ1}uA#tGS3CNLdw6$g@g_Zl5?BI9kUUHeo zi-Pg>cF1*kson&;R;P1!SQUW>Uxk<{UHzgtKwZYw=m)+BBW7Kqk34DIi@EweM3!H8 zeh)x~FmJ)@-rh+PJWi}_hg*TmpQ0{^e2Otbco#m#tetc_=!3gI;J0)B{1AnE?j-mk zIb<4`1e8uX&byD?`FR&#GaRD$ahNQmcH7)CiTD)6eaw%)Fqz`OWWA$uPE zgJVK7>QYt5-F^>lNpUynv#PeUQ5j&rXG!?Uj7JLI;1m>`<-%=ii>I4(fB$uT$fv;c|XNn)8xe-J! z>6_`L8?|@x-Eg%j46dD@tnyQb6bG1Yb9y^drPm=DCDFF=`3z$yproH3!7B5&rqRO= zG?RBh7UYK2MYwGvFS0D|{j7A~Ur!`qw&(fLSJS@=qWjb$NZJMFND{7zg-tWL^kNe=^#R3GtZW ztik@gRhnv0C9-y{QL-5p)GR6RQO(r=_p>dbRyaBW}GS5oHbfyyk9 zzOHDz&PJS?@fjz3M%g{EksZ3CaPVl+e#Yp?M$H!aPy}#b?XCIKp@5;3wofv8vNT+38rUIqUWqU@*aglXv~V}l zL&|0s`*@ zw>>RF*5#Ho5o`dNv5l-kG}X zXnLve8tgxgs~}L|RI4njsC6^Cq79Q5mut?+z{LyhTdFo=HzK{k%6WjgX4}+I>}orY z!8H@!>Ihc*5-`*HNxlV4V`bZczPe#)Ju^(k+U+etHe_FD(ONA(p#9aMNd+zjg39F?S5)a!KH*Emrm{$C*n>GoGlQMg(>>( zfO<#Iv|#DQ&vJJ35%*WI>ihEC%Q#?hf1(?p6;GsXELRrQb^K*`=)dV0ixmDrYw_ z=_Lp4`JzQ8Rtt6#83C=P^>=P4 zftzzO!Oe>FK}nVtXq3OEbLAzoymW0Nnv>)ErH3WC2dom^ySZS#NnEIrouJNd>fc{Xdb{9^dsp2a{+ zUEEknvX%##8>gJ8>!nfUqoUS0#m$B6Mjl%JzcK@snd|+lR^h&X3??CNw7IYW?B`;(MAH=~{vIODzW5iqxJsF9q`5tOW#6Wj z{HtmK$519jkA@lJ*|7MVWkhRi+hb(`!i{~`OXxPI_)YUkQ^}-RNmFD7DUmHTV!3)v zALq}kfxY6lRkT_@X=K_Tun*?HyN#=wo<8dE3_4I>h@n%>>Fa-^q@`l$FUrNARfQ8Bgnb4<)jwUP~r#iYekaqP&j?@A$(}@ zsg%{)HJ|35vU8+19C^1EH4>Pi)_GKH7b!soxHBn+>YD0m$}Ya5gtIF~Y$Qm%{=Zhc zs`&+2@k`e}8_5pBL6U1UF=xp@k4W@1Poyyh!|f=fh}4*j6?Ftpose+?@>thY0f9$3 zYk8*+N;tQsUqx1a{~0f+1-RZJOokSOjuM$BqxH_b01*7s(tuY0H4@)Mwgnl;06ztE zE&DNLaeeU{O6^}QXWQ-q`7QFqJU zTqR)_UfA|9)ZoG`vw}cFsow^Bu~E{v-yQvl%nNRcGm`sxD*1Mp`kxX4@CJ{q^}+Yi zHTP{!+bj{YJ{(?n<^FVkDv-WRs*l>KMU{G3e>)CZhuD%$nNhRtg%fdIAVyB>(liur zGt@`AH6S+Ja^?3Q39dtmV&C>_98X41BwZ)c zI!xYsZh5hc@$mzOq|zC(14U_vd(U`wB}EoW*d))a90o9Mw}y@N5p?n5cL9 z>Z<|mSt-VmHMi5U<*=1GkP<{`GcsQpl<{BwF#`gUGwXy=yia=oU|KV{-je9!@VXetl8X)W~9 zggy4Ufa}h4_}i)n8rLu6TQxHWJgQg?cX}gwwR$rZ)<+HkvtUDd&PNoft|Yrnsmr7I zYP`m#3$v65mnV#DfnKTZFK^`p8Rbfxg`xF6LL3*Mv0g0?GBY4rPY62Xlc)IkRW!-S zs>B(lK-NNXb>=c?JX^1ZwR@iX7YSb{My3(C&x5hgZTQZL0a zSwwjr=xMQ5#?^r&hFbqVEvke|ho5}MG_~qEQbq`O}sZ0nH&UU}>TdY>6M+zSMI!aNqQrTgnO2fuT z(B>i&ZrB&Y8=6vF4PvABh@@gNsMj=VC zL*TD5A)43t9}+42hyWULzAXnBPK!J8I?o@b$bV`d4ziutmqt5+XTHxaYwfN4M!)1` z&4oW2iAdX5n2Uw~1$~u~VKxE*Hd^bPVm3_1;{)BDORvj?~v^i@PVmn|zzZwe- zac*EL)ExMR9Pi3;!YqqZ=oF?16pm3ae9&u5-bl%hqN4|l?gt;4~Rf5sthy% zU~V($Sk_2rg$FcI0S|lPGe}){(_v-BHe2tToJ7>$WZE+x6V*{HK6i}y@Isp2$6hi} zbB1@EFDtrHj4qR>M?f~R+8WE&w|RYnKQ-53B-ROf)SGz`ra8*OZ+TKzbj&Qn9!q|7 zNo4f%0ByBO3-%P&+jMR&4+B+9z!s+ z*#B7+caWF-9Gf>%I?M9->7_lS?VPBk##1)S zuq#StOm9s>ximD3WvfJMCZiVkVRkca=03siHbDpoKgJ8L`ep9XG?YOr=lv=w> zPuP34@N~H7tEDHzu&HVcl?lvp$JL9<(A&sLf-F|=3)LwukzOroOV%;PY3(%dzveqp z9<#jD@z5fKqK@K%M`XI$+iLSUUD0D85KV?#fT}ba1_xQROLo+*GX~vK+5RcMc-{!m zso3A)qwOd^gliEkZV~W*zo!-#q)Wzco4VrHuOjKh1?VyMILDAr`M>%kY$|OT*dYXe za^IP_B8UxJPNGV0dq$38R+ol-@VmUSZ=x{mWp&;!F4|`>_D%oQRMs6iv2X>Xyl``KVQs@!zNJdO}Ta^D$TxJ(7)W?r|i<% zk^yd7bEiG6xViiO*$Ee>~Q4&&zIuoY^fPPH4Zm$WF=3fV9~V zIYOHo;#L(ZOPja#)j3*A5iiHBXQ{hYjI89@f4uFSke%lCsJuE2)(ZYvUhs=_TjWY> z$~<(NL7cV~ko6=Dd$NXYniln`877X&*nwD z;9{LXj(J6YkS5-hl;uH$(#_Tr{PX+h_a|RP!_9{>TXrPQXm5+q(?bK=Y*eTmoLS2$ ze9~BxcD&L2qVA+jb`b0QgZ@=!!+?3B@+^l6yEXxp&6c{Ny$^?bfZ38j#KM4dN-1EI z*1*}MiM@2Os;m7yMA4y@;2j_%9bmWz2+|*Xgy8hL4EueN!Uc~U5nztHOvMYzczMuB z&g;pDY@Sic3RrriR&d6<3vO3LhH3HwKExdLnXHn)0vzp=b3UzvG-fG!M|ZIby{Hud)2mU^o+-dy3r{vTpJqs@ zBYZ(P>jU=4vA!2MJ;3cEmW!G(2T->r!b8_||Dpl7Ajg|E^(ckKDE`Fg=aD5>Xi4hkl><8*eM3L234x{3;+9S=FVKTqO zjZ+qgcUgRaCrI`yuzdcr@_aaP^<=q_L5OrT`@7s|4s!gxhbDKpd3l7qn(F}R&oF^a z?p`B628S<)L@4B8>J`S)<-r&boJPQyU^kwoo;R}d@xraMSuHLu){#~idHcowwb_eh z#FGrZ5>x(=0me;y5I)R_*b$JL#?F|Lwr4QHghV=VRj*@IHD*cJRnE>*Xu(sQmsD+0 zpL0;CVUc2IKir#f?c8MJshnaQ5z5-hQ1gptdYr2bBF7FsFQvP(=7W-M)kP!I2He=1 ziU1cntgiB7TFfbx%TgQtlwX|Q!p|0T(1L7oXTh}O?@e)?8PTJ#OF{a|NijZaX(djn zHmfx=*!quzLo<2vJ&qkpQk=OP$-@CADVQ#CmXg&PmHONM~W~A%1 zu#gVSit{a|#oE+&7Ukmvy^eCH&$4{IsV1Bj8eC|DWSOi_1A)~DE3{HC9%|8i1~VkY z0hj{r{LimGJ@J^{++-4YK^GxO+`k&1z+BENU?-Z`HPn<4N1NRvVxd#M6K~-<=yEk1 z*$eB&AG6GLajg9cZuzx8->xfeWB+h z9T$PZ%JR2J=I9YgB0Fm?9*qkN-Nfl9Wu0r{Z}ArJu=v%cDO+Oo#=Oy`sjkSc=+PoZ zdeYDn$0WPuHTOVmTqpOH?->K|f!y{=GOwOu5426ic({;iS`^r=rLb?7{<+`y+e~kA zku#5@WA&2aSAv0%GO`6#Pu4?*)8Mun@uPnW&4kqNYE2iFOe! zk(8K5ki-6yu+eI)rZfIKBQ;G=E44^NEE{^Od$tsaOoooVNoFGc0^Q|a37X+4sEDt| zk!xCr`-)?e&N|nxtJd3j!;NbW{k%K|tUqV9Q5qhO#IKX#%_nupe#x+EwTagCWk%{zi&E(%&&cFKLapY6?x9<5YueW9R7nfgT zIVb>>)0vXF8)hM4It8*1CvfVa`U7V-HeI5FmqvwJ4V>!d<+z9uJ3p7_#zIP#oAsCLzfXUHhPXCbn@=BzTL+x6 z_NUb1mswhu*w4c^vtC<)cO{+nQ1WwrmE~6h-lrVec-9~6Zu;x|{<_b9Ltq;xK`iiY zMlcFlWD1#(!aT~}P8Sms>tE^7j6iSXJ(fSU zo3VF_(^>$lJTqGXBPnbZ`=(6`Q>36bf(G~jtDR9Z`)xvc_DlEy^UrHyAm3h{KW$6= zv>L9?d7jYI13P%-?7$MrDVFd$^4Ouu+%)QYw};j*385|Ai?{0fI}H9O3h2SzjNiew zq=j@C2mFka5X-1@t66H~?*SjfaUuv-Va=VTnmo$%D%Gt=9(6x!9mhD5v>&+Z4_?-M zb@#hq6oyi7;BX#DdqKiN29WE@=iP9uZh2FG7RBu=EC#`qNb*!~tFPa=sof&(wkWjl zVGk}M10y3d(s2hjRnFe44{S}TC6LFGUvX4oDPhkXQyEX9EM`i1CRR;tJk|LP5>0Ch zrVH!o>qoy)M3$8ZrPM1-w6H*phoWZ zFkyic=pekr1dJG?SfAQ!ylHdS4HO?oK7?X|0!=a4zwIp-dBk{W}aNF5jwsjWs81M{kve;fnliag>h# z=sumy;yxWjVpaBu6zFgc?CJ!xb8Gp6t%z{qDG^Cvjz@}ZK(3)Lj30w-@0s=B8)E5) zWS!`K8x}NX=!j>TkqG)7g|89M5wE(hz6MX~`|W-BxK9!bp=jqbBpP@a?Y;CYw!YNi zcYy9Do8^^~fy!1`bmQ-uZ}r=x#T-($n5^|bi)G`wM`l7+t|f=5U}a2p0at~at(UPS zJ)}4l^+b$-3I3l20esPrPn!(ywlK~6JDP62x9UeO13(m2Rr(a~(i*q;zyFNJA<%e3F&C z(9FenOigqX!k{rM^StQ&rQi5m_>~jf1MO2FtAAt9=Giy5!;D^ zML56T;x_L}6Z%~s`@q$YFqcn+yr6|{$_?;Uw<6l;G|NZpcXb&I^)XVL=qRVJOskoN)f%R7!4b__@#4ULx-xq zx`ATKSw{5%Qsv0$&f0aAp8eA5tI^Cbbd+FIdjA|{k#1bEXeBwhi0{%)_!!G!4y)Bc z_Vy{TI0W0Fr`c*9WatkRGzBHfyt$Lks9H3Amq_s18Uux&$(c>D)5zWHE|>%-YV$*5 z!Sf{$uxb$UP2!8>4y3rwM{O9ZlGIFQ&~2ek_sTYXr|JBoAe^eSAJvfD4^PGPQ@BaM&?-gJn`R1|U^ zjN6(p<_UqLC`vXVZmwJRaI`N?Q`Yc8+-uZGA%`qn##1};Ks0z?V7@4q%1v*f?i;bo zj^T*aZW}S^sIY+nGVGL(5 z%%h1g{QS3r;Q3r-Fm7F?!$N2E7|gVaZXk}O@6<@Mjg4&n;FvhN1{$j4{!C(K?`5_O zL}mn(SlbVfdn+g0>pM-heg9n&;QCzp=@p*Y>$6>ZS?6If@=(axsPr(`l@PS+GI@fs zrhr|SAQ$M_qfApuIY*+$0`_}0<co+%ahHtVAz}gwyunkS{0VIfKDn4dX0SYKrF}cQ3Wl9J{BNOQq4?Va@@b%?Mw} zbKRtFuT4+0@j`K1#%?dkI7SjlLgxOnp}xfQmoAD(SNHXPts1a}4qaD$J$uq=g2a#Y zbghVqUA9=@B`fCWE`@p~JLmZb_QdG_Rtr!x{SlyJPn_>8Bxzr^Mts$D-L8t+m|AcB zJ)bL0$eB7`tff(?R`%Q|;3g-@g0_14eZdllj<79j$I}Sb!advEMrDZ#Vyypq8(tT- zNI*I(aJ?56MB+G8HfhnYt#4(c<9;;O9;2$Q{tIN&CVhYTpK4Vv<>fD>zDZMKILka7 z7R9Nxa%%uzs*B96Ht_h0+saa6?|FY$W=0IF7-N?xN+)&e`fsPv%OnXVl{#`y-#q#= zU?LOxgsZPn89zpE${i`#6w;|>Ob{mjOm11M{MQqI`y>R{AT_{`ZkV$CaKQGfcYtf& z&9)TmCk^% z^@E3hnQi~wd)(F-2Bb?Foeb&AVx{fJB~HTU)@NU^+}{<_lDPgVYmX2mtsAO>M{xuK zXY%?}kgJ-)xeW%bs`!ArWwcj6(NxU6;B_Yuq+KQvWs|kr!lP>8A%a1o{G0MbTF3p4 z%@rWZWlsJV7>g76_OyNLg~y>T0qT|h1{G8Fti#VGcx3Bd-GIAAI`)_Siho%*-eQ7D z`+Qr%l8NgR6}qG_h31D$ zf~K&x&u3m;txqs3d4}!eJ22|k!Z3;oPJb%!XuX*;uS#GE$Mm{}bar0L*8U;D{}*{| zxqP@#bBTRzXVi8tyB3}kXezWtG=D0x2Tva!x(3-P@hi@2DRG$matLmHX4+Al&v!sF z(&Jm!8eoHPor%}JAIq=&7R89}=~VF4ImKl&56*Xu*Z=ML+lP$4dfww=3C^`M z$6z}KFsl4}Mwk`m(vxReu(TC4KU+j=e2@}EUSdmbJ*V8*CFoiY)uR~-x04u%mxm}k ze8t*P_QX>nN^(_^@}Tq;x^W)%wW7G!MXB4_zG1>oxxv@Z!Y`=$_w?1)&^=Dkm?X(c z7;J4ZjP&8k8Qd+W2097va|RWMS@t%9k}YzTxw;nE z6$VAg@Z_DIPe4;oJs0kfiJ_c9ua8QCK!A1;{08a#0&&iF)B4@vF~gi0OF7TceJA0; zSnbkWvz0^qm#3Wqsoj@wY6>8_cuxLK`$9g9WjeFca}N#Of7HI9sQP&=Na%VI2q%qB znxwp*2hmz{SqKDNY1(JqsyR-O$MBR8^1Fg_Jf9Vw-pdv(7A*^}LD=UM#6X|gok2UQh*ASna9$Sxpx|)A3 zuE<*;awZPqaw`|N{5G2(&jKjZ`h~@J;E}<5AV;0X_rRDB;{1(D%870s9vpQzrM(C} zgC#{p-MsGOssnjyxra3Koq&AQ4i6_;dWGd8C(!^Wa!qJCYT4|9N5j+>CyetAIhv4l zxGd^!h0IWx?3DthpX|lHu^{JMH>4!PDVlKVbCmpB@PuBUKn2|Q<7BZap1sHn(K>@W z48j{Zaj!E?TbHQvzQ%L$xvIa$$=*|-l8{5sPThf{($&3W>5$Es?r-JM8(9lNlic0> zM-s^%w775X(FBxwJ{8WT&gPETWs-t+Fn&X>7xj86y_{L-(ECV1iYCfvVg+i{J=|-i zHe(sB5y{!ZkXQA|wI%U+9{-!U@BT*3!%2t{nk9VuZjV!Y^fo?+`EBK);8DDRY2bFq z-y>!}>EN>(A!FAW@z)nn(CI2sH-wHemgUMWP->poMVuYxZEs#|o@`zus$PuZO^p5j zR!mBl6M5LOX9~B;G*1VVnIQPqbJwz`?YH_YNaD=ozWm)51aBmq;F&~lMwa+SXkRe= zhKRF-_tCc+fo?5{lXt%@0G>6r5#sJ<5yy4vepH|%jqX~H*#$#&*ea@FYz9f&IeqL9 zGdfn|| z!?aX!pT4w|n(Z>ygzTd$?cJbtN)Ww;mxAD`qXzpn!(6fw$e>{LYV$XfsWF397sqH7x@9k0xnsp-FHjdpjkMpTPk*#o5D@4H z!kemB_4ZZUO@r6ZK3)jld_D;*V|~YuJ)$;Ft#@V*Ajo=AN9D$DZR9);S)GB$mzTX& zsQfMF|F*Dyjl`$0b+HyXw>TeN%Mx)etC);&HS-N;7wlYlg$y5LIsrio)9GaqB;dQq zAW$PH&RDf!Z1lPBeJm)hr=?l{GT52(h=|PXW1K94Ee6pGH=Eba5q<-|wZXZBCJD%p zpKEpIdft|NxcC4a+D5VyUi?C*b*j+|Ltus`ibq5!VMY37fl84?+VoS)?nT?y6?pZt z{7l7=#te() zH-I#Bly+&hl>OM?qj+P2zmt#ttU-2JuX$tQ@1nv2GkXu4&lIQzyj*nWfPz-o=n0ISCgxedR>n|#oI)J=q84+Z=`TT>9djiRtwymeTz4Qhmd7+0Uvnz5jjYV>E^?sNX4PgBkKZSw5-9%S z+4COKDx0R#)Iq3NrtR?mTX6uSr?u=cy=;Ulp+ziE>rFMxtH%8JOHRk8%X<&oH4nQj zDt7!gzR%|;C?S>>M%uTY>S-PO)YCS~q?%`c2?g8uZshBy=w9(^meDA6uNuor3q#Du zY!g@C*~;3esoM`%CHS4k#;8&i$|ROwZfVBlc3myQl62zMh^~}(^qe08v)+I9FK}W0 zXko21n$-?<9wnPTwdSQ=hlntgM*`ox%>LyO+V3QtD@F-mA4MjY7OI%oqKot^o8?6O z5mlT|Nh-aH0T)@#Dbb`wN});h1Z zIyD04KH5T!d+nuSy)xIzdX#KY;M=cObF80fF3K9VtAA3ivYH=@`sUM$*$-Qp!^df( zeFgc=nEz#=!v8yOPSoE?1-~AW3j3GO8<>ygSr96G2msV4b8w#TW^{W}(ff6GSaqXK z{`kvHY*m*YT=9O~S4yq6>es|Tv;U4Uq`WvIVLVLp{Qx6VjirZsBK}>f2iu>h4rFJq zzKKsuy;Km);v45~B9yaFB)>7$E_lhaOJsCqop@w5mx+^JBA&+-H?n@xt!Q)jgsJ~= zFr5@5&>OSwhb406bcO^t7hz4Y)60%jLJLWaj8^pWOa3|8{8eW4AE~v5`G|Wuksh3l z;0L`8Dl#$v;-mkE-AL(jBYMnlK7=%z=VApaTr#d9@TeXZ6r{5zhbkUMAY8q6&nUc> ze-_>xrlWU|k*qXs;h!K#f@CdCEje&W>npe&VuOK~$wJ|&PU?=9>HglV_?J>uKXaTI z%PyRTe2;PD16v`@eb*6bu55d2U;`rl%qq5fw>Nk142hG^${_K?F4nozBkja< zw{Bp+{{f@MLD|Z}XlN0LipXJgoGp#_Y!q2s_N#IyIVw^#mQw6YS?l50Qa}r*V5|)<4M=|yLdo3QO)7)CQ@qGkB;-N1!42mU8cgBZ}moq%- zKBjH*zYqCgcDBFT)d~zeYi4XD%??mbOvmz-yA;R(-X?obw(a}%7n28ua{@W?)d$3B zUL7Xt^JD_5I;Kchln)24G8*QKHZ8Xg?8(myW-0-GD09j^5oFctN!r#uo$K)S-)&S- z2QACJuvum9=>L)Zz^aY6MEcOX7U)6z-BxfOr7dhna|jEHwRmoIa;_tK1kJB87C#Hv z!!=KfHofqSoB3$^P|j0DM?_y;ag+3vmccTEjxCsp=1y#vY5#%zR6u#=@JMBMcCS%S zEXDmKFk#Ug_FHU*+kmNW`e3KyB8i&Uz@T4OM9EAG7|ees*09!gZf-e4a|Ld#t_0)I zR<@LOc5P9!>cc@kk2B_DHX6;=vm67e6V)&NmN%zzn0i-=1Jwy28k z%`d2J1crPPUue(Wt>gpxA2B>kzsVEG50j`K;;OQcH%`!VTMhIsL_=q4p!EAf5Ki~f zG$o;`9LF9W4e-Dc87}#boQdLmaDZc^pC_wyFW9YC_N?Nr2i4 zcSRjWwswA*IFlB~0=St+d-dz2D%vA^QPF)w7gH!}X$Ki~dmThUeO&W=+h*Ub>TWLA z1mK=SzKt;kyI7v5LS;6q-THukJrnK_@8gbkj1Fh*MlR(x7S+tv*RE#-1p)-Cs9gKy z{bSyKkeIpjD9q`*NPDCJRGz8%bV(mfm1!JitUE)NOjgCAxhBfBJ%7BX{^16I3pIMA z%g-X}&jo@lCH2zLrm6ALKRtSUWp%-vj#l5FDT%zkrFQO^@XA{RT{SY;Xy*HBlChC9 zh?ekslrEd6Bc30I(clof7mP^1rDdVUM4<7ZH{(;#{@cn=N8(fB_PHsIEb^8pE08L^ zc(X}29Uz}e^jx4bk}T;npYa~*8LYrPp&^kgj8d=~(d^pI$tTWy|Q_p zdDb`3DDr|?{rw5oK~iwbJ5|{RC`StPfN1*~mY1|Np#;fKJHQFpGwf$N35zQzVfEfa zZG;Do26_M>!IjIfJonvcn@hR`Mzr*)^P zG6=kC{ zr@9Ga>F58zvzvcenJP9FQ1+T2mc@H@q?%QsdF&?G@ZsNH0JV{@ECiTURti8?86 z@9u0bLs!d;0>f!*WIQB-lyrvNp69W%2*iI$|EypX!&*^X5Xl0~XV3h^m7=44TzN3i zQ?PG)lf9nXT)@)8K^>&xuHQf_`~TT-PwdC z@OHLSq_cDod&fM-vgmtO`!gPjcf#0Pyt)Sg{c!7rMP;m*sjR}5_#C7t#rc$3U{$wE z)kq95zJ1#E-?61)c+#IFaox#yAog=B*t!o(eyP~DXj;sLCgGcZeX00>F+Z_6CDp^2 z%Tas^s)5;;|8wfQE#7tsAia}VF4^OJqaCC^Lv!XY;5p%Y;Izl7di29cR^xBfLl z^2lav+Qt8|b(H~8rqP;G5v2q{8U$(SZbVwTB^42nl9UcnMjE7~K}x!7=LmasnWGKnji7==^9?$I^>#q^sAX$Bk-0Id7ALG|YQBSwH);?HKw z?bm^_3wKzZ#hXl>k`EKs@DVew0n(!IA#+xAVs{ka=z#fhphJ)~u=j}dP9wJGo^KoA zh+39lhBvOouE(1I6Tw_32xTGA;OJ+WP2;YjXAL#QX;nuCFXq9CXyn85^j)B^qMP6V zk&Oqg6xjPqUxb)I42-u1o3{iHCG`UY3R;#xurBKo=J6R}p=D!tfkoOH`@o*~rx^{1 z+GfoRl*N1iMe;@Dots0^8_w;nJ=zzs+m-Vo48*`9Uu=!RSJ1QhVDwSXp~ECx_m)J+ zFVZ9k!=Fe0`|GKvjc?r4_abUSfq84JR*4Th7>KqvT->`j(1di&&xMgOs;kevO>eXe zqHIuq1-sw)8bkM7cwLYhKdy{{`5E1DR{>K%o1;(zb{T+PF9TKl^7ydJ6;-XyJBR-& zdB{0~qF8HYmE=qIgKm)a)WdsqWH>y5A4cd#SZN4jyMCVBshs|oNC1C*6K6rKjvl^G z+o0sJskTvCLvg(a8qT0UBLhZb89ad!)7;)bkUPgIP{;Ku9smz)Ujv&{6&rdC;gA>t zy^iTigiA-c(_~gLW7o6fM`A4_jvy+uJ9JCB`Z}{!Hmta@?Oi(=I&dViE{;7#-vtC1 z377Qbs@JI?^4EEK)g`wj_Vs+POY$*M!-Mn1nz`cwS=sOhpY%_O-fSoD3BKM5#)$O+ z)rIsYdZ)bcuLed;c~NO#fh#qTnT5wpn!R~g3i!RNlR$@|Qe2*U;4rVWpV9f5$)ula z(ipJv=g-{D=bTD8vy2KFTVM8_V(hZ9;-{hE&JUP5fHE@LD!mVS&Z)}$;)B6(!Z@AD zYDq;o8SU5S+ENb&T-a1yBkbmDnFMV`i-S-jx4R!M%L{eCn^mxaqic41`bJ(jKln9= z#+V^=!Zl4?$d>9k0nB%&pQusbqpaBusGn+I=^rwj*6dt8hY4c+r{#|be1uRgPuB1n z+zn#pBN4?yUa zZ=1IP9hdec_E48s`-jOl?h+z=XWJ|~tBT=XH}-{Yv$+!=a`Pdkpya(TbohwAWgvj^ zqotL#xlG$ZleWh7s-2I_GT*!)YgOVx~|<0U7^r zEvp0Joh*^#?BJ-4p#P``Slrv!o}qEiSkt=sn7W#Tn@sNk4_4kf?~iaqL&EwQWkv=^ z8`*08ug>V23ohQdsiiLcrIhj)0a|dSwJ!M|`dF$?stsX~h^WNw-TTJgp5DMs#^Fl` z6VY5r1$bhx_82Qak^7{5!kLX4;H?ce)AcKHCP9im+3fS~W$;i9ODj;_i5THC3(V;G zXMp2x$pNyR%aavMH}?;xKod%?A!Bsu>U6%r84{7Co?zF_uyM*D=kny_q4VL~J&`@1 zrR3|4WDJq3K-GnaguY+suy-lWqJRO|Yt(Q75D?sTSq|pSbe&to`b0ms=f{g5 z>b^o9fkJuSy|*fXPg-NNU>olk&v`>9wye3yl;vTRF8^sB)(APGPai_AT14pVcTL)O zVFXb`#27Rq#vslbh4&H&CnkR#?sZGnH@O4!5HyHg=v5PMKxB}+xUbV#J@*57>09(2 zh9ui=E(dP#>Cd_bo2uTJdHsx%t2X-pG;^hEw@P{GmocjV3M3#du%G(j_e3~ zaTRBlcK9zFb1OCLv_4jqvpfvfsg@vXf*Tpo++z&;b-LoU))=iJlO??R%L?hqkTQ2o zLQ(T^pw#aPOLOROq(^=Q1}ECK6EP(5;yvm}Lh_rR9f?&U^;*J9rVkN5nZQhb zpm_iKdhINR)PU3;ZqEKIIv)HpEpz^OJqeS8UjA(rcBRfei>sPP_w?Cm3K;G*0zEwj zWHj0^mBHn^q6^5_GQ=@vMxOBDWJThih&XfqV*W;?t|DAZZl~^at_>0C9!Uw}pCk40 zX1c`B&+a%?B6t_(_K4dNwe9TXRAzeAvO;$#QyY5+X+$GBEpG@f>h%&~1N|I8YD9qe zPN3lYdB1FNVV=Q^^X3&00EpDFd{=D5?AY73C==ZnsTDc+f~QD7=qe1vz8b+avZ0@; zwo??!&H;85r5x08(ZkpQ=K7QJ!e_ENp>8wFkjkR!RJO~p%KE6gO{Gc4dFg|lmAR+D zu)I2fFYKU)mGySn{2fUV$t?n149orSPXgC=u&sR-y89}P>ojh>mexu; zRr0kaYng`1oZ#7luiC6ikpz1v?~V$~eDNx#7XlZfCbWa-vCEc5m6_$Kdhs~2O^q}u zkQ@=entRnbedofAmQJe~Acbhvi}T1kgGl0xcWxnbRPAHKn%xLh={tRCw*v{V)kE zSq<=B^MhNqo;HAzr{ir_qLW;=g{9QKR0H5)=w!Pvzw+?7ItSD2sdO$YxO^!VZ`MU- zceg$r3iWi@@3Zmb^w>E$F7)!)2QnV2whip$D@j~d;yJc&;udOm^JRJ7v6&n{O>qB( z?Z=C}G)9XP;9@B!DRJd_F+ESP3$)uIyqU&&4r%F~ratFmT=BAHx@6&%cyA|gBo;&a z=p4C3sDloufsLPP&9wp%@dM>DzZcd1a|B|7Xbyucl9*(Tu*v$w*IZfwkFVm4Hxdvb zu)J7+930d?@#rL*YfZZAa$rWuJ&LN{a&{dSm+DkP`0Rc4NPRrywW5@_NR|jb^j)>$ z+6QarNw} zz3P8SFAyKE(Gu_$6jw!zErAQ*ci)ppqgbZDnbrKP&Bulcixda)?^G>A;;)_Y7639I zO+A1oY2&#g0xYEZwg=Wha&tz*YBK?_&ddfJ4v2~}bTQ1@1C&jlm6l32+DK4EfrOBW zQMQcJMSc!Jqe6%P38@yi?FSm_)xz|5r(}@X4AeDQ@d+5#j43X(RWC1E!AiJk&K-1K zy8vC8Rmu66B`TZ!G=th%8|8bh02)Sb(DPuewADR8o22J9T%X362ZDigB@BRw0)g1e z2GeUDY;PePcQ$#^sNW;G=$fq(dCTb8|KwnJj}|^fW6dnT5nS}e7p7%)%87w3=z~Ts z35JkQ)E_vlB7T0*9;G+AH*tcaMW9JEw8WPFPNoh$d(^1acebhLcb?~y<&jLVY)L9Y zDp(A94)1u#4v-x&BuT>IGvS;e+da$c7MnIl+;wEi#(@_8HZ6O3S-g)R=v%x?{6^UrBYpz8u>TUBLa5a*96-T)MP)gh4vZ| z3@XnJ_C6D)E1ex|=tnLqySf3DG;o*h(Zt~5VUxRSj6xKE=zUC=?0(?)^UQ(G=3rT^ z+EsAguhS8W=N(W!Bi@u3LaN|!IZX!4{i zOE{lz@e7`6c=)*_p?;(YQ7=BBROZwMM=m?Wa}O6Rt`kgepwSni`l z8^L=QD#mABcRbbx#9+|{Pz9gbXE<##RS<&Bteb#mA>g8O^>!MQTRPr9Z!`+cp_BaC z&KprACeAG}va+L~_y1%zIp&sFAo@rw3P_3sn}A zUYO*$`xJtPOCg^9WeIKxg1|*-S!CZ|iP;mtA<;<+MPA5syvH=Zg~8_J7Ra6N6Eq?^ z$P72BD^7^A!~I~TEdv*5)W%^o{Ya{%G#DtSxu&`9QN-5fhhkFASazq-(6vloDW@A# z`&S_nf--{d=>6r)Uomc#G^>Q_+h_*``pQ zK~4P2s`nru*G@_8mLhN&^aHBG)E*dTL<7!J@P{17DYJW-+w}U)HVs!NYAVrxUN-HZ z3rY)5nO|Gzlo*5FdXYY;=S%3Jea;3vc#1qDMm0Gdt6l@y!Q0jxvHv+$6p~!5iV-Pza<#5`$>*5)@WgA9 zPHHK%eVH0k^moMpES|DMt~@`CncJ{@FIV$b!~oaf`$+Ean<61Fdz}6Qgfssk;o?6q(IwaUXr8^ZIR0pX4SyDCmzN-&v=`)Y0h4v zU|4YBJQE;-$ef3=5q~i`eC8=f2o(*QW1;_4AKu)K-TgJvw}ezm**2HCxo89i1gmOx z^8g&1M%EgC@7ws@M~kXFGhZn^gdK&1#~mZU+ogCALk;FV+W#>;b&> z3>*0YU$XT_wEUDF?UoLR#MdgQ?EAj!w^p9wSq5ttvW)q=Ex87dbJlb#zP#wTzbQEsIq1Q z4z#2k{KY20RM5QIq~5j(bAZ|&H!Y+@6>9{p=}0djT&`($CHB@TQ`-BYsqQDXZD>(d z&(3fe9CGw}ye{Fq;{!7=guKORk8RZdWP$(wn2azdZigqUCxp>GVPD+f(wH~e{p`bo zZg1`GN+9w5fsJBv=xnJozdRcKW_w)%>|%zR5N|0R;liz7X?u`6O@8qA2r;5{U5~q< zvj(6!n63v%q&;c?I{H6G84-q@9_?(uMLZvC6e{)hq87I|wK$TiJCa9gai#3HFq7bg zy&$lVbm;Jc&zuE#o@opI%osZ}Cj>C3U&m8cLlm`^m*?yv8S!D_m#E4Km4{JlTXJBv8^aHJIq4_fiZ}G z>klgmrweLI)D3)8R!{do!K59E>g!*s2TTn)qN$fo5uXiNE`~9FBksY{NM9?foMhN7 z$nh5sj!{Gd6zN4Zi10}l#4p;U?9?-U<#2%UMP8*E&p6wzVA^jQcD^}>TKZ)qJGmPq z&xp#NN9e_mr}ZjbA2~9tnhVMBhdL)w6bHExLH1|brq8Rq=+UA9u;IY6I_-J)BU)Y# zVtpk(8~96HHcv+F{r$N00uoVX{a!!?{}llLE>`$hKa#mw@4PEr-!#~_3D*4ABLm!s zij<7hGW_D-=XpGMO+X_8c_+IS_4E4U^JsC;*j||%D_^GYec+e9OtHAvhRO8lyG2(# z_X{~~l~@HBhIw~yiru}oC}dfw_h2GVQtXB)%LkgR54~<_>+>LHm+``B-s6RBDnx3*HVgWvk@C(fXhgv zkavG*Kn3k3)yD)k`Q#_>)V+dDWjB?o5t9<#K|vPnsiKdCzGeQkDBxceB%@;VqWR)4 zeRuwo6aYM57(SOci4-gt@z6!j2zNV4OonqPXhb2JKfiW!F2Y5I7uza@V#K_oh|72B zUfU>x*Wj`5k{?ufeM;Q6<$#R9kZ$*lIekkeAdRPHvtn;MR3@^u-U?<$ePkUS0rc&&Y z{jp;tGQa=@7sCU#ASCB-6weZ$j+g>n=e%Rpz4~+w0AWD9!{r#n$3cCs zMS;``8qA`R92MwDnla<7AEkRPtC?YcyXwIq9LYW=vU`d6h4%a!cRrczRoG88!d*#& zENw3ltvHgM@W}O#J$4i|4!b$YztYZ^*M3JHU zcabLzcu@Mm+Xt;F>O6)?OoK*k%feSi^l2NBU4%wLcU(5_T}DF|xiE-TAz?LJ14CnItCjUR{`DW0wz?HN)j%`(L}>f|SI%HC2{&E^L(X zEy~`@WXlF97znoSvYKc$6AEr^`_P9T5L6(hH|`9u!MSoDUv>Xdqzkf5y~tQ&5H?iE zYPDXKGi*D^HiBosji>D?4w;vc0^e8%0F8amE;dEK;Kv1P*Q*?U45EKyJpN@UeKvsM zdk<<1@0+XsL+PCDf`RH`nT^E%4r&Xa{d_9rjyhcA+psb69$YOa>)Rqr_Cj9Px5)X7 ze>M)WOagG2N+@|`ujQBEFV2MM;j-bfux~5wbA6s=0iKeE%L*1UI?X!cHUJFwnS6u# zP3v>{Cu8s3QRsclEoC@(EO}^=A`qAtrmsTyDAU*KY~?=(VD?QxcnpjOequX|Y@C)e zVRX?vy0RaiD0KW^@ZOKc6|^%lkxRm>6)4}mf6JW@5zfNL;?2hRk4wde@6N*0e9f@P zib>xwS|&|xR@`R3%;;uT;Xfb4?pxWise%qoy)_MJgy!6kSHNo-|xeD)MBZ;3=T?%vf7R_Jb$61D#SE!!g6gc1NCv$Yv{2!i7OOB@HeQCu(lu(IvD zO0R{8$q1y{M5;ZM`9eR-OBnN94DqnTun{_}7izQsjN_&;-HycTq2oYqVhRKcw?|f7}ZqVW`ATiGQ=DhcJ4S1x$I*56jw%<};2Zx;<3i z9|%Aa>scj58P2ITV)QwV2rXj|@Z?^dAHF`S%&+Cf;0?wN#-0L^Sa1rW?5(LaJ%NYL z^@A+kES2lq*ZY74qP*DPjGX=(L5)h5Pe|e7;tx^Uk=3xmkot*DjfpU&f>oBo%(%X( zKK*XS#m;~!h57oC$`a12NNuTSijUQ>Ox~d3H((7tSEkPjF=GymWJ1Ac&w9nCrRu_2CpG^(G0vqIi9$?ReVCzZfS z7rSqae|qfiw;S1_mCbefqe^3Rr3WW2y>5o5mZMn{W1$d#8X5|%8<^-wX*YZfInwn2 z4V!MCe`C$P(daLlv(eD6ezRUJvY+)iw&pz6Xoq^mB$y&q+`~@hWk=O_PD=O3%lAEWXZw`6qPfXlbjxYIz}MtfVNXMg%%c={?Jv zK79!i>smVG`(#^a*6=v-jYo2?{jpz+v^~XhAk{6FkTED}(#xezOS`#w|Ie8t&KDoe+3#2h2^;#MitJx^lJ>$ornL45B0oi+ z7QdFU{R`>L#Q6Aa5TU1QQ+~|*j}gD`%>m>+M$WH|8)H9;KSq{MJ!C6uA@_+MoDNX= z^s-;Ctlu-t5>_VU8w?dRh@26>+t@}1&36n0LAo{L?mZ|ibv zolPBm()W8^(%yBlJ>BD`*6*DQ)*|x#&{95bperK1zV_K1roHC2BZzW^I?edgi@!rm zSd<~3vEle%NB;MVDayv!FfycR%J;f|Xvhxt!E2TFD%f48l7>z!9qx#7QQg#iQBn{8B@UIB0KCe#tJit<3D;%sbP7kJ&7@{ zinBZL?HLNappym<6v@kutGLZZUbtp{Q()>Ruiu1u=emxi&Ls+diuNhaW$zzH4$qR> zC(nbCXt$cO3VTV0SaSS-;-m}`bGd$Py1@14=LV#V7VYQ-Q~k=T#H;p^WR~finATdO zWpOdqD15rvcfU1f_9^9>NWQ$?fw5H_N|(Qch3tyj`uxa6%KoSxET(|d_XWM{3F+tlF85s+LprNH3v)x*r3S~5tH+NW>oxMEgG8sq1_$m8 zi+E6dOeCFad|_`pdbZ+8tIm(HS&c>Vmk9sM4&~9I2+i9pQ4M2*VEjlk7JocQ&>iHu zbo@0aO}tLsFB(PSH~Rj9EkLW#Q%L*o{<`MVjBZ3=t_>grkt(Tblt0RK|S>%nZ zYh8q@o(;Tp%;b|V(y}TR?b4K>qnfXL_6xBK&V=|XuZ0=WT#-PHnw$iz^0oKm!!wy4 zy!#BjRvP1$o32TCXPe~6ORn8SA2r6Ki^~Da$}vd0^OzsiZXgftnW~UWv%`zRx5%X< zxsbjl<{a6i4tDG)#Sn$v%Jttub!#b&rBZ}o`>69jSW~C}uxCE|)Qy1;>y-1BVAtQn zg$2b!2wh{-2>CTwZ8}5+W>67_(8z|x)5GRJtGSK1dhO z+Uc;K1}U87kh-?}Xn)Fom)tHKDF6>XzD^aLGF$pdWfXeX6634swY0|Pg7!iG!kOuU^_D%j-&EV`1RNC1Q~P(9Ygn@bmUC~ z_PP=Nv@x#LHA>(dHyNWVTN;K53w#H81<0kbhN5SXnnUI_?t;Nk%w?6=%2B1_sd9q6 z-8X4zn)dezhIKasR6GyK#x)G*(j+7H5y@4MaCT=n9{oC!-^la3uS6WVRoG4 z=&n3pnb!F=Y`0$h>x2J(QS=&(C%*ztVliKCp+y8h|9bom5Z(O>n|TAotbQBNk5eJ_ zJX>rV)iTsaj3<%spx+`j1Obd^{xwr#3*F`nus(ygdr5g20Vh0}G)>0TPXu}bmnl_~ zeb$=MP2y-b43p6!K-Yn=vi>+f&=*U+vft?wK=~v*p4snXURyw#Mf7Lz z@RtX0N8h`wcyv#U9DUfg+v}pzi)jv^(tk{G-YJbi_E7j@a)8C)q$K9?Er9{hmucWb z^a7ScMe6}nNXG0^Qil18qf2LK*Ox)8mu z09KC;`L-V55D|wxcUoUrHJrGkzEE}sl!pc7&kCK}m;w;&_U>Vdm>*e$ zu8xKS$ANJ$^Y6=l$iq4&GXL&|q#jGs4%F467ailhjr=dW9SJ4yU*0YnB<8>p?(2Vc z@OsEb;q|AL8M=@;E=9Bzq&62^js%Y%r}_Ym!8$XOwCE0lnkc&2H3}^9$*wEbo4M%2 z@KwioY=*Zzf%bI{eN|X+4vGX=8OGZn9Ll8X5vd!!J-b4wS{t339GQXs-s)b|kHYfu zS$s49)wjmC=nnf2>oz%n{Q+F%xe^ZaCL`s=ELLoEIzo|#?)z;SEz=0xT}dBl+2TWI z0IP@((2{QV+@>HbP*m{uX{aikReg&#IYF08LX!IXrJ2#?sgI?Dg zos}1Qldm1pFiF}8{*G7yp&LX& zKH)_p>!m(?6kPQ#<5O|0%}`lWd`Zkm4|-jCSj5*O-`y`HW(WuOgA&CK$H!g@XIV|B zr>x&>)*qNxbFc!po<$C0kec2$t;oS>&rQYcLMPgis z`LWT$wU66h7vfqn)c&J1DVQig`8-o#GyM4lK01qjw#;=WmSduRFTr-C=;pgX#SIlY zB7^aIFc%1^pWIduv&nNu&D=KvhqrD}&yGom31X$nb37dakL z3;YDE?`Q)$Uj~`GGu5S+F7pUAWNA3)P`a zNq1)3`V$j?;V_Nb=tSV2dccUi2!x2y)_@M7RrWT3X5j1~*8|b;yB)2)jD7pqy`O&# zAJ|`UaZ3cz26R{_mXdGIljU_iSr!3J{f3`Wm$`Nf)jH?B0eai<}CKw*OFVXYI3XC+dyHLOs*6loGbYu4s9!z+y|tW~0b;Y9X8Od=5BWVh zt#t8g3gDSnaeUh^GB4G$xXSf6me|R3(vLbvuYU!eC_r)#a!73_aYIWW_6hJ&KRvJ` z!jAb+Q5BiX;1OUi@{R5f*8(lAn_?e4S1epgEbHq%?3*HoBDbao2FiLs^NhssT@ynX z2#&z@K5GC#3=a~J8-V=>w?hy--4T0(AdxZ$@|8ee3V_-p@D|7jEGE*rY!SVb;1Bl# znzM{#_Zk4&hp7IqpQ?NGkfE6q-n$!q4KKU_J3OHYmI%f6SeLk^Z-Cs# zxhLlN{tSTwEp-PF3@nNF{f!RO>4dY-06c`c=bf-Z$P@ChqxP z8{PO4ccDoQ3wz1+4N45w+!2-IPvuc3dF)lYl|a&1hyAI~3AdWghEn&G(o|%uitCkC zqVep|`CUB@)(nMZY481RXU#pWtLRS(lj9pnpL;fJWK5HKB#i4UAlknqeB2^U8O9RS zlcvCGAc=YN84%jF3q@@#-m#;|QZ8lRz(}8VDv;n&T9ldBqMKnRjiMF+k=o~vZEA@f zt;}2<5&hf+KfzoFx3@?l;N($hvo(I#JyUFIyc~L@@A5?=s4{B!Am@O$Jn($2uQaFb z#~vw0vST4uMxr$cA-gFIC<*otH|Gs>%B+EX3WNi|83Z(($2WAH_$GHg+Z(RtyOC^m z3FSbHEPO7!-v8PVKfColgL+ovVoObAqD?XwR0R^=dUGe_v%AkOFp9tuxE%QTGLKbs zkx2k(d&_my&7UXEGfbbg^VX}#k|g*(-1O|TFi@-Tm5@%j-T`a^J_3=y@lECm=zwZ( z{E(5t5}^bvG(aR#f4Q*)EDp;P@_tbWQhxVO^v-60HsM^XP2# z06KK$V~wpN^KqaUI|Vh!^B^*oGD+`p1=iWcJRdvo4+uExST@8XPg2D?Fn${p=-tk? zCf50JcR3pHlZIVGO8d}}o!6P(NXA(KN44!9`6+AMOP!e~=zF!A$M#LV$+;5`l(R@5 zZQxo3VfnP0s!YeUD{O--c~>g>zy|EX+tRhCX=D};jm4?j8Bm4o10cq8>WZ*e!;dL) zX7ijdh4s(kNB}qT5IMh(KWSKHQwZ*IFft0QdtC~%VEP@;{-8yH9(E0a+X*j|_Jd3# zMWZgdlsxUQr5goj7O~m@5P>?WeJ^_(UWxCS(S-|vvZV=)Va z7`Bcvyt@JexgYPQM<`C2F&W;pOb-1MP&~v&9&^TGJW@p`_ceI5!Dj3%{KO!TkL$e> zA@{r}$X>AHV8g-R-elR)(RZ>F{A9OxkiB%!RdbM^=lM$8WAda$0NrRFC+$3)kj! zXO`p!KY6PEL(Eo9$#N`SmL&h3lK%K0By=+@9O59|Z^hC^d7V_sLZi4g0-7432ot|7 zDK>x=E*+cH<{TL{ram4e4E^!HL4l7gRioz{JZxqOH~;SOi#cF?fk2wfafJMv-9pCz zy#D5Dmz8X}U-)zelY|t->!dU}O3?{1<|7xI+z%B`WJOwC2cXtWUT;sKtX?kXrA>S& z2Y4+A%7$)`yKp0!!=QNE?)A=Yv?$CC5$W*VU0NXYmssEh3@B9T+}>rDdWZQM<{#N+R^EOjiw0#W_i@7zp zX|*!&yxQU0@ zEN`b#D;o<`;9#GTPKohdfD0@yP^iv*-C&mXPgzOZ5X}<`SLnRqg*93b2(2YMsYzGq zoJb38SCEi!PR;@Q@mR6%9GBJ$$t;^mC)}gFUfx*~y>ZE|Ia>zMUJ%Fsh#J;m52A$t|VQ?S7 zo-j`P;ZZrQXNU6GQ#R3lKS`wyhX0~)n*w2C`Ai&|h+}=HO29mqT|}(gLXeDgd#Vzc ze5`7W!2+19ww6WH0H=$f^Ut5W#5CVe~ z>Cn`%?ADrlRAkFePUOW+9Cmeu7PCycHC-1zH2nzf;IoO)<4wA-iLwByY|`H6Qi>g~ zEfyjB2_L}7$ZU(2iHDx6u+-ksEutktl-T)KQYlJkPmI~w?!uNlD%wnx@R0io0d+IL zqX@8$(`u^wr}C8fr4e?07NtK)S`|LgSDxldXy|6R7fV+Noq~x-JLPc;WAvKgk`p; zAwsac_fy}P3qQ>m!sC*wcB`R_yK(KoNHBJL4GkkjU1=1;Roet)TMW`iSX6|f>1V4c*V%-HsPW!`VkKh+;5_EbS20;GSJjf*VN-t$Kc zCeiYcY8J2hQ+MDTe>Xjja=l+_z|3QAIPB@5qd6=vJ)!Z2HUWZaSR_CEmPi3U-I}RD z;ioqjHa&K)-IiG!@WDnu(2L$5Fn2%LC#CZUXm#%;juwE+^0tZ8DDg^qd%Oc8T@SWV zcCX<%VD-&aM;#rV=ziu!`e?;mIf>7krzsZ!Zn*(JPIE}nqIk;rar9Q+-D5e5u(wB1 z^l8*zTK0c1a{KpPBqf;-gtJl&p;LG zLykpmA>$_x={&c8j2;ja7ty0^$@1>wG`^oLBSt%WX#9AevlXEuaLJUNl%FiYnzxSS zpD4>N-B;2;Or~Jj@o94K@^0qap^psQy@+-1q_*&w&aD2)fFb~b zxJMr39O*#qu^Eh*_@Nr+8mIaVHzUM!cG!;#zDj0wA(WWERJBb>WfL1;(Q+-0c-tl6K5PiDR zhLy}uTW|%Z)8K)dlk^z=kMLP4~G&77(^9dtJ9$3>%@-yEu)<1*bLO#SsDcaM6t zCd}s^bzn^nprpL}$DG_+%2D(pQ-euDq;lHS3R=J6S8y# zwc)of<$;$PP7rB>DBdYb-g5av{=Y=%>M2g*VHp3*KyVJ9tJ)3vM`t3kQ<}hzqifw| z;@{@jC>FhFZ1b6$!&XGEtFXq>c{sNs>pIlvC25_xFz!N<(PdVMr&>LspF_r^*QP;` zyNs2CF7Q@haoH-;3o`ThNBs6>OPvye@$~b zBKCzO;fkop!EdTQ9|e7yjae%3x1D5Cuss|6=JsOXkq|GAoxp|(XPIY}@c*VcxBa*j zRy1d+`LY0zEI9`yvC(ToG_I3YQ>vj>>sX#L-h9^f_)c@8Y9W4o{ra4;g6>pzM6}i; zE(Wsucn1qgvj)k~ReYc?|56MT_>up>KMVY0pac;oG@`%4My|sF^%N_)lZ|~y5?)z} zIXKhfsAJkC?|b3x_CgWc4(=Nt?Tgg^Y$k#)hW;!`59_3$4Lyj+hk@_@!K->irSRi( zpg)7t32xQt$@vB@=h>MkP7|^jBh6<;E6A}b_#}e1qD-gA@Ezu?TXX)Jw_mS0*fg7` zRfH~SoPot4YAh3Ln)qMkLmw*2#?lKpQY6rzOXZtm^u58m{)P3D*5Y_*OFhV4z(xXQ zkx=U-p}>)t9>HA9^!Ok#JhUInM?WdZeIeNQ^MiMc-~@@0(nQCKeQ40ZQ833 z(D)}lj*{LkAZ6)&3L2VHmX|I%ZuEVD`)i-Q0Ejp~+E~6xIW>sTG57fp6`Lm&Tg(91 z6|Ir`Zk}W@rEn7h)NXX`UHzZFK0VOLAjKiiZ>XyBLLb-T3J zW)jmTi!43BpEZBlm_QX!;s9&rwo&|A*yE_1o@@v4whrDAox-pr$UeMuB(nQj$Fa;?jw3P)=OY*y&$AB&r zX+`-OI-DJnS9opd`vPaQjXPNXlO>DPabc_Stsb~wK<5W1R8M-EKAAh1743r+4BRj9 z5W=+?EkSAif%QUo&{biO&-AuUR#aH$t<2jxZmQBgp z1bFL^g(08Kc|Wf(T9wku0^#{clE0rTw2BqF(ox6Q(U4j(zI|(ZdVRf5XdPY3ZG1~j z18p}5bmAwdGq8N@8hzC|oTFDx zS5ih8=w_GpHf7He^*3uHs@`>f*UPZcP2T=j11J-ibsN~YbenJ}2y}akI0$Vblta>2 zN0SprI4ZWbn<4kUd}yeGI69+*mQ0a5s?ZAeMP&9!3B@SJWfLR6$&<+I#8zU=!WUOKn_f`%1fpMmj_!iEcEi?XoB41Rdx6rv&9ipT?u@ za0yC|?Wi<9MW2PQ1|Cu&;vNCugl)!)b#QPan6l9W-Ms$8w9Ss!=YB=U_9^xG3Xy3b zr$%(FT!GF3ioe#e>(Eg>?8j(5%cfHC&{Wc78cHe8qkpH|(2i5wo0%04@;PtP9$g1SLh+&QiLrIX> zL5bfI{x@oJw-MTO`(l@>6EHig<7z)-g7rgwxP(!}$z(gA$NN+%{Nna6{ybUDXa5;YuZE z0X1AdHk+tbi9^kEqmLIJ@pEA7lP0u0gHhJWDb<&eZnSRLL~6}?{eC|i3V?R#_gKn~ zm=7BtVF`bi9|S8Py=YxtG>bU~=WS3mCXub7@t^w__SBVaGIiH6-xKA3bC&CwThCZ9 zk1&+dVr!(m9U83yUBrwamy^s+wEX<<4A>oFM|>#lg=nfGCT+?ZrsJ;VARh6u2oV^a zZ(Z^hv2BbgTD!k@(gUvDzUc=ibvOy<&!FiUu9K}?QNI1}(Aq?-MQVGHl*M$jm&9Av?KI|Z zIyz;+k<0nvt;990lY%9fIak(0!7{EQJ6nHRAbv`%0mwLgtZ^3Vb1^LOb0LFT8J~tZ zVmRKRX|F9eVQ?USXAZF{D})gm%VEk)mzd)NfwP(%f_tf8+ltS-OjqmjcGM02Wli>N? zLu<6d8Jzra5~<&s`?~*9F$L?SY(!S95^%Jt?-k~N?X`Yt3pwNOHpi9j4zr&8%+tj% zD9tITKtLC`0E%>+&#WBt8<)}i82r$Y@J;*<1D7IBOg!WQ!gLG$4j3dZTE!%*#MqTF zYd=RQo6L6D9lfFo1i1xprvNRMzK2zB%LfHMXUN``#r4XOV#cMbYb^3U^joj7}w1WA9sP9WLHx6Ptozv7M z_UGM>PLAfK-IRaBCAM+)8YnAUddA48Qi2!qmg1)R7&|s*sZ$h4p2-u0g6Y>FF{!y9 ze6hwC%BcN*MKCM6d^b*_?+^$_IApG@E?Ff~VXAh=LCS(!sS-Cdhg~PbgQRDhuTPt@ zpGVi2Zjarz27noSkFbPu=Qj*G2}h1kR}IE2_>VFYxFT@1(U5oY*H_k(clFWw&JXMi zdzzH6aDT&sjAn5b&iRG$bh+vkpOmn!n0~(hbj-z*<;O7@P(vwS#F+3F))QFrz=6Wn z0vVF)to82cVAnNlqq-ohtr!KmS;4#OJnB_}@RiQkQ!a|;``IDR4e!-#yjUT6OK!BS(6Cm_Dd z#xzU1ln*lr5G>@K=RgwEZy#pPk6!etxma8g(GM5#4h9{R7hPLhKkM_6ZVOKBXq)m7)xk>=~RkGcQ3|yYm>)qcevUIopTNsNEin` zgZWdO)4}hHD!~3_76n3q(k+NX6xpG?mIla@a4{~>G%EF^`Ito867#H;Qdv zBq=|?F)4pmE@YtO`GZ4>-+>$($^**AYsBdpq+s@c0K6zVt_*K=H6J!V{}nUYdHc&( z>0xI-rjjb6EKoIUC)XhHn?Ry}xU!|*FC-*F3KM^wP^RaalbmI+#*P9cd~VXGO399} z>(2e7x$g>5Nk>1%Ol$NjaJ9A2egd7hf+wM@7~_}}t4Zr4K&EdCQIG;Qoc#1nHaN1! zj*A8%DS8ee>hR@_+_&cHWj2#bkj}Fck;%AVc&Kdy#>GiTphGgrQFeD09Cu2sdQEvbAK;-YyAnzT1=Z9sK?ay*e*ps^13s*YR5_;sp zK!~&Ywr@kfV0zz@yIU^}T-7h^@I1}RkrghF%hPObp!gtuWxJ4$W%BQ8kR6T@X7bgS zs)ULv#63PjgY-S`no2);MyJpz#XrNYfQL+QN5gft2sTI#`jDDbz64rj~tdL3@X*zWO>U00#hFC65=PCVw(gr@=J9V9^Jf6YE1 zAEF13G&bfy;YVA+`NefY02iQiA4~e|7{(S@ChEVeVfU(M6qGpiQaG=8V`@4bvvg|R zftpPnZK?i|j?al)h@Pu$MuzQ?iWXgr-?cGXfpki!uq0`kBV`A>u4#lzHnK8AafD*xAh~em0u{m~?Cq=@HdHPX{{1 zY3a^JLPBO=fUi$?Y==|O6L22@Txxftpl>~=Ps~~kJ{?j7j^c}$!%9SNs&!;J5B&i6 z$5J`{dA-4D5p^Mh%DqLo+}f9^75i6;24Rb zZ|F%2Wkp!&#~lP3KVV?-^d0jGZrsHE=7#q=&3R>`UebJ>DubE{bB@#*m`y5hy_I5MXNQI)02ipEbFjdSk@ncQnEP)QVX z*^pvsUki`Qx^MFqwRM8Kd^FP$C8k}U4MkpBtlyP6=lk>l?Slh|U`0B{#)UGRVy#y_ zi7u(fR}uN!GdFSrh_^j=KCnS?DR1bRx#vh-gPA-1p>)iX>O3J7yCr%4LpreYSsd$| zsnr;G(ZVOx)U5+o=a7Mcw*x(HFGDZq`OiSHe9PTXR9!+k7(p?l66SQK`idp)N`rA(Ap8WR_v{;#GCnsG-+ieN%m#u zp~}tOr}|t26n2qBfWoIt?mBWj8nTo`jFPH)a|wB@MZR@?*0CzW31l3j!<}F z9Gl>Y-E*-B^=l6I{+{dYIZm8q+!lOmU&}Oji4R-<; zhy#U~1rKIR1NUk%3F>vlnd|ltn$^#j3lGf=)|s57Bs9}VBir(vbELz#WuQ0V+C|Pz zto$ztv;if;El5mb3;Rmpj0#6`@%%w->c*Ff^` z0_Dujd#f?whwWJ$2)hj4fV;0~!h?|}U7t`G$b!HvKLxVErx~HV3>r^m(6l#Dw=k@6 z`N#TO3P+;v>!D1AR}Um4sU=CL>@zcjOrzaul#)s;1PSuwnNJd!NPGy9j^t+2%{37< zEZ)<0H3jxI;3(;R=zDVAui@iDbcgUdtV95OCnz{$j2K^p<=|^E z3zhx~Q`pWJr)Ic5W)KWX!9vMB@P+(p_qj;-C~t}cS3sNb$OhT8p>*nPKh@zwZ}5zr zC_M>@hFw!>`K7t;JY zN{t*J#6cgCy&HOBXQvONvL1V{y062DcJg#Sy93^rUv|_hI*4Htt9p>Kk$%&LZJik73 zPDatJk9=;1Ou?YGv*P8KsQf+ES|}7|uPj;;I(16`gHUB7`EE{I3#~z9VpepFbAFw7 z7R|IQb5#OX&Lz{jto(X}ibA+|)IAM^b+@CoP}% z&Z~ezcoff%nqwz&Bf_wd#5 zAH67!IUtuxiws3DZloH=uP96f7TmPLebMfUH`xdB(@rrpWQAm@e> zGZjnU6h{xbKn^lJ85|6%9Qc<`9kue~hCf+J(>TqxkG@$79v&jKe71B8H1sZnV*&hp z-xIq_ja}BJd~nTD1Rhq*FLnBTe8j@FZ$8D!2Zn~5^JOs$ot%8_<1(${j;~pbKWptM zh5MB=SjwhYwOW-<-wE0m_BtJ^H(6x<=9c7atJNrAknZ?JU2(p)e45>fqylvdMt2RdHnEFjhW z+Gtq8O;4f{*T^l&HIw-6{9J-6rs)%w|N0)lb}->U{m%s6UxB$$#aK*`@phJ@q_Y|_ zaNto#yRZqNb68Ekdr3MqhdrjYwx4#{z9BT2&+oBuzco^%i$n9qOH0pkI z{eps>AumBMPyJKPRH$O$WYFZpoBN48~3Dy~X zw*?pf(abax=U*kr|7<8$w^7Cz4qm17e_kbk3jexj-hk(q5ujRS9h##FM|`OK>X1+U z!oG`kS(_SpM=U3*bhk9fGm=%wPLRM{&i7c8#QT$wemrTe^P=}XirFb_#GOpCY>)={SJYLj}&*S<&`cpT-4Z{u8B-}{EvA%&YdJo){~5sU8dpp z_M}YlFbL3f^oyIgtuIPU=t@JGX?~~Ovg=XNJRDZelBEPuSDnw{0aF2R!k^I&DKItBihA zjhB;mIW28fkac!3s=~N4V9M8be*-;{s^2ywR&tBAagCLDveJe-aR5rsKrp_HF@TMn zWH7u=ocI9`BJo>$KUfeqHYS(E*Q^T(IfN;RsVu)tZ}rR zVFHi`m=cy>3d+WkxN~IdGmf`bJD*g}Wd1Wxy-i$9>eoum+2hs`gf~D?b~gPjpi$T6 zX6|muU8NFHsS6-*0}E7u^tzGh*4TdIYGBlq;3_tci5J{i z&quVr!i}#IqfISz8cjHwhgvdOZF3cUw|+31 z@;L>neu#>dzVJ=>XT24QH>Wh|PcG=s^B&28SJ9ZIxf;L=3oGh?XK-x9x8hd9likES zTD*ABE<(b9zpipSqI91M?z!USd<>o>4JpCI@elJs=^71v3!#)5zNSpGN+7)y)Vl8^ zrQz#X?_b7t?ft8(0|r|Ls>gRgd2o52bnR1f;$ZadAB`qTM~O@k9+%EH4oxhdPgOps z>NREh18gXcxK)HbreaCUR5t+7}2Ci;Ei z-L7^}4|v_@E)iD$WZt4HKgFq=ZjudN@SKR7y5JR8>(-MX#&n{(leLx97wx>;hC3ti zj@^?Q=fHN%+h=tr|LF1Q!L0?Iz%7obLf3^kJ>5+$0o}TItn14g(zw-0LNs7N9xH8tAcLR)3dX&1pGWQk#-z6|Ll`IW^(V94&n#ol)j0DLjSr1$%wgsSH_q>j*N*AI^` zJ|l{WuRr^DWDV(ByJ6O7bG2fYBdtE;U%}qsO}}btB;z%cyFJqC>;T}{VQ!9pcvLvBT5_w=jk`P&9~}=_Uudy>%L6HAT4k^3U0#Om1c>>`L_4@!Dn93t4%n(p z2+>JxT%bN2oZQ%-_LUu(xZ?#aT6KOTRCr!Nxd_pWO2;XkiQQx1ZZHTmH)t|NvY&(; z0ry=eEs0`~?3T&rlAKwgo8eRjv_Tg-n;jH{(rS&P@? zpbWe%i`Cf|7NhyRXc(>|OaN*ZOc9tZ}8(m^>gQ_8QP`_Si;Y z>+tcHxEjd9J=Nx|x>1j@;s-$sojOumEE2cX4ry!b6wN}3`42iyh)Un?-+mM)9cB(D z0N6mhaHn_clT~rkmeXONpgILqH;tTnEcYi{)YW8H9E522g+P(JlIdL1BMreRKPSvO zpALERc28_7B(Et)HL4ruRTda&D*LaFGA|w+m926EKc@3O;@^qc@vyq}(|}Uge2+U& zMJxxihKr~F-ZMShD>bGMJ}0Z69{8M9iSLh@9_6atgl7tp!F^v3{bZ8%hDqET`7R6DpI$xa_*id`?&T!^8;@eEXp*V~N6-Em%&G zSMdAhdk1zt?uSj-aTQ_}K1XI;cSoxSy?JP;G#6>k^p%2L-4R*)f7ui`7W$dZ^9$mH zKf1z;R;3QRGjW}I&l5aczMib>;`BKCRNmyX-`%q~(F(dG!pnMV%~GaL{6Famutyz014Gb! z2D|`%*QrbYHMh;gPngr>^c)x(vxv$N=BOp+%5=lL3j~tR@=DSvGQ=yyk4Oy^*?A|3 z!ZgF0oIU#y5hTUIm;5t0z;td*l}}nVTO!>!+G(*jkr5kg?h~88J4V(0-1;ktcdLWO z{uJeqNm%b5+QP3;6)`n}`G4=v|I`LR@f}koS-R>Aj5F^C3k^t?RS$S==t!LXe8+y2 z?Q@jv7&7V_&i>UAjEU|OHhE+X-lw#ru5;$cIM?RoZ@IPj<5uHs;3%yKlp`nIXE_AwY(79%%J|DIX|Xt41%kS@a${n$l6SVNSBPK-sy&doNHG^)6eKi#;}Z zf`ic}n+>T{c%k1br)75dXrfXkROg{>V;9D;(gs`J0FGtkLtluf#G|hcnJP^4E6?Mp z1UJIKKktBUZ~`cIyVusF>rwUQcKbP(Qbiw>F@!p`s9g6eTiPfIw%{qE=SI}6u1 z`s*!MN%0GWz#Y&y;yJA&>Z?PWxZJ3#>O8JTz(E;JVdsiWD%_#rT?C~SHqdDu3K5s zpevfwSSUH3wJ3OoS~=G>B_%vf>e}&^aowAk%};aU(MF6qB}uQ zj5TJkLlX}qQKs@W;*?rAv7HJRli8fLXjhBo)(miV=5Ffm&?Gc33ZS)Kp?Mb`z=FmJ zP=dj>a%%Hu+)YD+h-+>nU&Af}oo9OTdaKZ-A@2`lR_3@&hTG@H??BdY4qO_d;2AET zp&sNi-V2vl{f5RRU%UA#)Wks@RmLxk7*WOl2|7`Y6iv6WSuv#);S8x)UrSe~hU-;8 zvWxQgEN9;r_DCegjcX&$g=a-&j4CHmLMq_r@)%s zUmWY?-MA2kdW0%}L@3O-1e0?w>hNHMw3q05rqs44C~0`19e-o3ctfkJzK1%HE#rq> z72gS1x-_mO3M&iT4@|!|ttIaD^HBOyGS71WGsF+tI% z%vc6@l)s8lyOk@}J&cn8nwFg&xJ{?g`&CT`cna69`>j+{TkOljlDxDY zO?}Zu5q#)zqj=I=u44q%e=QYhm9RFyqYc3N;_2Pv&f?@QMmdv zmD+AnnX>6#l0b-%V#%juwEf%v+sDxnjv3jYN$DCbB1Pfo;d7~G1n3M@E$gg`8Y;y> z4tDo-7MNsw(Ab?ukjKH1f(r;Z`yFxi)SKbg=Gp0{SYxkJ?V8W#fp?KB@DqQU1boKZ zG8p&1p}~P^f7m9Ibo~Gb(Gi@#h-$XYFsT@*r^Wx(6ISsguAQ?asE|7m>SfatzAyJNujtVM z=T>8iuDcGnD-WHr!Np2ohNE2T38T|rn*hmXRP0F|l<9bIv73-0ihPB_&lQN2lLR;o zrsm5B@=7en*LaBOUpl>_Bdwr};4?qC_nnq?kb!{K%0~6Y%%Eqa<5Fxq<1#H~auii1 zwT^~7>G$fq@-@t_`4(cw87bNU;po?NNQt~RQ@$p>_R8NRP|$MqJ;%w@ckfIS1%)fS z{Cj~Xidw=8S1yXpsOGD|^+EJI@{pylBaR-n>=4*jIc=bsR_7D?9YG$?Phc2VLv$y5 zryO1Cr{wnISG5&*3jiH^J#KDI2aEm+S8ybp0@eVEpTsY;55AeL*VY;!IF_pr(#w>x zU+6vSNq27OD$qf|j1dVJ#jan)H;z(Bx{O?=+xYprKB_$r@!@@_(LYo)+%ESUVfn?4 zvWusZ<~eOX;I$d?Hy1sdW!#r^UIWUNMe5_(qY+&NW|`YM zYo!2UvC3(IeM}`@T7J71)g*qn4s1^7Qtf1LA{0S4*EDNyM4VvB@`LcRrKY6>t8Pi0 zjt-))B7#FS~0k_GI))t1DR;BDJYX(c0m#N1N#zqU+`B z*MYANCXKEiKiAoDRRY_M` zM{rg<++<3zQTcNX*$s29t}Wr7)@ z>opA!`aM#CAJEW%sbl}d|Pw%W9A#{jww$yB05a3I=zxe&D!cq6pJ(6rm!?g7O8UN5g~D& zri_rBSlpdg8k3@-BM9};*67C+E2Un-=$drv>V+>m!`#%Z5fU9VD3gfOET>p!?FbF` zF7vf2oQ;PsIT^u(Y75+i#h6w}#v;Ax=!h}N^78&ix!VO&n2E3A{l1=0{rN1)niM~9 zdntQ47~v*#3n>S=&NjQol|vT)>pdpjxwF z(rc^VRE&4VzPjDigxILUTQl>l>X3kaR6^h4dW$_VZm!BL$USOK$Qq+oT^#sYuaI2R|n0BvhO)$Cerx{Fq19Ea727ruMu@*dI)+l zb?z{V<0skTPzwdfh(KIg2>r46GINJ`B_hW{hgbF=G`2_|8z z4BMy_&XPTVakOxr)&TU&zoT?;(9!uh4oGkC?xNfoJ9CvurFAi*=M(4H_jsiKa_kI* z3MO}v)QIQq8e-gDGm;Bn&QDLU_3{0&#)G3;anY=wEX|TPNIMcc6Ffek=&Cprm}guf zbu>~y)xcWE#>D0kP1Ee-l{<~JmeKKXQtsaCs2JU|olso5{H^aQ{z#M&yRsY~Z;ee| zLZuJoo7Kdln-E7@;llZ({KEw5@G+qL(kVnq4+?p-C zH3YYBzW$O)m8@jHt%&fpe_bFXxtaDyS-|K$*K4H@P-1PO9ks7glkgXrX^v@ac<6AF z0FR13$U~i^YQE3+=U|-}r&{POb!^rD$T$4=2wnRpThaYXE&mP&(?|)QS~71*udbA; zF(klj-igKN)Cp_c*ELDdqueaD!$M<8#OJib1L(DxEUeoFy-|m>9df_2wk768_8P4; zGE=ov9Gm7P?;0d|t5OYMRuPW-UaK!-)FqrmhBKo~4a@?3XnY9Q`4VpS!I=k}su>j^ z>Jg|y6$ZE{mytytyTtLP`jz0xA|+z(Z$0q~9Q%Y?U{kC@s>79Nxiosh`9G4o z-Zfr|X4i=AHTM>ucK*?!E-xGgljy8kF~$~G2fDw58M7neU%e$=i5nQxR|kWCZvqLd z{N({!qp*waECyJRYfU=2#PP-|=fZmlleqS0{xB$UuLR9NW<*|qZ2|cTBYwfvX!^ws zD?|05dg7>z)EDAAGJLTM!dVd0q|rxInjXvzuDU5ucM78slxWjhqP;!y%0+s?-yf-y z5rJ35oLiA;`)|JA3P~t)6Zj6l9e#~T$j{X1@hmIv4<#tfrl%5=i;D!IMq%UPAM%_L zE$Z`%vQ`~WJ-*Wni>pDrUZ+Q?_cy8-CaW8Nh95?=X+W|ZZZ}I&9l$4g7JrL-h|lO* zHCVMQuk#$g@y-eM)k)hw*om#m_Y5_yxIfw@5b)i<+X5$^BBWfRSFTq{(H(mg z<}bHuKFr$jaiaB&70G+0$A`K}f|eJJNn?z3sp55HK$W@DO;prPhfF^ie(`Mjt7(3~ zkDHA1k*s!)vur|Tfc!qf%gU!IH#){NFgTTRRHeSV`7eiUC7QJHN8WrBaRbC`+1 zzCBVty9~AbEh%rH)tyl&+&RXfsb5(_ACv4@k=ZYrrEr{dZ>qWYa+}qUr_3a%y&wsM z8uEcG%FvwI#HtIH(9!H%*aU6nG-d4R(>At*}rZ-gh@lIXpUB zYgrjA5N+*0BTuahwzI)VI-QESZS+Gb*hYz;mV3H z^CJDL{!$i(`vgQk(GoW!Fr^{2QbrcIQITRA0|D?+Gt5Yc&8jE}&Xdg4vCRcEx#Of3 zPM_DU65o4-z-V3EQNNY;t4OZOXDG6^SwzpBo=&}+G2!}{EL!e0)7Umz_p6jbIF{C2 zRpRERLKkPmn@2quN7z_UbP8A;hOH#tiQFj(jZ{{u;FgTgFlY3SZ-t)!xP!^c5Eox6 zZVY0zGWE+BE}%Rqf9{sq?k(@;#xqvNRuHvDS`nKbvh^*S&^SX%)d-pa|12tt){eC` zVeEPM;mPRr65IKSS-a`liu?cv9!uHk#vhE6WEK@LAkCT!n|*D5!}Ej{0{cDPND^RFNyUXZ}> zY%dgurdOo+-pCM?YqX0n8HZZ+y(bN$p~EUqT|IitJm4+4nR!5j5_lXc_Nfb`A;wV8 z>y}!#<*hxInh&*P^cFNAdAnuhH`XnVY(3j{HFM;s}u2-h_i-iU5d+z#+2bUs^ltsUJWh>nxAH~F)Y zsp>$; zagfCEm>XdrDXY(CI8YT>l z@2@9Tb5bYYoZZev8t$Qg7I`dw@JjL&jSd=Y{culxEn&>rV6+sAEnPQ`j=}oiZq*fh zmZ}LjInpzh<_6je1%UcE`txz_K%4GpvHRL6u)bXPM}LQV3aHhhhb}cB+7T$;$sCXJ z9zo8$E9O6W3a!$s3l(TLYrS{}a75PwEn9Ut{kYLP#x||7abmDJh49sn1&6L=gPy=7 zM4Ye4)j=Tv+`SE*5*7jv7!NeBedT0mNFq;&t}7HZ?0{bvF8t1#r9!ID%#ieRE}Sv4 zp!=PZDTfJd!6DN#@s~8!OCMchl2ivU&rh>3rT$6=79u0h2A+z722x2Dn>DJ1o>H1< zNk@5AY&_*b_!Qtf*vM1Q8w&l*+LM>Abv%}GHy#G=5J(*4ax3JH|{6*HQ z&m9k&KWFomO&9inbAVZ+v3pn^)%s!kah<^yG$o0bCp$aTW_B^Au6)A}biP}&g zx%oi>no2WV`zR(2xW-YuqR>tKTMS}-ZGAy_ScN#N z5AD9~4-!GI38`DMW|jAahi*Ns=pV`K>z<^fclX1OGf#a{&|*BY=olM<=cMI%_%5IN zagR-q)dl=QqTf|48xJ;=`iMFc#O8ukQCVxcD}u!nuS^7zmc{H;Ji>}fc1KgqN9r`Z)~$6#z2Oqu=Fl-s^w3!sO=ToV$o`q{ zW=L^2!*cD_A2m|H#ofR|Ei83Cmvoz(6NI>qs-5J_UOupOhy9?8p5Ui>(%EPKSNB;W zH||OWU{VV~)oKJnT%yQj?P}J49d$FbxXI=Jr&PhUicQWn0X}3XmfX<)Id1dor=eN=#Md==HF7c ze!2Hv@QAHF*(KL2V{KbMSk~6D52H8HDN+cf_#&>QOttw4r3-=8-Hd=bPF^70Zc7u* ztB;TE9(ohoVEvPgwfHq*eAH!k2tC9G88Y!IYRLf31U)MzAM{k=hQx2Y6}~iIJv^2S z#Ft#zA)*C9Us!45>e)4RXO_%9v(H_H7Q$yV9KceJ1br5M-(~k4!nof0=fDFFIxEUY zT+>lgRI>@+w3>M|KUTp~prgL8XPD|(ubwPdMp@XKTBa|yl+Gp$Wk9O`GeRPJgPcafdn3F*~E@VKqiTHUxSa z8NxNZYH!q(lqi8exD2$S{b6H#bU$8Jsx*7XCX-0FmBd|(uu!Jz&?|M#<-T5c7S(bi zc_RA{gb@zxX!x(A1DG5%&*`M3pV&Ffcd4kb$5_`5BdL1- zE5!hYO5I4&dsRyc_)+qS?q?@ZRG7x}z36IQ2I2c$ct zF1XIBXH;z{&=I;~vbVlJRhCmzU0xL38y!xqk zhRH?{ui$Za=olxygYr+a6S_-3_MAceTqNl*g!jIcp?P9gg#x}GL4p32CB zF1H(MmAy38`8gzp(P5}jW^qoqNaGL;=aQ*<61OYhv%r7!pntfDisNfdRWAJnq@}8q zDOb_|OfCU)o*bY4>~tmK^qX^M#u+XabSq1&A!iIA1gzpXEN}bI8gS|K*aDzzqxjjM zoe-RnYlYjEe+}?K1BS67ruWuyUTDuZ8xCez2LNDXj6)zUS^&Q~Oq&&$AWB*5eVO|V zr=T17{joOqL$wBL@gqX?cSVZNl}l@1dqDpmE9VYo9ESG-F3129J3HairVr&Ga->Ss zBV)3#6niT-PsG^iMJ63pczGQUl5kyKE4x$w3zGQ6YqD%g`**pnwy{99w<(3*YBD%9gyfSXJYI=3p z^%!So`<5)oE%;D*AVd5S7vuW9zueq&{M;;;ILQ?q)P_r%rqXol*6Fs4IH#F>woEa= z1>Rrm_W}liUOk04-#MNQ$GEzivml(g$A|Y2=YNWe=H=A@fw0@qotPp|3x|IVe6Kc6 z9pFp{FO=F$R#G0+&7O#&*Q&+>g6zd5H|ux0oit8A&kLm&rt(D}7s}s!?G3K!>{i1;Ic(WjF4KFKF^>6(rwM=S#?u2w zmd4$_OjWTSTjD@LRGVz#bFj5=#zkN@PRPTum3omZ(EPFaJqAleb|qqYEhctJTPLQl z+RcTScv2xrQh4)+$U*S3b4_`cRB$+#`{!WZZuOqhY0!YuAg{7D{MrkN88Mb)D7BZt zrW8iOud|vOV~xuSDG!G41@9ET7$xa~Nav|#Q`2GfDDF8oJnr|3Uk!9h#gXU1Xm3e+ zKMAhkJ&lr>zVW$8I06vnwlKE`Z3cu7LR^fFs#TFfcHJ1=T-Ev2n%&@zdPJtE4Z4&v zVo_|QO%kwEk%H#*lL~Mh?Z(rXgJ0XVDjDnc;x^dBT#w00jlzI zju7KPbqD1+3Lc}QDOFJovKg*6A`VL98uEXGSY(q`(}P-6jOiRMsrxBe3h=a!CGkDC zGQ|Nd$2okSKd-jovewZHQJTDia(~MC&fbwT;Pt#SJ*@#aU9i{wc=nXXR&Tfvn#a~5 zo zt``J#%w*M*nbC4GDK6Rg&aFa|c@$iSAUyXxz~j=c&7&|K5=FuGr7=vY=$^ z^pnzK@rt8nfPqV3+TJ(0=ITSCZ_qg=4uCqk0G@OJQJfn`e?WAATu^tIL-(k}7fJNa zFjle!%%_IHb1;djJ_vRk@fwz&wYh3MF-UGUMV26bHJ2|(CvC}LH=hC~;h-Zz@FQ;m zoyA%E62cph5Z~?v#>d!D-zAPdO$hI_#QbpiW-n)g)!dcd``Y_c$eY@eJi)&03BS`9 z3sk+A6t$y=+J4^_^q}|BB_)#RiXQ(umAkLP+7e8ivVz%n}&I zaz$BGJ-q#*;L03V2d5|iY7!FWYhSkDHMRusiv{XLVav+jat zqtnAGy7Mo|p&YDzsqJ;&*!9_JlT7wL|NCT~-X##90XFWYwJH>#70E>t$^3qRiKeC) zLk3k^k3Tje+KOsebNT%!HL!unl-v_oj6=_f4!|bRc!E>PabGPO-`M+baZ!`FrjN*$rF0)_o19jcdL{FeN4#kCC1qWHgYN)~CBA zar61Ojp?;kxIU@h${l@&MQkx$6vw0L);LP{2cfpXH#>UL1rNa*GLR2S4t?D;v@36H zt+-@7EXFw?Dt3CuK{B-9$9uTIb!gMKkgZ(Dd@-P&NQ1~8G|~TAkVWW9>F@>|>mL*> z%?pSBB>@*-Biy~$74nSyxM_u~lU|el6hH; z-OhFyANVspdt&S`_AG$Xr2%W)!?*D>v_tB&jnCU5Luf0-vZxHu5W<8n^Z6Y1geXa( z!S2~`h(H4=BTtLzpElE*!0Oh0YkJeWD*wWy9I2G^{PNof$?Z-wE=SpK+QegB*BkN+ zkiOjw?wAsJcFHlPqxJj6+>n*qR&$qdCoVamaO#gd0^@>5QH=$C@^emNdhdKi~?tVnq;#Ec5GKcFZ4C)x*$qnBST9 zDNmNJU|GLB*UJ~IT8J*S(m{Q+9`SDQ4CD(Qx}H4j?dl#DEvm}2L(n@S!}OEBe= z;In$v{-;oct0TiU-|DIH+3}+bVh<@0NnA>|mqt0V9oEzpnQy0^ysSM7b(n1;FG=j4 z(y3iWd#fL$AUrpxZ;Ew*GDX8k(5r(Rw#!*FVEyPD9f+f+oQrPs(B`U{{>}|+w9a_* zI^5E<*R$}X7un8-P22 z3^zll<&ajl^5h|i>#cEJ5=u3ocbRfnlzAn2G!(&$3sR0g8@Au>Vy6aAwjrc$dfm3o z#?~ou`(DbQi4JC&IebzMMi9ynYl`=$Z}yM+lQvvR0J^}=%gUjZTc_ zbhKVotuAZ;MKB8tqhl9=*R&XS6I-X|8Kdi*UmGrJ;GAysT_#G4y;bPfMQW%Pt^=a8 zSs@rG{RKH4JTt9n#J}344Z<9D40U~gi*8{|L)baSr%N17G0sQ2bGPM7uB)+OLRA6J ze%9UtS=Cp7aKJ1S;hh4kG9G`%I#o>RuryahJTSkzddG<-5NgRf`6|1%Hw=%sl{zOB zI4ojB-h}Fj5x1U@lBNZ#YCtdu7}K!2>UQQgsja(jP3SX>MB5tqxcKTosvH=x{bX&X z&91PcvNRrUeEnOBPXDTb$HK=}s3|+2IS)Ox`@9v;!KeKMackvpPtOKJlaLBNWJCJ# z-BloP!(Dzr*9|EBmqNq+@&&N66FMbq3-p8y5rzTUbJ7|Nep8V3+Y-NCM>YkiBTMyQ zRo%6dX7Od2>)=X#hj7deXGv=jHD+E+TDRTJt=f2YdTf`pZq|Xq+jD9ZUwa_F`m?n| z|KoTbpq=8JG&6b;D@QW8wj(FQ^Hktn(WPw**_9TiQy?9{`9zlhKU*qKo?FkE-j&TQ zOCQvrakk$kC#(9KTbh((QfmqWyp4bMNtr8eSEvlkP@A|uv+-_xNj#Lb*z_#NCV>Jx z44MlBYNY~$OW--sx&grivRs&VvJ~&kfRkW?BN$23g$7Q8hW(8>g+-2etLLIaaLt9M z+Puu@ZTNO9&_0yuFusI%mEDag>*MwCCKUg(lyJlKV=xrueu_vg;~}U`?L_V0eYZp; zP8x^KQ5#?R+5r$545CZZLXi=xl-ll8JIq!&dER)XSrNtuEC?gZtqXE&T#==yk?@L7 zCy#UxBIbRvAF@lG)qBc~U5H;5bx8+&lsNokJ3)aTW)NlHLgugc^L2H9E8S(L}0Prvfhz|?~*@f~jx-UqZaIzQJY6{>98E_9jmYrB5wrGh9d z;(3kwdTrj@xh&J>7<3nKm0y6Jd%)+ z#*xK$1?oW>wM3s@qs(DX)1RLFoms|qRjp!0>vW14R~~#D{bz&(1oecztH0}ETvfbH z)~$(S-5(^)Gphx{1y6#uxZJqjl=#5Cx-&D~&K37&gpUbpXN&|7hA-}y>0)$YutbI; zCEwI{L#lfG3LhwN0O0|-6qeKT`VLKV!d)<dKa#(<}Ut6>%KQKZF zh<+N^{715y#sn-2+;~UzpFQVsymAv2cfnvMOVO|nnM#?Jny(~rHIV+Zu#O9HYe`I0 z-QY-!cAO$#3~vfjM17~`h0A>V-M`xhY>|W?jq4!e!R%1CnO^$b>(=E(O@n5CqaA|> zLVf1x-u1E`B0uM(Ho5GQo?T7gM>>|I$RK(fGk-}Bmctun9=D44eUENDJ1Tznbx*z{ zVprHFRKE*u(JBdlQ&TR?2P|67rxY}N1cs({Yxq_dt-g$2>(Y*(_qT41(Xa2>-!rE2 zVzw7oJ!aLXof7sUboL2Cn!niJyh7i(bNfP}EKp9zC>=ghSc+(;BJKaUotHwd|5<=X zr(72Yh4@t{$~wKEBNJwC*(()YoVnVo_(Z(Fvi|$+szS`sqIkHG&g~>7$)yBS?RNRq z&ban}9%pE?J^7%J^Tfk_W5+A+dIaoqkNJ*t_Z*W>$i++AFY|T%a#Oa1+d$x)n$dNm za9zF_XpO z#GT|-9ZyEj=&HheQPJ{eL#4x{F&VDm*`flx29j|ys|jNjC9vmo*s6b;QSgnf(c$mQ z?*P5FlzFP`pkLy`w>Wyp{!pxA5Z}XvWwOtuNalu#Vr8lRIw&Dck${#5Sng8fj!8^1 z$!H1kOWlIBN3O?*lJO6(jA&Pka5y6l3<34ro|Pbh)oysi3uS&s@) ztJK06ajdB0LZt2hm$>5P5;po-Ey06@KuS|XKnCMKj(kGxEk=FKJa`Xa|3JP zd@b6E`AA_P^tlwoU=cK4Igjk}>>j72>{XzK5^_YM1#?$*4{;#fzcRF^M1MR?By)au2AVd zc-porfJR=T{_A!6k|iw;S+*8kcoMkudh3a1(n&B(l;DSbrA5h4Ng&pvUab9KI9om= z!pS!=NK#1Wcar_0%}dHo_NpiA{tu}U^1pyxX!9=l;1SILEMeuT=I&Z}6f?HC|E)90 z*P=^V79HCU#MR?8R(edfO6>V>XqdGnaQ8m7{HJXF{iR~|xk_zAwkoub+yJ<)yXP}TD|RsYtcg{J~j5Y(!}{c=>FdI4~yyql>c>bg8|WU_je?#Q zgPHaynr>+F`uP49HkE34i&-eFF7nlBUoX6w&d_3;!cDw$2ws_8w z9(>_F=v7F1@NCxcWF`Lhbk=9G_$H%}Sg8Xw^XWnAYY$faESB1LGRqc8LGz|WHdHn1 zC{Q#8b@+j@xJk_GW75Ij6&D?lyefV8feMwDM}DNW)P7~8wN~X8OQ?7=FYfI4wfS0p z)!~;6Cuz@f@W3=13~o_fw3a=(kJo7B_iGW_PEM!gw$&g!bf|Fu6=)a%ASklOZC@yJ!0oG?G(NkO%m*S9D*(;fdT_;hU=7Vb~&} zq~YjjU?YrJVh5FubqPuCsxB}MDMt&5Qz&e2U3@W|w_5cvR@M%Y-w-#5*$h?1=n?H0 z$M(AD>Wv@Q>&VUT*^Dm7nqFk1d0XPK_Us+B?w+UO+yBh(h@0zI$l}YdbJTD99rX=_ zdAR`dW>T)#Lj)5H=+v#n-~laU#2%&=VS}#B?COi=I}8;ae?IoA;UdC$!fPA#g#7vI z+(`gNYRMrg|AuJ207t1M7mVej7iH6QdSB!In_?n<7#ExG+ebaYPvA=*^4Cni8a@0m z0>PzpYA@YptF^#1xb=u%gLt4%OSOSHRZO(yGq2uu^BaF={NT|#1AMq@cQU&2cTX^0 zt)HHlNGW%%Vv6ierb9t7cLP&4OL&vromDBGWqexEnp`R-aAqMSqNJ|3`N@mKzN#}C zGkOa>S?M-Rl}3ZPc|zets2WIS;ZV>SVP-Z2MH|a*89-0yqW?LPPv@wz}KHb{gD=vZl#^Z{q2B@t|8}GSdE`#Iuv#oh$Y2Wi`(v z4<6JA{N(+83Yx5j7$eU&Xy)T5NB8wBi#p*;!)y8a^aMuoi;n38vX<`YNyu@3=!Na! zScOgrwmW=e6)k;o=d}#=YZ{IA%i){jC;`ZflCSX9HZP&m-|ou?KoE~Vdc4+pxPE#4 z(aOU6?<`)5>4z0bm;bwF{?@Y}lKqiYbRzi!w6EK_KD$Vt>?A-i!FtVdyAnl}ISrJO zdG5RI#zuyKo-shA$?eH^DY<#+=G;!~oWHisyg1&to2r;<;_G|1|rw?76* zrF)Yq8P&16{wi!)`9Za?84mgwIYfw5_WIZi)Am=s%GpRfrAviE*{FukUf%$buY(7; zKgTB1CWEoT(Is`4oTGKw<6?=u#kAb&(K$YZ7g11(AWq~PbJP&kgoZwcn+S2DFq+^@ zDLH+USQj5y%1Snp-ptk*+hpAX!rgqa4W$t(>5U$IpoZrQ#E-L_J0z26ZWu5D|F_G7 z_sKQ{g=L&o5$Q{*-ze*SV_q_qO^M#-W3n3tyvn|NayZvma@6dc`CYfOx6j`YOc$Wg z=@ILHo^b7j{jvlY`EJcTbrmKn+|?w!RcKa|hE$j0ZTP#nRjetMh3P&W-)nA7)OmT_ za6M`+S~TYFnc4%NT`hQG554^G7v)c#Pc&>Tk-s+WuDwfC+8@0{ zsV82|AzdaA>hb*~)}f;(MTJ);U&YdgINUT=Ud!xK?&$pSWl7H$*5wn~C8KoN@$Yax zh%N>nk^Q+*7Sbd?Imt-=hp%pDdXI(Hc!bS+;F-<<|Amjndl)3PtBc6U+cW=8TleSv z9>KM|Y59M}on9mOt827B+r?=4TwY>?dWdrRteeweR$cuZ| zAkM5$D*h7k`1}4mnK~ct5EbH%O zXegnwNWU4!hKr(4iB1KZMD4jufeRp3VPQq@k1_m*Pc~~7r1STPw;j?YEBQ!>f0uQ2 z{hOyb73YUjZqU7+0DSVY-ECy>i7_Fz)2IBEm8M6rIyxcm1;cq>k z&_C%>>N!NV>wpA4(;-@e0scCb*!dhl)@@`2oeGbbiOxFMqr?)5{BEyG)w*PX@WJql)G+b?0G z+YU$?b>A>qrUDFXQ?&40p+0(k39akXSXK0w-Z*mhSCQ*HA!)3U)4cs zfxy&{uL)Vw^aD!=!&fT zXMAKXEnl*PZ*KceXe;gt8cZI5p*#SK1# z(^rT+5a9{X_D`%u8MM6{w%stfAj$E(?iNst?t~8zb)k4rqDT2`Z9mI*e^v^tCEUV2Ps59%1t%kWg?f);X zuU`b_yZwr6Yq$Cqb(j5bTRPXHzRJ5#(TaO(geVQghg~)NdJz@7e?wN6ipMFzS?exj z^6q)XQNrl{G@R*YYtcGmuP1MUAQTdU%0x22TI0B<4^ll?=3quGAVYf2PxeXlkKI+w zj{9a=o@g9-;ki|T>yED;&DSyo<)I$9bdM*4i=$&+^6$h>g34l@b6rxPFo?ZE?} zs_Wt91~EOxqY;3eCY%cNocj}Ux9jT)urI*DId;jVCP@c~a~{J<{HFjzb2|^(xMcG| z$Jh(8?7L|B<}XB#7^QPUh6}9UVw*=Wa+a|hWm1(wOW1YmM_D>Mj5}A=STc(P`xPA} zq@7i9XG!NTO546G=KKJE*=J4NQ@W%t>7>$k@V1Vt)V6knlmCUToijfq=Tw5{GR-eUV4s~?BK8utvppZX$3_K|Xx*?&s00pG;93D{y1Z}4 z3t5+EMw7ZgNHLNduEYqxpLPi1lP6iLbCMs(Rmq{AuMkTaNUU+wh|lpMfqxp#O!+w< zKL*&vIf<9+=3PL8?AgIDKH2PjKwX4xPJJWZWCOr=0l&5pAO*q>jT@&SnBC7tc*1RD z{yJjlvCN#K$?uf6lsV9#$0D+b4%#DT+gpU9cD~%1gVYJeM^baqT6k}>Za?|8wZ0!j zf+IJjmQShtqw0MX*@PCkeG|l zvR@E}5U)AVv;%9Z`Vq?pcV=`plMa>1m{xOp;SG|XC?2Lh%ie*j2fxDCf<=KRI*(SS z;hOw+LqVx#&-czomap4UU`Sy zHC;0&&NN{KH*AWCk$!2$faB>viJhz*@lP#NvL6~=k7c625U?dI=i#ceAAMs)T}n$ux2N zjewY1XN_D}D?Rh0l&?#_TVjs<^1(J*TF|C$t6t5(`@6(c)YbztigAuC7%ai@S#u2x z#_(@&(hm?ph9#;_`Ffe!x>N5&B5Ts!iL7;KV_)~e7FTul=ySzDwG#da@j(}mOa0nY zo-7FqkeX$YInaTm+r_@|W$MTlw4-)f#gtglnzsS5P!d8KY>z3CwT$G@1wE17;W8l!50sFtKfKm>NAu?1QcNN2%Yr?k7WG zbs=W9_j#BB{{!G094TmLM)YrAw^(7%q1sul5tM>H^vZMX3HO+Ab|QPnF!%tYcLQg) zG#%{G3>C4Hn7f?C^ioS4@;inPzlj^0{oPR0Q9emBTIvnaXy{=&8JQg8)8nttInFPss{TMMp)i z5GCEZPM)%8g%czElK?mPo-e}Ud~kF^>w{+{L5{oP6WI}H5VMUw7QKzC8bkF>dZlUrWh3hyy|$0#Z-&g z)eNUD{?G2oW8Q2ekPy4jeVGr9!m0PM3R5zO!PibSJ)dqrkRylEwQFh{NNwAumQVuy zCm-2#a92V{#kg}&-Lq&*-GI!X3~a-LRk!BP=msoG-{q~2ljCK}8g9GtkTB4Logu>{ zJkd+p;?uj_)O(p|@Q#rn`ZCp9(@Y+{cm7tsT2_)N=;&7EY`26kJG5M;?vW@0ekMX& zp~8=j|9sixaYP|PPup7E!*fFvavF|lwD+&zmU7|dQo2Fo{Fu?$Wv?4r-?9em=rGQQoQauZ;|crKdc;ZT1W}~X7uAj z8j^d@3a;)2GE0O~%+ z@fBq!GvyIs==pp)4FpYhWRaa#7_k42%T;l*Uhb3lYC{!prG>982#Gb5WV-{-fx7E0 zu$08by*1r6=Ydo~LNn2nf*NAz&3xpG&E=3OTgmi>E~_Gf+9zPe5vZLn`?;0sHsUx6 z!w|MKw$6VwazztsiUR-8ElywEKP;=+euOQ1CdlaH$Cn*cwUW37dldiHNFyi~Nl6Vz zedHWxH64;oBXpX~!%zF|JPr!bY}|Y-&}Ih99|49f^=kZ~`zAEFswb5iyJcfiWE)Si za~FQ8m15Hk22?Dqowy_cQdmL`4>}8dFMXGydm|T2X zxpz-i^+V-T+}W_-Pz029|7Pcd_1o0IOEUqz~g7RuKMOk8nI6rkWRMUw-tW|2lW3E!X4hZClEJm#(e zejT{;+<%U8e!~vP)DV~G0@=sn{QEy_IsZ^UGN`0tF`jxS<&a(@NhTe08-+?7ZDsNPIOQlWet<$2b$ySz-K zV@Q*6#BTA0**!#KrVux>bxk++{;g)%<65-wOp2NDYy!tbwz_YPPg(I@EAYPD|0 z^xCU7GBVNzGIgG$yQj{0z9y@noP4;*ielGGb4noiCt}mlf{FiOH&X!Q(olwEkXCqv z!SAAyZ{Vp*#++iWYEpNYvxJh`i>(aDo*xT9RPX!p`~IPpr((Mqe3zdYX&ZzbwAC>u z(+&d%`}g>lD0yy&JO<*S^pd9M6kw(G(2qCk;=CSzrOJu7%w5&OI5lTuf85HlN#AO8CYQFf)Lo-zarwp+jKg|o1Ej38h;Mb9bz;M8vC8bbb4?3e| z_Ea!oHGT#YVXr3zS?^%$fRlxuZOu;F_!?ESGd^dNBK>i=w^?=hsbUuskX{gjlk>o~ zu;25Bo59_*I&SEaLIl<~lmBdyY)axVl-Z^`Ws7P60Lhea&_}8d3EHC!X~A$qeN;UA z+%Q0S{e6U}NZc=+{OIj3m+d6OQ9pZ7EbdzZ|Hv~QV`Pnc8io&DiiYXkfWZ5%LnW-t zl4r8*e+JOxD*O;;@M~g8As1MEU-VMC!0)ft)9iuV`t#(irPKG?R@uIEdEbXV6>W)y z-&#Ds!B$DlvL- zrGW=1>8UMPI)_G;BAd80iYit{bn%W!M-&tee}1dR@n2`qGwy}0yyI3gdnnOV=y0xt z6NxQP1VKB9-8STzyMh&ddf}OJQS&>Qr{3r_ly-mcaL0Uz_g7*L>yE_eYLy-Z*(5$Q zFev-aDAx8GB=5BT^OrRgw%GSB4#a#7N@n5qvpjFy<`k=X?O$c1v#g9%ksydHr)xfbZNJ93>@;Oz>wOOE8+R9E6(vl@B z!8#0K4r-ZSX`4uYD4MSFKC~`Ja6u+>H!FlR3~~{PQ;$Q~Y38&Tu;J(al5#V6>9lIV68vQ4q&K}QlI;}-VkP5=mGu_z-KU>M>tq#9U#XZ*odn#O0lxD?b-cKQE-5zMgrb|lHZpj9 z4X8(_!XLC;rJcyIWu}PN@ZD?rrro8VV(LjBHKvVbYS9B(Pe}b5P0wL|r5=3#t?umH zcD?Y{gA`{&9MCx@384ir<=!LR{&F?Y{pb{gHqMaX1hQu@d;#ij4}%L22`@H#W~cj*dP zDXv6YA@T=FQCtl|QV;`jY`3oR%sp}{A@3fg#A)D-fsi~IBCK?~(-mmw_z1@+A$(zU zyot>>oT}7I?XlB0F|*)OIrz)a@}}}ZVK7^bV?`$QH3^HOTi_SN+*>P){`n;O4FNIn zCTac9s+0P^TZ{%WOjdN=4~*icbEgBgLEBfhQ%AZ_D+>c{Q|=GHEQQ+w!J}6cWo;`3 z&n434uS}5oO-I9%QQng*tzd{^ky#U`acO3M^J0AHFN9WiL))Y5?LSFzKl^TI}u3$XV#y&1f!>jBh&CHX8 zbhPMy3i&McZ?eRM?Am)@5!>OA!YwN;>b@_012*fv<0Y5kl^M!F zcG}4d`J2QCTQn#LzQ%2ZNW0k$UstYH=$rQ65eaJ;L0UVd>Nv{nUhD5A}|9$#2@!KCa=i%W*}SRaYwRyk5Vw9k~+0H*O*6AerEkKyjyQ z+LKvVs_TYY6@04$B?t*t@U4tIJd+yysio3o+DMvK$&#KHVsIIt3%qf;B{pAc1tUO zD1e>=Legv7b+aNy)>(qFxTwv1<@I$QX>(~zTufA730-mq7`u>yz(bR>%3L*6?Wad^ zY9QDB?&c6if)lOP*HL|Ww1tf(1qD(6i-$tNa6Ags&$pU9jnkq z&G*!7>`IbL?HBaA|w0beQ2*ar(kqN^rD|QnIsc%VR|yhGtAx*=x^T zsLZ1^3`NHoaT3@0&z5|sUbIyeg%yi76-I3D(wLx~qn&lGY^JjPbm6Z$k)o^%eHO&g zRlo5T&;-ZX7e7Wgw7m0ez7zkOMLCSslg9LC8M=x`q*O}GPqbCV7@=?Y&zJxV5HQyd zKeXDv&Q>a1_xyw4kw!@OI<-FH;`h1aK&{MKmYo_qGR~>kHiiA@^|DeS*@_%~L$^P6 zbLD^z*v!t~Ko|ciR4a4{y+(}i>eb0ChLjvNwpuOoM8P5q6mP^LLhB_W{~HkLva^oUbQItBOv^GL8dc zAZaLzUgC*Hr|LQNvMCfP_w{fa!b)GH$QF7J^ff!dG;ud`p#sub!AqJeMTx?R5+FhC zqqdTz{06hC_;&sQvwGLl=w@0y;Pu3}kRI4~m?^XrP^9U${%S69*gS1afTHhfR9?UZ zeA>pVj0c7#QMTX<$%#WoH9oq!bqT7XV8iJ&;exAlRhTyG<+3M|U-{1%?KHbaSF4u3 z|KD^Mu!;AgG@7bjY>!@$~b)ZpN8y(8fgd(%wuI{0W?5wC*vllz+(+xPd5c)B4e1XQ)9|;*y-Os%)#f*oND*BK@s-D27O zk`piSEdS+d)j(wy&)3<~7o@LKPB9^@?2M?Xgvf z4x1FOyw0viHY@XJ2qwVkXJB6GE+_^V6R|CD2NJC4TN!A%nvq^>&wx9sDCx)sqBaWM z)r169HisOInW8}VrN%y?4YN#)w5Jxd#35_qoinf6Oc7yPZ4)#JL0y7+GHL9`MY|Kv zp9_Io4*=V&1P z;F+d2=>j=MjVjV6)M$F`8It46DBZL_Ut5lG8L-v;7d{X}H3-B#kk_Xw5ILA3qA=64 zm)BrwXg(v`LtFc_Cz#whvZ=RI<+j4nxz&;`-F73S=UTP%Pjwvigz8$(UXH*IVFGKZ zj=_vTbHuXJYnI9?XG17jGg?>?4Ba7C2R7S;i5iY8kgD#_=PiKLWtScdYHQKE_u*-J zHkB(6Wb|(7iZrI08z_Vh{Ol7BDNC=S)94HQ8{H_d)O#x2V(CnqhL1fm@2riE?<$9@ z-;{2SNK*&hkY7)=OOT_{idBH%)L1C+5>-_fJg-nTWDYDI09CTrn8aX#`&XSE8AJ^3 zILoRrM@LRKt;tz=Uda3Qt$7{sX~SJn;l#7Fm!Nkk>CZ=~YF8s_n|f8RA=N?A-W=J7 zBf<qrRzg*a~0jcW7c$`UlAcs=v3s*pdXQ>wQl? zOFS{uPY6gOJlf%Wyrgq(qkyjCX2WK2(X02agH-!!oCi1FP_9ogw8}=?-?)RRSi}c= z%I`L=U40+8@gpy{Moksxs#5tXWKX6_t)cf21Q#a46nFULJ9djd9(?5J(u>Z_sM5t7 zE=cKpOY3f4_nD&Wdp|e0nzCFdt5Aa6yc$H{aje45+hU{&dFJx}whFP={G6Fm(%7rS zjPLIo9QA8EnM#CnC(S5yC)#+(j};nHSS2h!;d$H>cm4N!zR&aN+loN3(#AIcoLKw{ zS?li(@zR%=*`<$|lf{P0IlsD@l^~KKF>P4&gdqmn9xNF483c9bfT!7Xa86db!PwpD zwzLW{zy(|QHDcSE>kQg|Qe<@dWFl7tN|u^medAeIS&~WXisb(xRLc8;*K@>GOHXU} z-{=4_Vy7pyoVYX-8NZ>@4gc*I>%*H(Gm1#zpQ{QRoVvRV`XUJCFkYO%-0kp^PsvhR z?7Md*I>@g{bTpHyPG`nb_f(kJMt?l`_Q6KI**+#6eupdzx}oW%(r#cbYgkS^c1>a^ z?m)ZTT^}#U9@BV5+;YiK(vPXMW*WJ_KR^!0q+PTrz-W(cW%pBggL2HWz23SOlzF&S z$Er9GrcgdKcioSRmHIcsF!XoC)2@sK=h^n9G_=U4dMN7ZlHM2BuqaDSrx`M4r_0C> zBr^^?jRug6?QhPS#yQ-bX={70Y}MlZNIck~B%>iVwfg*h@5MsR489P|*z{MQ2RL&% z6vI@t#O*|2dL0TU#w&1U-v+iU%OTzuMm?h+qeh%zGDGuXhmk-US+TMSJD3bQr8CZ1-*BXI_J9(mo0OlCbkZN$2; z)+AE8d%ssHXs+=fHYbSk`<%Q(3$DpBDhWIC&!0v$$ltZtJ-AGv93$j>Ag~ zZ9JfI%f*}9&ZnaT^to#0B13Z<(WWQIF`b_y*>i>kLuk9r2(0zs#;IXChsb9dTJHV1 zkLEr@aOXuU-k`&$i{$youI$Z=P`)KGOCXq1d)_A&#q^ile&M~GK{fmO zD`m&C4#m)%L_SMU!+Q*!A^lvK*6`H55-=+Yyzz7QaHjIZ%CahuLmu{Fxt^&zSl8MO z-^-%Bw4{cYD?IbRD)>9XFcC@fy`x;U=zK9Z_J;o@AH&opV~bxdeLKx?Am}M9JtCKd z2$pzW{lO-YEi8m@m`FxtgwQj9cskWNMB|~1PFTyvtp(eR_Mz0f7Xy3??5F$tqqTFw z%`e~_k-+!*dAJI;u_RQFZALFs7{whb)&8j)BO+-}mI+eb7+56*D2!EPgbgEav1;+-L88LoKa4YC^d~l2exa+7m7&QF43lkGOs>W zM+9*@82TR5`JHA>flGCBJe;9N3mDwyFRBp`i!1B#;_r zi?KIZg7D_~stD{b+Pl}ql!Oy{bK*%&Ney~WgjaexKVw3sWM*L}0ah*59w;*pcKyHfZUj2lRpBc)W#)7*G+ig_idA@ruw@>wK4Z zIwy(#-tw%`+@(YM-*bGQYk+3$3dGI{u1vqYSY)OS9_ehr<>s(j!3A3$ZZK^7!8jp# zT2lQwCWZ=OvO|*G+R-TfWiZmoglc*fTESPPt+vX7zOmeP8nu)z|L3a~ld}p^-3?E6 ze4UYWtZ%fzE}ih5Ze9X$8g!^nY7~$Cedk}mQA290MNdTgf;O!eg&*!qp~c*COUFwjHm*AJzwrx{YmgqOH_jiI7C+uw~ARLv2LVt7$;Yy4*JVfSuwP?wl0EX z;|6ulp%5W1YxAL$VJ;}142m}4NOiWsfT!O8Xe+Ip^3DA|Uix40>6W7JX?-fDVOM=U zxcTL~{b%fGI{Ai5O6~ez(8r4dhl?u^cwUzt`Sz0$#xHIUUq^K7Wv%mq?gou@v=miV z!UjuB+euHRzRTm?{(XMuCtcEJ!+|6=7pO*@;yfiT;Qn$v#c$c?+v{FWL#duQOAZv*z}^ z+n*Jct7agh1Cv-@6QjrDdC-i_m!kKk7KQ6>w!k@c4a$OTmR_h>805#*QwP-m^@LS? zU(?1dm!Zt1M$o&CptvI=tQe4^WTS1+@a2*F#LiVLZFpY*AD%&r_@ zoQaPuDD+XAgbp-NxB`J-=FXZPA&wPdKxG*9HCbt;T7Xcrtdqq708Qk7q!-xKA z{ef>Kw!%5Fy{Y$eh#UiCZED!oLfLHD{Gc@}z~wYvGl@gr_9@32M+7>?b)z97bDXU!NKUYc0K4cz+ZH9%OtFbkXw=P^!IWm1D6UIxQP1A{T%+DT|-I4a1| z8h4cwoLY2#aDd4yd~m+2B5of0cacCbTpE%W;XX(b?ih!?V(i5~`bL1Io{m0pnuBL7 zZ!=J7gk@x*NaPf$-IjPwPk>N$r#(ipFbG@M?qA&}7 zHoORvZfu@xux(A#V?YF9V*OtwH>EWAVKP>+KN76Kel`R6XX>q%QWcx%!>^%Gz9JjeCgQfuca-!X2kgpHK)i6JA|D^Ob zc!zAZ@yBL?e3kAD!uaP!_;`Wv>1D+{OB&)bq@!}@`Z-2zwpSJ5ALD(} zj&GRn@^O937Ex8BxPK6Dn;t%sAdnR1l+ix!xej94Tg&1z;U2o{^6>_Gnad!o0=mIC zwi54>rW5g04;AU|k3+{8eH#r0X<$Sj#-8auJvy=K7CQT>o2XfBU@oBCru87xl*mdk zwoKfs4NM^9*i`>KA}}qr!!EmlV!4uKa=Cwx4~H*t2_UqsYA52WJO)5QXR~xlagFYc zX~bP*+XgFp&aXlH#w#>Ki}phJ@YOHpY^FUQD4L7J3`DzJ zaK~QyuzEwmz$|XV%<9_+cuhfVWC^!FJyoaX@tkv_D$u5L!Gwx@x6FP_54skWcgi~s zDvXRZzNn8r=NH-VCT~H4;%9B|6=y0`4nNw3kz(UIdB^qD`jmNU`1~cz6ectcY5$%E z0DC^I3wLKZ`Ovqhd(>?{Xq3JSAzB{4X0SQ!x%MIJO4(g zQr>(GykZ?eBuu&^N+;l|< zVej`X0ZQHX0KU7TH`Aj8QrHrKsJ)5~5&V$u(5?JhT!kXqk`-hW_nZ1 zVxygJ7$6duE)My#?2W#695%w`uY-=#i+6pawHz$EDtD;+B`&t%?HY zHn`X<8QS$SAIY{SZ}1kI1AoJlqbZ2mgw#^Mv*4L7s6lPK9%@#?Xw6usi8}=9G3vcrmu00a!asn=oDrR=!39VNjqC# zwZ;`+HGVyT@cDs&s)i-J=IZj|7hkhtJET5ry|EZ9feKdWpA!^eAVt9p1J3A5BD#YJ zM*d3I^`mG=NJB#}2{}{WnWIl5c3kS8()1h*HA#o7Z=S0j107Ot37fNy3=S%6$uL(k zXFGp>apN!ZI=pc?#s;+?-FSZXoFpM^bBE&z_dYCcKYkno1evL+L z1bc%z9x@iQfVo9g3#+A>q+kCM_V}J2PItjf4mz`T>OioX?Ab`jMZcXjiI~9c&kVu= zgmKuk$z$u0m*v)sFxqX;5!wUoF-4pM5OC9#XUN1Jupiv&yttO_xLU&3B6^v$`HX2x z*o{Pz^tz|?IF@@$rA*G(Y)`S>IR&W%GkXA0cxzl2Y}3wlNXOhev2s;^myGKV!5q)p z*+yjOId8`c{m_}n{Y6zf&>TOxoQbJmpF)avm{hGrbrX1o==|)-20w9`@ai&@(RD9j zq{QPB_~sRe1r@-acYO|t`9o|9#QoaqXVGsH1R>1)ksGB6e2k*zK-s*O`irQFMrHo~ z8hW$HI%p;ZIM14T`bE?=KGrMY<{f#h2nGgL-wQi3_H~$2n+cq8W~3eleOLRvevak{ zTxf;P2<1)$NX7^tdlB9yF%6gATKtvc$zCzfwDiKjNz>9#KzzpeR8?#{5~L|dt4(QoWPEpN#k(5VS55V2Fej}>>4Jl}_3qYE{|82_ecr#~ zZ1*!~M*b<(iQifZk{6}_+!=oJB%<_V0hXsmZk)Iw&iE&HaIrVD*`zdEsK=s99*uzh#O?^3GH_k7KRm4o9J+Lz zkT~e72YZ9+^vtgaeRDD_lyq6Esb$ya3R$%I1fkenECE?BV_>AMdI^v79+{c)SJyc)0(Vxod;&;*|nw(%6Rb_$} zYTPIWV2p4Fk_+CY<2+uIu*vsV4dhpUZoV-9$(B>y02_6_6U^ohVfsA1k&%)iJwGq* z^%%v^?k;#OPkY9kv$%&|%D!aX&)BfHR6j%FMKC*7#kvbZo5>!jT(8uKdy;pCbVYV7 z-q4t~{~(~7?K;{u`IKqy63WK#DkOyF-dO>rzEZa4h{Dxj%{TL&BCAJO>HK}eRX$lO zV6j+E^XVESF85RN%?2~6HUT?5X@m`^41c-=>5zO^7?sr8tG-K{&4)L~WyFh0qXPa) z0zk_vl!P!H3v>Anp0IVcj}n2sraTcac^Iqi4VQNwfj9R|n^f__o+pzTqetN?`BdG0V?AdXSatGu_Ww$%3}1HsNR^0YPpHVgL5k)! zE9EsY1Haoun!2ZU92+n8O~rn?qm+Z~BX}gm2D0EN`QH2&A(z4>cC}$gYGF#pG5cDd z+K)9oqXZl$L>Wjs*kzphsJrFq~5@`bYA<%m51H%CRp*y)l@{rTr5`7 z!L6V8Q6R3oP6{PSSQ|KRXQ{;z6m~yaIehiOPc=kiRh(@sDyz<1(^)JI&r`ZoL{P~( zx)d)`op~UBIEYX?ChB*Y8<;&6J7!NAc|mk9x&5}?Rz0!CksxSWCMxWc_7mt<@o>iOm4z1%)!UIWrGd*~jtPrtGHvW4gG$x6 zws~{8ezUeIkK7=9a7-478Eh`%J()&0xn;pR|JwNMHr0dtQ|s!-InKd-QGegoD}rg< z$b;5ZCuPe@!2ri7Etnv1f3oNOOIOd{3^DUZp^KdS=*=}f#}JzY4U|K!6~7tB4Q}`- z^WV~aup%?sm>rSfxc>DV(L|?iyPnWBZsLS+VnXu90b^udF^j?J=qlG@e&6^=>q{>_ZIG?jJZeL zpn3(Q#9~TU8IUmq+%>vWf5&3`se5X$nZ6gn*)pPmM##u~oHC!9T-=lXW5?bKI|ZGFG$ew3)7x(QXm4 z)QkJ#C7^!%LJC=Dd*N&T`IaZ~CCgywFiA**-uV7?qDjnVw z(&nCsSz9@iYxLa;L#Tj8cA##5rCMJi!lh>B22r5={DxD9z(96Zl3t=?8<%LY?)x`~ z&DSKT$v1A(PfMJ1!H5U_|1SM?*7`bsB)3-}GU`__g}W&qU!rUOc6LA+Y=T|fv8}C1 ztJd~O?eugnXf7h*LOwJ+o>=WL1xOY2Y=!)74OTJ9)hP# zfR(!CRkl>Kcg6pzaKqFex(mKMN}>yQ)Yk)S4_K1Aku$6P^!9)A`l}Eb8j<%~qzUjq`jJDOhQM0!1(PB)g2eZF=qbl1rs@ zoqBf+Aid83)y_CKOlEIU!T{)?Hjzg}FFCwFgZP+p^@V&44CO~?pK~|yYu6&~y z@BTd|jQ3_(9@1y^BeO&(6|v&+;rWuBaa(A-Qt-*!=n9$#v9TRhS%?BvKHk39efNEl z3~O(vKgrP^rk0JHDvdvh{Dy*agIub@c?+u2TWWZpd2>P-?Z^lM>Jxpd?HNdKO9T%k z1o%W$m6fpkax$NF_8O7D{`Zx*O+C5b3|;-8^pq$n$|1rA-G6&b=+|zd9L5H&c;{>r zm3%-hru2+x`ISTEv(qqOW4u3%*ErtLAe?$bbPCHQ{&S+EPDEgJ2V@T#WS?>-xxP3N z8M-$xj6Xing9GVNO_e9*wK}5Mo+w0m>Ph-%}%b?6!p zh82pdEK59IFGd}q>HOFF#6$@9HbM&k&BM_6Pn0_U6I@3AO2Em+B+~SZ8e)`CiF(GR zN%I@*b3+#56hFx?;WlRotv}cI*O85wf(fA^G$D;aSA)bBUmo*{S= z;i$Qn)x}K9Re<|s0)BkE&Sf(?ODbb)UHnx`>USd4cEI6i-T@nMXavp|9ilR3b0bT( zwhU;t>vVihf6ONxCyM1)ytk`$9Nhcj@dyG;!$a>dMozi~YqVCZkCK77LiP5!3$(;-X*!A<&~Xwc;`Bv3hOWYG9(rF+{yo!p-rp~UZAit zyQf{-=(`=`^eopF9y9XnHJXQDe~;u~ujaYs-E3r$IFoS+_hII+h!TXnK*jjwTjfS8 z-4BgrB?H;}D*POq_XKGjxGj5O9jEBa)jWIcUhV#U$?nLIJ%>^Nz3ps;i0ZYLE+J6e zlbyYMPOAbULQvcY?6Lgylf2JJ$?SUV>?aQC(Go_ocj7qtx!Kf(Pe$r7;>Lz z@wtyDf@|^oTZhdK2a20SdZzn=eATb_xzrz)Ats<%Rf&zS@et%jJBtq2l>(MIJ|3{ZO=;jm zH>Lb5=Dj=%oh-hJ~*2wd&%(fqA_$KbgTqOt2(03s5V8qnY9gXmgsTvHs zRlGtp8$LO+0QG`;($Z@eHb%5+7$BZ5_nXT89X3Z7WXj!-_a}^r9v_?X_q>O7_JcIM*P=FcPHhjGh2sIp-8h$1 zW;Jkt9cl38_UoCv_1T%n%*ID=iG6St9OB5>^uJuUd*Xqx}%3)3s{tQF!Vvzf3U zm31Ty6G5MxyZ`&5uYse++*?H>RxQW4Z(I{nWy{j9a){MN_d@OAdrX|wGx6T=P7$_r34?+E-Le+fr@EHiAsrOCxmaa!aDCOAXVkU8E4W zxi*zW0tgQrGhZ!$Bg?_b{T}-g<5?>|U`b8)Onmo4R6eXo0(6fBXfu{JZ2zw*t_CiH z&u=m0BmL)K=cBOaL|!A!tJwbN?#Nur2bf*v5;1H(%~#Jkij{CFT)c4m>E?snHJTLm zg{8vulppNS`~t_-@YjqeNHyVE&z>3m-pAGHi;d|wh$7#Eyj!yuM-6^1K)yLBs`|U0O|3j2_=0$`B;YmrA+IYv}!NVdCl>_{SZT!Ka^Rc2q4}g>ZcQ|!Pe|BBen#SRl9T(}tmm~hQ7-eV=|*-3VMTbm4<%N!e0>jY&28Uti8;z(IDhSOelV=9 z=Gh;+TJEq-Jhv2}u(Uq<$QLE^1MOj6h`mE3wSx;f+CX&pJ?y31Q#7UV4{1afrisx z?CQR!pQZyBTkheW3%~Bzk^-hBTZI?fM3==7I?tPA!M_1~LtbQUL~q5S zIM0Hxt=POz4>ukrs{GQ_i`od6B*Qwk+WnMl?da*^QJ2Exao!dItS9j3=Y!{)6l*=%y7rCOFmB_Ea zPq=jMdlc>MOsV}PeBP}m>MpBu`!?M&QjLS{B^*_IOrx=c}6Gz z3$NYs+0sw-So~BrDb#S%jZ(iu@U!d#wtouVIjGiUc?Ew2*5J1QBVd7FDH$$HyOSue zG)+iPhWn18v%=Av>`evlWDDh>h=)R4>s=w5mSZAB+#bca#uz0g=byn{5Gq<*&)DknZNnf5~xI zMF517@%R<8i^HR;p^*Lb1L%aX1jPZ1e2K+f2?_C(473lh%cnCCw-n+tK_Td3_8ING zd60N=%)tQX_2|2crK4)>=ugz1*6azq>5Zq5Yks_*zr$8F)9pOE^uGcc>>w0#%caQc zIf(d}v4wM^|MyS>;@8sMhP~9SoMXcjwI`0GR|aNizZXMK!{Ty5tn4cl7RaZGz;JR& zDN=gr8TXy$$*@*izGX>W)%`Y~cN3!M{tTdae_v=bJ5D?@C<+JbGnUSdY;K^9^qDR0aQ_gT6PT1f?W1=ym)zZMnhcr!Gw zyWiuUGxyFtSnhiU6MRavL9AWlxC2=ja~;E*(S9TLlQ*V_eOh5B-Ro;fd*!-Pse%n9 zyiP+EC^_wX%I)O*t*$qj$-5XjEU0W`LfZuv`Tw4?{weEJWLT)qX)c=|4K|`@avnR% zqg)F_sZjBu*Yz03;HYU3MTS4%sLyFnJbs-L!h5l^0P!t9Ds-!}ase&3Hu@9?p1r-# zp|pYc8JCO zA;E%2;d_WP61%+{A^!deP${2&%sp_JI^2C)8u#|aya`@UKYJmi?lHIL&RC1b2g_9` z1}@i}#93BVlOb;<(`E-h{ds5U%|8TPSraD`6GID?;Q+`yl{^D>N#qIxgL~Sf?5@WA z_fm%wewzgmn<_l}f`#dhcE&W+!{2BCH{?gP~l>UJsP>6pt1lGFMgryr@$|@=9%FI+HDU?MX_R$jZ(xaY-!R0=5saQ zLPnq)z)NL*6@3=1@gAXP)&uO+cF1ALf!gsd@d8Mk8F}6#2ig6wYIk{Px5i8A4LZN6 zEVXwb*NsVQw}0TT!SHS0AeSUAPgW`Yb|?|1v_e=ljGrA?>h(=YKLx6cwE>>m$|9E~ ze=o85O${T*pS1rtBcSIFHmn_U?qvQ}r~ABt%FT?8U0QTQh_GS7LQ02fK^5;xifU#? zjKdj~*Gz_peG?P!PmwCCrxo!l0)KV|l?0X^Qw#EOLpk;$a!c0Hpt~epS+B zK>pGG<(V_~gQMiEt4?aeyQ98FqQgS79PJWA^D3#C^77bV?qJy1_%eJ84G6p`syqC8 z9BM>(`Jzkhz_bff0x}-53DSKE#{;~(9}W*#ifzxZy<-6Od0O6#(Y&}%3ClbEP8Jfi z&$##`p0}{k=E8!4L18|O@*K$EAf$XsX~V*>Qu^VWn&?Vq^?Hn&SWH7$lJyrzr_I8{ zQ5l~&3-&NE4yjr@5zBgkG|r-*tb=giQrPcpDWWuXVPkQyCdjfrL3`FTh=tr*$(c!c4t&v6uWVtbN~9In-Ub%Zv5&} z;X*%sHu@8sY+NRp3}lG~9gx^YqTqUV099Hn0k z^`p*NxQ7hwD%wSSEy#DCB=?N7u=2^*YOeV9pa@M>ku1o~dnu+g?0e`@YCO2$C!Y0P zEYFz_hlWSW15pZcHH(&NninI}Pm`aIb?dve6f3B%z5d>eP6GZjOvK~O46ki#B`fPy zSLmA8442q?UpqAR#H8_vE7ASy_q8X(YDfI40@>Ba9H^2W7e^Rrbx27B#-VkZPwi%y_G+f&usW}K@m?E=pKGg(~~hsJ!o zaNxx=h-p(!uI6D&mVn4wNI|2DjP)Di6>ypIHufd7Gog@QCruXw;Ch@!SaIIxMP+x{ zZUtsc~jTu?W z9-W4I(_Gg7#|*nI48w|?@q)R?*}h00S?B7b&ni)<2059Z+pU>ZsgB{7(#Lb~(K^ix zA{$C=L+?tMh zSR7ZZeJ9Ghy@SQd=2duP>fCG%qd98B%YI$Ep~j~=crVg?^3ic}2fLcfuF`BwjlLEB zpZfS`0zxlhay6#ObtdCC`wOryV&RY$)ke6SjhuFiH{SS6;n9Q)Sw`NBXSRcbbM-|+xm z$;F{6;e%nXv0QEN?>}QwKpc%=t=CFjTMkb_#>H|EvQG&Wbk`UBD)-*Y{4hMg5U_8q zcY7G$d_`!-cQpz`WZE0=Je7USs~5U7-&*=3iw7UGkUf1ZFR)q&S@*R_%%Op_m~rxG z>rA_7UQ(r&&4n%tQFr6d%}Ek*pIBiPUEybuI`*Oi+0W#dCkYNVodw#ND~=>II6a5C{j7texJ=}+@N z1r?ZJ%bzO_;cLVyd+I0cHkZ+N8tuGh(zP5>)S|9B%T4WEnTspfedeo~HWE3IExjmT_0EG~+U^{@^$!bQya5`JIB(tPhmV~Cgq;FjR?g?n@?{?;^b}2;+5}jk zW5`X!0n(foy#{ zC~R$cggM%MSnxV#TWy>8X^@9}ueg8^2*s*?R^-94x?~x)cg3IzmX+KiU%0O=LvNmy zCE8u)(zcCzD!>ib^!R4;O=087b0IP_Y7JEaGrc&?M`bD~U@NrpzL`ad0m;#Iwut9E zTe|FQSd?604G%6c1>bgt6-P&|{J5?1I&ix1wCklE?CTTo(2dBa)4Z%YW`Xo~@pn^^L&AYRebM>S#kWBe?uLhfqd|u<`I8kopg^YT|0X{@yCNKa)#niMu6<5oDP`--nT8DJ+EZ7+TsQ*jhAnl z5&n43M3up%G^n!kQ-xuV9S#_;CiPAHxnfdwUsECkjPErFd#>Nr)|>yf($8Ku0I{GP z=g>{rWs&pa=zJ6OZ!q5_PI3D`fZyLe>qk5+v5DDqqQ|^JEy?@NDn##D`@4<0=bNZ; zq8tZ6gR>Sc(M7YNWImc@;PSbbWX1D$Cw`z{RbIZq$q=|#1N_lV(R(=}Do}mr&nyu| zdj{pQ| zlxFy*4>!i$Huwce8q5!L*KDRya|iMHb$1LzfpIkYX1Rmu^J&n5oBSLEIyb>t_4a`r z-2C2j^oFx8;>n%J0Aa{-YoEg=L1(zZYnfE*xCErlA%8?|i;_rrZr%CfZ3y6$T%9BY zvW*g$wM~@MhX0ecsY8I}sgVJF{(g-`WnM8O%d@yFNh|8R){DFKU$u7oGw&K20tmPqJUEcbc2kNn8iZcX)fuBai9?js?zqg4$n|Rev{#CAS1M-XQd~aGPBYn z!kOni*ifMraxmZ*oTCDP z9OgNUy~r)3aSV6fG1)LzJl`>sl%h<3NL;{?5G>r+)LCiJ>&sjwlY|MoVr~?Xlug7TOYP39z z;vfXfeHTgpvQ)1pdjw)_pWKw!fWc16+Ks6DqAM#7%1Sn&ZXXs^2=^G_(4v~%N0T5u zu*_6hb#@sZ|-rYtE*3Z4{;5iCGZ7unk2+#*o3MiiktH>{D=C3Jq!$NgU-kryQC*v z{-dpyzV-2RIO#R+flu^XibHc zL@AIx3$#{skBQISP9P$5meJNTBR?r1IJ3rlm#tO2sYKA6c5^DgATvoJeZ@HWsLD&> zh|J)z>%cqUWRe^yIT4lZsBcjDBEz4Dt=-WSG8$$EKP9C%x1KwvM|=sV%M=}-FE9OQ z?7B?e_a8wGJhmm?c5NpvhS0{>nuh;VH+4X~#0GC&vSw*}yK1dGDl(XzvaOVDgei!+G{zAdL zfY)$A*vAd0lzwZZ#O&xTvDvy)YJU*_N)t8O50C;xp0rE(?9j7KmKxxEiQz=CRtdQb zSSGBrQq@0~=fX_*u1cC$3NdCDuAGo~Qs5y$fxB$j-r&#yWniu*Z^ni>8WpJkO5= zXy3eVFJ%?s&gkYD%J3eE`XJC-Mn^i2YHgP9$PevTpf|g0n_?|qu%}E9t(3LF#Zh>m zLKKi4{nNvaWvkHP)|+a z(&FRpfH?=-IMtMkXvp)JtyC29J%ab=Z^6MX*1>jn4l>H?W;?8P{Y>qGZOQXG*)N@# z`hYn;CG!+$u@0--lY|#IxIv0RY;X}%vfl9uLsm0FVDuv}O?)D0X5`a9(%Y(p2rkdy zF8Re;)m_NVodBhaTV0IU;u`deXhbpD|MZ%w9#D0(vsJL`%PMb-|DURpPm3vzGZ^${ zuH$`k`ZKt9mv3d6W20a(<~lXr&H|f59p0^|YJ#^v4F$2R!u%Ic`JL7GG$`Q(EP^{M zay`Bs5KWM7bGK9j(Ih!E>#kE=a1%aBi+RGYA|dkY;Dy-hf*^Q+^SW}ElqBwnz##%C zXzoWQ2{6D@cIaGM7@g6|aU0UsQ{AKLO}9CmE1*#luXyT#+%KsWT3Y0aw=y`X2UYYP z1NL*&keOvGrg|L1)HD};QB-_kJNNQJnkQh(CEjmKEOsmI8lg0<$4 z;iP}6x#Y5(b^7;3z|{%=Q05Lqeh?d$69(ie%VuLqKajR8zzUN!+5cGrkK@bM*e`z| zFyOadV9{mmzu3&Gs-M)KY4`1ek-;G?sr6~nokLOL!4};$l?6P$^0(&tJVSliLDKzB zbSma}nYgHacU^YGxA(r4FDtR?KG1?1*rFmN=kKL@#9(lg>zkJNhuX=NJxE#vQHg@D;Gn!Zch8*r;?R?MdE<$0qmH`}5x_5$Vi=p0s@ss+RsfmLEx z=*gFqG3Tr-^QH9+)k7(V?x^nxl&3W2WmQVU@sO7y-w%JE{EZVC;1#=0mf}-sk5AnG zch&?qDc3M9Ev0_ ztXqX(5J~OtViiUsl%RhiJ;9CwznO{6d*hIwW!EKgHlpuXhV93{HdFY#A@ZmIp-9;- z7;n`Yb1@S|s#wpQ6Qm(h-?6wl2FE=kho6UKE`LW*1UlOf@}~Bid4Je&8U;jbp`j6$ zw9d(_kskp2NGItiNK_Fm$WU2PL3xbli-@OQCeML{^fF11ZauzuCdAlyO!eN`=P7+r z*2~qnaG$t%zZ}=9ltISEGc3zA%NQruZ@0EMq+IRqCFE8*mjwXf@WaRcB4nh-v9^P~ z%PV1<;~)p)UBGUprG|vn*!`|$%|w(f+HyvKq)}-4$J?~2W5jx-zRlmLBA^|U1RO_4 z9=2D~k15EBinb>AqY0`Iycr@SE$2NRz-<%0%1ghhGcF)pp{SSf@mZ<6iMqpyjn7Wf zL*r@BVtCQpdu}VJ5fFM+>yq`8p_r+Zj&$G!hj6_DiY#|ISswRfzg5mqh!0wBGd>ho z_PoeScARl;HSFbRi?E=^v|cEg97w?BiG2LC-|AP?mlE5zBp<1;^!zGe%tAYY$p&NU z>aTkuoobyw!1roERm@BV1el1ebWi$rYID*kGoH?GDmGtyb6g`zXp1RZ*P>(1X%HRB ztB6YL>fN7&e^lsbSy3$r%KCZUn^y^o!o+@IXN!|WJQ6eYQ6>1_C zM^Ft*@rz5YDYk=g9z@>2w0H%dre(8>uR`BIyZ%g9pnnX-E6#SMBgLWg;pP*!55D|J z57GZCa@RszL&zmCL)eigZS^H@p|}0XY9|wo1EJMQb?(;gZ!?37I(KC$Sq8+t=~8-d zB7^QPoN}(ey`+p@+?5xt8*`|)i9t?%FH-SP8skQ$r1lJ3>V?A{QL{pEFadH>G-D2AO?Yrhb^E$VDqr^FX;31Vdj~Nr1h3{#^w6CUh=B{e zLnqqx{QdOTX8+^-+;ya#`1C|>?1k7S z)xNvEE$qCV^JEIVJas3lBr{1ZRYNQ%MhH|^ybCH9 zw{_>FDt-gUid?2XS387IH>S$BO`p-MosUKA1j+JMUbP#(yeiE-|A9$>rm1(-DwHw2 zX)cW+F^r4BbQh>s|2#4mUcj-5Y>)QWu2xIcfi)S!fuFA)tGJcOekQ{d?;DVM2A-H1 zE|%@0Ws3c8y)|JXZj^Za=*~Uj?XX+I+pM3k18;nz<4jcJWiY1oS&t$-k|~4`{T3P= zd`>Yedy!0dqjM`LgQ+g;S)}k`OluZdB0WzG&se7AZ&}OaA1gJ4yDOgTg==N(I+ZX4 zFjD6-IBkEXinMVZ;fa~6EvR~wIt~VD{g+;b9S@)q#CH}>{hx|d|&7&$&fH^i8n}Ih=kSrc7TG9Z>P3U}-X{@Fvc>a6s z;v~_=Pqu)zGHD}|uzW@Gb3J`d<=L6_;*+d1-}qa|TxHroP8FNG^yBZ8Kbd_Qkt4jb zcn=jK>YQ!$*KpUy^8=NQj17&*og6u>BzNWB71Oj714`i~hR$1QKd9-=2gNO(_t@g- zk(i`diY?C9TNcE3f9j~ss1oY2h^gsE)BJkc%CI!x+?`Nq{S!>^x z%Ue83m3q3f3_ZlAc~Cd1z5n^z=6Lp>H?Me5Xx5vW*BIjVgSx}Aq5?F7R4DsML3k;} z=_^7_!bdv+P2T{d(Wz(XoagbZupekz>FRPuRG~0)$VL|k{rEVcbkT?lQJMZ5>N-tivic?u30_`)|Sig%DMned3 z!@Zn3|4PD1htDd;BglYP17_o;ORt;5#rSiSJ86)Z4dwN5+1IsE;=PPbGAHmi#+H5* zFW~hueg+59sf7C*z`o#sij_bY^_s@K6YKIBTokZ)4#VfdD8~`Fz4r-RjPiJzL&@JbR!48-fhAu}n({-LB&GcFly)#N4 zID|y>t|KpQc@%;yT2Y4jP4HdyQ~fSZ4u(;iZVjg!*9K$L_A?D#u9$M@DClQsA|z-1 zzFeKe<(`D`xE}~n&ySS2I9^3Fy=?)5Lkpb~Dv4i+rt}v;W+a_j$XHJf!i*Zh-6O4db+-@2X|@oSDM$R zE}hGO#$YR@vBA-KpA^w#!DL>@OsvW##tetQV=6e>fyDsp_c+JufwmoB=7->M8Ed;U5vi1 z_-57O72#`DbB_0;CSP8@Ib2}V`xwaDBPIKOF)l|q>&r*Y9&BO8JEoUgTXh@0`qn)j zX?wqxJf^&ihdqV?n^F9{XFKJ4ujdg7CJFJ?XITfwo|n4evsqHiQL;jL-w8LqxTwx= ziVgc9B>kN8Vo)p%>t!wBM%mXOhELD#R!KB+bt@1bI6ZtcLBrH@YLZ#?ygy1dzV+CH zx3@&Y@mDk?ni?XZscPaCJ<-RCg@{uST|Buz#S_U?pbo*bZRu9*-6E0EC*l{CZKKI4!b(yqP&wjzyt*g5;}^BOKSph+F(CA+9PNz?B~$A88+>7Zq{&2sJQj< zI?wK_?xb6bvoq>=P-$U|Io-XFW|1Wz!DRT=ua6`t6>W-#dS99TGATZo) z8RW~Yo7nG5P0!@JcQ8V6d_aQuaMGn(60 zza>82Gud`nWu{T`Z|C%!MLLM~?A0B5=+3=cJzRLv9-5thpC?Fw)ssX=$UKJ710q2_ z!0`8P4_3hIyI8fbCTUtMW;)4xbp&$*nu3m%%h=^3D^a3Td9!3KA&AHYQ-xF6$lOn3 z*VL>n1CACd+f~P+)!Lg%Mc?FF!1{1)2Tq=|TaDrlb(=-Sp!5_rX$E>L6HC(0w*^Dm zU5zLc(+d(ybIay^-$^sDpSznHro5`sJ9=n!?rs@`{A-xMqtyMW?~ve2HmovMVwi!JjGNgf$KWwB>}B4pYU3f!`6JP8eyd8N8Bb0 z4~4y;{%D}xFKxa~%sRdO zh7TTZ-F{32^@BE^y8Wlkoq;(tFhbcHcfrQ5ens`DH}=FW@ZgFmPb)7FTeS|t;p#w4 zpOm2Y@%DC99&hb-U{m@IV4V;){IutI>7Q|DS7_2%^XqL^2jjkAs59hPAfN4x4*C~= zBacD(=qwkx$Z(};IwUN>IUcvZq)#!yy#G<<6a zCL>1^_afzfUSYIP*`Bu#Ej;sTyP$W>Io?dgM^(DlgK&~1SsIuc-ODL#R7gbRt8h(I z16}M*&M*;|bjTJn75uj?yUnW&{k*Y&TY~nS6!r~K1hLKGnopqT+KSt~BqLk(wxXx; zPmS|N9j6664JL|=?)RKW`Yat;)#4z3q?|lzT&ymc9TUPofckf``ji$t^of`ZuQA-)Zf|_y3%KSbycOmr{ z{VJqCpWqod{$x3v3*!y$UG8Vx+-@c!!d{uIsi->`(r4`96>mHty41OLeM$VR-sIxt zUbm2O)=hwI?;fitqvX_u>3dQi3FQ-qps3W@uviY_v<)0&LJDFapR<3t14i7Jf@2q)%HO$GeaU2152E>6SUMU~*5sH2L+x1)b%uzM3LS zBNQVJDN{yH*LCu3u)OM-Dxap}{MJ6CB=sqDYgj95cr!{U;&@lo7QgqFZ;%<~;jIg+6q?{jj{J zlwx+ir%%JYeRl=3_S4$c*v(_CeNLA(`%iAtopyPr?Ooi)pzAMv_mbA;!REmL%Ylk0 zi9A!I*}a6Fo%KD$uBne+azEINVCfAcU=)p(*<`tI)D8!*AZxc2loCz#6%AXxIPuYi zWqAb0wt9uN1}Gew6+Q2iAXFcXiXTmorHOmgv7`f$(!2P*;<@QI8arzc5KBbR;|gAS z;wGdRnbRsX8Xv z@Zw~<`cYPuq?d-(&LyMYEyjIMHz043LQL#mS=kzd=H$|M@G)t5@#1tl=O=?34s>5q(1cUE zW{t4Mhpug})INBAd9{2;ueU_f)n4-wpTq0OfFyMHq{m%2Jrw7qW=nj!+7-mT>Goda zx~cWZTnWmVzAa`L;?FNA7G8aYm*o!9n zgPb^vvj=`{&y4WnnX*qy_xekCcn~@K9)+!Q9dhz9Tx)wdVO=eP=}*sdIWWJ$=S7~| zCM|p&Q+Upbh2C>_ur67n;nUQ9641>ZNTfrjscY+Q>?(WN%~f@Dj}YKV>b_aE^y90C z{o9~DdMvHZW;O@6QS-#-)q!dUk+dTHGXlakW2M_aXKW{{|AIftXYLTyA#T9MbN6i$ z3nLPmun!VH00)cj*)bnwu$b`S?1L72thYh4uH!aKVWsUF474ZN%AZUDH}~_TJ2Il_ z0}%|>N;?LWq_-hTH=#279&O>&g4QF&+o3On|C;;W4hYtf1XO&Dk&iZoX&$*++hep? zgiz0ri#W_lv(j+zXRCP&tZe(?jP*@9Z?~Nwjqs!>XySTJ6I-)yyPUYbp^G}zUprK9 zu*2TqylNye@U~x6kyESTP;!pvl;j$Mr#>T|sZ&dmRzvK*UqODGs-F-xD(1NQ%iPyz zQ1huGa{_yPxNUU=E@kUx4~8Q`IYrA_pRV87(%o*7wN>`gtqfl5ZgnC^bE#}q_1t;s z_)uQ3{k| zfb7(`@0bQ@PJ#_Wt%dpW)8=FiE?ez8e9GG?I3Ns^6goze{_%mOavo zPlpSM#)LLVn6N{)8O<{1sy~vkEx?Mb{?LE$FAXBBeZv}80#3|t+`Tv03V3O}BY=8S6Rs72R9sb0_L*}_^(>j=1-`!(r+(p!YOqd6KG zPsoucEppgJ?rn1nQIpNp4}Na?T(kHV`{D1StrQDWbtd61GIY|^Tt6VT?P_6{G5fdE zLr`&Ka?_pcOXJ~deat%CN0qA-0rA(j&$6U>kh;vN68CgpEojAY#A86(Z)A@wuggIn z^7F*MvOJtoI(+xivTUgRcCW0v(GQA8!~BM@w797ou(gq19Y=$#;WLNp-_rLF7*)kj zG3f$_{R}jHFL%fMkrYh~UEy9dt0rnbzH;!ig$Rru0-Q7^VGH-ET`|+7C7S+UVJ;41 z!@c+8Jj~Mn>1WXgbQX@iL}awv%IPw4UC7y+{;>f#5lo(AxgYK<_)F(;=U;gm5{Ug>ATyg%#X zk1e^-JH9U;$Gxt{>T<`RO@W!$ht?-*^DL(C)ivKlE(}dBJ6`B5 zoc=f^S3DCE5!_^I%Z0w>b}zm;=RqWv-NC4Ox+b`nlkw}6YOn!pv8oTwg_2NWm{k2u zWLBEKuvQ-R2;}saZ+b-jj^3D-Q~p<_zZbal=d$a^r=PTF+0$MS-}9*@H7r~GIfh#b zo!R)FiLjpYGy$PHICD#1UO8Pf)9aO(zC)|4ALPIMNIqojcg|-Qd<%Dvbj&@yTM0Tq zbkzX|M9TQb#=m0@A@4l$XtjPsVHlzHr`6EK+pge>I&WnrV0%NQrucx!%p~RhP*zr5 znnQ>C4-C%e!B5%?RQF=VuutVZd>qaqz8M>*$ANLIm}o79|2p z68i+OiEd3vzwzAx#sir@ub>1+SbdwAW+wf6kOo-Mjrp3LMenm$+#QdpO|3d~$TI{J$YJ|Ss6+~;l&CX=| zR>C(laN)F~Is8PV?G#0vVr`%Ev}d+%qo9q>jMpEDQvZ&$oF9C&eVZV%6$|wqP`TIUGLNxhgRgxx75&=V@eEa-+z8B zk~PfvyLtRmBR^nl<(`eiI75sbNM5sISxD?#8`gyFvAGfY@5ii`+6TL=sY;y~IK&#MnrF6LZO@d75rU)>tBq%hmR&%^~9smI97F zUA7rR>Lp)dn{W|A!5rbdH>!!sx|MlM<82If@aCqe`!^izLZQq>GnK06V96RwT6kZrby2 zDZHA>OE%xj$wV_H{xl5#{+EEo-|J*T1OpB2y!f&*;nYiDPr++s4JhJala5{7E&&gH zP|l!rE7}0Z21}V+>n-gO*u_4Z7a-LW^!Qs9lqJ{{H(hg`aA+@hN|X?qKlJgd#xPK; zMl~LRgi+6r$5D@Nvb8_O}%Zw&W)uag+UgNmog7_q@^R1LgGt?;gv3yg%j@q z_*K#W`^)tu;l$8Btd)`-@*ZLNYXZ3o5n_iej1VGQgzy*JCl3Fa0`7lsJmTS$AJ7+rt3)v8@bB7}$4e zr7?6dI2O5-w8e0$K!?IGOcz+oeeTyp5&)xCeS>@YlYs`(ZH6QIZj5y1WUiNmA!_f# zp}vh5d`w0X=c|f;1NC5Gh2Fs|Do{N%Khs)oJZp&hTKfOK;l4~9z9K|06n6?w$dlHt z{v=x*j3qCAtQAyDeS`R*boBN0KkArHQ*h~+j0ldtBs(2|L(HQjF5Gnr?2fl*4?&kb z0)SA;-5Dx3pm>Fr*pv-w1>gEh!>z)0@`;CVSp=}^e|8FBh!N1En?QMb@EEh;=zH2*dpy~#FI;;acB%3jex zUI`R?H_z!GzInF16+*%kxBtzUZ2`%!}QvG7v|9tBl*An z!#zM_D}T2!(bA~@B(VA^;$C7R@if~~y7Bz``eR{4+%Kh=(sbo->5+ob1>}5?U%E5L zEp3&F1p-D9S=Bi9HZ$I&si=>^^Lr3I8QMcGWL7Vnj-^Xo%M2%nM!&>=&wmMTPGQB= z<=8vL3{PyDN{*&)$CIqx5_gmX(~NumPKd)URa50ftlTIcAB$txyva#JNGB>V{{oAYlm5pil8rO)02RI7ix0{-tJlV& z{Laku0jOh>5h3^ucD`IEn30(o`hjYIC zJ7eRMMcM~!lg}T{pi57aq;pHfR+F1pJ@)(G|7ooLZm%}*X3S&U@+C#5ZY)rC<{1_} zcOu0UJ3U^4F(>P-%$Z%=c=@}jZ5-upFf|_h#QdU(U#Xp?{N9I~ZrgZ)ov_*Wp z(zp_58TSM)GPqg)I={v&Cc&P3i>xi&zvS*V?cB+GfyGVx$O4^`FXDuBg3BgSr|O{> zFIh2cYkdREpEmNhA&aO}a87F8LH5qJNj-5^g$3<;-ARd7{v0v4)%=f!<07^+%sd;X z@0Ujk1Vu?LApkVkN#42Vvh=%99Txq6i{`=I=sY!y%bh5{n}qI_O^_LXY%K?jx3RZr z`}qQ(gU28g+Vd2+YLKtbwBXXETqJPozj9jYApri@bPrxB&xATc8GI|zH(@UxLh^^8 z&rfEU-46z}p`qgWhtO>_a&v3n7D$EoR-m`eHXAN4`VOzNr4Xm!X%Ld>y+daQ>(>B? z^vQHR5ZuzFP=QKz%$HtyH06bBAuBxD6hKV&xB>hTK^_LE6$MQ%i1W6#d z5|1+ST*)fFF>`jZ*UaJnF=YZs>3#?U;TOa0{bEc`Wt}fZ!o)wSND1R+FrkvdGJ(|A zAYJ;P4?caiDAYvX<7=PN!WJd6Dr*R!JDUou{F*~Ha$HnQ~Nay(<6W3g# zA@n!SYa(=d1#Fo26#P}62y2nZZHulti1notwc`jali0QTn|*9xiuRSF-!t>!r!Z0U zEtN48+&pBTH;6{jzk?pS}pwWGMk@r>RB4O9L3EzKXy_f$5sz%6k6X@k36P1l5+oKjRg1p_?U`jCN zvxSn(2tY-N`E=*8?L0rr!FJe9arUp91(-3S0(*~5$6qDeJTc^g)>cgWKJxxXAHD9+DM-6F zgCACmf39@?gJxyp-;pfwTCV@L!IL>xD=i)71Plm$(Y=6t05uSoV z&yf?z(07J+W2Q!2v8dbQcT(MIG9gNA=*7@2jWIkr;f+O=chXcv#JSIy0_;h^7Aq#b zgO#!=2=2X`Z@AgpHotd0j=PSfh51%oYq8nl?W}T$8fq@K!hTMQ9-A%l4kiZY`lR<> z_v~I1fZDFZNZS;>K;x2D@V!U0X_l}j@c4`e(i_$&ssm3|gJyoP{hHS(QpT?FkeBN3U zdp>FOgnb8wW3QrA9nkZgr`M<1sHeXH4a>&s^T6a_o-9{8M4cT|6~`-d2@}S{LZ4Vr z>c_#0b$6QqOSs{QqufEpLp_^}t?C~j)Hq5Wl5kKf?-}&W(v4VhX?$y%oXc3IK;n@o);ZsQn z`lVuUX4WZlPOqbmCuT%Q*JAsryhN6;7I{io&dDCEHn;8dz;opo{)2)*?cY_yI6{n& zOMf22Ii4ULPTtRtW}`#tEBN(y#!5Y%{KcsysN_qLtav(enw(WP0%14IS5q)IO>Nh`PV;Mu}}&MWFOVI-pRcwl3#sDb{?D7aj;0l#4R7I0)@j0jD(L}b6~J^GPL zz`1G&p!uesMaB)j!D<-1hSdW-qHA1_4oA$S#6jm^dED!fKSha~CVjlmtyOo0*^M+W zxsj+}Lvfdn+!bahENQ#lgvnFL#@<~$x^$h4^{wbNg4^V@$9-NQgQB3<5K7a`zsCWzzi*{#SeS#Fnl zTL!sPNGe6|o24!b?%2V?iE51o&~iz(Gc`_+nXsIZB4Kt~s`^x>R(MJDl`9Fx@XOHXKRSh0;pVI(pKOfMTt8Zs(!RuabYuIGa z&IYOWf4+YEGOB%(Jr^-~dei79JsF#QakGV+NL9$--pmLt-8^SeEosW~OffF5sPOWO zK~Z7sIZZ;l+8xt+8wq!N zfh#Azj|;%7<>52nFSTF<4jca~cfEuq_Qh&2okGk}?rO zCGr=cw}|+Ezcp<0oh55laA~6OX?x4a`c<75%RK*YH5&6-dzwio04D?=G% zzCA9wgT;@j`a}ILY$yL)+%7kMycplII-G4QxrLUqqrt+{*D)ttSFUDv>^!c3p>cQZZe@M3_F$>; zT$;EU$eH?g68|T^l)-Rmd6S4$gnibpi>L+1L?3sl&T16l$P_scIqm&HZV}0V~?Hh*HI7F z`^MQChC^#AXHX$^R-~3!{zi(7d~8KCEo>~hp(H}}>RjdxmIbq` z^Kyy0D~_$PmufjrxnVXyb7@x=)5A1C1(2EF+%HBt;QH=hjiFiBhUH~pe8ds6D*E}!u7G@ag{D?c;!bsjTyRlrq=-byhu6cp# zTawxpX6zFYFcJ3ui1+^>O6=$SJp)DdZ&4F?1in2EBl*#?_`R%dtN9D&a+D?aro!Be z3Wx#5SaoqG&a%uh>QFCeMb}g%|DqbtK*Vu}DEnB2I6x>~^I}tfN4`ULw&OV|SiWt% zu5v3jMqT1(c>E*WvZDFF`B{ycTw zg+9H+)PeU%ny86ctucr&*XqMdlQo-0j{p6&#VG@XM!TGYv!4H5qPe;fdenweSA#h^ zFtoc-fux0IQs1RE5w1cyh{OrD^qgoe);8)2mC&v>9J21tMYhs<5w7d6PZ7FW`=^kI zCbWlfR`;|yCd3b8AJbrJQMQRk^@MqZ;|)BA`*_V%kc{S3Ek!iyWfEeiES zd0tYe6(*Jyr;z)PXN7My{dg5o28{DSjKk`Mr?mK|3{nL;pw)|53xk%|=@4@%{dD7tj`bg)dWBT4c|M$gq+w>!!7uv3=zc zzrpo6TRi;2XLy^kZPz>^VI!Xsjhh&0!}y3b6w=hJkRhp-X2vF(>|Xktao7YN3+vk} zLe&S)S~5MT+Nu99l3Q}?dHC{W94p+`x#{j?o>O!U?$3&ZwDlHKDdsr<<*q2EX>EXQo^;N;$@oza8%%9Xz8V^LZkf#)n%6#?StpdsjaF z6k4H5KzShK5Lc=C{UMZ?L`A++59vIq#lR+-r%2{16d=#Gc_&>hagaqVBE4gn$OFE+ zhAQV$Oe|Hn%MmA2nPYfZftr}n%bwU_oiFTPeHawkM|*RIH|ohvgNpb4ad(!9Kt)bJeUjWKu4X8P8B+?XI_Llzq_{iNZ`Fl2G-Of{w4Q zItfG!!|2g!Pvw!=HP9?IH9%W3{lee^=BTl5^u;?KZnlEC^l-LQkwL2+=hp^~rRJMx z9-COFq<0>Uu@cKQle{13z^&Nd`4t&X7(voqctL6UifY{LYF9RJQ~*r!sYZ8+i*m1^>N}NE}d(r7NO!K8|Egd(mR6a;@U+L;KDu&CsL= zvU5ZGoP6Y8t(@l9er$`-dx_$<&rlA%BF*UDA$?D)Rgv1-qZKPIi<6mQ#JL3(1dey- z0qpjlVEg0xGBs>IyKq6-csKGHv*^V@x1Dq33>S-{)ZqhwIA&43-5|K|LQ5^bHjsGe zj*g*)l-Ig$+W2f&x&hGlI7~J@cW-2}>hI3f(#f}ZOxQl?;Ng!6`~&UXmSdkSYc|C?5P_sws?Y(eY0P9vYx zn*P@Hja(If9LJ_C6nSyPqpFj2PeYbyRmLA8Bz)SDELZQv2iDr)%Q$(Rt$*Ec@SpEgUMt63 z>@9nM)0hW0+J+6WEUTNg1h^ih)kg1eaG zdH@IC68m^`xHr_23(5`=&(%qx7mKsV^Da?9Vr7=1{{39i4q9fQZ+5IB1A*-%% zV!z5+{)h(S$fA93t=dOXq}7V)E&rJRy!#~{f2FxxZN2cG7R~V);>8_<>rW*896!Mp zn2;C7Uo@Hd@EYgVxF0Au*XG5R z$4^6>B#qMbsOIi}+6f*dKFryM?MA0=i8#Z%ob}$uwTtskAW~Z>vgr_^-*_U#d{Iu{ zQnc77DvVZN>E1`*25gvrZ3uQhqUkU3Rv0$F3PWIOy}90{J_PLASn%v?D+M;6qC}@U z(%y7+^c`2QZNgP!g=&FcleahEPA&ihw-}JOzxf`np_!qFco1+XRWWJ-P!~?l0z2%w z>DD+i4D>;mA!4P$1fFcwo;1dEYV8Cu8zK+S2ie_Pa_oAmS_mjdUzVSYe@#&sy3#Hg zMNTZUUbrl9^KkRM7uS!BvV(xcsUS`sUy0dwtrpC*Txc)P;+}{hHwQU#8tt-R8ls}x zOT0j;-A?G9tb;ejISc zp^RIXc{i+1r%NTFBjA)(isBrnK+-Kpkc1462D@D^LHl&z9)J{pG>CSt3gB z8SJU|9XaMh+!{I+H}X!dHI4X9$9dWVv#XW(MmxcY{lviA{~VjYGu|dqg6w{z`u8?) z9%vgl6uVn}ykyqc4@^x@B`E+}O&iu9=LW#}yk{bVO?Wf&<+~&bm0P2J(w|m78)XxQ zShWBL^48wF55Bx6CY6*Q_gqPfF(S(%A7|?|@)$dAP~nk}_s~w3LR^j|XnWi=>e{86 zX&ZZObPCchwEvUUcsQX!NFUq3-ie8e(Mep=fug!rbP;DteqfP+0(f{l9)(Nudp1Ly zJ75zDMzgASr9f9>+lA-Xz{u>+0wp7jrO6*ixM;9DRG*&-M#Y75uiauZ(q7x%bgmw! zMp^ZzTvWL{(;#eZPL*`QgI)>aIV?S3kX6mSo4~!iJ!XD7oU@T5|I)o-Nyns3M_mgVA84!MgRrKd&`y?}7xc|RQ^P&tCKZc#kq_mI_J|HqajaeuMHJ=C3p`c{m9R}shNSQoP;R1<{Rq-4yOCA-5d;ppPz|4qG0RUNC6I?gq;Z3 ziRYl^P+(cWr}%)*lq+k)*|0&lMRw>LAg(mY_d)^EH|&!8ahrnwQ&Kfc#BjTjf|MJS zU~YqX9sb25a%4VoTv5E6VcyxE?@#jl>q4fGCibeGwo|VJXldAkDB=zX!$ImRZAnr? zd6A=l@yQ5p2yI{U`gShf-V_*br5ifc9)}5+D$SNAB~8gH3CmtU68<6) ziXkq40H;uL;=$&SV0Tc@QR6dahH=mBu9qUOHN{I8{n&0jEi-&LV#7U9`sxx}UFb9Z zP(ym=qNs+8p{g&-3%a8k&M6!86lLBpT=_8Ws?VH{XGVjy_lxfRbICOGV9`$y75it_ z3~0BaD$>Kay%Jw8Irt=4z{azuR!mO?L{8PBnO(!`!<_{yDK@gR?ml6tW+c{L9Bc7Z zA={2wHP^V${Qh>heGQQ_>#;Di?650T;*G!zX@&qJlz74v1D_eQ@0bg~T@s5R-GHky z3&|#k=rN9cNQokD+weNRPDkML%%akDa?)ZZ3{O?+4ua$wHfYmLF7fBwsbEul+LzX4 z%SDZae|{9$r@OG!zcW0KFnZ~gWiqi;Nv#m#w$WQyX&a{Q_OGV`nA@#Gc+Cpla~IzR zTmVhDLg0yL;NGdoa~x9xa6SRV8CWj`vq&pG@^dPL3um)7sS{XncY!9f#u4D!(?Kaw zKUsj;wFFL05gdy>cm!C)dHmC+D<|ie*m?>w>rM;>f44JD|S z{-EB${He^5QSjyFRKcLg{YhIrqF3<{U$m!m)T#8GGKGQNN=$KN^>d?LZ=$*6iQ?j( zChQ???WomD+T2vhAqtl5hiny;xX6EjMu6A7KHn#Z2!6OS-he0PEklWrl8Evx(6_k) zo5d5PO-96fD?!c^yy|hIW{vJU&SVeFJ~@wQWjxqRBU1)xLQ>knpZ8G>Yas1Z;MFG~B1|%2~43 zM2XUSWw%mp1R>Tl-@i_~{NE|F%XJ7)_>q|Q&6;jq zsgTA>5pgX=XROUihP~(92-rhzIW;tf+c9rRaV<%7x>nHBXO({4S>Ulq+0)AKS-A4p zG*V=Qas||nIk6Yk)|L&zeg8=T@k?ChDv5x8QAXt}z?>|>jIkrF*kJ^hxn?z3+ixDi znbdqwf{_s-DL16JFS~zp)A}9ak)|>F{1OBwQ<7Q$y+IN8&~8hhyaP;` zmmCYahn_psoC+vl(p|p0(TTZq0=>Bc%N=&jbQ9^SB#7nxI}UUsWjYHrB6A}5K7%T( zAMcyJOgff&>pj5hOjy~nNT)d-<9sET|2w2-(bXK&C>9a#{IpRlUay2fGQd5kQN~c7 zG=7Jy$kiFEie!`#L;7=^TL`BNy0m6O#PMMDfJ3xd&a)_4pod=4No^vi6tyOay!I`- zT4T60@oKmT4Z?eYm`fCm`XE>@{z%gg#9&*s8e?MH5nU#VNTF9psQIrTOt7RpDLazjB@8nX;H2GywjIJiB0uR@_b>&58*G$(|Wjl;3V7go^Y?* zUV{5eg1J0B5!uJ2OcO3E>hKDFhO1(%ZcP5iBAN_^eD8TFpcgsFhp>dH6Kw6384yH7 zl)Enuwvn~)jmI|^Q*KW`fG#^sxzq#joAIUhLhpM6#XGIP3Vos4NbO}HV-proB+c!F zVP$>W62wYCxAc=99^n4x5y)I5*^Z$?B;URHrYmT5qJnGh?#YVl63^EkiTaX;75UxL zbU2m4d~lCB3^3Pyl?!C6brp*DGIk5-YQ&zplV;gY;BPMYvFf|5VD0<;Y4faY4td-A!0ec=}o z@;kMZqozhc`lnlbu+#2G={&r!0G`SwP@8SHGtYf`Tz`y8@*W0E8TfOb4TXLx~R*}_I(WmukN z9N~$?r4&(!*m!oNPxYDJeOI_ag1zn(`GC!}sUAU)`vlMj^lqZfyS&!PP$391gC1$o zs-)M%`INJo>mwq3-vN5gYXSPefpD2za6x=L>r;NPr(UeYR$ldwmcS zZ+B{E@%>R{D=p?`LBB=5jjQ1>e^qWzCwI^1a_(Qt{537%_A{G$bj2`ZErBCQ!YQZ_ zemgD3_H~a*`JXxBGjo7J_5<55hyCr9+FA;9-#2ht+HWolh}d*;4S|UXj|!p9_p1*! zH-SCy0H7M#r6;<-VsUBWFzKlR+}r_>NZO-)HuW-nVbo_^LC+sH?eKB2B=|H5;2ByQ zR3XY700XH6NHibcIl#su^Xvx-*in+JH|IwxKjIaFMQ#(%L{CT^aZMQmWs*Bkx*S)K zFA@X#le~AMmoQK>24jS?lk2CaA&hJZM2_XNZ#A|6YQX|M!0vAWW#J&01LJ@%G~^6# z);q6+8tg$Z+Fh9)pWzFA|vSW6|_31ngnRvz4FmC++p`W2z~!Gpic@A`48mF zPbHQGn@VzNv zP;#wa_utDtl`ac)gaw;oWZrhIu3mN7xPdQQ=al`cE3JDl^FyKG1802a$0M%CG@RHt zH*mVsYF634-c{eAHQoNPp?gRMQkC*+|eW6Go1(e0ryk>U5jq%@l z=7`|T)(`19`oTE!gb-KmEmcbwJzEv5+BUToY{S6LsU=t|K%K9Fw0GeENZ1*h$GAyW z@iJgM1DS&Vj4v3kA>+2$?swP)KbTcV!k`BoBHBC)AYW!=gF^5OCn!na^S=cg;UrF- zBn92D;OE76xl6?A(sGJ;2~>(cwYhgV%d;kKv-^m<&rK*RzwRVb3uP4&W7giMJ>6$9 zLmf*@3}5O%;tWzb%EXo1g44c-$Vd?Q`E};B%08~9sp#Ev^HHHG|l!)|Ix_xjCI2F1bz+ zc79fY+k$)%9iD{5C6!dj_&@jtDuOy&_n9uREiIi=@yvI52AynO8u)|+Ad5K(jinL5 zACmJ_usj_83e@~)MkvWx5mFroprWAQ%Otv_)abo$pj;c0enJjV_HqCAB|KsEq#(C0 zEC1@j!IB>pn*Qwph{y8M%Qhz2(Cvl7_Cfr3l!#~TyLC)d=c@Ar=bXo05 z2_vRB!TIL2CriS1q)5LE9x2lBAv}aC}*%76HW*dBZSN!33=xQDr`+_=bj0iBg z@^?ppC4vopSd9~50^Fy3tgt%p1VsUZkw+Mm#<%1JwUsV?kohI< z3g$*QlnI^)Mp|(KM}sXY2Rw`d`hz8%NpK*?ls|*+xn2`W1?$IK7nFRSXX83a*DJrjnDs%My?^q1O92`D3G z=J9HP?a51U`IETo>!s>wc=C2VmGtoUY~5`5)UT35W`qQLE{HC+_{CKBVhj8CF3$ALrmh{(U-9$tQ4liQUpng1z$ZP&1^VT?S zhpg+zHcq?D$@>-$WowV#`H@7uP=fSn5jAVJ^mQg}U9k>yZh{%A;1)sMZxK3cO)h~? zeRgSTHl9!h#`X!)kK<0kv#?OhBtLgPZDaoFHLn{3=grouEy6Ig7;4uF9IjG|g^c65 zy5dmko?kD8Zy4q>pP(YfMbQ)N#(!7lN}T?+ujT$# zYYQGxN%I<;`(t%Mi|D+dLKf)W>%dP|KKW!=fSj-<9AR%G?|jFe3~8UJdrsNr=+Pky zwOD?Q%pW6QM1Q>=d)eY?ZNPn8;zOgfkNd-U?Red@TqW2vg9--Q=7Layy{#oSu1;MO z>K&)^*G3sG=%3nc^w5OEuT~6L#vdoH$uazRf=4F%e8?ThEmnSLfg}^&|3Y>PL|AFW z?iPL|ku@_v{di%0f2PwLFy)WaVRiEJB4vQD%G00EuIM6x+VDzg<(YyrZvGyK$sBd03ov6M(+{%*fx2dt&wl&DMx=}umj zm5ldc11k_mcx@^={A4QtvX!6UW0+j_+HUGTm!X)&xYw#X>d7Hrh)fdc@(wYMjpbN! zXjk`?r@)-SBIz;{k303XmL@YecakikVo>VZx08sfH;xiJ97}-yi(FQ50YAW zyL`R0DWt!491~*M{wM^9f zG-tQnYo|d^oob`b`PNU=AGF(I?BSXBV&Q;Scl%#55#w<11!0*=w!z9PhyW?ETjiJz1b6YsGgGudId`@G@C%=eyjaC$4iD`mCP=PF6hOC-03(@8zQUIe!nDQ+|!HJ0MA-LQ7lf z12Xb!H_~llANYc%X?K`buckt?qQj2+ghxV~Cftsx{|B}sa|!f)PE6==&$Zc-OAI1! z5+5C7^>}{mGlNPHbXOc!@D>X9>*L5@;YOj%?!j4x&W^^&m&YB$DB<8rVb7iaQTGH) zbJgIEt;554X76b!OU*S_mOLW49ST8@@$Lx3tRoayP;0guvKs(@KHA5C{~IpRK%0W3 z5IIhhaV}sxwk{uoo#@ypLNPSA*y?12%Xt=Ouq|?Ya^smT=;w+K{N>RA8ag%Ew8S+c zlCInbjHPFF5pAOC1mPi5!Bp_=*Z011E5AI|s`Hj)G7gfXcvwC;F;n_eDESkQSaTCn zn{DaQM7sZg`(NK7!6v<6Zu|tZC<9qu#+6>;pu^iL{(?R=u`9x}TMi8rS&!b))*L8T z?8FEhQ(-$W z!6e5qO5_w4vrEMfgXcN&9Tb|r3dh{8vdMTDO}hSmrfnfn^&wf+_0b6aArd1 zR6KldaExV-z4i*9>i74`vYd3k4_0eS!U~<_);-S_#(ZSdXK<{+y?P0g#`YSM*V-c0`hbU!rQ zjEc^nJkSv^%yh6;@zZ|=VcB#N;80VPVhy9lr9U({b0d)$qeyboUQv)VHipXvbsw|i z#;VjJLYy;_Ox4=ed4XK^*q=K>9gsjXw?`ao%uO#-Z#9Y2V@DiwR;i#xpJN=Ea#P;vA;H3tTkCXdYDo7lL? zFB}ZnJXRuX1)nnkO4`2(rqQ(62eN4!lWCG@ZnGzc4dpPyrneeKd>fxiR2Z^GhiYFE zc)=CCKU*!p&>}V)`9`0^uR}2M#o?qzTI^!&GI1|(iwOAm@2)DoYdl8;l4^Sg2Zvgd zC15(1kJqc5+yM-eLOk3J8te}Lp?;)LdjOO+xd4XfNb(pbWd`a--5#prV>3V0n^r^! z?s<@rUusfI6xRdkn*FB`n&ei{0k#Bbxh>47Vkx7U%%6hoiTIx{rolj%P_x>J`AVAF z@-x zFOrx$&(+so-2C|Oo|7k$hqS7m(S@Ch4Q0IRkt>?q=%LEVx!t=CXipV+?M};K1!J8I zT|fOtkj*PwS5a{;CmJ6x(i$vW3lWynhi~Kyqzj%0P~`8AT71r59l{2HoLWz+6+W%E z{XoHJhmWOrKR4hW;vq5fI&h({`mmUS4L*!lF5>Y5eCoRiFqS$nXEA)4Z!Ag5J9zKh zFp>rE{YCE8_+z2KoJ3dxNE%O~2^+0It8F^&F0k>6OQ8@&eX#i_J;=B~pw|C_vxP74 z>#6fPytx_!gPNL)XFDW`oG8bF#a|}$r^8Fe%K9WZRhRhCoa2hvRAF1iPE0x1$8}cT zbP&H`HOfeVHlMsZf2SPy>i{LvZ7XuN+@cYIsY{|*90e=}i=(b+n2_@cco9_>wd`Hw2Q9x@Y*W0#sJknr6_okFZKH}$*`Rnut zmJ#atVATu(gv8dsv9cI1-QC3FB*TRZ03iF}m~;n{e}-NA_~Qe-cEc$b0T9svP_mtZ zLdKvWrwNV0EzWy~1^qr2d-{ua`$rnC_Udv@;=!%O@*CHM!bQGmTqUa>JpCiCVpEUZ z6jIZBl5@t*+=0y@zmiLlWe$HQn|KHk(tXb+(k%q7H`$e}F>f6V=kYMAzw%EDk1VN5 z=$m@b{!Yh}-G71zLnB>k0HB5Hxxl{w&>vr^I}n1-T&cKii>(NXNvL05WA46nHn%4b zon->JI0S32TWH&DW3*!&KWZ2Tt#;0B?#fL~NTw=N*oPK$ui9)%3RZ;S-Xd5hMoze4 zW{Wmc|5S?BQiLCHi9#li5}H#1xR{8C+NN{ER?W~9YzXyEZ;LIvc469 zH-!_~FC>Cl2&QB?Gm^`J+Jc4vmPOS~H01uekp>=QV|nH8Cu5mFdyg@_ zH7X|o?)F&mEv1i)-SXKEcES&=&X0yL$!lKIw}_u$bFDVc5Z?bp1G^Q+?8PFt_ zAbQIu-8+{{avKE+zaTkWmz*1+#b*B>JOEIRg4a8czv)fE?VNLnXaz;%Hl1f+$j%V4 zD5aZYdeOC*%qRhsnx&;8nC-G>)s5V(l@BVtl8y`*=Y(^|9Wn0CRm&d!lfekK^u#_6 zHHqmfESC=tcPJ z?gu!+h_55opzh$f@7~y{kmw`o8)1wlqN9jH)mOrw%Ux$pL4%$WO>u+O9Mv5caXk}M#cj;|4lk$OI_rz+;?e$=(EWGL4jWOP z36siE!&WvsMpOo=d~pn?K>I?B%ipY9Vtog8t~~ArUcdEkc1;-xRxhU;C)QRA94ZnD z6+4~P(#v?=hDTf|$%zmZ6SR7sldwh~tRx$E+l9s`bG{wLwZ!O|jwOMq!hZAj{8@%q zhXhJ1ujnH?bDr)^ z3{k;Cuz*O$1waysx9yEitF#@MOdeGq1VXyYPa3Ua3TONNH zx}tkoP!UCXDcHm1vw#-N6!gCy;rb}I zE8z3QAy&;Ubmzshye#*4M&;u9Y{4vj+m= zQgGfKXGX9xTHpY;5Lg410>HiU;6ZIIcGM4ePY~mc_A1V6LHQXr0kif1?LOU>XplFI zHx7dD>PMf3@Tl>ANpbGXJ@>v-7D6ag^Uk8?@-&baf511IMf?sC1_0}?R}0qaN(77@ zv&Jw*dsb!fa8{{upY(HQ6#r7rs479|vfxf{PKytp+z6?HS~CsRog=!a}RI4d4q^uufPy6 zzgz^AX3|l|s`XCgdbP(4mYN?H3R>z~idQBuJQhD?q(2Y#I1+4C=$26!gL2Av|I24% zmh=Ile_YL%3p4#0ro`Ah6Kw%_*p7)m|`3>}A+&g%Z?{fLwynC|Cn@(TQLidJCkM4J8MMvH6PU!$XwDOV4Tab}Jegtn8=gEz z0Q=0oY(xzBjymBNb6WbIzrY}ho+vhWnC3@NZt;yIId=m@@@#}7A#98MeXD{MycvPI zQTb2G{%6PjsBPDjywuZW+rD_l(W{?7W^kNl<|dIxt5&bwLP8WWY{L1B&l|%FtGa%b zjXCZ=BcnJgL4|tG;+A{My{AiDY<~MI?!bU$2y7yU(Ve!33*Mu9Ni4t&!5!FBKXzon zl!H;Tm~$$z_kgKhCEmQv9RNRjPc?Y%`jI}zJdbHfmtYgv6e25@T*NIUe}B{pSoU14ip+DAt3uKzB3|J);R5nxMO zQ|MPL*I8i_+1Aegc1kBp{hb1lKJejUBWjim6ZA#ZmC4Ud2=Cr)sgM-9#rErTm4|G- zD|3PD4~-sL;-k#Y8jcwG(JINa5ids79>HgzbyVvhM3g_W2#?$X>=_)nfe7Q}s9~&^D9)5>25$T*s1^!JB zPmSG`B=T+h8V`ZlZk+?8j2r*+R)k-JJfQ3$vk4FA*4o|aewp#FP*m?QraAh z8WEh*i}5kX*SnxK49vYQIehq>HT}$zo3ULCm!JVfV;jgzR7JlDXCH0_vhCLx0hiU~ zxDhUvgjHa{A!m?pdYu;)4Pjd9$%?52d!o0}dkcFiw!69n1wB57MJ|K(-2i-Vfey}8 z0kQ{1ofBN?6?sZ$kgoWz+7U*y?{?KUug0Tb#yYA8{yC>x+!}r7jcU*x(&d1dPIb7f zj+$4Gb}HEAGrT~i-MFNb{45+%G|Ou*toA>@axW(q*T>Id*om;$Hk3rQ?ELV42|xfN z9>utJqnu%WqqY5yVrXD%Ro&8Z?y!&T4*u&m2EvV6xkryeBcm7S-poC!| z@nR3&f>YlIloXJ7Den^1MDZT{sPMeY#@R#WQL6SJAQX{b{J;)7G&wB`JLJFy(-BDD zwJ~Fn^vJ6OqnK+c(bP8%bPV01rS=vI@tTjKn0lTk$dgR83+q3eyWYt{`nZkad2cyt zRz&1zB;^XKyZ2zz#pl!^)#<*MF_`B%0;5A~qUDDv8lm9MeUXDy8@P6dS}ZhS_AQy> zcmF!1ey*XPe|nzv_hf7>y?w>0tzTJM0ou;fb+P2h;=2Kb9-wRo!Bu~!k~XInfl_`V z6?H1Nq^&fn_`Y!?CaO6mk_to@>VQsC>~Kr;F?M+EN5KB>2mJy&t+y)qpoQLAwej9@ zK#?c$6!e%NaD7^~!Rq4kqWCLT;lNs`GtY}%`uM{0ZSrS*oZ)v4^OUf7e`LU($ge!y zoj2L21qXYOWIsEVo9k?6nva_-z}4B+0Z7$|@k+SiXJ(k7R-o7!Z9wV9aa$8EXfZ&g zB%Nbg4w3b{_WCIiOf2%wIn_6(_zfDqzdH{;DCJE^^)uJ*K$QpMW=TgE7X3d6oByvB z0(v*Lgt0@J4f~Y1{HW;PUuH7?RX~)IJYPS6g$=CdppJ6NtQVaXH zeY@*f9fUqPXsPQ7u(>zbpa0U{d18-Wk10SM;mVP@eSdmK;!saha4`6xenSr;4k-ymFw!<}e*Y!`yovep?D}%X zuG9b*l>2!;fQV41@2!`W-1lFAo4_CV?RkBzJ&iqYG`HL}DbRs}*?R#2V4^K~!_&+5|0UcI? zyIGDqbl_Br!nX$dr&D|4=TsByaYnoGa)xTH-XM(&RA(M3s)8K^mBRjr@cK5ptJmEDMeD98}`wlRczo&^5kvUOur1a6x%UpYUm5K{(`;Jt++ z-M)imh~H{w&uXAxu6QRgAirt@Xv@jyU&%ThT}{2HT&KK8c8e{>#b)zoCMIJ{TJGMkA@#qzk|B?jsL|Hky?}`yGN+j~<{+puV-FzLYA36ZstFjC|rS9P5 zftqHgSiAD%j_^b`?&0mZxZma1UpM<_x(JZ!wy_?sYkDKmrwV1+$OqVs`I#R*`h9*1 zzl3}jd{;nt&kUK|`?I6}PLX}qgmQytHp*4cp_1h{B*jMeT8lMd9g(2s4Sbh>_YeN- z{qXM!nI@YTGoBe`2&huWtk_Q5ssBMGV}ZQ^7fuKr4b|JVC=qaf45%$IU!N8dLr)s9!# zPWPd*^=W?|?g8UO@SV@<%>;R{b9&&12s0D-9Ya;6r^PYgw2W1DT|yX?3*m#EtX?2$ z;ZL)D+M6zu$`7X3c5kS7OvSwM`fYjpcYOSF-NW5+6C)q?cxnG>L+I36VscDcK5ekg=xo3}D5 z36(xoN>B0Z2b1RCRqCI$23S$+xv#TRbM#+Kd20!Fja#z;qjtrZ7wDdg7x#`1M~~!h z;o~rr&I8zQWP{%7q7(x3bSbi00nH-QP$!gRP_M*ZFQeRh>pNZ*7a6Yy3@}Z=uq=bm z7JBzU)#whIhl8<~%reMm59&WQ{7djj>O=V9`FfB07t&?RI?)$+f{&Y*L?fRM-0els zs#5-E)A^y!FQ=?|e?B>I-^GFDxbxr|V3Znd==@jx{pXwgxM=2lSMT1oHeDLNc3I{FoYB=wt`#P^OaptSr#B=xwD>c*H^)Z-5nf}}kG9>>w-rE)w3%o!)wvL#3KuTF zzsZ-$*7OfL*hEYqK5^^icn$LM7a_&xRGo7{|L1o7b(5$F+K#kDl-p-hG#s9+q_ll7 znzQ4(Rr-j;zlK6|_#RMSMdPjN7b*Qgm2%0R1bEcvM zUqzs#rNb^QDpUC`SUgG%{_eih*~exqbJspb=n~LC@pup5a6YaCT-G0Ee}|&yAK6(0 zWj0mf_E2P_(|)Ss3v)rUzmDI4Mw%K`vrIMbJgYcMZzo5nO|if~@b8~(_#YPls75Tw z#%&ax#^n#7+{@AMqf$HjEMEWHk`-2$xgy9yZ}8V(U4?{)_eLU8+NT^Uul$M(8l zJ+Kn9qdTXPw^%wlk_3FMK7xXA99I-g>OheBH#`TD?4WPeSVY-vPr#zDDBv4vb~ZG|v1O3+Pq;LT`RX z#{cRRedePa$jtP_e;vAC9V19-bm zli~{BgWaI$4WQs;fPQmQ!B&m~_{xOYq3C&SKBYLUR`#&{XdlG_yaH=yzk?-LQrkFCj^NRoNd3QE~iqN%ng2J35?uCLO{6>n=|L1i2^;a?`go1K8k}&C0G$j(F`5KoDv5fml|w?oye{kchGCIee%|S5lBhtx6LKQMY!wlFIm-oz2Yl@ zk4-(f>EG)tpc(Y2pE5gcn<`=|xMtm=H?F1q2YvlN!~W0JtHgn2dD=OiA$UPB6mGYZ z?QY{>d4X++;~94TKv%2mVy0T}1yHzvr?B065#ATtKIlH%$qnmzUpD>9-8dxy< zr}!8pj2*^1ex<=?YAlvSLC%=<;94TdUn2xg#kK+gA?QR^M-ayYAK)Ml*2CPpx{AmlR8&TKYzryK;->nOVcD zRrkXce>%I>*aLnTp2HVQY*mAwlQc&6vEvH1u>2+poOVDrVyf%5gl{r}7Ed?uS@hj& z6s@93*jt_*1=e?M`11uEKRqYWEkT7)DCFUggQx_kN*4j-Z&}R$8Q*-v_ldCnA?n9d z)Vc{2BoVZXZeYGfIdDQg10}U;OR%XArm%a+`^no7wwjy+DD)j2=+sqRA18d2W4dBl z6ob^lf;+>)c#=_wJc1$x@>TQkkI{1d| zS2Om#?CXb1Bfa3)VVObLI}meC(xhir^zLR?^#Rwi<@hICrP~+ktRCK?&)Rfb9K4#{ zjBKADkE(X+42)vYK6Mv#gz~C^U+JT@Q2>f`9rBsbUX#Le@D%q?OgyM_tlZFc+Lixk zovW3umiz@E1)}}6pQ+I6-#`mR{>);5R5o=#z(qPlyAyA-daX0_6I>n!D0gicb=y;5 z^WAfb8OPM*?%Coa8Y0iEF{ z1@t;8_EE-*KZU=>c>?kITl6k8^6a@}6*fjs%NMF+4J`!J7l<;uQy3@m7f)DaC4@3?n(C9$*{aJ z;#y3|;aint*(Y`ad|GF0Qm?Hn74G-0B&1uRtk9KP1Gq|~+{(VS>u9xF1Ryu@q6T?p+e=r&N}M5K{X?h}a zmcg6D49`xcDA5`xqF?_$z(t;&_fUD#wPO>AwTl2P74|ieVS)P$TduLq}UPDp)E)C0zXHH@E98Zmo+Z zqRbhuR(G=Ev@&mJ(h}An%GtLGe|+JoKp71W9c{wORartoj{m+1Ki)S$?ku(X@Y|}} zyH_0Mar|5(3xB;6gfNYpz1H;kq3+9wfS*5xHh-Qij4i0L=Hi2-2& z_|F)y&;DA}CHmUg@@pbbcI)S_4P-t0_1*q?5`WeW`yE0f5(awg@gXx7yZ5?2W1FuX z-oZ_&{&PvO8d%&$w?*YYDZODNUnlmj&-CXFUJuid zgigEW(Dty}E1T2h8vFe9@dKo*yeSKpNBN0|u6&!ug}>_C%Dm{{cW#>ZJ(YzNoY}kk z%vhwgvB_S>gK1txZWdO~#{G|zTo#D8?G_}E7RJk6@(@D5eFE$#qgjvX=>>9D@9?pb zyNTX@8UNP}B_J_>B_sJYv6qZ&=@BMwre)nImg@JuXr<|Rm~yyx>&uyWn31uQO1~QJ zyh;53W9+KKs!X>%ASf}4l&B~m(kLiW(jZ%5`Vl1O!Atq>=8DZje?$=}zg+ zO>@`hjAwM>-1EYyD~stHt)mnzWlr+Nm2lvo;rJcgfhww7xwHe@oa9 zmm1^Jae+Ef>L=Li$J`DtQr}6R(B9bIwC7sc*|XP5)Sjzv(UM9kSSF3)A+z?y&^r=d7}{ud!<}tJo;^ zORe==k}QKQRNyHa>wTw*!a)f4kOzk)XhAP=Y=zV zK8{~mL?7dS7ASQvHq7X`e_axDUZ`h^7{eHr6uek7+^0gPU$jcjk*x*!-cQ)3o0HDy zs;$t)3vBV7Q&Zhzw-LC;>6iG)DN^jQO~RI*5oHCzT>a}Jz0k+dPe{3M(44D40 zQpL->V3H-&tAp)OR9>70yGscw)>l(heFcY~$;}Az9ZPikOP%SBn^E8Q@b7<_kphbq z$D&v2yz8Aop6KsSSGbn$}{KBM|hbU#Gs6OJ2`m-us3tK_F58lPs zU5c{T>m#3(T*y0_-=gBOJsUg~EP42bsK2q*K1N`}I5JwA-%>LHow^agRCJH~%y82a z>GHLY6>Vi+s}Wi=w!sy#wSylByQ*{?sPb7%T~n+k*+lrC{CN^K&|wY|Ft#1XpfbCz z)-p|~T|?O$)6DyyXZvGyxvxr7e-(Htwm;#Bw*UQX*%lVAF!mtaCgmrjbi~x;4~s4< zP?Sb~y&Nuf>6MnHT35m|r@kZ$4V*eR=l3M90G${0v0FqfFF8(oclYIR)P`pI*xaxh ziI5`|u^PR;f9Bdms#YZPfdg?!^0np#CjpK_;) z6g${k4X&9PKaoh*rqFIY6@g%p#?WXp=R_S`WCo! z#9Y&F$xVqByD#L+O-(G=wLU7&22-ZA*;6!^=>*KBYo6zlp;II`ytv1)C4Q6W!t zjOYv6<)B2!b>7%#t-gHI3a9mBeU;^?jw@vb=PkigY?Es?f;%r|e|iHGVU-$o5eD6w zyEnVI-dB`a(O|sViQPdO1tda;tKVcC!wFbb#Mar1gi;|~kI#9&E|_gK7DEOKkgN1@ zfY1z4IS?J4;;trUL!FOhi*@4TO6|2zXRPX&lA2qKu49wKe{!mDd7S)l#thmPOO%ft>Q zCG%j8K)_bI&0JUh{VJ?5@Dp{WskNf_?I zlnO5#Q?J6rTUpqqxfwJkk-4!lZiO&~nWr6@)dKcR@_uda%k$tVww$vD+tAhx6?@;i zA08H6M#ngR~O(n z$}q-R`>taDdf)%px0twrd{~-pi5+>&RaUO&(w=ZtZ6@h(U--VhnqIq+e?fKL|%H0IiYK8{mu@I%>{MGJFu-Wv0VUdvh zy^JVde4IvwOBqSZdCM9i`U?ZVS(+jmN~&2S6lNfXOf~oU)Z0((VAO%?L{re9LmC6> zd^oR7${Z!YS>Ao{ym?E2iuQgn0KC0SDFcxf#Cn(jH-|Lfq-d5DO^XxfD}cq?goS@S z3}%aTQ@ZY1?!que1PrsH5wiIW)S0CF%Ww%7fC{CsXA@K;j}8!F=O&y8Mb3!!&g?j9 zI^Z_DYRwJ8cy#9hjOUww>IGjaoP?w44nL`-Tqb6-B@E)c{(iy@^T_-YLO}~bo5g|N zUYx1P(|=SMe|yvof|!F}I?MR?KV~iZ&#tVs3WY6d%tnrau=R%ci`*%l7d2V%ptj14ZQC+AH( zImo_y(7?UTsKo@^qxK^8g8|5}EESGgefD}X2bT&Am&LC~_%Z(}; zj?BsK*vA{Q37lnwtG8TcE>KORQm*zdiR7`<8Ja&fi2AAW`~DE%4v3QVS{I?Ck^uO3 z;24KzoAs01)tt`vft!D}dnJAxT9ZOTRV3JG*Me>(!tCbEkpVLu%6*Ln}vm>C; zos-(xT$odsWA=)g+yXFWoMw6E$Qa1o7ohpigCXf37nH|t4N@!mM@cxB4%gjw1zg#C zO=CA4WSAe6Nh_dL?oLmic2+SXrJJqmk2>7b=hl>6)dzWnSw->HL@Sg;#|u>y`#Bhj zs{szv7Aia}1-}9mv9_Ozb`W@q~lz!$o(}*NMT~LaoDr-FApx zb?zNfj8Ri!QoU~>==7L6P#ur;9nVW{^;@&~GGz1jZG87)bgM2ZL(x+GM-~8a|HKnq zxx>rMM4{xxGpdIIv)+g1ckrgyhvTE1Ry~;%VDR0-fUDI^g;cE9L4c%aKBTA(*ZH0| zh6BZmo#ktSbumI}AWl8cML<3I7&v}A>rfSmZMEaajz_i;rBe=^cr8F%X=B#|y;Q~o z$o9ESKfS#yeQ20Uvl5BcI((K;v>v5z6tDn}^ar5tO8?M#mY9E5$b3V~bVqjJKEBR= zeMZbV3E`x}FJ#ckD7@5~{4aIBTqN$(TD#T3 zTge3I_{W-&JR>Y{7{Osi8=Um^MYc(`bJf~%b_;BYU!J`Ob{vj^lB;2Wlm7vu{bK($ z4VDsNIq$gbybSC6GpA-^NLToRV)mJP)ZwbvIT4__05nDp0B+GweR56PdYCBEbG1QA z%JelVl?11nrHR3^8_2eKh2dHHej#v@N*ChB)R-_MBaslXJjiptJ$05ib55DoLQ%H_ znGQpGU2e3$%$}`D(-bSV!5vhr<;aK{8H*%~WeJfq($lN59-={;3;q27zZgl>Q;?Dy zoj`RHw2U4#aq0tQ`M_z^RDiJjm|%Kr4256SV}Pq|wX0xN5B4%EFTs7U141gxY||?- z@2dw{cB0ydSfsOO;dk`KuXh%cFvwWTW8^&-drABoB28zNc&~sXdC8&W`V{GlOs%Z@ z?#OxB!Vvk-RXDZWTWn&_rRo@g?U<{ho6G!qGuN!(?wtsBJ9MGFRM_rYZ7QPXtU}aD zm5a9-tR-rH;+OM~lo)fy1DV&7cwM1OkxY+I*%831Pu8hT;VG{YnTdPAkKZg1XXwS; zWR~(Ovx$*HnU+zhPT4SDfvKsQ*I`w+%am}d$h=N+W~#MRlCPU#oCjhL&o-*0G&LDE;#}(&>bH|&G?qUYYzKD&(wSDe(Q7j z1o=3-)%9l7KUa{*9%07C#8E+Ht?Jf%hCxbkMXTDvI7CP-#5sKWd?vgV)})?pE!PFC z4Dn+dNUn(cG0JJIRPo!1zr}{$dQ6K$3#@LK_cX&`8}Kc&9QV!qePlt+ud%}0roi+v z4C@V3$`DYhx{!epLXea&@~|l@#Z4cdGBMh`jh$&WlJacgeB$c$(*jopDs&r($!BDH z6@m=&VUEuoD168{q}|)%!jQ#2F!14F^P_T@X*XQNr(J_fjxB@EB(<06WA1FrVOmy) z?;)*IBgvblPClr@(iU)dCEWN0wu+HBS$w&&zp4!RW2ny)IEsOPh*$KGIG&=z#2ZAc z!p5P&kJm*SFDt)+t+hYwvYSrQ!&GK^B|U4ho1km>en*iP>+6_(CXXn;(@EP%RznGh zOe2Kl*=W*ZZ8Z#@H7NCI=CK$>dc~j(%@$DAH}KFf5Lk{|W=y#3cIV<0{!SYCHWVgP zPW}rpm~3A2_3Ev;qf>47RzU-EIn`y~jZxygX3bbMd9m-ly^lY}3eyGnDBEgIl< zctXTxi%2$$e12_QjJiit&~t?I&n^==Dcx65!XBJ6sd1QZ?lcf!?6bx-E7Zy92`lH8 zE6NIV`tpq7l3PgKW7R_iotDF@MCz3ZPcxZ21NF=PPAdHEo@%29U$@{N2dM~H#h;;WOTv;6!w(O zXuHmfzg%qi!EPR}H)h)F`wGmLXokgwlv3}cj~$l`v>Sa!UX1IApM?a*6wN0iL#;Ke zzoJgK2jbhVUt- z)z8AfM1p8U>OhVoCt3rXY>`PCIK0^7am&H&AI>8ye@NOP<#>Jk)itj~zgd-hMLp?# z$f;Ub%tUghSTQ2}&M`U%3i1ho6(|Gg9c?iuX)#Q`#ys;rnKWYACL?|$!O_yoe%hP8 zFh$EB$4bMRY8m~srD!vbbk3M$#j9F!w)Pb1=h_b{QJP_*=GFGX5-r|QXmzA%USUqa!B|luZjxwV zFG4g3<9kN>f|^vKMChF$*w*URj4&QUfb zK4Qa=s$nA=jf(p8B4T~&nfi-Zs-x%>DrSK{5wkzLL*ZzC65zU%`n;E7cAV6DZeHrR z1q~p+pRk-Y`I0IhdjBAyLI`d1g}&#GeMb+`uV)D+?zd~0@%;NF8H9$xcv3Rr zK_G#P5#*xczeMUU>%TxcA&aW68Y*+_2TJgp0v3|>+m!=KecHfxGE7c%aRI8!(A4Hb z(=;pta&74vS8(vy=0lvaRg56_b{zwXKHaRd02zdMmdj*XFqTCj%B$uvLxlAkL^tb z2+g^}UxpRdIAu|>3D#i&zHhH@s4riO7aDr-;=DFK#{N?;gF@|Ed_}KsP~R4o*)9zY z;$ehw8c#NxfQ%)9w02vr=0KLh-1lJ8+$(ObSsTdsMDT)n;Z|Sv4dSyJ#IG7?QhEC>3@Fr z077GP0UIjf(a4%yp2#SX8F)vtecUzF`m(z&OgkMAVm5kB!(GP$pC1iM1gXb`UFu~` z&IoA`9LNl7D1W-^RFXNqk;=vsOch`jkzB;i>9J2bsCnktFD3Wy!15yQ^qTYX z1zBrK(s3NWPAY!F{@HNoHc5|S4z8^eEm_Y~l}ner%8{>FTFWc2I?oSwq@Nb~_m8iG zv7NqEL|b`bB+*obvE{T+xq6N}t?Uy+CTxt0fJeSpbz+07#1zG;L&MJ?p;fNE6e?K_ z^>iHp{;7!w{YLvp+? z=dmNLg}$wRV)G+C#QB`__}vH6%H#nanOZKjb?d7-LDst*TC8=G+Rdtt8{NE-ve|Vh z8YrE8{)&RVmGG8QmkjH=ouGYHp7+Fl*ssJChGn;R*^)$&^|Du@M&0qUXwk}rWb`C))q7owk>c=Q!3AcT)66Z_R!DpHE0yZyO_x6}&zueTtmCK$T8YEA49Tjc9(Nm*_~MOj*azQwL6)>ACYwM&>S> zTJ?|5tE?ZWsvCU22Hn5me%l@H-U=SDd8LO_p{o1FxRu;xd+gS`dxraMJ{Ow(r-||d zs79|o2oo!xF`pK5z4Zbe=D8;3qNnX+Jz1Gq)A8m3Y5vMo_H0e+!GqDMtysDWUx~S> zLx**tu+e)Z#|vut2>Xw<+Ju|O=k7H<7Md8Wa6(t4Wm2udmi}c`32!q*uNQACk2$BIE^(C;U;Ll^!;7k~GT=Ef8;OMeeQ$Y^D~Z_10*HM6QYoyLNBg zQmWzGs5L^5$IMFMuQm}#?mZQLPx6v@1eG^q=SSJNNyf{)dCc8QnELB*c+tbxO9vC7 zgmXVVT~aR0!4ekJ2fjkMZ{MOa>WvxVH{Q!lHekOH(ma!VK3%JV67#-ML)Q1z{l|GB z%z=6MqAW8u-2m9|whJS*R4aA#J2Y<(glJS|oSCT{qdWg5+jEM|nJidX)tX$e+n8nRc7DYevEI}#FaGytLz38cV`E*Jit6Z-|gd2?X|Wzg7j%EM>aoH=iEVWEt= zHR{-$=*UOP99inlu$;yo2A$XldY#}`((G1xMv=N*Z+s+jj1SDpW~}5a$mdjzr$={{ zkNJIcK5%O9bTOBAWv4SqM8`l1l?gkCNr7k8A?LRy*0(}P%k;NzjG%K=xYK;@G=ol( zn@8^n^7s(`?SJHMLX3+)Z9OkpC@`y@U$wdHpcEAqbrx$|R5-I;Ng`!4`Ll+msm;>m z+0keA2Seer_tiSy-B-)ux0a5nv1|0LXAKozE=H*Znrv*+sc{Q@(l2q^l%};B)ne`X zT&qd4aBX;(dtW>jI)Y!(Eo!;b*y|ysj9-tfRrfHL{N)cZ5Um7hi$k_NkSglGIXN!( z6g-;~)vnWov&xEsTML|;QS)m> zLQ&42>}>TGv>Yn9#%}z|2Q(34Iv-_X7Frx*`g<+t&bXqE{L_xBW1dL|7HWdmD!Ji# z@?l#mvV^=SU9t5PdpyL-$9JFXdyUX=3iHtJVaMZ@otSyUC&DP|=n7BmAbaPt0j>Mn z^J<2@WUE6GpXe#Z^N#-cB))}WU{$JXVRrqc;Fo)jU1~2QDRzE0KZ%!zK4rMTfv@vB zUQH9tc46ZsLoq?0gQM22r979>e~51Atpkr^I=6T$k8RYRy}^gg%#7)rFTY}Q>BROx zRBL(ATT_Q2HHS@su2#&2@9#p67i)*_Jm&qG)9i?R;HRZ4EQ+PcIAttU8MS+F;ebaq zQJX6Kqu=+-<)oimIX|voDF8T=ur7I3@HXq!yaoSKD3Q6lEzTuv71PXz2#6vg6X$34 zRt@x;?qh?ESM$}PWh1K4{@DDL^Z6ZeD1yo?b<)VW z>zVL{lgsEs;suoLibRZykF;*U+=0}VYUp*d;t}1Fhc)e?G%P6eWAF8wp}06t<#wdHLz(OLo3Pw>ct}!)kc=a~-E#8iI9AV(RpCsM0Y1 zUf%rvpaK0j*bQ2px=yL@A?#lv65-kS?#M5%p@9l3yXMi*jp3zMolXFXSY%YjgoK|P z`}i1nb25B?hdI*Jmy*0~SSz$Q97Z(*rSs7rAOG?ue|o9NM|U_mRBtL^2pv{W2O11r z$(4OR(?Z{nnTz8-wQi6`Pl$K&88O->O;%rw%cE%tOWih>07KH?=bB|>9y!_r!g)gk=aY`)?*;h#93RBX zEkM(CdGs@4gaEX&jHDgOjga9PeM-EaEz_54r~|19FAz&|Eq$xdVQB zGRMC&9eW3u%CVK?a|lh@RqpJ_2SA77nTO+bJ|AW~Q_a~Q!eFhIVBhn6eUKn++LQGl ztmj*d$gtWe@w>?stE9MFY^E`{Yv#Oj5|2aOB24q(*nUNASxx+?Vrcdw=BWCq59Xdd zbgKf31RgD=7a_DGt&i{AJRxPO)>Kvgh-J)*NWW^m%L$|JTkNH5rRyC6@43?DmIC8b zuIH3@cd#)vUWPFtCV*L8cIb!tYN#l9%@Ltaa*F2T@JSOxn=Tb@Qei0uR$aL(&T}l z_)bfrx)9k6jokOm7ftIfw&dfvBiT}S2AlZ=CflS$?)a+7lyf(g6Z3((&2kSKDzE)= z-yFp#i7YlF4Rew-lHSHUwYz(}ch{}hx#X?3(Gps>S}9972W@nk-!*QtUsoY5F`2(m zcr#}g&vz84;M-EdORfhOjNz#>%=(>;<@#@eEMe+1SciL4LJJj#`-^8t-(2gnT!1g0HH@|zGWP;!uG+|64x2wap20-Q1h zWlfd=+A2}_HS56*hQeLIP^7+MOnnX+uS?dT`&yhM|E&rbg&vraH0$*GbvAgnzNcM` zc`_gZf)5BQ;y;EM@rhYC=ek=_r#kG(;)w+(-|5T)!)_y50aL8_E{uagCE|;$p%*fM zcMjk|PNR+n(2I=LydJZd4AH*_=wKS)gx^Q0fK^+!uc9lQ+gwu<7h^*M6RUDKle6{5 zMx_;}W+Yv$FYh8f=;@X>MXvp8gKwL5MAWwZ=f-L&%Ukk<(g@wfgf8k0Mor__s0l$! zcX|DwEsMoiP_I^VDdJ$io$7^gE8PI)Ir99k4n0Hta{p4q{a)Iwy1UWLGR0?zIZD*d zdG|f>KZNKBT_kTi!u4P~-)Vbwtig>Khq+A?0@Tz6Y-W`ho(E(t!Lhg89+X6v6Cfvx zCEOW$LsTqCY_4e%LKm|^iQR%4+64#AJV;NtQCo_-mB91&|B;22124B1PEj2#*Zn*a zo~{&S$&VTih@}b{4UVwkWsHwU0kM(V?XbtA;hX}6fdME$%DuT5FT9@&D3q@jbv9oC z^~5$)gq+-d1fLzN*{2B+c~)|L)8DUEVF&t?qzqf*7U}!X`u7}^v~o;WCT|z8_2%h) zzkWvW?BxT62)jW$x*4u=3F}~k=v1rTj=IwRy&~Fkd*{zj+!90w*A=wAB1^O%{}jXa zHPmxwd^^y<53f>r#YJJ(Qg7vi4GzyM+s zN8y!}cnbnQ!nfEA;&O{ZZ_HTJz2ch)R8)Uj0qEVSTg_{CoA!9{j*#7z^Iwm2m^@Q5 z=EdTL<2H2DCe4Oy#k|&w3=UTfl~#z!KMGDK&}&(lN;Idwbn@Ck!zt7U`tyh^*%>Ei z|DH@h{A+g&J5z!#gOdyaU%>k>1D)Fc92v_4ejCBj{-|#wyJ4PmlEF4Pl}^Qj|GF&nGn4Ke%uo*ml#M^_hIc z^$xt|eeg)gu9$ccGp#s zIZCTylxn@0lagF@>V5OzKlIs#3KDBrYcn_P;BGI*edZEg;{o~eFE{3U+wsU5q(gOY zyf|SKgYrBA77ohba@DiN;Df9~9klm=pA(r`uCX)9tenE*J_^d3*#zZvAy3-1_tPcM zXbvZ&1$mwlo@7+Q${wop*lp&f*go|sQK>K1WG(q}mterk)usfFwOR@aGiWZ8)3i+?^!<5p`C*=utcg z4-D|?iNEm{BMYi87l|-P)g`%biu>DgrL$J9rA|fA9c-R&ZCmocRXdhWk13BP;6t70 z^Fr4IoBd3Qt_j>d>~nfe377U{>bTP-e=afAv33T&35xA&k#f79%Ro^NOtva6u&9U5 z|MqiLJP{zAgZ3h*$~d_ht#X%r_jVn9ZxMbKE0LemjO@1;890BoSHdH9Sa8$W4r^S} zDj(py9j;R0;v9TR#ebjHLM_iI$XKn=;=OYEz0UX^4JbIpbrU%XiCYan7u(FK(+Zq7 zmV$ayeY3aLo1C0rZ#@E$1({TN;Fv8!@fb3x@U$G@zjr2d)shc2@OJjj63>XwGZ4jY z(nirrC^>h?-|BrybTJJaO=f=g6m|4cbx>OYs`7#j=48$9%b1Cy*?`Ns63-_H?EPTz z+1N3Zm&6VJN8fHE7GCdmoZak{rSiwo8rQ}kfmNY?RV1y<*Btmr%X+XYH*Oq49^_`C zPuQ-s*3F;L!?P)!qXI5K`HGk^oXZrn8 zlY56vjCXA2@bcm_w&PK4uYwe}`30k#C2Lsymn^kTT7Nfq2tT8xb`PzwX094Hsm9G< zk-v{#9Mtd{MN|i`w%{V#*irM>1Hi0FRJ;C;eVXN6IKW!qw$x1ARZrn#d6pqwEmB8%7_+8lFk$`qoX2U_yi!z4QtC_AA zmM5_Eq$<4+bETFGUj<*j>TIEb!Vc%N%kZva5V&gylWO*~B`DGw9?PS6uccGTaGui( z^WiGfBAZDr-MheNF(MpHvID4iHsaTz3s!&gy8TBbuBiFkhs^70*EwjnIMUG|Arh%( z@Jclz^lJJyY3#~BgrVU*W>7#QfmGxY^7Rl%V!0g^Bzbp~1l^-Uh6!lWO5! zJyHgM^0<@qp}a#3x>uh{v3wSiGkQ??d$1TaDD^w<-6{$v8_k51VX>NVYJGjCU87|R z*r5Ii(;+A-=0W0_kLY!WGbj$=_x$*2apIK(eQ(G*4MTKI_nR}U=i3KPeEg?3`)5Pi za0Kgs&qFnXcGiy*bS6q-SdF;HNP1~!$-Bk4;ul)$@&fK>Gg1)HeW)w695?a4oPWi0 zsm_ne9KW##7`r)0*+DYpGNeU?gZ@*u!T^kU7>HuCPmDi=kXL$3ND&L7*AMXJyAUTW zWQhhM(KCx*A%=~CU1|DOI-T+)NYEx%h0#mH!>GM>2bck!20{>-`!6xBgs{9k{^OD) zivX)fSWnFyn(FZr8}|C-7PYULM7w{%e*S4amy;|HlhmzUI}-*qPRdd!k!)~hhyR+_ zjUil&-OUAKVoZrZG&0ErY@AfzX~Qc^V9k6)@tRr=<&Oan<+rvsKrl9gzL_-u!vf|s zS;+6fJ*W?rhBpZYbG5e#=R=tLCv&Or@rKbZI2hBwo@Q1KD(c?PA=e_Nv*42qKO^RS zSPiB_r4B96|Lp5uU&gA!wYk@RxDqN8(eiFNRNe}^f6p?1E9=CMpZ?=T|Mbp(`nY$X z@d$oN2?QR(^r5$in2$76F4=(&cK}q(rXYx4e4Kn)owjWk9AJa*fcU$9A-bI75RCmF z6A--(BG5Drrk3-^Qho?*)sdsWS$sW$%mFNhSMm^?Hbip58_Mz*#g99RF7;96|AWCqgjG^BMy?s?<>eABuC6R1DI2;Bi&ou6-S`C~fh&bXjh2xcyXV!`Y@ z>gYV^&g;5#X}(wPwcRpBU``{oRGYK41sh1dLx%#pVlf~H_ zBk^>vNX#5z*dRZnJp>Y@}w1-_$63oEN4teh4`ZB^?!r z(LNo@o&>>kwyWRuwQ&J5Id~-WJ)xY&H2d*HSSJB=Bff>gi!)$hdby@@DL~!Bs+PB_ zID5xqVMyqAq~gCn_&) zet^7Tm)F@a6SXwsfF}E9k)O(_|NqND&-FS3!vb285OGYW+MsKSV|LqQ8y6R1p9;x8 zZ|tEnZjV6`c;kB{HMldEv-NJr?8P7d&sqJ`zWtYv(N09mq5fi;DjX&D{BQK+-&%cp zLfEHexX0M6+WPi7zh|2UfskYl|1ouep1L6yKdL*Ec^syw5Y>skIf)1q4~bcvz%71D zsE~8Xb*cZiBP8&QyKY4sA>ZqkLK7eVOue!@Icww|8W-j-1a*tC?@HIvw;R8maB5t< zB?z#J`#SC(G`wc_VF`INs6tr_A5ynU(+JK!|0A?(SNr=k}e z0fA+R)gqL`P$5Q0g<-Bcv-K6m6%~s68oxlqe_P(va*Wt*+i!Ryu8svEUr)Tg!eskx z3-8Q4gPIP$snA%$o0jsNtMn0^s}4dZY>(O}eyYusJ^n{sBfN+k4pA6)k`?GEu4+HE z(uVTM7Dl%90qPV=mMFM0eQ_589kf9zn+wuhri;c%HC9Q(DqQw-h+C~bZdDO zoMlSADr>Tct-5*gu6xND0V!n_Z8NVvb-goL zVa?r-`*eBGGI>?9Q&6L#?R+r7VcT8B-Jf{~WMMGUVnO5g0ZM@)M>auxEd-at-f?EW z7fFdI=s7|5Ah^G-h{#g&48B&sWz{f3G>&t-EA>>SQ08W2!k9orRT4t8h=6X=oK}-k zNEm!NAT%yUDzY&19Eq=n|JW2X-m8Xd&D;e5<@LZ1eh9JW?-8BSCWsPwDbmwd5qC$f zVS5_veE>RlwJYczhI1Ic*$02cCWun_C^|zw0My#Tke+S7qTO$lnm(Xji@wUEH`PRh# zZKb*A7k=tWepw)%xTS!MbrqOv^=CbJQU)ST+r5?g;M44n*9{y2lS+dyBHPbb>wDlA zASd1jMA)e5w1PtPoC{D`=EA*Rl#S%&0>nFjR5)<`Wg%r5%$PM)^y-Jl)%&e6K=O0u zW@6;yruexagJsYyS<#e3+0wN>Jy=q}R{@1V8f1wfJ$fN0=S@-3ZR_dxFL;S!oRG1% z(nP-*15Bo{Z=?k=jtQLLx#%4RS)(nIVhMmzI>fDUnGcngp2Uzqt32JxtTBE<^5>>- z#}}=1-Hw>GZ9fy7ezi}MtE;K*>_M@vak_Pv_zOGUIbCBZ-rdH@ch+4^r%K@er!N$8 zIv^wSUf6%VM3%7;b!7mf3_1cPw^VpyET^Pq+Mg$Ycx#*%VL2auskUNA9RiHBj8$20ePKQ&5pdy`;7G>he_z%<#r&w_8V@hU>s z9Heq~TWTZQM&GvXuKd>Ct70Lw+|w%l<4pL+J^RP@{r23AW&@ed2$0M8xz|Z%ivvYn zapEL)Fk@R`{sfFKa%Q+pGy;v|nfNoI;@SG1Uqa+_*KA16SR*{3;tO>xsW#&0Rd343B9O+^e*ia|4nZ9_XhvZ_1|zA zGrj_N1%J?@+SOL`<9(x@R}3?X-w2A%08~L{PbK!|afD4IIl_E?2?p4^HllstA*uoOy>tkDE3&yyRt zrxvzhjaWMpE(#e0Ea=c+JPhNR`v(QyUpkWCe?$|Ldo;#H`|x?&9L}2ua0e)Li+2uv zkk(hvudCcerz+hpRqT~?_#lnmm~}msd_nFMcAF1IQf?zkZ}g`n;qPtpDS-Y9hfz zseT4QPc?IfR>jaW+22G1C9)ObC0KtOhqJ&4_5q$PITg#fSjws|zO@nJlnuu!-crRl zeF;@uU>4e2e)4H1O)gL#bv-7if|q;(rbp*e3Op|S5;D30e|jUhq*y$Few6iBox?9n zJWj0AReW(y?8)yS=Um{tphBK>yykB*8_Y!boh>tGkZiCLWl1KH?WQ-;5~mt$jO{D(vhq< zSO|*dLW1&ZsqW=*b{?Qh&`KI}Z&J{3VReNc!EHK6u5rp!~JFS}V32!ebeOwC*Tc zrQ9kgukRvfHO!L8hiD_7m8dNP=XAs?Hl+qvPs9Q^{0AK@{C>d{sROV*qON+>H)i;! zUtMKTGmbBoN!g4v8ehehy zR*e&aWmjAgL4p}=N3G8Ccxd`-Y%>9dO&S+j{Lp^A(7}Vu(p9ey+mIb^jEpRve@R8|Y2iq$-*b#G;G z)m~CmtwmU5&qG}X%l9;!jV}QE`shhKk;)d@ee1|2X68rrDRvMqP;l!i5Adf_2aH(n zH<_T?@-AbY59P@yrbD+Y8ye4Cj0zqe26iS(VCm!VXnW-^w3L_u@i*uLo;Gz z#M}~$$v{T9;*>gQQ_c#k}q$rTRFdsFt(IzKw5XlXvJG zgGARdzib99a!SAr(-Z$e2I(|T|LxTL$Ghrx0meM;ZEVJ7l~S7H_xx-0G*!u4c>)o|vXT}IcSauq%3(7OsPb@VUFN3|_y+{nt*_mtTVtv# z8IE2_!b_tur=6k1H`8psfEgDV z@zlm=gFR`S13hxYTfcc*-$5FOv&=XGiWTfx4^tWEX5pe*%&9)wz2285E4W=V+nu?T zQ+0HAe?{80zcb^EKx@=nhs$M3{^VUj+eYp<9v(rV!gJj3?7__3fQnf*{nHV(E)MFHD+RsGo4F9G5A4Z9&)g7BkIGEUXl0!Ctefw(jI zdg+xfXUv@xo(c(QI&UZki*6zFE!IQ~?dJ?Opj84S#Ezn${(Cs{^#Q?v) z05LD(qrhEz+aiP?I1KfX4jI99w^_bwgRLzMqF4a6D7Zcx*i;Btzsx+)4)sM!)o_1= zl90@vH7mHv`yShu7~Ke&jq$9)Q6)MU7T?M56wA&Pz4cuFA%X&_l+B4W0l zd5hU1)?mqD7S3c!go&4|r^xSg0Bi6sF4fEf=)k0=G~ler>= z7#Pfv;$k97^*YhyWlEV6vLKDnM_x_@#MED+B6C=VP=(9B^NBWe(|jv6BoQa&jCU0N zaQcKZW`Y<9IrEWmN@m1@vp8}4l6fgFpMcHC7E3h!{dyVkl6Z$bKq!S&w1`ETGf<0| z_K(Rey8W#QBME02-tN(h}{rz~3)+VwC~w1TXycv zqyr{^8Zfo6@4#0pmP%;T+Wwprz#waTSFzI$(3L*8nZ3+PpqM zDCT0OJoCGam@L%fFc{Cj^NPWe2#d#cYx6Dxo>fM61F#^Th>rQ*52n!E%hb~?ye;te z1D`QFj^Zh+J_c)0mQA0#2ah8Rq(_DhQx#q?xBTPKfye~Sbc3~cy7@P_{D z2;@)Cdl0Z@!35nV!L5-a6uj0+B+SfLLfMlg7?Xs}qg)S)1h&5guu3mM=@elgJ0|M6 zOZrm!Y!-sngHPQPOAdSK(n%{}>cJ!avMcOY^agGi)_6&<9C(+A;ztlC zZ_|iGu_*-Z&>)s$yT=9}%e80(1f1@IR{I9g` zMsqDa5%XrQI>-{54R(_wTwp8AXMIX(M9{)EydhUI{9@Aoot)o&7^7@D;H@mUwf%(O zI%4Ani~B{uMU&oP(?0$$iFAKfLJ*WxJ}t_8%Ep$!eUFZMzXJ&72n-qRVkC4MA+Cb+ zlzPSCe31y2SJvhs!y6{OocU;?34+o4i_%h!f+NMhu9C)qoy5f!E{r~y0yz^i3xmh> zWRem;$0Id`GsxRmQ?ZfJK!L%f^2Dzd~vrVlT6}M zMuhmj^bg*Pxk1E~CjmAZh8-71TbNhu=SyBpYj}b{g9vWe9Gm1y+|TrJ7VyN$y*JH> zR!+il0GJjARVX6~B`O)qP;#vaQW>+sSFU!W%E`7F?Tc*bBQ~Clm5GhVn#A@IHiL3#ayT&Hac_DK5s74xZ=2+ZWvIGxyY4 z`2RI9@;^7XuprUxsgj$ZD6qy^!NCflqb3@J!p{`EZ#Fbso&%(1vM3rpy=xnWEya;8+FpWNz2x*7OBS!0kD} zFTr-{eTNoSdNYr6LZ)wJTfV(Rg|KD5bRs?sQeI(O zOxmV9xz6?_$)yjdo_UflRDF$tIZNkR{(tZQBo;x!Im0>$7g^JiqqFs=a1ZSCQXf+O$PU#BR6e28>Gh6qEMcA4OmIX6tU2^K{I2x@mmg3o$- zQjE6Li_21j{ROeG95K}K#qgxMvQyk3la!wE^QaOUMn5~Xo%$1%E%#+2PA zhZC1iQ9_H%^L)@|0d#vZkbgP?O4@M+f=`-QqNq(pf1k;>%ksN{ufLM_BKcbf9Eo{* zLzfaMvv`0qbH~43d6=z^SM$jZ0(TK($gV~_AfgzHOEbp~oc1-=(7I5YOZBpo2%xL^-e$46k?LBsGQ1F;<%_8d8Xo z=|){z*Di;qyfDcFBJDWhStw!rj}7>C1*{jnu>&uw<*z4RPp5md*IvgxHY_n$`xLJo zc;kY>&;(t-tH`fky-j_@JafXbe+zLY{5a_U^&>SsCcAQ-*o|Y@P2p}aQ=M?#!Te6u z(zjxdl~a_i;*BY>Or$FeFs}}K;1Hw%OF&PtVO)w0LR=R42ovYo0jRt~*#nzy-jxbs zQqG5Y0f8h$O@gWrp|No1<^#1fl{%>srhwsCk@psR@K9L8xqG&jSnnEf(Vnm2(PX=CMt@e|-W!9+7(v zy&PqY%|umwPPxoyLhw=v(^MjU^=TJqu%(N7bFe)?hiTw0uLps9UC|;(f?0V|ga-4i zaomXS9T>(IuW%2kx1(BlkjEZI-0-}CmBQGh_}ahZWd7&)L9S;hVWT3|{t(tv~G|9@2T#0|Mgta}qOKE)pv}ZC|PRXV?Fi|M(uwYlwxI zF}N8sDXSkz%_*dBIEB~J_mJo;JHF?7+#)Yl6trpe) zVZ}#V;L+}|vfmZaw2#!Sf7ZC;&~~0G($5O{tIwo>p(1d2GLMk{kSV(*bg4b!;^4}R zulMS;04r3pECoEZ|L^&LIfng041vqkyC$a@-!%?NzM<9|2!R#-8V|+|4|(DL_!o+? zJzS*{cyBT5C}wZC#9gcNi~nD`9p)46O66)&HwVZ$wRp$hoFOTd7`0J6PZjR{-L?Nr zEE-VT5F(5NoiBq-mPN~LnoPZkI(tB7^n=(V4dmTsTTMYM>99c)q*HJIUwhXc4t3i8 zWtFTg>9AAITWO-A6C#Yn3NdX?InDH@kRddU!x&@>4PiIO9Lt$;jvR-i5<-!)1`RVY z#$cEkhZ(=;+xzbC+V`-(-G6`A_4b$RifiWkeDCLZ?)!6p?$3vR2Cxq?!<7Xdpdnad zh6EFqWdJR#;Q+Jgq8dk0$R_H?D5-eYUKvojmO+x7t^;lVCZM?(qsNPRW2t=+=WaWa z_qo2>tPhTe{uf^Ff6PVS%7#?zXvsP2Qb@gA6_c4t)Xby--o+I0?vgpU2`fOqAkGWY zpW;AJedovD2BJX5ICL{7B0}&$8ZNB^cnsFp2p@_4w*MHRDh3c?=T7J3V96aqojN`cneuBML@d`_||!pOGo8V7Qm+axM;=^8O3Xtga{ z5wdLiUk1cj#`=xF*{q0MSE8ro0vwI?jOP37$g@`EzL}oWda8 zK`6iWEpJ?%IWiysCDAZsO%3UL0xUF$1Ay`x4#f6>EEgc{2XHey?h1r*`r>o zieb}g0}fP!P(`gXfbeqLT#pqvVWefgqWNea!@~-0ak+|%-O`$Kd$!f$4=I$tYqU&- z>nV?kZSkNokGsNj&3zoS&yLaQ9QyRfWFm51oP!}i zw`E0#$}?z^1ro*ZjYH4O2Z_=TcThnm?(wS!6DI-t7g7o|`?+!Tuh=cmTQ zPa%6{@M6{a0FaPoVLbC%ajIZ;_^QK$1Gp zSV=A{cLuEf5YS*8U5V15e2L4Cn9suvIDk?XS!fi)^j-aH}o{F4UB`5`30zA-QduMjL+s!&v(@P51Yyq+(loO*U! z!u~n`ZGE6>qU{%+alit~!&#MGGr&R7$=u;p%jLy3>WJ*Wb|rae3} z-9ie0e;luBOjlSQBv?*1%lZs9$EjIMetQ_M#x&cfLaBB&?OYXZ5tDHe$FlgX{ z;bnK)u)jAI>d8bWANB!+1fNy)8)5Nb#|8{pX^PlU+WRc1@)K~Xn;vH_7|Y@TDP*|8 zpZ!%AE~-S|3Q})V@hG*-{`uI4v{(4LGALPn)}_@UvH{7tPUGW~xsZ%3C<@81R3j8k z6KcetTnIARv<|`6X-BMBKuYF%#EyJeEbfXmzfd2liu^#v+p zZi4@89N`PKxfn{t<^KxE8Iz)$HjJF>D6WK)Sikx!&zl9;J=T-_8Us=+8GhKF{~+!0 zZ%5yxMu=sTd|aRlN9D*`u8G>Qo8W6XcxgUUU zHjb(Tqk}Nv%c>`djO>ayEeqAXp%rmwkiV`$E;j9JM-}r?pjPn5g+%MFx%7Rq^ zvPUIid;WJt@N~aS`?VfZ)nn1Gd_m8qZaHqVwGt7jIkg-9rWfCnV7%*)$K(AxNO)`- z0?TQq?IV0@z~EsW_bx=9BL1Lw$kOs901G-hk8Ih=QW_iq{FjC{9^hn-*;Tqu>^&@$ zCK6(@M&c$s@w=>ba{0({hmuG_2WS^xJ-oRrlcQ9Wt*<$A<>{`eFQ8#GeR*-lu;4+E zje~dSy?pUwR?gt=O%t|I)hGMinNZyOPG>H-?>aHdSZ85j4lhJo6Mm@)f zZxV75ltI71a?Yc!q$}7DzaKaFr}Oo9B~hHba6ypGUek^pYREW`_bQ|ErQN%h)hEiE z1y=$ohYI@}rXNNoN2+g)gA3!C8#t#+r->}Uf&9FGtsq(%(uKrWyEy=eU29~jBe5Pk z{X9GB(Ipn3NqGtD+{`1~Z_3x~%T>6wcs{upOnkW?Y>dr_cmdiL{B9yAd*br1TQ1_u z!Q{4KGQM9EP{uL>9S}c6VPId9J7CQD1+j%Xd?7wY}9Ja$;gWuyLiadu<&>k@FDh{P?SFH zdKLc=bQq^_31^P)tBH6A4=ucnV(0_It%+OvawHO-ATj>(9}nU3fdXE1v^;{6MskE% z7ld0ubFJK$GN)aCQObC}ZVAC*9jZ)oNGJVD{1+NeoBul z45Ry3`XwQY=M_v$C0S=*l?A?6ABBJ0i0IQrsJo@IEM!UOz@H2Sz-PftZdlT#6nrYm&UVn!v{T11-#)jk@SWbx6AVyh7p{ z{9yyno$~-&po#kQZTLxY(E z>be)4+hc8E`0FMJaQT!8iie7e3Orgydk^u=YX%^j6>!FNuXXOLB>FlGFD%e4Vc;X#rI7Hml)?Ua86}a-3m11By!TFF-C)m@>NTPG z9kJU_wj~wj8(t{3i`#T#@0P8S1BT}!?u%p_AF-5Dt2o+9bbsW$^BR2AS02%&g>EEw zYYq%>$Q-tDBZkR!#4yPj0j!1iuDrmKVfz2qSJsMN;k@T$QM`upM$D+l@ z7EbtFTAvnHZ1jT2qDZ5*t*ZR9HPByqC}GL?gifDK)VU1c&-QsRk07zK_Y=ql*WxNM{rx zfQ39Ss@nzb!gdIH|7Z+c@>FrntL&CvTQzAZNx??ZGW$TbLY@F42viL?lGUMcpgLa8 zI>>c#rF*)~N%@qR5Ts*Kh|@qn@AH{h%n1`c;eoIh=4gubve}9bId5vt*VNP4ywii>!{*^r zu^OdgwmW~hD{@d_lh_44;XnAXMoy%JPOHRq=~$i037?1?=nWVi{jSl-B9P}XTG`0t zUJ+gk(c|cYBFT~3%c|fi1w5$;p8=JYEZ0&I0qp8{-p_|@;`ug1)VM&K71=gH{_MTa z)LZNi&C_kE+G(n5!ndP8mb)ldy?du^&2hbQzjzwd4`t^SpTZV;ny@Hhu00Tk9VAz9>L*_w#YV!cM+Ga{w`dF$v+Qgy2Z?#rtJ-F4KcG14;{{L25i=Fm$i z^WA!V>}4IM#HW44z!Nhm_(Zs^-{fj1YW44cqYcSQIkQ+QEjFq0oqezZS>ZnV=_PIb4 z^gv$M;r}Zj{}HjHH6j&ax8}&Nm74m7F50+M4V%ZDM_{Mv^&fh3-KGn?3!D0rOrg){ z7I{7l>*5nmq3WSWHWT}ZxQJnC;Y)MfEd0AH$hhbLwGS1;w&J)2To(yP`-6?}&pSmq zWENh)(sRpPwRms(^>M+O+d6CPi9^%kb@HB-|CT z-Z?JO8Xs!bY>Do}q;v2NY)#LS5@v_D^~kB5sMm>&OZdXCfB1*n@wcaM5UKFb4n&sc zRo!SniW61*sS z*8joHtn*Z!oM&=1pCQ9_%PgN!d$JGug+D$D{_)xF`U=wYnSdCkcbb)s`e;rytpJ@D zEk_&BbR}Bg|NfC)S(0xtXn{wXZXh1YM{iZGv4gT=y5^zf3pe6CP3WNl3$YvnKt}GR}H6x1<)IGU;GSE-BC+ zxU5s+;TVO_f#FxM<4N{al4W$PL~R}_J%B_R zC3p38`eRQmL&wjop|I$7nZ2kD3OB4c7T$Dfo#V-;p&EWHHHP+V$*#BZcK}K^bkkh^ z1pOeqq^B@Lo5bNnw`Q8lt~wz9{+|CVUVuj4s@(n*I^HI|E*+7>bwpdj!JN>zQPY-@@m4Pt_)E_rSGR#1tlljE0^!QEWc^puP^}AgeF4 z>IB_QEt;f}SdEId#eHx0ISz$+_ZQ+egm3z!5q?FZ-P?#)3?uh_89L~^h5kgMIBJ2E z;8b#N-vWt3BU6=*;lHO}ky){V)3heMnC!mLZ=El-rnPjALjpH9fK}9wlA8)?bLS!NZ zfxsMl5g*jin!z8+kZ#;q^>smya_5+SH=vLIUL;rG^>Bo37v*$qB;346ci7y?)4Gqg z=x3W6;9bcZ?kz;K&D<~+IHW`E@Qg{2r*b=&){p+RnGgr%9#rmeDDn zHj(@9M1GCXZ-kl7)toccm(&*3`|J8s&b~=ulv2PaXm%CSuN>|`QwM@ON{s$|ZNU`; zzcj`=z;a~A!2+wlPnlMaT3)vx7u+Nlp13CkE&v9;s_W}r)puwfphXe@z+Iafs4HMk z;Z#aLHF4L1&!lQE^mqYM{x*9kc>}C_HE7T&Yh}5KKQox|k|Iw1=!JvAbs+Y|sC;g* zhVLl87G8;w4WO6UEgmhwc!T*BE3L+#{wUS@-3>FiA>y}DR!u*ELefh}XzV&ka_HSj z7{i}|>(f?adNaQ+ZkdhPWEG~*WE)a+{Zlu;dA`__&-WTnhzai|(DK6Ub%79$e1^^% zXI>B!tSwWs{Cr}N-x^jLG}X~k3goh;Y8&)+s!AV%7LX$pQVV;TZc}BYDTse{WO}pe zsoUD|)u>XKhgA{%iWN!qFI)`{(!$sjZf?nwfnK*|Kda~_D1P5}LC)R7(EzKUP3{bG| z6?JjCV6OKtF_jAFv#xDMNLlfy2k8$Il5A&DsVOOTtF)Oj5>*Cwx>bXN4(;TxjI1>U zujnqQmYR}+HUg@cEJW!yIP2R>Qn9Ta4vXrG_0Y#Nk#=;=8Ab?19>SE#KWH?AKlBDQLa>W9nkko{UtvYYme;1H!eLE3O7h29*ZR zhVas$r6RktHr4qkT~Vp>E*ER<#hC#)!nM+pet6y%vfDz^p0rfC{W|~sv;Qv1f1~98 z!NysavDg(A?3!iU_&)J%Ky>*??Z?-q(a48y)_!RSPZ~aoCyL`%Fxu+n*$C`2*mXCR zRR`IDbH`4&+#dfPG&d;Gzxwn403(%x%m4rY literal 0 HcmV?d00001 diff --git a/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png b/docs/en/getting-started/example-datasets/images/superset-authors-matrix_v3.png deleted file mode 100644 index ffd0d8a06a9c662070feef82d76cda5f0da8cb4f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 324610 zcmbq*bzIYJzdxV|Oi@%2DUlnMQa~i65hbJ>sUaerQc?pXEkKd(ZWuXWbV+v%Mu{{7 zMvd5D{Jwk6xzBSyah{{E-ygu2Ili&$x<1#tK7LSBk)s6BgNTTTD4)we(;y-u9G>qd zzX1HW@WJ;9@IsB%)2C|BpFX{<=Hy^*Wot%6B>y2QhD=Ly@<#ALy--qMG!-#Nn$Qyw z;>^H75_Wkq1{MmINGc7rQmz12-t^q}FX@J^5|eh2biTOSE=Yc(N#u6GN50E9Y+uZh z-Y_wr^+Xp{A&@BV$;lN-i#gYRBOHm;!zG0~bR{B0aj$Cl`8va|rhMsak|e&98Y;r% z?&QHo!S%-87 z7P$;tp5;$%>?>&-NpW%Qay2w&iu`i^yRj0Fj(J2{8m;be3Dg*-MX*`Z3Hdk5-|PCp zeJaV^aR%$X)V@f+ocm2PtNH8oOPEZHAWpqRF*ffAnUgN>TsG~ijE}m+=tSQ{92=gg zElqzGHu;wL2WXMqbun@E`HfDJ^m}^oGcJwjPsC(itJdqh#|L7T=O8tQ-*QL(8JoLq|9w=6i@nUJTm(} zxJIaT^Duk7fc_b#q=#t`Ir$NCmXy-9()C`=&~i+K})uu1+=_agRvl?Kbsr z;)jVNqtrp}^X4>1{yL%?Et8JiiDV9eJwFtVsHTZhLZHoWe}Ioj#r%n$k*2ZA zzJJj2Y(oBV#QV(K8n5qa_(lE^CC@bcxq_wY3-~qp@%v_z+?yA^-&puS@xAS(d#r7* zg3{G&MrW!KKdbiXmVJ}phsLGUg}25ppx!OLTqu>^cx5XAqxK^5BJsNFMJDP832B{Q zeS9&If%|5RpJMBSR+@RY`Smv|j!D%%)8x0p5zSLS>bK|Jxxrxr9q1dL-#Ei_34YgxW$gEFpLk6I*orE(@)n< zbJZ`12^;KNWAwaGAL|o2D|5!L>`QMG{}LH&S(01{&Yb0|hj=0|vip?# z0sD8cqWe6nSLg;Qy(xz;!RQko-sI1u_hk5Zsfs>Xz`K9&Rmn}d&wR36rW5i5>=W`P z6^Ps!%h8FJ$KTzH-qi0ry&gd^a;N=u!ShdDC3$IUwj)c~mDz{A@6DCUwaPcm?aa}A z4|*4R5eY)^lI4T2A1@a5NA<-@GGAa{m>cNm&%#>t9u^H3x#_vSXdUMs&l{&17qWfH zpv&;Jk2|5CZh~&>QQIR<3zw+~J3W{%tP%!sbE)O6t#k`@oFO;tN{wuIs2OL)eb&_1d54Dax-Fh>+psH#x;4(lzU{QYG zoWguFWejb$(T+i)#!!9RD!cfhT0}=@2ugEpZvxR4gZ64VIUhm3*FEmeV%pG)3%h8>x4 z+uOHIf2>jcgXxbLIhc%k#toUVXZ4>BK5>UNFq_h4ytn$G{V9prfC(qjFIjx2Sjflx zP9r zvskk_vj;t)(NMkFD*wt?Xa!5N-R@}Zdj-2Xl!ce|A&!(*%Gh{g1QPl{59eky4PO>J ztsC<=og;jnBYgELvu+%E1v?V%mxz@o65D#;V`X=ASCqzw1|y1bv!6Kjq4h=d%{0`a4pz||(|c=jL*e3H0SAIw@w(Al zt(!fQhp&A%@UWCvNp0pqaW(fJic!`1hc6eK$E~Z@*Xiryf9hB3sEfK=;?u%f@gWK8 zwy1;g5!A(SKh}?|1HLNQ8kFVgP_${UscT1H7+UO!xcVTGz>6)+ z?(AkYDbgZe*jey+8A-UdFLZYm^cI97450#sZ)<#h>c$wWR@B0TrX6dCM(1aTriBak zdX-ZWRveD!;4TTyIAoIWTansVa8LNk+rxuxujHA&YTP=wzkUc;!2FV#T1?CL1y&xd zzT}|OICcm>NZ#5**Ww?q@0|L8&pyqWHTvG_^pf__z>aQPVIFl@`JJlTSJAn$BF8kB=kzhl`C}-4cG(tYG{8#epSDNf$Sf$x9WSmg|H1j0{TM& znB$nIB-$aKBJiU}c;`vl0-iqOr;QRv(Z`z!fm};rRV3u=jtU?mXaP~`oz(iLWJH-B z*ToO2pIo0KT{w@wuDU|`Ax%lPqg0RsN!z5NqL8u?(&2UZ zurlYQnQ&)TJK@8jT^WWR<(S*-3jgG=F5r2J4X)zEc@8(#dC9IWg<@C zn4E|>#ER%Va6}Az(F0#ZL?o#sf4%}DN%E?vlSl^#NZ###rZMy z{qtPEnpf4O6#jl?ttse{Ga+%t{u%SFBrkODibs8Hx%i6kfJch+h2{R36@G)#vYVvD z=c%NKe*GK0;Y(zN)p)6?oXu>V$7U%l57-vnc-GLDA}LvDGm;%@SF;p7US`&V6uEfg ze#!FUnYDQAli#=il~igWiDC?QCB^l76N$pkN_*IhneTZbqW}&P^4oqy zzx>VY#?g;}E4Ez`cMCAmsER(ta`QfZ>mO&ql>3xQSAXZ(fm@p=pywnl zjI`B`pvH|pZvDb5i`tTUYk4ciB|Xr_G5!Z{DWF8$Z`mII=0mz(ov^E#DYO_!K3L5N z52i{b@v&khk=X5It(3o>zR*CP)0&}}%KD%9Hfh0T`9-^>mM4kDJb!_x7?vcfX72i{o@rQCt z#!sf-KU1s-0pGvBk>;>SL#s=k%wO5`zsM~=sXtkezlkq^;mvEYS>A%#swwt|)6U^W zKF3Q3K#f4Ir1&NmaoIlog>sd;NJ4))@YFAa$t^a^%#6EsgKtDI=*+%m>Bf}XlpSgm zYSefzR+6HM8kPU*W<8WHGehDBI+$@8QDz}MM*E%#4Xr{ncz*~r9xG4aPxc0I>z_4A zo2cu0XzbOSULI^r%3PrrJD4yPeBDW<*=ar?e#lSx=|y3>UuM7E$K5JN2VEV6&?6vxj{`qn|{lD!x|FG6QFQd1cIW{SInMnl*1z!d+X|Q83_0)4dZCM8U{^jnMXww6 zeTLbS*o=iWhNAvQ+#-@!uC3Lm+Upw4EWi<`o*g)xn>cC*6OLN+yuoLr>A*VWoGY?k z74g#tn%4+t#y}L?Q$DzlN*ifjd1%ck%G4nzR*x#K-N-K}E;AfRD*n)*Yc*$kcQZ~0 za+aL!;d_ri<-V2dWO3EOfJ?Z?)N`Sta9qZog2a^u2jt#0V>T$8v(d#U?7q-Q&1;lH z@k%=`7v_3q{rG1^ggF}HB+if@v%G<`$@kKIH&Wf%+lZcaUkIJY5A$|H>vk1xCTT#1 zdAq591-*20q>6Pt>|ivVseM~T3E1OS<5kk~GE+b)dMpYI?9(6VK!vSLBcT*O`ec04 z5+<^q`z3eZjUV#k9ofbQw&aq=lO>)54AfWrQb#oc>@^9L1zIr1qn$=K@W((_4A`3N zmV4?c{;;tMhi;s!0B@e|#x#CTxR~~BuU~XvRE~xTxA;xjHN}*A$TQ*!0H#x8|VH!@!O7VSO zKB+YYzn|l%!eCqSuF`reU+_oHJexpU=11v2*pgf{$lqA$b!cwq7*yG3f2AEeWOAEm zrsgy>4HdB6UmbGWX{Sv9E25Tw+AWY|Ufz=3!&!!w@HlL7i4w~pZaKjLJauKpLV*=>#@up5Qp(y}&TX8#=voSc)8hO^_JNWp;* z;&h{!p2HcpStT?R1Hv%*p6twf4fXIeGLnSCRWaV|U+xF=Cy9ojok8yFUg@$gMqC>g zBNR1KYAYI0Ui-JaHm<%E@3rvE#)vD;m1Ar>4q;m?bTIoF5!7K$1uALaG^R^=1$Da9 zc=6=c?zirPx>Ya#24m}Iy~d~Ox+b)zOld@kEt8%_PNgX^75H_`?y(rp`b%WF`%;wd zas{HHCOOo$A34_p)b~v)KE&a~WhRlp!3Mm_$XqiPzwyNLEQKmu67Rk}CcN(T@K>0{ zyYJlFqkB9P)f8uf^&~Gch;L+AmL!KUiHzpeyOGLxpA^2jZwtG}n;jei>JHRG!w@OY=f&Red16VJ`o5`2fat z5mmUrUYMW$vi2iBS$oJiBmImtZ9Ao9543v-VCPFIg^l0Xe+9MhlM)A=X?g4SJC9HK zCg5y1QNv%+-YcRkwi)X0w7dg_uh4lU(0@fuTBJ~BsIOHf0RtujeYyW5O15XXPS$rX zy(vUCZVCWm-rAd^a@8XRUIL&`{j9aJSwP0WG^rP}hw@ z(5Tk$QN}+lhH#NR{6&)ZZaeW$*b9pT??Xae#AdkYF-pXGjU6t^0=vl{W%Wkn!|5qi zGS)rRz1C~{l@I;HpjQtfG9t3XFCI&H`P^4tSW$yYk3p&z!bFkrHpO(Z_ae(>(=_GB zS|K3Hbu2t>%fuJv+E>>pHSjT+a;5`3oGj{lx9~vSQmq5=-rq@0UVwWtd96Bm%Y?}Q zVRSbIt!jmsn(B2qTXNbvrO7`X3iqLI5WvI}CSLwzoBX*5{+l4gs=&OZr@%qK1SD7UX6FE7Iay5eEwBE^IEd=Vs9 zhc5keX&2sSH7O_wbhVK!nTWx&wC z;Gy?WX=kGL>~Wo;_SaySotF$c1}4!^=Vu9ILP#FJb(FL&D0xjkKePUinlFM+S-BcB zF-BACo{cufq#pb9R?KN?Y1~nU0}8p3OFdW=)VNkV9R14|DcOSQSNWE!Nhb_ag8K`_U zQ>Vr1lBfH35#_YnMM;p*tY-|N);KzhrJ zq&}UemU^zi-H|y;e7*2U`BxPF^I7rWj=*lNtlemR7Mc{~?lmFi@_LCD;((A3q9=Zl z0bcit1Usw#io}+B?L0qu+y#?8K8IPck(5wDWT)6M#kd5NmQHNxrwcwoNf_ zG&WneDxdu*$|f_{yEuXDgnvDggiAssDx7z9ipFr*CX)uH@mS_CpyfS%J*S>`ilN=T z!%s0Nug0Di2rbB1t`U9E18(>n&AQXmE0pc7{3&RGgOwjeRkI-F)d_23V*ViF5e1g) zhQ`B=@SLU+X=UtQ)440J>y7IvuSk0QP(_^{57uF`EaZuM)1HO|<8+`&<6J3^XwbSw z$@SIBq!HTtNE1n(P{#Zpc`e$&mhCkmVL6z(eX5WVZ>=R2q=^xK&PwJv@kpz8;gjTB ziS-O=eIHwdy~nu%vR%#JScpu zh^w{!BbpsxgPT9B*fI*`%SSM-lmWXN_4-Bz5$BJb^wh?W_+sq|eJEag$oIBdwYJP5 z-!CCDClGr<>6CqKkR&}XCwF()nsO1cSV-41`hKxy(Ow_w5lO0w=Qm}+Qh+0{BsR8a zTd~@4RAj0bQ}e+w;|97En@2XH0`M4c!fJE2Z7Rsga-W5UbcVkhp)54>dL&2H*iI<) zAUj<*fA&oW(|l!`zQp2}r~F;|Vn1?-(q!X@ZGsnmLYcK!<)#SmsUyKg?tLvlh+1tV zbthxXQuik8Yd2fu4QLG=-^~CLNH%0C=HR1|K$`CUx^^?bfZ>p%`H)JV9d>IUcIGjz zIlTMr=Lqw~&aWnsYz~T=lEE#qT}^2%&pm?*{8>TM1mQYRj=H;x`u?Y>e%G0ksC>9F z^|TG5*od6OS`X}_y6!XbJ^(aqc}I|Wj9XYCD=f(&;f>_TGRnCTC!RxH*1&OWQ+fKB zeqNDgwDJKX_x>qDO@23f>AJ?ahN;9AGUJ&L8#jSXk0w* zva6TdQ%)5O0s3GN$h!`Cs~uY>9I4I;BN&g zTGOv?tkSgfRXnb{Go_Ov058Tjf9;V#l&j~K=Y+-g8_@g@)Q zT5bH|^_c%V?XOVtkM~pOjBJ|tq88POyZSvyP9h1q5%@@Q2ry zgK`|2>>S*n!?giGA)@u?26b7TzaKhrgEr`$7PXS*_H*1FMQ^1bn|*6zQYyI1+O-xx z&~RqZdw??Yqh@NW)@b0M(Y)jqSFg=V-N%7d32sV3*8K2f5=Rj-(hV`Gj3nH_CWF5a zecp5E?x?&o=WI5~V+M!1(okkeiMn(l+uofeK&IpmeVnV7MDoc~V9OZ48-wIEae7Po z$mgh)IamVhz0-cMUH}b7u;9p5-keI~oPoI*Vv{v=Md0VRmgG0;P3GwO!^^DqIbW`@ zM&yNE+E;iLYM7-K3FE5H9Ga+j&?jmyig49z8h{3cXo@vett5Mm!+gKfSp@19NE|PI zb6d-bi7$@wH-$Sy+?^9F2ev{`{oI02Zlh;X&eoVQFc+3tG@%);SlKL`-eE@Z*N=zK zi6v?P?evU1LkH^fa4Qp<%) zH>rS~Qlj&IQCkP1-~+S)yY7gmf1-8>Co|PnO>4lP;zD$zy4UbH^p&SI*n*UWgOTJQ z>6H3v@tG-4l`**)H0T^^Qag8YV18;2?>5nneiT$zcT|t+nas8tnCTOmV#>~CXKyM zTemYC(Eh#Q4ZI%zh@SkO$Yjh7(w(A zZXZii_^A4HF8tJ41nE&Pyadhm#S3r+x(tWqY|`t=vob}cvnOU>Z5ft5*W)S{dItum zhRCbv2#i2}WXMp&RE5~vt!b+D{Jgy9Q;BdDJL?K~B?2t!>V;G|UOyT3Ucd$&$k~fX zl8y|2Y?0mVgBh)8#@HC7pl%#$L+^~qhe_-j)D=1GKF&DxJTPfu?_(oV(GUkv?DiZ` z%jm(RhB3EOW$~Iv4hSe@?D7V_C_NEAyL)q`VgFvIZ97??#^uz>?$qv&wo#M8zvrgi z1+9jbd$O()ybCIPPfqx)lRYa5Tx6L_J&7Ne+>YKn^yz&wHYcAOkh$uRWV>96=Lf45|Avrx$r+0G_b z5wRJ*c=q?#f8i)Kom85rB;8AX+Bul@a$pd$Pwb0*$XYlO z1L#oSqaPTUTj6r+35F9;nykOwQ_i2U*!kHmXg$i#@bJ~M7iV2;DWmqEEUc}qm3&Vo zkZu5+n*>lUZCIWzdqEPQAWi_D<1M#MWMu&$o*Pv5yz%N~{f&haY9tj`aN~jgXma*! zJ~rIz8f>{pZ8m7Y;DN@_&D zHdjG~VQgZ&&CMnSc7i`Lqvo&it})<>#Kf%o0;Ly#N;W<}>L;I~zBXF6EdWhVodNs? zHccuXJLdZ2qo1F6VBr$jPJ%~5buc1)h(Zk90_1JS49j|CpU%ld{vLpukOxx^%=ugu z?ts@hQDHSgaGTT6A|YEdwNrpYl2Y^>#_~I#75K-vfrG$OH8o6hIo5Oo>U(PQ*kY-W zyZVQJ3EpcCmrQW%3JE6NW&>`k9AWG`b6N7lam3Q!%B;zcSP<|FO4bNI;JF^FFVzwd zY%Ex^&=xUKYN`x9&{h|v7eH^-qG1<#>Uhsza5db_u(O+jQBFnih+;-HH%mKiUDPWD zMuBg`_(M$zeHJ@cTB65t9E(`a#cZNfFouh_-X1ZRSE!Xc99i#m2y^hoO=Vvcx1Hcc zO;tgQYL?^kaI5}C5X=}17>|&O9O)oyxeU_v@5Al!;$Ci+y0jX=ZRq))Vp{ta1bu8D za0TT*%Nfo9o}<1?jba#ZmzwUWCNoAP1E$XtawfnTN}fHh`8!Lji((pBnARy-{7 z-K<2|ZvSYWC*f*~v0@Iou+kf63fN96)i3gE0jno2T%@||UP2(1ajn~C3uX4gYPJFw z2*(kdJAYlfU-ImP_s$Y9mJJH7aXJdsPElc{zcJCs`M>y)xYAoHvxu6tjRO)olGUb7 zoU$$37D3QyWl6Lq?5FIiiYR9|Rsm69jj;tzZ0bx+DdRss&0y zh$&}6_7IVPd!m!GNU{GrPT=#SgpAfXnISv&%+>=enmHfP!pbC+7+n}ld)m58 zkl*WC_0Vtn`JGY@5SIS%A8_H2OLVUQj4epHmrImEn9BkI2(@0i<$=tW=GreupaK(3BUoh$dC;MnLH`d4Ytimax1H_MFcb0-vkd8Y|l9l zi07=Eyp}W@*f*a9hPNN6$RA3;0S#+M^cdB1-|MmK1tzBi$8T1wmf*X6J*y}KwnCFg zY0?{GFyyYr^BP`E(RWdutu-A3JF`dTz2vR;2`YmLm_;mfHYd}f%mqCwKDs&A=qvx; znp}P}#g5s*w5W2t%)q{;-TQq9LDR6+C{gn*qY0vLy);7vm~qDL#ufIwK~$$kL&!70 zwk;b`l@vG7{H2u0pO}Qc(og!WXCE4HuB^H}2{umUxpu4ldItOIX6~BBYa<3G0yh(O zU(w&(0&*yq9bjWFDUbt-j00dLn*K2hN4C zB|7>Oip|0Ph$=A4W^PVH-{8pP0=EW$r!1eOsq!~?=H2p zSO#**72)PeTxWYoazJx58v8!WuhlIAs{w+-TbCZNp43n8s@lrUNtr*I?(YVOxs;!udL}xDY(pX;(UKJ?3)3I>j$}7KviYygKJXs6*UeYhAdOQq5?_=kH_(6ZR za>5O1AJ8dBLv~du%m%@|SivuqP^RAX4i7V_`K;xQrTgWX>OY1p>sEY*v2|2ZcIR(OL6!-<()3vfS;uS1^ zFI1<%nwl6*V&gf&X{P-Sd!EQ9-ug&2&?@7wNTKK2@2~|a27Rs)oHHXZ$(Gzp7BfOXX%oWBG2$ZxB4kw6-}-i!0wY6ZBJW;e`B zVh&QBeZXotD}4#Se>%UWv-2%nerHuLW@~~j(jrfe_jFNhw8q{h$mqQLFG`xCxsBqj zoHwyG&MU~pu5UImdr3gLuAfsIz+y~cx-kdA0M7t&lEzu$#<9F6{>lK-r;Yn9?nll3 z#Uc5A(WK0$82=?i#P5V3009uQ=`Duup$ryp=O9%)%i=QtO0hv*rswa(?yp_vHlI#LCJEZ<+tpXZ z0q;O0Lk{$8fJ`aUoIsq6ocjdwre1vb9b)vw;P#wYoW8kZy2UYibYzj+dZxTBII&)=*@9|QM-Eijf@qF!q5|s>M0(UzAvOMEqpj>L`7v#p*3p8#-Xk)7%(5GUE&AXEanN9KVuW3z;f{b5~w?H0U;kWw;$+3JxChtUgk z-|{&!Nsxee<_=fd=z5`ws;pxmlG%SJ^1pCo4iH0+&Ty-3n9{j9v+i$en}yvr_1i5M zvvpUpmm2IFj?GbH1-hv^(MohYOWpjd3CNV04ZlVT_mS{QqA8M~qap!4Q+I?nlr7o))#HBC zaPb@5u`M1z#H9KZ(7eG}%EmVznSZ9d$=gJT2bCzWNI0nOaRNK1UM!H-J3Rn$S${C% zA~jV?2?26UK01RUUlnx8dbBB7+$-2w)2YJiaO(h&Fhj83zoJ$B&+0yc`Z?)`Fgm@5 zoa}V4;Ic}hU^A=EJaaW zIzb{D%z?~~gUu9%E_AwTKmr#?y%KR(v1L6zK7Mc(m@Jk6H?2rxCJbz}%IC0U|Xv?YD}6GWg}%aMr<6j4`sR z0rLO{glKM&cmsu|0*J7VAx{9m@A6)kSe%nyce>S=Dj;Vx0iZ(*05KvtYj`O433MS? zsF$-fODYwNE7*t&TgZw4>IZ4*cVlN1)BP1K+UKk|N;q67BwBIeUa-IAsl>ZM zqaH|&De7F!AufNKT}{6&Hp$};NkDFHJ__Ddu>z198v?;?l>)L466cARzIc;5TDmO# zO##@qw+y;IW^c&Pmxj$vP{b9WeX%1c8wqnK=GEIGY^3*T4EIM@k5^KdWiLzvmit1s z^y5;Eme>N9&!oS@yFQZu@eCLxlgMas%`!cK4jOr`Wx?iraFYNFQ=j58>(GkuCtlfl7Z4k1DETW;SIdC9r^C)!ouf4xTQNm1YD!3UmpSNRC@4nK^mQaXY#lWry6Beu2#;$ z$lxc>l`?dWAivfEfFPc#sX?#kl$2-}=u~bhC5Z^sak~StJfXe<@%j{x?nsVG$eyc6 zSVSV|4v;Q-cxiJ2Xb&1pZt;U3SsS!UTQg&HSJJWcbZSDu{W9}y1<(eOeNER&qfJX3 z3fgE7?EDo4#qfML_AX=?Z@_PC+x%~c>uGfK9#Xbs4;@F+DMqU#i4;Y%Y1RV*ToAT9 zf#aee&?cbmWZWGOlsnTjqxu>hIsvm#MQ0X= zS2mw{Z)_pX1vUP_Hs(or6kB=A+g`YCbZ9#{k-3#ml=6B(Bm8X?CC5e3&$U-k8Fz4b z3HEL8uflL!wIO~looc~v3Qg|N-1#lYPhQCVrPrtk%(@qEh~umka9Wg~0`@}s6xE3poWt&&R4R;aUYq1$L&mitp$8KsdP|dK5COBt9#ma}*ipttIr_Ce31~Y)7}h z{ZelFPsIB!m}bzx<_h3eZ;vA?SatQkbJdZ!o87d-@ayZN^?ui*kfF&P;POLBo*1jE z27UkYB7fDs|HEsyWBdU1zX50rF~m%S{l;VI7fV*USEp1x#G*qJVX>6BD^djTUS4L{2JFj-I#_Z4*RuB@1n^_g09tHU00I5V z%}3p)0uvijFr{7j?ze;zrvC{x zIaQ@R{OBPce4iV7%w)>_Ik%*aNxcIVQ=%K?DCf?dCOpKyHO=$Y%8Gb=%yfp!m+x;tt+4H|U2w zsBth_Hy7pkKec!%vx}pv;!XchX>x0&G%4i1ONvro6LguBK2$$-Pl5O2cK$E$I2d~# zV5a&WtV9Iw>1LJv4E#wUz<@3M)z$z#K7dh^j84!t7PC6k`qMZa+nnODOoy7SudTG5 z9M*j(^XD@GErJ^)0DI-}VqQlQpS^txc^V|*V}$I485<3JIQ`Fq5fAh8A=6pNu+##z z6LdUJp&cN^kXWt=cur9Oqw^3(nH4w!9d{Ez4#pP1;n4(g3~0NCqU*UCWt$pkJHVwX z76<$Ww@#K2!X~I=gDxMCC@1(r4Y(O}F)Wnk&*3Eunp(9~RtW8aUwS_`9iNiHieyjw zh|-?xq-0Ha^#37%M6ptl_<0(|okiqM8)@iGQ?a1^o;kKqA`BWlCO7#Gjv*hD7w^KTm{<1@JH*0k_v>fDCHs#(ti0I3E- zMrost_g1EWz6M*s2P+5k^09`KUBv`IO@I&*zf(XG*j6$6ekpLzNZ^jRMYpOKq>s>m z+*R_GRW2BhCZynf36{y};am#Q%P$KT|FSkijZ=ruee@ySFV8p|5;@Z=qej)89^Gu{ z&zXefUH*@Hqh&8i_oVhHxgVh|kP>dl|Vzh`_us7o&Wq1e|qXc%Hucb;-~u~Za^1Bv70jO z{TDAm7p)hY38V(lJ^toBu{;f)ctXbXTG#N6%+0UcgxIa6BJoqw3uKpWz9hg*@>8pd z27t82`}+Wj##?f>WH+Q0%XE%fDsr%fgXoArmd#>}%sppz0JVg%UwLH*$Sh6}fU-?- zjk)_NR63A8?#n;c0xf?(pZB9F)|@-5A16rykT)C#Ec7me2Xc1c?z29sJ zT%`awXRY(mWyRXT*Rtoe8}y-rNzz{QOsX=_SzwN++5x7r-NnU{}z9 za*Blbc_PgE%X*a;P^>Z+NQ%=jXXqTzwK=+teOZ~&lpB1#`4bvNkRLQ~qZ_D|OxCyBi-!QDwVJ-MELDDwYA?K&R- zc?|%&nplegq{WTUBs$=8Q0~|SOw4;SOMZ72M^Q{|S`7ODA6qAU@5DjNtIT303VcKzF&uMUSlObk~er`QITWh-PqQ zUf`Ajm&?klaRd*;Rs+s{gv;BaH}-101h6->6gUIX!-O5RKfN5mS8eZkhi3^ZS+;M4 zUNR#X!okve^zk^ zb4j~t)b(D6MXHqe^ zz5ub9&wervtgjN45yC3NfPQck=I{d9X|)^#hR~)4M3<8#?xzUYL|0mk+$$WZV>SiY zV;a+0`oTWEiEma3RM(GC3+5lMe?qF&1MEKtcz|A@1CncB6|&vH-brZ5NIMMc>o3+bz($3azGm&Z+q0Tnw0cVuU%W!qgHnog7`y z{xVlmdhES7cpU7tNi{s;>0mH58dJdsEb%cN2cUKw0Bu`B`+DyrD{08i3r*QR0^p7p z`L$x$qkk@30O%RIxa$1bDMrae8q*`GxGMs5AY$EX(ccOa?CKA4{$i8-qw8JBnL4c} zbDl_dQ^!Ox3oj*R-w3ppvKZx+fa@AcZW)jtY&M?ZE1~CAU|G5pHq=l4&4!`I{m!XA zLU`nOJ>Y)SWA+#F?*|WiszR%im;W~eBb7zL{#Iv!Iax=4g|k!=KmvMZnGM)|btqN8 zRs@u_Un?B*51;$r@U4{QRe3yFcdBsiMGF}K0JX)wnMs6(8p&KA5dE>q~)JtO{91@Pldk!b!kctL#vEx}upL+kLxbO?UQ*g(W zg1roh9m3-b0#h!ODy-`42Ui1p{!HvqYki?3mc@e%? zc0b?FsH}0vDQ&N74~oVwo!r$nt%nNp{KLsoSxEPtO-{;9p5B@YUph0HJ{Pj1<$AIw zXq;kaWQag5>BrdBB2)j}4t6C+D!$K^3k$Zp#d^xA9Br05!Wa z0`neUsr7BZ{4`yzRc*k4tVH#6JO1f}p?awiXU=gF7V`L^O8Mfw0oLJhi>c_MqVL{) z!b1OW!vt273nqq)t7lwsIq`MXLFWxR|MxpO@BC<-9z1ZD(1eUz(7u2AP9h~SKWI_J zSY;|24VRu`cY|QQ{UaKTRFsljXp>ywcac1Cb~&}(lr*+T&^;@&8yJJb_MIasYsGW$`k$u(AlC`4^!jTtmm_+Y4$|7=Nu|F3 zxiu(mne>K(qJfHmE&}rN^yje?K08ZQ;_wg($ZSqm%FMbxkFKcI z)EL>n)XsM57ckqnu~k|te1N49!`adY6mI z^?tohJkL4Lb6%eMlei7Z>^4K}d0bQw!F9#%*mM-O>Y?<(CaK$z6JoR8?HXxkIrVl3 zJ4mR{vt!thSWOPd$V8da&h0eZp9=$2NUN*5c0BL>F3-7L!$@l*IV?0j3}P|BpKWJe zttJ8f{lX+flqh(tdyCq0U9z;LeYi65yH#tP9;veZm4O1Uw`z@cdXfdj$E`|&dw6v! zVIlFo)s9(!^@(oh9wBtirmOsV)k(UqBWe+N7cC)Jl$x$afN0GIBl|BGreefe0n!CO#_fvP%(BLT zLCJJ?h+{?J4euco*KQ|4-Ti$+ftRPpU#EKnu<{TmoDVDm(b$#;!^bSZ(mCaU?zOjLWV3@YsN+YDPM z=(q_v@4Wue(IBJ5^F);#R4INs=e}xK!?Fd~GRNam&h)BWPZeMirk)_@|M@hKaX60v z7`b{X)-fp$4vsvr5ahs+RH*nKO8@m}IdL?n*CM6^&D$lnoyGs4eEH+J`Ib9ss$69q zFyKJC%ZnYZUv{M@+hTX;uI5>j;OVYjwqpAHxGZc#ulv7*T+2@$?<8-rmiDZ2>679Z zY7jP@Dpl6Ok?L#5?E=4u`eV=JODWbC?Lpp?ozo=;>y?k75=o}c2yn1o+0GjT|L9$Y z$#sbs#M=&5R2o7n5>l*J2T0E-Jv?T`=`!EhyTqF4wcB2lHT)=p$v$8@3{pP`t2(b> zu3_>oD58D=k%ANU_Bwa9ZmZy|PMcLc#=22#Mbapu+Hsv$I>NS5e1zG|&`ksngo+hbwS_xf+>bdvoCxJ$H;Tv9#$ zun=4>alPq21e=jvy<+!Djw6?wGzP%;nGDtPZ4lFD00$3+mapR4BK# z3Km+T28~kh!5Ti^wm~x2bD@M96bt++wzb{ZyQ3LAZj*)7;B|J zjMM(ZhCoE8C#5@|mbn#cz~#%fTaWVQ9VMk3-ANf8@oT$F#cMAoC=c}Pz+Z~UBA%RLoDn{>o5Vf6O4NJK z-!LfDMxP&2{04H~zDp*#?C_XJ`n!%Vhy0{+_EOy$D0`z^owgCC&rK`;N8+vdT4Dzq zBY9pH()-2V%qaDKkOO5#z6Wk(m`%~3dqBc9%33mqw!?!U5qHi$7h5rqtQ9N|gSULk zEb}=^=vN-pnWnKgobqT7@yhF#4#UvN79RK?$z)u;;qr7^wcJ6-l`j6iu#*Vu_&Di3lFRoL*!Zpk3P{J#T zLRZRRyQ>yaVkeol-(I6JiS0hdraQCg@=g=VHq23)1!;onv%50rZvA0^4}^eK{Si4F zt(`=T!oTReT&(8wx0LL`<8ZA+{pSyu9b3-P1s2z>f&!7_ZX=O4@#s_n*GygdkNWTL z-%w?7t91t^>0^r5u+iJIv0%$Mn^!oemR)+y@%u=$U258}z0uUJ(jLQ^NpvbZCyfxRl25l1`R^L2X>##Icf3(b41MEZyq;2KY70@^x!&Mouk*WaDY_FRn3S zCMP!`@VS2aza9bu;TS6wz=1ew>?HP>qQaid|F8cg;tpSvolEw9zpE+_lxxb)toNT= z#6ps@fv#-qajqwWgZz)G{c>W@(X8MI4gl=0##-@lgekA>X(lX{xGIi_5%*uT(N3g-<>EQr@FPbqcO;KVMTT$(yL6N|tv%_X7O?K1JYUm9DDaG0nPV z^f=TqjQsjV5yo>_SVw&W&-=rcms+YqQshu*uZp&5cucz5va@|bPM5@mpQ~~Jxf~UD zt}Od2&a>ji=Do~ZCY1G1w}8w&q&-d& zU3*G8t9KZv5848a6QtV7--W@y4g>fx$i$mQXUZna*(1wNT7$vT`FxrXML&me$G8+VYe(-$pj2zLyPq)^RYFEzG|$F0TeJ2nkcbCd&7zm-yEBtgzoAa z!O_~{a*9*?nvteJPEZW@ry`R|Bs#!89j5QQNi9nULIC;jqHjrxgCx={sYpp8nz?Vx zSp$7G`9M*+p+42HwAVoC$G__=|1!O4+T;W|pOM$$yR$Q!Q0wXf?j9+7OzKB@IqXV$l9*FZS0yq_ za~A4Z-O#{WS5~-TZUwgd*qa^?8)!{ct%6u&BAqYv>_tjJ*GZHb?IinW%i=qZs6JA| z;d4*Y$WYSEklrLkYMYU&N+W>l`VIvYxf5iUY7Uuwr~w)-edY-}o7B0~y;V{uYPqc% zF`(^HAh+g6iNpBkM#M#_XL$e#mXG4dV4<}+P-k>D_L}K#@TtBto77N6Il-T}Xl^~y zqEyP~2DIvqO)9TS)hZ8;t(VT$jo84dCpKNOH{FdnRQif1HFb78DhttL(4cq5DNc6` zP#tBk*$<2uXmuevKl`4vDoiC>@xR>8{@)?$ITA;xm!zN9yBHmpy|NqsZX;m2phz>r z-Fm(rJGPD6=ooCCW^5z{CflN1mMp6dEzc?pr;#=ua??p-LKnd(LP5L<+PPq!-aaX4 z*P5<&%qv?EnJXUze!RV)KI=}J>K&#k)dVyGoKslY8Kw4mOe)rsW?_hV;#~&;+N;jY zXJ8PMeSuX68t9qr&B;wq7ffb)uy$-zQ&o~JaI3go3QH<|*SZIZluX6g__rwl^Dnechz)kXOuzZ9&@{KaR8bNcCS*DNFZ<$9V@jFPSg!GC4cI;NGsiw!|Mx+$G;5T!R9uCt?zlzy* z96-Y`KmZNI(Ssq_y6N=waKF>9ycj~Jp=++Ca;tT=+?rq|XyBuA7)u!C{}D?4V{|f9 z5>w82agCrM27^yBv*$8R2v2}2I$-sBQ`K0yfTBpzle0Ix`x8lJ`@Qb0Y4C38`fN`N zssFVcs8_38IYgpr&f$P_G`MiJwU>Et|M8r@@|q_vaGCzRbQ9H#nT8M0V2{a0Vme&( z{dd?$#41BRnpf@IU#tdMzq@5Jt=Y4-t=we`&4;XRsQ07vx}1`At{879qI(jZv2#aJA>^Z2TK5M>%>V-{NNG}TIVg&ttGbZIGC z0PP=Q7R_{nz+50mECfxJOGD~XhpbFAi2)<8us~Jtu~U5@`8u9hb4OiwbHm|JHgTFnpYya}PELFQ<_4SjWhk~Ab1#Q?zgzYwwwp3NeksBLL zTA0CZ=YOTN{-Zdca$Ww#?gM2qbwpJO=LdQu&`#r5gEBsPf!;*>4p5(sAoalj)QxkI zXU})D!~>nRnzhbsEFqFpQ8aYjq;%i@L)DLwAAWoOCgB&e#WV&5NRn$(z z*nnt}*tks}L%Xc!0!&V_$BmThE$FdMgIG6wD@&2`&uWkV|L^&d2|2+tQ^5p}{1|ac zvys0Pm~u~=A@J$>;nBh|I8d=P13Dp5q$ctR&*rt?xSq4zn^^htQ(C?Y`@N%UEOJ_7 z0tTcU3A|j5Fw(lM?9`(X8{SD(QXNdwfn0bsX)o!i0OY-A<2&eug0S_gO&gs+dXtaH zn2mqF_J3P7z#pW^kx%}}xJIT+`O)}FTsAPSBc)dCAYe?yQDpEce=v>NeS<#Q)0tw8 z1uj*|XIWl`RpAz1rSyz%N!6|Pyr(&OWZZ2XY#cnj+OPc>ZZF*B*|Pyxn7;WBI{S;e zKw<~-s*hhu7Fp<;nEOY=XwrDd4+qLT($bT?w-|K8f~Y788?Lpix?m4D%SE|iOPL3; z;q@FeBeXq^&P2RrY%etSM4>&JP#LV|)WcT6$@TkO|0Vb&Q*`rKYrHh3SKo#=Gd-Pa z#taFJ9gvDt%GA#OEGU}0PHGGyO?uLA)s@@DI{^+0-dFYb+hq@+!P6FS<0i>Oz-XyR zmKTqBN>@ZU17jfcEj&uk0sOfES4A=&`fCk;rYZ%GeKGssh%&iA*(#DNXE^{&wd#3Y zY?JU@o+okVPF?CU`bks>YQ}K%dM*>DMGOUrwXLXhl{3~k>A80LUw-haUiUwLTa+#YIEet3u#DZObb!dA|s4)LDy)xz%+e)Q|~~ z4QalV=%HViI{EV^JUG2WfA;*Hz}xSkqwBWu>RAB#4Qy(?s{yuT7J%!` zKFA;5ZI|1z#df8JkT{q{^Q565q>0_27w+(-^wfkWK)YK)Hb89z`c;biBXo?d0R$ru z#IOaled*jHQ=e|5^e#e%G9K8HJu8&BLb3M3K=Oq&GhblpeSf6vhX)wwZo_bNP3zl; z_3}U#r^PHD;X@MsiQcFD-=~?e^5nnf=FETf>W)uIBKemzid)Lcbk_~vmm~@(k?CB$ zMoHOxjWtQ|)#+XGtKKyy8ikySqpuTIySH{EitQe_KwW_UHl5zC#=|`_sxq90S2xlv zOS{D@GtEM-ehradnDH#d#Bz6pb)SKceD>>C2 z`G5V9A-D!;ZfcsHxA*dZrb>TLcb~YzqDSa4Av{<&2XC17eEM`fyK%EHRW^HUt(v}S z$#m|bwG_AsJucJsNnI;&G%=KkXl zi!L`=0<>3e{VqH%_+{OX_sXt-5NmW5Y$i?g~G-7)3tPt5u9F^7mHrTHIiOl$voKXw&w zQFA(%-u`c#>JfbN$kBIr>;)v`u4xy=R`sVZ-=BGv4e+C6RlpW84sNOub~E28=?t2#Ubj;+$F@M+weQl~gX&4BG zZ**{IO6;ZMgMp~GfjzWjr8mz1eZ*08zY=zC(EU){&+}uj^9+WgqmNMRNi}yF zfe%rE+X6e#k(E;-l;E(=6$W!?ky7&N9A{Utxc%z!f5;55A-yU;?1@wM8O0Toyz5k+ zg>6kR;J^l-=)oT8l&FAHiSSSKuRTBcJ4-)FUpGt@S& zjCJ1Q&mL5Kz0~;TN>8Todd93^h?{aeeX@d_pv>Xg66%z%r+5BW%=_pN_ZtNiSa6(F z=*f5d8qZ@U*+&@Q#i~?yOvT8F;(`uV)5Eq;5}*NPYcWDb3zx00UF~HcT=^e;0p8s? zreNSgoHOUyO$I2 z6C66!?q6SIjO-Fda@BF;VV@XO4^3#*sKIA7djNN0C_BlemJaCKPR}uKph+@&H}}~C zZgDxF0~&n7zEg9N1*TB2?1d=t-(Hhu&Qo8`l#*JFqKyCM%wE<}8^2|2=MCj3Zd>5YkPul-gHDuEPTuu{8VT zHSM~UZx@+`oqJa-yDs;c_g`KPO#^MN89Se+eXNmETiyI9vJ~b4h3%!AEI4%|;v~P9 z+r{0lFXajs8iMSe0apMBtUdjP{bM6~vu?88er)|8y3=dp&-DNX z=tpe_sJO^i20NqX<(|1RVtqDS@Ts?ewbFC6Xz=%x|M+5EL`@mp{ZNB&bULeU$|x#g zSlPgA>s$oz8iDc&Y?>_E=@)cyGvcFya!n1;C`-?Ny=Afv#=l|iuXfJ!e zHB)7`#gd4l1MkJpDS8nomg<)V^Zw(*vFz&C;sd`SIZth9oHJx!WPqaq8z2w5dAiXF zl7_2K2;zmxvm}+Jn}%J5b+@@YOW5~q7FevNU78Ghr}T~od}X7Oed_P+MG!94 zxSx@ZynIyX*xcn~eq6LJ+Wmym`@veEGkrQBnd5>Esg3W#Rq98c&A15XFdAP~E3jqc zBcvDb<|yBja`+$959p*HTi)rz^5=GTN`8dUaPc99h$+&^g=^W_Y7hqt&rVU)*{ZgO z-=@7D#=HFl8vg^n6i>$AH^fNntbJ_N#Kl4*t&_pqF%DmB{*tLRV5uPK$mf^fQOT^Y zPLKgD83x00<{}}$rX%1S>dglCcF~Sa?r%AUB#2`Il57X3ofK zKdt%fM(=Z^s}A3vAnHqpQt4MQS>PVxYiTU+jN}F1j$v%cs#_RaKO zF@05@^GU&*&bLfxXss*N(hxWhxY2k)Jb|?at{Y=nqTDrtlHZn@9Z#K-+b&zl(yz*L z=_!lk9hp0{{x8QWK!*?vKcfqdP)usw%-63F4x$b6h71ETqk5p$wA z;$8xD{k!wkJbntL>yVidDMhKVr(|Pdi!pCWjfJ+i&Oka*%)q$F~nQ+o;+yhMk3uA zBoWT&MUe@p<*2|T*T%Bus`-|6$GG2zV|oH)9w*a{9_Gj3mK)f-@D8-%^-?1lNaMSnjKaBO`FpvG$t z$cE~cDpNfkDxlCq?NWoKGcaSM3;OReX^$nrj-;cb3x_9z4SJ`pVfxsJfAGa7H8RibL1-Td)&(V+uQXj zdKc}6Vt3c#B(I9u5PV`6gl4R7>imurY@d^@GAa+LteJH|&`lK^Y&nwcfDo1{Sd^@F zC}CrBvx6=5R{jxb`MW>J%;HEvM@>G3?z60wnQ>d1N9JV3YhiNycKuZbs(X)4QDX&R zN;I(!KK9AwTwvSB<+ff=D&|NgWN&GMYOJe4g5k&C(MHQpke5KvtvV?#b@pI4#ed9k zJx5JNhV{T$jN1u&KlusoLStO)s8IG=N3-JjlxVM?C6}-!=p2g{-sbn&7iYizl4xzV z8tHm9vRtKaE5I6+C|qgGbowP;J*|FLj_1*ewdOuq^LbycF%*}{f}}+m+&Vh>@eZ+` z!?kWK9NF!L_`5Q8T_Y!oiwc^{tA6?!AE0jMxEft2zx&MI)rD_p!yT4glG!{P_+Qrk zVe6v=$fvRdlfe#UgNm$Ab8hQu-cgEt$-KF{K0N-etb7lR?#zD* znxr8~-W(bEu#P6o>Z3#l2%Z3*f$$;Ic&h^MCA88z|Hzl3I9PWiYYpC1uk=&al+eiM zp!=qbn6qI1#Vj$EKNd~`zhPw9mnqf(HdBHm6qy)6wh|h()!=NMYtPhH#WwUz(?q!H z1oa!MH&VC>-W~B2mS_z5Mg8?;hoy*I{WKI(#Ig9gb-ie<)<@f+{7Jtj!t>(a^NAmd z>ezVg44B7;CVZYNfL5uNs=pj^coXDJx%A+CTnFaDzHTDe$%m6NbFRlgH%_h&+v_)r zjW;i4t+`pZH(mPh%kt0hAzB$3MlU=1%H2|8`_zdl?ar`6$wevtmge794dF=R(O2>}+kayH-o*9x6M{han{7x~eWM^ln5FO1$Eo1T2IomW4@@=MKo&*(Ok z!_L-yw(PfbwfQvYaBu<#ZmaJJ=uM^YtF73Duh!|*I#&=-T6uOSZ&s=h%T20V=|T(L zibF86VqeCa2KcO!93jnVvt2ZbY;!VS=>=-16y-6%;otIfX2eq|_!mOYi=_}tcBO1< z4Z7zw_H=OUo;nC9FwRdCqpNhHJNLkcCxQ_bwL8e4 zNa59(l!vBcNQag(-1=w3KMge?lN(|3z3wHebZfYjMUK0Qya+R@L`_El+lXFv#5@}f z`DmHKR~T0$f6dl-q}$+FZ;#brv=~CUq(l+Nu&}Ln?0H+E95m)1`4FKJn6CK(NeMaK zUh(*BahwZseJ$Q5NRSWvYeVLw5fxz#Fc#!7#2flZK1|{XaY8}oVCSp9QwIzIWZ1Mu zsp-}VIp#Dro`k|8I@>N49#U#c1 znT}h}w{oZyRpUCD6_DJS_TWoFXn5zx~Tx(~j+3?^kK* zfohlvQ>-j7m#?=P)H^pYzg!2f(!r}_y_A2X*cgP?mR68XrmZ^sQwQ^fZ+SK0(kjj<#btY@A>TT@E_`y&mtqbV zZ>i<5@n`eZy50j!D$C*PYP56jttx)+xh|M|%HF(PFM0$SDQN+OBn6!VE7_<>UJ^ng zowWLy#+}kYrdpGBoiTo+0iN2e!Lf5eY8Rmv#DiA?J}D6-KOxZ%o*NhP>L!UVt5>(heS#IZ#{3(Gc_q^Q3helY8hZyues6YMND$0WfmAy;bqE=(G=wtY>uPukN zp6U=Gmi8mRRBPuJ>!RE6jCnNQWKAoy*{!3G__6+R<-w}6kn(ifQMKVjw7}}ceYMC^ z^`1`WAj@h0b{>#=Sq|^S6XmZ)$qnrqNBJS0w2BhWZ=74pCCDXKOGk6mPAT^FYaLUL zdWMg&1DV}9g%@0I;O%PgHz*vQSj1Y|?(YLh8x#4-rx>Gc)_4o9nTv9dYlprm!n&)^>u>oDDTV zo@zoVT(w!2Ue<-~8yf(=PA>>4JyeeSC{y0j1RcAFl!+OosyTZc3rq*9Dnd{)Cei68 zT)s0Tq?D9>5@P*`!OnV%D!wsFWbKB={WlMhn7E$s5-9(PM?Ge+6Q4NVp$vEOxzrFT z={(gR;4|3#0Uru4l}0*foYK0U7Zl@QPV>X_yWj2(UQNVzNZ?Wqx2G|VX0pD7ua|4z$N@%rvOG?5 zS)F|~?&J4&I2+jm@g(6lMKQ_Ge2jM()pnap^}4|P@Hb@xD|T-~AS8v-JT66sr;2dw`O`K}Jq83d5p`vGgO_=)l?$%lq~Q=N(o%Up5PCNgIvpvL#eqTyku* zoVgaEr@d#B(6RDj-%g#H!ul1YandteNE{F5qRc|?_zxvM-Ujk&WuMWE#uFNpwq@5S z(Wiy&FjNA$@fl6;$y`o=$BmO0(j5y-eu?Qvs_?}Q!NF$St8FE`gU)NFf{uTjGx4;k z=kwRn7NAD(J;vj3-f3r*-HJ6gXdCy4GDjB4`hC64>S0}yYypp25|ariuO}JSud1wN zdeNyOMoOaN15=z$;vWopeN76^BTycMU4!iBY)UK>M-1L)-3Ct3+=A}n!7KnEp{+)> z_Tb#3cV`UvbeF77hpFVs-Y)ITGb_$|&WaJBt&fm0GXqqU9zkg{wsEa+N?GNTTWRA5 zZmP}Y>20B0Wt}Qf>hDv(ylE0`qqM^vNyFazD##*jVLR;-!-AkwpN||k+t3<$il|!U z{zpVb7ON(3%yx4HmqC355xs^k(X(hI87;pnH3OMLhgybpH@>`8AUWzhMoToqLfI>(y2GcCzTgc%O-^jCo^rmm6^YbkySF0IK z0q5fMZXOV{= zbg9Lqu#o$x!|5otc|@!QD08#11Ume>D@4RCI1B%Flg@53Le;cyKhgcBiq=> zseH&5xxwTY&hOi90XLBHsgJ8C0`hoz^Ceo?HO@Wp5iOvqafp0wnogs8?Z}xuZC${OZ%ZkQdIadk+V|ZB*da9-MTZ+jVD$`6~EcMm5sl-z+u z)9=C3ec1xmJScA(mY;EZTcD%fG3vTg$IGRZM|O-yZ6Z6vM?2LLgoYYMik164M}#bP z3YuG$YCU_u_0sB|8mU z5!3gd3h@_oMJ&lB>*~y{8pNY&o|j=KXf6hL9 zI6yxQ@UzCx(d>H`wp__HYn^J*xHrn}DALXVW?-ntzCJ$-efnqgv~$n9)XvGoZ(Gpe zHAhFdN`Y6~QuvPCWYr}O9u>xgm9T1i{aL=%4u6~8_zD#yuJ1~%MH{1=HKU@waAhk- zdpb*kXqDI9Z%Kl3Mw7UdF#8^rKiI~>^>T)G4f~aq7MduRx2%ib zctl!mTg|v!-_%>l`~5GIkHu%{8r?XZln@GMT4b6Mq+%7((RFQ5Wbx=6g<+ zbYl9(){!U5pRKr#Mjk^fPdBdV4ILzKvXs1CHl$7*-x6z)N#d>I&f(sbsyAiYUpf=T zN-(37NEUfH&*HF9wwkNaB33Y3_ZPGPoNm;4pk~AG!C2C`B5V??kDBkWJ8cSi-Xv#e zI0gXJ`uTH*_u$bFzgntO6bF$^%@oxma>pd5?`cKCZFR!2^n4%e<$YO40m@)5gqHbu z87IYO3D1Z_u^&$C1u!+;R9QElwwJ$kvEngP7M7bSF|xM6clQr~_tY|q8Bt;5x1gt~ z7rJS~sDq2OcB5}dwr>)C2F@no!hIFBjUii&%YjpkwMc$T*?F>4ex22Z?2%G(5I&s| zx%Hi)GHEaLJWoEs2%K{L*UtTah$y0wH#;en(XFqPz%ZoLFtwrwb5CYB`3QVJ{AJDpH>*uy9HXbvo=hiEFYVaTsGnD~V>=dws z%09o}AMz}S0(Zu=DgZ{&Qg!zc+w%C^qrbc~mg8y73%x%uv3zuP%PmUpAJvBVhU$x zdtVX(QOYAw)W;mUy%fyEG+AS*&o~l+CVaSv*mKaf{_?{&v5%|M86=*$W^Har_4WDf zdu}pNMT=5hRb!gQ+IVI98~MAeLeJfl;BUCYE%t--8s{@VqH{n$oQNvMxlH9g8F>2jcrBnr+9}_biJnZJxBI zRpIM!)#)pI#19rh)gy%$DoGz9T&cm2dQ%#@BQ;RhPPxKw*#Ror=4O_+AMDhd-gAd@?-({91y&XSoz`G*biT| z7(UDcdC{CLbVZxQ@M?NeP9u0v0|oRzlXpGe#E#>5P;+cdmFCf?#73QKtD=O7gS?vM zX^ABZ!Q&y;2fEI4&s$O<{7@{EHHz;n==4+8pMCU`HGwQgk`au@JxWG17<4M00*GH@yUgl0n&yx49ZPRS3OVOig7I?oChsC7 zW%HWa|EQVQL{q$17Fc69($vdi41TLX!VoJcNs#$3IgGs`;Jc|gJXsPYB-1j|YT$Rz zt6=!UNm-XfV?ZJVWI@R?QbIGF+mqV@mBxcf7E}|Jy(z5?QYY`7-~LsxL~xxp&BdE4 zUCie{IMF+%=MX&k(_BXt7{W}<7BU3<9H3lBB2y_8uG5VA{OL7eAakz6B7pNS6MH`} zW?5J6CQ2lj9^aQGX7XY4SWo07)a$wQwh?6mkrS2OgAbguN*i2t+GE%RE`snT5nz!J zmQowybV`?jM?PSZ&Kmi{zgR7r_2YOsNPZbb87BqJU|s;InHZrtHP{#}RC*4c08B%( z0R{)ylkWJHPc|q3Uf>zY#7n5${x~P zjddFrf$vg2-JNXc-X@n6(pvm9UrnDc_BtyTs@$cvm+B;^I&t%dqSu-tc6!d9&}-k8 zwMVg8_Y<#phs#;StfBm#hW14i9aNp_2*}zqLGD+5xwNzLXv^GyWLSxeL(N zDx?5`NXWNN=vV z2+&8}lN}7u$=RfmBD$8Wjw<2Q_ziJ;$ZTNmO;Jd`87ypSy3j7L30M1x8O+sFDan z83BA{Y3||Al*mhU(r4fLa)|HZ{U*&>N}7{z4?31DJg`%eWf)`w@JO?(z& z5<`}saUBtQb363RU!f12DzNC=Kr&rlvR2<&HQBbpDsy2HXx}riwP#+9JsdK#VW~nl zQZ_toNTUTAt)Eh}!k>?}Qt)K*1wV>o9N&`7^Swm*NRC-j)@e1x%OHX3=2G4%(g|yF zEH&#?<-4yXu^4fWR00DCbC`ikH+5iqnxAr7nhl* z?y9-puk++VwGl|CFq_N|wiBaaXuY;xdX;Fe-Q=RFyHW4lVlbRf z>=szOI)H!2f_qnH6n-8Bko&v8WdU{5uF`e|m{JjO_ z8G|^x-cdA2Z#sl{7-o|(lr{hiTf7vp&EC^#oy4_0>kkL|I5L>m5|8$zPM+QbuFa_O zL~8ocZfwOZ5P+K`W*IwN6$ox4{JSgs_7}gh5Jy>hIX0H9jl&Ovw?0w#LJE5}&zdZs z?Xkz#+jqmahZhOM`X84Lv24k&&yNu2d%6-f3UP$m*FJl^M@89Ljh+$0_mKyr4a0U( zm@GY~*CN>t!|xmD;%u+S@{)40_x4KYio*@`BPI9w{GNX! zVKgYkhMr(36@0Z&-Y1UI&QZy9$cV<%?t7c{%RdTDM=phi9q%_~?!3j>ZdkfT8w?3R zofUty`wN9d8Q z&q^WC)F7@hLloymQE6;NZc1Z|90_>!X@eF$H_J1*b4uRh5*D`;+$nc)xx4z6)3S@{ zYHQ^^*>+G@APM6|oW>}5a_%nYjCYlDw!>$Tp4RjCx5SD|RqCdlG{`2pKEYI7dp#0d zs8CFv3r;A5@1v(&(#}$q&nDTap7;V=ruTA~SL_dQ9r&cak;FoHoL#fk1txSsFEo4O z`@G>lE(G6@g`JUD*H9d`dDGMFD~^9zb>V|K(?USo!OpQFvN`f{{SnH;SE9K2ABKa* zKOg;fo?}Oe)!+AKFw3@`txt!M*;uSw*_hv4(muxALz6V7a$l$e-*8O5tlvsHfrPb` zwL%0Okl9vBlb=&sR7}b{Y1OK>%7=v1X6r0YF;2=&R9KWWNkPT2v$P{T!u9xQv3(!_ zL@U*}%<7B;!km?kn(JrU+0Pwr?M@lUYW(E-U2hAB#_)6{Vg-P8vU0~^B$9cYQ+vAc zvcHZNc&2^0p_%5K^3NQ0lMDwf|G6~dTbHjzL>T3W%0Pl z*5_G+s!4-cT7m>Tw1Wn?$6y=yit1JAYE8nzBGHaednlYyvY*-sxUfv_ear& zq>YhRY80M`X~wQDPClu4r=_V8D&Qvxw-r_!+bIz*95Xrabj}fEt6}%Q@gMW*pBU+J z2odC)(vB)IFVx^Zg&(0V(dd)6G$=DKELDn+-o3B5Ph}FSvfmaO0rF;LmfKdz5pYgI z`0_6GF#u|sGJ<5(jPYYYI&jel+VcQKZO`G)`0`}yPB=Hjo{nXLrzL$hkC-Akg&Vo&!KS1 z%;BUSi@90AZ}k8Y`E&L*zqP9^HziDUtCXK_rPVRa z)6$F3wBaNJEHw zfa|&qxVs&Wu~PwMktE)L^w%XhGJI2E z#=ga7h)DBu)fR=c!gQ4~iOJUMxlB}4wOxxX%s-|2#k~3yQf2z?QAhl{vZyqlIRfT!%kPdD3ed&+iSm#>Av83*F-7#HTKdM1zclvKl^K!Vp+Xy z`M`a*6<)Hqv~JF_*OvxCgf;;`WB90k*?v4~t^f#@R6y`&uW=L6Q$On$&n|Xy21M8) zQOrdVcVYeG-ET_VMla1qVHAWbGuzNG#s-=Nz_R7e5;v>8)|W%PV9B#lt4pP(;f#Z= zNUv=qX7zoXUtgqEyu(x3)yL7$jT-bXh(atN7$UXy>Z^~Or}U5he>oQga3ZjwoemO(V%p=niD5FY4720WnMc)V2QS; z+fi~czgS>KdJGeL=(q|wLq8FRYwY_{b;D+hYTi-tdq`jTj zZ*oLI{(Yxh% zY<86!Z`=WutFkUW@8l%}qMia-MdV$R56T1Y1A$ocF~|R7>$>Bq{{BC@6{YNu-LeUF zZC4>>COd?LWRuNBnIXz{jS!Me_RO9sviIJ5-|O-_moz@#-+vxF?seYhea>q>pU;=Y zC?tW#Kj+WoAjcEux~TieNKI(z{Zsl0UwWR2XN$N$6Ywd*?f(=)j$8uhNw{%03dp2B zzwac*Ob%U8Z%{K4194#$l&~=newD6f3rzpfE^fS<*sEHy{3Gkou{Qxy5P*fEPD(oT z;PK|{RM|48qC&A^<3&Fhbvx^hh&6o0^(O~jB*C73O(YsRVUXFokKb5uz|E(pE0koU zcuomRmAK=i?s1&^-v`M3zK@z2Velkr{hH)gqYua6!pEJcEkdTy{ne|e$1~P;!imwg ztc~Q>@3o($Tc!+IJAq7?O-x5wzqDd1d=8<16f0E;sGY7tmuiCzU%6SG0LnG^`4%87XHghC(G*4NO5Yt@epI7lSAZ|gBA$% zj@yg|>3+gyVH`l!i*;ur~ym3kM1yu%Pb$}E8p!8*e^0C(%@5HpHBD{=N16&_^ z_bJ@f{dJP6mYlqh6FX1bYm7r=MFR$&2c437iAy7W=Nid3(_5P;z@GNjlJsfFvQNa1 zXg^D5@cpEQ`D>#@ah$|-Xb_dM*TqJm6A2a_mnu#Ahc(5|vbg(_h<>|ty|So{E?dqS zS7V?&GkD(zb9yI;8uvd35u?x_SVYefQ=fh$JqSw%;vF zB`pf5mhvF*KSfnzASEa1-(p1Yr!aJ!Q-QPc8Ip@v@j}=yEG~tQPux+1#&5fGpSbrr z&QzdL;-a^VEDtSqtk|Brb!QvTi?Z$vb({ftH-~1S>tzOx8j;*)s8qhAn~k{s0cX@; zZ=mGQ1{vY(g>W#=)MbA%S3GwOj=)EkgJB9U5b#7SO8u(nOA)uU{$mrsdVb(AqiF$dfOnz6Vuj*_}$Gjh!xRQ60*;jdzP| zElM}`P8%?022@_hmopEd#z?a&+K_lsdd6f;RRXv-cl_UL_Up+-$sEri0 zixkCL&{0#r4Yy>~X;tp|Rnq!(lrXzIDC)L#c;gus(^>sM{K7eNueqg!J%cVFtjoDb z>;9;T^FVfRrc`pfl_>NLTwz^oO`d-+GTpzA1IGcD5Ja9-*0Hp%$YC^a`>7MSao~$v z00<%rXOw%A?XMGZlL;F#ym#A7-BP5~FX+}A_kn9nNm)dz2Aa6nz`pU~98UiD-?iP> zamHbilxd>psiJ4}i6M&(4O@T5A2PN(&`gq|9*t$iT0u(0Pq++i%DzOmW+3%xwvckU zphO5e_Eay^#F*_~aF)ytf3A>|G{!pt^hE0lL2g8Eio4*ShA_SR5({$mY|w$1qUTMgMJfGgnjHZkk0nCc6c-v$*t4Mi~4E0LpX zfw=RXc+7YZSNkH5?9bOaJ|cSZu|n!Q?|mI1(O?LK%F>RvAW6BK?Quof z9nF_J4qp2P>`7SP@{a*05G~)wiIfhpcI%MWXd-bPbl96tQY5&sXNDU?xg4A#Pm`)` z$B%=Fr&`D~b4gd&$*{sb47vW?6ZbhnQ6`@RYUr6evLVGdu06EfZ%3B z&U4S>LyqENsmm{z&|e1_{HRBw6Kwrlkp(3YMyg*aF~GvxJ<-69{@LemoXjXcXyBOV z%%fc`%ezHbc~Q(MdOV!2>|Q}q!id1^iX2ta(7H&UzAh`|pj?K|=aTr=;{n<%R~lf^mpwJIzh-&zqCtQ0kGM%A=5{{a?d{ zsTRmXu88?2(B;yR`rk5yFFO$FOFWZf+Lq|{qMNFGbcw953vd;Cyb#-+-V#~@eQb&V z$uddBWA?73sT8k9xHBoc=Th80Vm8fGO6Lv4J}kXP&T?`40XV{Mu~t-J*h19u#(%xm z-bXk@O35ghkVH}+3wsSn%_a&{HC_UfMi<5=L#migkN<9o{kez00%j8X$egv)fY0(z z8M{A~lG|d{+E={Q*pQ8S(Ol!2@R5T21|S>8XP+%CT4U5?*|f2g)6z1%J%Je0Z&s9S z{XC@@mK z_WI_#PW%>abslI47Nxe_`RLD8&_$bH!R%k3BMuU!ayJd_`4XUi&a%;GU97Jo=I+R% z7z}4$UMHc9JluZLqsW(4AHmeJTzbdsaTFkoG%0$KZ?&^%7r?=9{r~*QSUIO+tVsi5A}CDfib7hqq8z zU%CFuE-;xn(R*4DqGa2LEQ}|R^4YxB2~xzdenkI@#G?vVRp`3KylVTw-w%)RkViBO zDR`r5!eZeCSXuOVs894+ewC1e{k56tEYdgk6^18<-;O^SC;sM=#oa%h)hDuzrS(2J zphJsCWXxf%#max3_K@l_Uiw`(Co+=4v`ppBp00a&ha1SD){#pEZ#z@jY9{2gcVxSq zMz)eWSubr~1+XEhy%f^}L`ss(7V|iQNN0?z5qmCp{bu#Jzrr5^ z(a9%(mrlQnr7k{s!!m^frmkP67ou++B|Oq-I$lB`bY^K=px7`myFr+yuA-KNd;vZ5 z?R)PS?y8K>6n~JRcr2d2X2*Nj*qEc9EHQxQ3b{_mr5J9DiY*5>|x8OJ*df=P8qGoADk(7Vom z^fP@wIYtg^DtcWUHp#OmGd;(DAwvo)QY&QJLU~>z^wSiI@Hxx41Va|9NVD?;)w3T7 z^fxdM+e?GfDu2<|kngSd?+_XMe&EiBrOvNOqf|+vHx?A3xI*$V0?HalUkm1Q?vih* zQ$4>cE?2tzOSw66b2`G1AO?(SRXk*vbZp!-J{>77x8dh!@*q#hJ^7I~Dat4C6`iHp zO4k(sit&R3LoS78LMaF{$5uiq0b{77Re%? zhtb+!CHXCyaU_xThOcwL1e7!~9BTnkVBvxOe>N33nZ2%4lb00_-%W&MLJ)-t>wYaw z@F+MW;R|_TYHRpvh`tH>-iyo}T=6xIH}EWTfg={3KB6uz1k4| zkR(9&Z+B!8hGqSdbS9e%mw7|lK@&2sNT$}o>9MbdKXIptOYg4FlE5`pmH$0vfFGxJ zXx>Rck#d(Z61Vv-SQM2+Lh7b93)<%{omt2(u_Q%HIiE6rpAz4)25cABa(MYrBS*{9 zcZ65&?eX^kN)3#n&j*K|vt2e)zkfN3?QzeXswI(}{;)bVYLbeDAdz`Nz~yC{r$GSM zZB9grd>naGC!;ws{Wy38X510)rhCKD^&X49NI~vCLwdp5pNW$e2H$_kK!|M@f`e-H zPUeH6t{nwne;e0ca-2q*+=})&_%ds2C9=^v$h~&`>VqlzuK01<9pw;IQ5-Jg+w7g* zeJ7J1>(xWz(;|Z&6Shf90^|*t-X%DKxbCN(cr8E?(H}4L71(T*T2EU>g~zHO!P4dX zshgKNvl#lHzV~C7Rn_E?r>q}IPoWIo^k*N-bnIVOxUr^>t_|G+V_(yjaw{0tLv2Fz zX4ivVBfsdkgiJM8%>Tj@T>ZW^H;Ox9N%TRkZ8cMhPClAXp%O*pm55J~R3)3PoLt{@ z@Ym)4^&|9wTfX2MxO$DeLVflF5mQsiK7vSD(gTAfg)dz>OP2gGoKh}K8w}@J$WUeB z1!xw|p2L04kaJGEzHwxQn#N(job2LR?2CAPjd*S-_Z|f)P`FYPO5M$eFSAMI>cnqx zxQ={bm}f;jENDT$`*H-$n=+riI(X}+yn{^ti|Ge^d$@L(TB$yAnzbe5z+npE&~#L> zy(xhsRIK-ZuZZ)^-Xm6_4 zA!>~!5n8cyoQV--!Q;dr8p7~7!ww|oWs3`t1LKYDHk>U<TkTx_k+MeJmf%X&(^MK>HeKe>Zyt%g?Z@+p`?8W!FvXrC_)|1Q!T<; z{pyw> zmadT9Oy#l0LHvY@QC!9IYuV=^geu|ZUM78~b3Y`FYu}s$se1m@+&YkzQVUb+ z*Z-5Z0P?TGwx33GWBqe;#De(nWpy!;2|wzm8W(y5q@0UP;Va&zVx55c@f^QmoT=us|1?fF>N-0+n)&+iv9X zYPwBbLz5^+5wcYeJ43ZF2***t6#~N=7rug%fxyA_N->B!=Ptx5Vf zJu!*^ij_Ho=jLJ*`3vusJp-6qZv<_qaRr{v0ghRcYiJReqvG)ZZHdsI$2UF*HtuaO zZBeYZ*`p*j(wfs+3qg!GS#KJT$S5)dEmSwTDd39Vv3TKydJ}Mg&FiobcQufj{}C#s z+Yt3v()+6#LGebkdr`eo@_<_jUHXt97v~x6J96;@uBjY`*LOHbJS&P=c4YTHx2;g^ zJWZ|j?s&qJDHEri$bXWl^%?>OL{YoyNY1aysci%n&bQR_{6w=l7SgU7gJqlg&6^p) zCA|903%y62`b>wBUHy5{#*)A@?h^orVN7La!t_iLTleLzye9$r0UglKfPW%vWOiqQ zT$%j4koe2i6ok5+bKS_IUiWuqrFJ!_zHA|D25aoPmQmZT zzy#!QvVY91nIJ54yFt8uOU(BoLL`KWKv5M}$aYc9g)ObwNr+S$cC~5!6Q7oB%3vg-XILj8)Ju&~JvN>)xi<7U#7_(5Pd1X~mfdu7V9cU|Z<|9##zwDj1) zon?~KBKt|#MRs}*epuF?Y^%vt_fx$21#7u|h9_tJS~i&E6k>K)?Ts6OpKoPbOsEKJ zMw(eOzJ4>*>$7#2qt)F}p!o5kt`wS7qq(Aq;^1|elad$OlX7j&FW#KN7`^K>y5^~b z6v?Lg0sk9RrAvu=R+_z9G`!arNVgon$h$32zEmrcsWASI_j$5}o6{937t3NhDcD%f zN{o?04)4igK{S8r1`v4M9(yEs@Aq=)-76qS7XPh`EOU|RshkksN&w8ex{@$w<9>VJ z_rt-Ub$fp8}-Hcu1o(f8z?DmRRn$o`*%&=i~v1KCAJ z7nn3|-x=f8Mj{J^(aZf+l|o-R;p2JTbS1J28quPYPfg!A50a~``&$OG4QojMcDwFd zBs#GlaYm8F0|_Tq5A!IG@k|~SQR^D@WPP}5LQ~x~N;Itp0zn-5xXHHZvp75PvdBuQ z^37mHdW4s{2PQ-la)KG|T*(tD|JaI1TC2?w8E znOw3W+D}yS=B9t;C_?8y5L%h>tt9^4!G|y2xmPZ|))9z1UvFxr@^R#THqapuYRZ=i6oESvUJF$gLnUIp@0hHD6EPV5J?LeYquHTTEryQl6zKg z%apuvK@>v^PAIfhX2cZJjJHag>J|RY#ZHmx^+3gYS)1C@fKq7Wt z7d^@$h)u8YVeQ#XDSY~z4M($*X*IrAl7bgHmeXH&+(SAFn?Ts{bHf5Eu=+MZ~{U<{z^8v4@$ce4bM>ri45JJ_oB^752BRRSwp4Z3d=Q~ zbaPQ_c;W?w`W2w-4*yHZ!VLnrxYqNh4*22+D*Oxs|EYtxcj9Gd8nHm8Uy5o%UnJ|O zy^jCnBMcOd%V}Hg;iW*V9^b`x$k8)l`FegJS+ht=6Imf`rj}tWQ@osn2*}|!3v*Qg zwey-|dc!Xp3W{hEs)kxkj4lmo2-EXdyCs+c_0ey}>2&dpNl1}8A&UDJIqHQbEqsMJ zaq0u%%`OV_S1-a;{_@saJ6HRjLzaUl^-^l#kp!p&44 zQ_7D->DSL@+5DaN(;MH_?Dj#uP;*LHRn3fixcs^aQ~`I8n>Ac3c=nR^c#tM9KC-d7WIK|_Zjwt-B~wEc*Hs1)$BF-`jpRxfsrbg zKab$X8y&I0(6$gOYhyB)jgg3xz-hLIo5kc(;kM1IY=>oKve#?5>UI96Ng=KiCPBMh zqietITgKr}=nwnp4!_lR@ox$IaSPx-qev=rTcbqazla0B3_s8~Dcf*+aG>^TPg(G} z!#}nFK*_GS5zklrWm>(9Lyb;AlIJUt`f&!xi;`_|Z#Kg<1x|`4<;r)TTSq zl^+UBymY9hSkJBh$tH*8wd#LC``P*3oPVO@^w(AH!R?~WM; zKLay5L*mXSx0_6^rcSu<(^z!&&@WlDTgl_cD@C;N4=STWdf4RX7s)yMvc#7oMVdWY zCYO57TmLrL?nBptI?-yS@q=N-$jks}vx`3$T(iCf8Dzr{*14iZ&IaE~{OT>Gf9-t$ftcleP80`O;b+YW47o-=b^WVeQq|;nf zQ_6mLBF*#LUR7B^)D+{@yz8>>1P3q;#)o8vEeu^q>J_gB_7>`!fZuq}=N^t9;qw6x z&5zFOP|*A@QYlfOU;{wC77|d*z$){GDnO_^rViK^s7l)ZtQU#5YSw=Fd|f4TdQMn; zl~9t>svfn+Oj}muLav!#E?vq#|6#_9Gp27`e1m^*+ogi0_isy8@D3c+5z{i0ZS{Zh z=m>7?kY^ZZjs~(RSx;Z=HU(5Np(;Z00&gE|V3-caiUsEO9tkV;t%tRE!BrDG=! zlEchKxNgFp#;Cx%?H2hvi&U+P`zd*>?bSl!<^tkw{e?vU6^01yjZ+9P0NQY^`f7+y z=Dzviws{xLihaX8m^RdGrYD7D{7>Jo#JuU`=x05kQ?n_~?T)XvT-tyEE z05*1d?BZSS60mg)PM@D+OHL_x zS2rFNjNQ_SZGD&CrRTnm1}^{dY*kUyo}gMT1vdd z2biEcOD~@fkR0DdP#&?ToAy>LatWOZGq=otudrwwJ6qW56tj13gOv0D;08FYrKW%Q z3ro4k9==d*`bF+KdVQvlhh|O_R3mEWP5Hx@8teTNBHKBE$}m(OOat8%xBp;k)f-lJ z1nVAJG~itr3dH;=ZP~(GP-$d0V8s4$BnJDM8N23hs8m?KWGe3$M-Jw~9d10f%#K{a zAl^|n;iZE~j^C_f6@^iK?PdmjYP)W}{|)efeySurLf}i|^us;oG<~R_*CIc{#H2uX z&xRVo$xjmmUun7VGe{=GxDLvGUNmv&^-=1ggL?N$U?= zX0h!2`22(opd3mknR6ie&g_NPjyWmXB5^T%XPpH!>)DOP;BNV88zS{})Vyz8ijs!9 zEVHc4)sDIFm;;Xgz^!-zF|t2BKS6hBZvzD9l`+DlFCtwl&eRI+#2w7Vv4pHv03dsX z0H)^AxxyC^NV0VVmETYanawO)Z$8!$Mhpxli(~{^X|JBvemJy0&6`!aAJtQqTmq;P zTJ1Z{T$%ud*yW1uaqTj_$@`IR(^rh7V%xZ~pah@+)rEn$OV&(4Yj6%lKdQlXzhQp6 zu+qhJ1fYeLqpR%zEVFw~Rnt;pOG0~7BEMra0HDDImKEcWN?QPlmn0Gs=-4|_MGYbv z$AN)il{ixS8Nrca5yCNy;Oghnp(QgL0h>IL_zVoz`}6Rs7ccel$iufT)YB_JB~cdf zuB(Gz#x4(wnsf^1xcqSeM@Ps>%B{|9uXvQYy!{M?pDZa1YNWe->X1KmrNs2c$ z;umepmS|L~`muK#kLECs;>}IgW*1qB%6O!A2i3wSCN!7)09kM`lW{H6&X}QpCY9Y3 zi{KG(9uqMYgjNN|?N2%`yVS)?DxGKnVDy}k-GI)4oUF8BlXvV0{)~|zRvAat0YvHfsPCGqM-ck1okVmCUEW=Tkn1moMsX?W1!7T}-wx zgqki4UVT0d0EE-B_M);^!e_FO9bxv**)vURSv6hnYDW*YE5p`EU<&ab z4XF;!BD>c)9Uf$qdE<;?B$dy33QhNfOq~~tz>p_0XM&34o%%cHI}e}iR+D$f?N9O4 zGED#w?~b5eMXRCP97=fEGH_lJJ}LRRbOgQJwc|3f(}g0T+pRkLd8hIaO(x|Lz28t4 zQoOSulK5~}cw|GkOa5R|zT4ClgCAZZKE$vDY*4K&bBB^5>+j|j0v(mM>|A&3wo*N# z4R&9VmwiBDQ1i&(uj!P}y)8dj`0V>B66^8h2W^%D`{UamAw6<&m**pvg=M31_ZC{ zE^&^G?xxCt8)!u79=XpnDqh*lrdI^}hl!XLjqvpp(%b3lZ@7(+} zgMsmu+XV?&rtXl$?Lvkh8xg=ZjKnl2F{`>EK!;Z@A9G?b0?w5sm0wPIL!nhCILXZVlZNR_^y_^+hi})w$^JHdWV^*-ZW*0FN`ENxbY2n3A}$v|(RubA5j@jJ&n~ zy@chZ`pr;zDoOVrHZzmvoPtd5k&qIICC_=AYxnprh0mM{vlyQ)7Eu%!T^7C71IMp% z`rKyAd~w@XHOIoARUafnRRzSoKbc(>HWSgimJZW_ruK%Wh-y0PPfR=c#ou!3E5Ky4 z-w|1QIF=Kxa|O8sY@3(^Ft$1cR`s>F^~q8u(eqgcFEUzpFpdR7$VhvvupADb3!iVt z+Gx`1a9us?nk_;Z*gk2sPQy5zL$}%}DUz#_F!4;Zf}(aXngsn6Od3#j;*kt=_G;{i z+qX<5&=(8Yf4<-ctkDA9fb&41i^dlF$T~aO*1dzG7_iwpJG5%Fx zF0EodjYW>;8tv3Q5mXD~TLpV@{vdh$!z2E|)kpR1yEnt1I2sLkW2gsPy$^*ZI~yq0 zEpWz^TED8@%}>Su4*!+pokXU+$~gPP7o5OAq3-ijOrtqLbdS^xDd9{Kx8%=0v&P59 za=}oNP}1~LifJM#K}NdLQ(x6B9}H-(qlB%NgL6VI6RPYeQ20MAWp&Mai>`S3_6Z>U z3t-2p?{N`xv=?y6%B`{v3Gp_V6s9AJAt|Vw%%;cbGI8(YDv|@4kUz{d$Nz4N0S!C4 zIqTe^nB|u#kD={@<+qrsj>O7gSuJCyoVzb2K}EJu;QUYI=Tb>oXSpmHr(t++Td8iq zY;DW~AU$F2AmGM*mY;H}=ZUqnzf8xt)yDnq$4!`iY`?+t6 ztbE-SsHwww{-=un6!I{gnGC04I7s889osySP_~Z2h787X>%>5UUawKV$a_%3j-HgY zk8#P*K;QVD#MFCy_)%WAf*WvCXCgXCSY z8&K&1QuJ);KLta=qS*w$dBydI_7mO=Ooa>!cSYJ$OgTYJFQ#8qtKfw`y}nNQgmhx} z>tP#UWY&~CTo(lG2%D);RTTnxldaCHEwgC+%{(S?Co!)TZ^jcGL&8z?h`~U} zX~bTX!+W_hu&uqaIQy=FHqF|pljIVWLWWR@U^_l4Cs5dY18hyeqb35_8&@sHB$GFB zze&BovSZYdn;jN*L>wq|BUdoPG`4CPMRE>EIh&tv^s9M2?0Yh@j=?(_RK9A5j7)LZ zF((qBF>J8~`Rl=1ZRr}=gr4=MAh+3G{3!vPq{yX^-p1yvpK50la2pM1ZT%%2wBK)f z)-gk=%oNVN6iefj*Z@V2g&S4Y);@-e)s2sIv)i<%jW)8J77Gf8Kmj6KFMRB!fz=dEq$*|+nCJa6Gv`|uF*9?7OK zjD7Q$r>|~Q7BK}76~xogbo=2`Ni?n>3xL9mm3#od%JexgwF5w+rOZ3*W_M>M9zE3c zQ9StMZ6hJK?U$fEY@e%AcT6FOr;TGYRvH{Sdur{1WX*iCf# z2bU2b+(IYzDvj&g&SxPj1*VHKoGNiYd}{C_+kndzsMT7;Nj7CE?IPyUZPu^18ap|n>!E2}fXT_bEK_NWHzwW=u^RlvJKo`a{YnT?c^PqTg9#v1w%ylz)&d z6L?pdvS&J_oLgkdV5{Z}`f3o}-iO_u9a?5GG{B}c(cx~DzhLUum#uj)zONmqYu_FU z%GJ9Yw<@`wYy68r{w{$PS+xtZnSU{4|DOPkI6}zX(M(2EM^7E3MpHtE${ zr#sQRJ^QZHakbNSKF5yQemiAmx^5g*3JxISnSJJ-#p+b z1VU7~X$S2q{kLPA?Ta|324fa_^c($NX|qYed)4(z=KIBk4i%Y@IX9|k9$^QTq;fX9 zsM8S@W*$z--prT%XL&kKxYL9siB||8l>Qds=e=OxoB0e33e^B?ox+GF=vOvf{Fss1iMs8n(;yq}-Z==DqFA3ud z4HmWp(xPt{RwdMA!RBN{CK3P0!9t;O1)-ajly6Fay^OXQI&=v+?cJDnVRMf` z1|FiwA}85IAW_+?9P5WL_?k!I36tBc(K0aiLj_xb?n|4m`$CBwZ`+UUe^v?VwH8%Z zdOh3Bo@h$`OYOU_qXp@`FL)GYA`~Fv%-MrJnyRkPy|*|J@oL@Wdu$ZVZn2X&-&l1u zj?&{m(U50hF5YWTYVY-Z+~l1g=Fr*)@jqxi9$sD8g&8ScncVvhXdaq7OaqF~8B_Mh zI^vji>!$i7Doml8+N7IS&2$G9Ub#YciL(s?^anPBye=psk$+AC7$QOVsRzb}qqU0f z-YIYrnVnR~=^?oab_aViJ?Gr|K2CbPmo>T9aYM?t7gK2)#u6u5o&AxR4?~~eR`Ks( zc}6o4aVTnssH#|o?s13>a})E$U>!>89ddHX#~3Os3W5Q)HjBME$dAU;KYN+>lZ47s zD)(}v$1S-m7pedlA+Lge%AD4zBXJf~wb`~>Sp~n24P5`vqNRyM&rC78h2J$~_|IR6 zXQkaP6di>DCIVp?3rGE<5MAKh$9B}?GVo|dw#%&^0*YdaIFeu)z^8pQ-0Vm zsg2pAmVH9L9MIbVHeucMad$J1n*+K((fxQXN zA@!|m880cFLeQYYZOr3f_Rc$Bf)rh{Uf87@{hwO4){XBKQTUsl=d1Z-JBjrKM#rvzmiJ)rCRcp{2*Et?v9 zI7ckn?2^Ql!QB~V;v3q7Q}=Y3@mg2@sMHXt2!kuR4A`2{M|dIcb6s6fxBkU$@NzJd*d)!3=s~Sfi-gY zG`Xh5ZmJGc4Uv+;_Gl&7p-=oS1Ca*j7uZHq9@D9WqE9D;E*(2OaR-Yoeq8G_(?q7{ zUi%tLb57WXuOlVv_cN8&Tjt;jv@aHjcJ7iMM38@L9#Db5upHEwG6WH~Uv3=CF|xm_ zKjXk+MHf(eUhcs-hdSg(@CxhNq^Ja3=Mmh}s1q0^I0)H7*xiXNSRC!`@BcYBn2H%T z&YsdLriJWk%TASL@-uTFJLi4dIL7#ncN&@mB@p`jR;G6Y%;faXqW5D=-@Ib_zL^uk zsI4y9OjOvC99^kT+mPanVrsP*YA!a6<5tvbvbl#H%5S4NouO%`Ih~_MiUsoEsyWE+ z;n!MWVVfw1SVadFFkdcZ2F%J}%wXp*DvEG$h??7IzUHb`e5}8F95h_SyGbW3dfuXk zeAWBvujwy;J%^yIo2(RDO0dir=_P1jPD3DF7O#@RnMo3(O@%+TK`)?kW5`xxfCI)n z@v#cAnkqJP-+c34TXXt_AH{m&rXA>-JH{5??~|RGiV8_V>MuvbEw$Div_lSr+60~f z-9|yky3oirQE0Qn~qt8-TT-6|$6# zyJanK-pm6`-Q!`bjpnmVDA){;RVKv|mer3@{aSGH2{kV)YsSTgZ}1dEeOm^Z;DdlA zVwiNhp>B{>3T_!LY($gHe`e5~0f%KiH$ReYL{q;d+a2R}mPqq<hD$kLWDl4qZbTR9qwejki&yTVeoAUL%cJg%<;bfau^NECNWSm3^HgR| zK6?2ngnK1k3D7K2$$&O%EFfpLo_-zxj6s1X7R*a< zlx{eDyg-v1b%I*_ZN9}wMn8re1)gjn8^E%J_OM?|ERVib^Ef7;pXpSIg5YLo;qQ1W z8Gw1V{i;WTHhAV=C?Z*l@%w1E3}9z?!*&nbhY?kS?@3>x&~>Wj_ABLi*{|svuY7nu zNGR?}PoF?^T0#P$N`vY0 z9sL_fm%WY6-BQKEl=k7F6jXG|djVxFWv!OZp{}8>N=34-^tzEvfrfWH;6$kFF0tnw zY-NGAc;oAt`v%CGV@NmV88Mua86Ti1Q9g#cqZ2tZDq&a=)<@h{M=0Z}~ zCe)lOeaK?Ny2FYVn|Zg6d`Ylag{n0kK6z~?ul(k99JX*fck~VO}v1bjJOP?K|B# zt1wuB5#5!q6B82~94qk9(b)NiyU2=$LUTzqS^!RWs} z!H8ya-W{dPCStoAyXY(i*sT? z1r|HB-f*#bpkDG5@^!1>!S}Cm0#ZM;HC)0kOgP-mlb8sj({V6cYz{IvL5SY8^cz%1 z*(%F#ocJYCoYH&r?HPWW{jyO8B_3JhVMZYEy~{_#pJq4WYk z@H>g(k4<&SjVgbZ-p&+xRcF3$kIGv=bAO57=P~^Dvp{qB!e1&n{=Qi4QhQfx2%lDd ziLtthtkH%C$wXCXLf}Mu`g_Bf6?H-8VcXYN81NUXvjPoAY&>7LEb~IHww(Ms9cM_h zVT1$xMXX^AS5a!suFvDEzzwHE?BzFFs?-ACZv90!Jo=FP4`R{h7%ShuZWTZgM9?}A zq2R@xW7r;>0lQ_`eYegnQP?Ft;&?-K4DnVqYpwBk7$>gmQ>~x6p}#H@wvIXzfW>|n z7F+%lZ>1y1T->#E*S~?kW5$77nyHbC#qjzwCzXEvnxjU?9&(v@ZXz# zOL+6Cs0gX{!sDLLuzS4kOa2%O=oP{oZG#=E$BW_3ie}=jKT|msT^}-w!*SQ_FD};5 zL-lMceGW>ZgtG%nA`z z&!QTKX}bC1xQ&-#L4;CTb)gOch>Cm+UKuQC4%K-WKCq^kw<}%9UdHb4(qFwG`b)Nd z1@9+Mr*$3$Qo?F>z4>anTwif~*xo2_)`ssF%NN|uwvkZXnVAB6R@C(@lW?aSCXu%m zd0DmABg2#ZYXTp@RZX?FN@aS}Z<&ti4TmC`xBUED55rvo5+QuX3As7Mimx@6my zZDL2pn>3-Pb^Yc&oH^d{M-98-HsO7SR0*?O>jiR0RwO$PVsB@QH;NY8E^x%cad=Mi zfyqzpz3AR}JBLYl@*oGg?TfEpOODwF??!CsTNguc9zqVLMfAbQu^QX?a6ZjuHrZm~ z7VYJpFaw1MPMzzP=o9m_$m9M9zp-U61q{e>gd#XZOtPv&XLy_I*IOICXImc^Nr@z% z05)|lL+qDDWyr&GqO61r@iRV{wQd&r&IZmzwJwHVcP1Lh4|umVWYEke;p#((iD~B{ zvuOoCw39}vVrJsVC5`i3Rp(R~XK9IxqC_)OiX3PgZ-=ewYg(B~&+L3ib6EHVxZjlh z06%PdZ7|{)iMfI9rX-E$8w(b=3f5a(`DbDY!%OPvcy&z}O~?vmTDY2QgU^ro1`fys zmt3PICn?DQ4(GEzXGn`D&Mz_Rv{MvT^P{$`ltx3q@J@*Q?@0)rRW^t`+JqZd?I&mWm= z_d9rtE!ZP`OF?)Tvo&mq@e_Qh%!;PK3ZN*7XVX%UtksORkFu z@(#x%3S^yoF4e>Hs(p{*eKqeY?h(EwUbN|)`FYj}K-=jr$hL~>sAO6ySqq7OS0zF; z8aei%OsD?4y@-j9y~5k#MIf1Zp0n&GqP$pAQgW;6jylv+%W*DgO9mg~m@sOo^V7VQ z4*7C12WxbPIVEkTW{Lz4Lju$oIkyi-&%)GtmM|coNUfzA?e`c2N*ZQrcG10PY|MMN zjS+qFCFQ5wazimAFZxR&rAo4?pSe>u=T795c?+v<5v2vwP)FRY1rPRqLj@uIK%?(| z$P>&tf^bw1x(&b|TQYoQoEz@Z1W2*V%U>HB8uF)J3*HqSMrc)-H74q)AD=LHFq=+@ z&~Mh9Wu?YnV?%}{eta)Oz|}9uPlzCNc`vn~v~GhiZ-33igXt(O)6vGe)nL|8>z=dj z@161_m2<%KU{DodX~J9cUb4B09nG8vmMeW3bXj!Na7=ry-|oHTtU}vlr{YXLHi>aW z+ksf^jA!>lXxa+600Jj46?R4FaIZZp?G@7s5KuHf(rDfJEIfuM|H%`M(arl7x&`K` z7ahR`=9RM?mDvkWm~QABflS2|%!s%&Ajp`aj8PhEp9-AGv@18>!c37a%hrctI&sQi zFgL6(D&(t|YQ=hJ7e=XV9s_gJ=%hH-@tQ5i0&H>e zjW(;S%%R&Z*A@38^sAe{EG*`=9#D#YW}W&q)=>u!$650pE(I6;-Ktjn2&Z=9K9gvV zz-_~C(W&e!G0uzi)3Ftn*ap8!=s{Jtac?TpUiwN`&A;3rHhh?+5KngM6K6X|uX^6) zjSB5I`}W@`7UC{^G8A{+J?=q&hhf(%7Vm@>;IpD11ZNlhv})aHw-SmjDKOXQN1Guv{E zSmjnt52hnOKkmnwsf}zQM`N^q$+j=r6^YqkRPF%~2buC_ZqwYMMMH%SsM@6+?gk!y znuy3)0R!zdXm;|oQTba9z8L&u7DmpsNAk@}{4i%QEd6Wo)rJq%D+kLfD)R94;Dub@ z-Pf?)*I=SGAa41P%VFz0F2@D{Z4&!&#A4cYz^QmoC*9r`;=Sk}%|t`Ir@FL#>PY3i2ZJgBZ!p z4aWw-VTJQZT(M~TJj77?Tg*&trE>I=c~+)%%z~2k#%*oI!-)-d0%x7A_u*w9k*RUW zR7dR-i;N&_tZKU9u$i-xPF^jmP21w?m5e`6=Q?-8Ic7UllLxB{pS6ZrZ+6$Fe@f)& z9(||79+o5^ATVz6#vdl&;n7*9M2Es{zD3&L-l1PSa-^%D+WKBm)LVW#s4LmCg)sY3UhpFbQiX%fX%cYodW;Wxg7fvdYD zJ@_pZmJPiXnG50?T$ZKZTtDp>nb+5jy!18q(l2Gt#IKQOp^<6+=)T~p!OPgYPE zS;a1JNh_5AGR0L>cq6hs9gabxfMMvvN?Kr)dKV8sp3XCXp1s3f-q%)m$QlOQwN**v zy20z3cLc6XK_M#l5c5acuDcDegF$#DdZ1Fv53&MqcwG+M)(kjx7_~qf0Z0G_Xu&cb zw_D4oQn{s3sf{6UBQVeN1vTuO_cmFzLy4b|QY?U$JKu~;R`vX{_` z!+-+F@(2$w!pQ8K6JqVaKERI0)zj^|(|v-k?7*{z4=Pqr74y=So6-xEXWPj>{F;%F z{TetMT&0Rz4Fv1E(so@I+ol7Ix;D?`Vb>>iExE+ZcYccJVFDGP36DP>_Oz-svxaWC zLv?3}uh~}=n~R=qgNOAk(Mk^Q5HO=lhwLB!cIW?);HT97Kc*-&Z%fhUccFPeTF z7U}dyGpU0K5+68mh-N}3;COxmmn7B3Q>g}`&{rSQ?sQZxT;qnz3rf4z+FAM{41+l) zm_N#@t+UrO7fot-c{&xZlw}*B^R#)&inWrZ4}Q(ap~gmzouaL^Iz^c|yZ!#jc5b*b zr}NnPg|l~SO=Pi0F*?vaI6`OeGi@@AyQBvXdIQmFG0PZi8;xxoYB1$epG#l3i+UZt z{g;62wv7&0f7}Inr}-SG;_0i!*YORv4a4qi>p}1K+hefE-{`o;Fu*Ql?HwC!RrGfP z^uxMVPh6rsw;F$UAnuDw!*ubxJjtJ^p#jFuaH|+Nh%GoSI>IK1l9aP#HlZW{7Yid( z;YYL}{+rAHT@d@st%BR1zP{EnGBPQqIZU@09~kei=5`Aza%^yiSMx=q201pS)sIl@ zI*X9mI0xRk4(~ONrnq{qOQ#;uLcv=F=D)3%|MBs3k04QP2!-L>=d|Q&+QX(umb?{b z2i0bi+Ao1KL>|_IhgbIz%{*9J&#u!SG`c0RA6-Pnt5q(Z6~+9Sq1pbHp#y7%ajE%z zNGTm!_x1l{>&nBST;DL7YLqgAEXOjIkQ!UcZYbmwQdG!JMq)5z%Puh?TI_2J+GNcb zON?D+21&LdTlS@~FT?cvMspF8f@ArMr`>gkK-%6k$$KL$@81)T%mHDGn zkK}z(Bj-2z;Jqi4A;u~0emp^38zjQB+`65Dm1Ng@GcKFn9gvB_*O*w^CclgwLqVr( zl7?gKJ?VVY-DDOFWnrkdEzoPby=90|ci=dVLdSW1YTF{VT>^t+V`8SW(LtjMfzJf= zY#fkGUq_24*ZvGnBpp;nYd3Zm5km+jd3Tr)Gc3`)Sm4yO!*_s3&kOvv7F3;apnw>B0{)4B0JtU)TzBxXqG8Mq zkB5s0Qm>cj(%iJGe}iKI-r=t=$=XyAy)d!}K)77~89>{Xw3S5NS{VMng55lO&$u9i z^z&4cnATTR*BMN>$KX7+&!G%R^yeXut?p4`5XaHnmd z6?(x`YvfwgWb8tph0NRA4v}DP=+=V6&Bom4Ua`-s5YJ`-l6Kz1KtFlXt}LRyD7^qz z-(XdKLGZjtNFjUk64Sh@u1bQw%CUhS)7e|w9f@sbZd}TkD^gj@!XrK zz6t1ct_JBx$cqAYv6-iNBVLRix4(a;SSe(e@_;s1<9GwSbvQoX-(GUwZ`KZGCcZy$x;);Pwhv-u9aEh{D-*$ZFmAQGL;ADOsqh#rsb6b zfO2CxZuqqW5&=$3 zv#Nf{ANwkMVB;g6WVLPMb~!tYF}CN8qIS-X~{Q|$Y1=>V}gBwb8n@Ny)huLDU> zZJAw@yX7wU!REEQ)(Bkjn;)GpJ%F;U0=9M)8&TeNuqK9j64jE~;fqb0R#R;gPM&go zHOD=IGXum;3jPX-uDh1Dr<7wdegRPLymIVAC);EK!&<#>LI<1GCVSU7Hd}-af^a?~ zxE4opN4xHv)@o}=+&I&JJAWm9`{d%7%H$DH@Xhn^bZVnQRpDk&6XJ^TGJu1!dKWE4 zn6QP67plt0t?>&8bjfWC-SpcaSXs8^G_m1aE4A7n%{pdeimCt1ygE?xJYvVYtdxL4;baszG}d*2-rYQA?S+< z0LnQq%eL2U(azC^X75WUlXs-8x_-l!e@vghMBVNW4-J6v5O>RM=`TLkz0 z8cb}^)@TXk03%=E>JvxRp7wTwfN3Moa(#qS;7=F>G(geLb{z*T{CK^vOF!FI04(5^Oo3;}dRyr0{@wl6` zXhgzj!40)V=Nc1b`h<0sm9cSOLDKptQcsACK3j2bO|a$DtQh0GCh%dku<*NkZzR*~C2tIq?#G~CxE z7MZctjx*mbw0HAdt77BM(5Uxa*6neg;D2|MT}SeHf9wH+UN*Ts;beK5Hh1`Xhy2Hf zIV^T&KgXQ-K(?N628wm$i^T19d|jX;oe92EdM}Z$r^GR#M2}Y7SLkd>Ty4?|HEqH5 z$A+ZRw|-WLcL<#gzdiEV$Npz#i|&jCJ8X>!irbLpumwm!QulAmmuB4?2^(XY_3%Wy zg7fC95`+%vkSAc7U9g1SHK;n}^Ie(XC3mo<($>@gUGe^3a9^Czf8idp6iZ_S_- zU=AKKcJi1Y3_b4 z+a0*cuSgWG&87vN-I2NQ_;~Je;zGsQ<|{w3iU5nD4_0Z2QJM8@#piYU{*KBgtv%Pr zlZzeNTBGF8|2nL2Vjog_8e8M@Ss!_z8R;hPB4^>D;`?TxS=3uEAVIInu*l8DW635< zGNwR#Aqx<+2MFm7ngg{)iH4vY=z!-zw3H4D=Dv`v1|2R%AASH#CBmzO*}#4WuNn$G z%}-ufu|81sx%>gUmVQD6$Ew0idxQKONiXOv{d2Ks1|#IYrsxNK)@0APsrgQUMH6o* zFLa-3*m&`H47Va$PJO}7+1GWwX*){Io_{@C?0nTk0kshljvHS+ZbL|LEq7$IM)Jcan2L8d z(DRs}p*9y2#Og$yI_xKLMF`;e?1h)>w&_9`mbCf~cmwr_-g;%<{-)mS7QYZl8Hv@+TS&A! z8fMc{pp1w8L~a1F_&jsxD9Cqx`hwxm)cvx)sW~|8WOo!Xpu^3 zjmUBym9I-7qa2HZx>rtAlfZ8CTK-M4Bbq|BOvppFe&E|ljD(Im+1v%OOSdqqCf_y4 zoX<0aSk1HcKPso$y&pemE*;@6F-V**JtA%*xIRiF%5`7TM}8YCduCqi^J=c}Guz~C zh5(DtsW_kzIFyK6M?b2HEl-+Q2Y=ZcJ=HeDgTK@WJf9aY&yL>KUK%0+MjrIz#rdtIM+0WJOdr_aW!IT zRtj$3<#+SuC820*q@ek9&Q1gcz<(OwVtZm~>QB{cb6jxV_7a0Q)o+q^kQJ?E1^b0( zPpj=?u#SLG!nVn0Ph8AGlb96wZ0T#bbN6KsPTE?I^Fn!+bp?x4j>A80gFa9pgM1V& zFyP2dJzc*%3}D&pf_Q?dvDmI}V|znQY1)nzuRi8%eA(5QvvFdrZSRSG`|YKUlS_k5 zsF=(3!D-#eJP?~bvzzj95A5#}$_4lQkWGHeEEsmTUAmZh0PQ;U=&!Y-^)AVuSZfAf z75Ag7?Gx1D{y8^%q{et^#{!<;0=-!S$T6PvRO(!-Ac>MXsU@a-t0&U}1s-|3_OIWq zL;79muPQ1Y_~i-^fW@Q5;6xUIqrYxj?EC;wm#!T3v|khwhKZ(-nKLKY6s~9vtWSQ9 z4+%i@p+*2s)K5-k8*Wt;C~SZ1lLltDYn+GSo)<3_ypgMMr&rOrI>DmOauqO_#5Tp5 zPuG{;;b8VxfO%S%@0AgOh@o`9{tLoXl35Um&pl{d-~iV>P>u$J?D1~F07FMY0kL1l z58Mp^;R8-2&|Z-&IL^@V^gjogba15N(38tQT{!S9=-DjbQLeL1j%cdDRQnK-Uv{-r z;V&OEJ-;S5w*U;*-DST!LeOO=JGufC zfd_@mKtV>FCD~)DqcW*hpdef-Z9!0OW~A0Ra*x!OlE#NYj@ZxhpOR;ewDm#f3eyYL zN6)X6li!JyUX7acmZ^MHFF&YBpwvUZ+u!%I-N$o)DS0Docq2u_u8Y4u4d4j1m?~Ky1WHR`VCB99@yOQa+6C#EmA|&VLd{mAk2@t{ zSCiiXT`n4s=cTFnVX24LEXC9U>ZdrKhu?BStDLjN~;@C*cAL z%}!;_yDL4an9;dI$!0H)$1t8%2zx?&_Uzd-Fy!@2YUo@9a4s-v)8S>qLfV^dG(4XVW9UqHFv6s6l?n`|gYSB`<)XS~2`nGPvI7H@oe%XmFLg+K?C4HWm-(=@8bT zSEddhpC9+L*j$h(9DMvjj+xc5HAlSn z)J125ehA}an=f2%-KqXL&=C2WBmZO`}agX74x$H4P z0;o9a+4ozPcb1{>e8N(88`7^Oyh^>8YOgdn9`Ne*D*EetwY}la1|)nGRenu3EXdhu z^$|mxX}07Ye`ay#$cc?S*oT~8X|j16biLt@_SB%KJoZQH_4^w(62!}GGD=(e{F*!c zG@@mL;MffREpr{g8XcTJCW@f4T@=`%PrAk_G=1-HVOu;lw&|@jT;5UKPV@_mAYU7e z;e~`A);I5Hgn2(w#?k47LJHY>O2+)lV$Vz__HJ)hSqaB|$p|NVsq@Yh8oV3YB{v_{ zQ{|g&J}}Ho(PJlzO^TeqI7n%-x4i4iIqn-zd-Imr?ps(Jf$)n|9Pnm=?aqTf%|EmB zpZ$b?emVcShF)N>wq6mImr=VHby7=n6EDZ z0g)^=O+-u?S6jKU*Lz~+rV!+mRlmFq%qB@9fzxt^v_Qm! zlYpyvRqJ8-339WSaV;AHZ|CSb)wZB#K0=`Q*TX{Z*#pk^BhVkXxQ~`J7fZ8$6&|Q~ z8u0ch0iKztXWvM~5~H~&LNd)~PkZzujSW7$CvhXeA;futFKzLp+WAdFcO-3Z>V#tk zJ7F#?(S|VLa_;}x4*l(0F;k?TFt}O8R?YqQ0dh|X!X9<^lyKI?!%pxdC6%Tt_M3*Q z(z&w8V9BIx)(__f!&xWcgP3=~@NnAn0c6*46~pd5dxi_oe}?X1Ei!me$30ElsQ2Viu&r`t zPxRa5p0059Y%Vqp^BXIc>^@)n=-_W8Kfp6Qc4Dd&5ul9f>U3Wx{dOn5n}3K{h<=#p zM)1`@6H-&>ah4CY+FTZwg5VA>wd;NTxxbQSsvahTUm75HUu%3R%WZ-Z9(f$#7Tqi; zHJpvw?;2i6@4{d;8ERnLw#bdq&Gc7Kte&+wgqIWNb-qHsdvTeCz*~a-R`_@_<^tSXX9FymNb1qMgfbRd~0stOnKOoz3 zgkd%`%XcS}|9=RkUy1f(A^wN0o$ z)&-u0HICj|jt{Uk3GJYPH2wYxTj-*Vl7jFwVU?pT(FJ@UaetfCSg~csT#8FF64z(? z@K@gncnB9}K1$Ia7FK{KsW>MX3hgVMX3>vM)!Moe__tVoeFJIlm*WS~%^yBoL?u5t zDz|H%>ZA8alz8PNM!X;@Ec9+>yw@jpeQCh+44Lap@_n!hJ7f3e>(g68imk-J(&f!*}p#CHx^YIYtr@88AxvF42*WwM9CpfIUn5-NE`CwNJ&O+z1Q zRXZB<(rPjmqx3Iep2mI+VcdZ&+vRWM2`i0YO3IRBnD(Rkk`IJQ0C zmpCvV4!ER84&1dPP5LPTDhd@s&Qd>5_rtpL#PEtyatxLv&3_~+fB6LfSz>%hY>Z%# zv1P4i4B!Qc!bR?83xH1ozBBx;wN0<`+ki{rZT{AoV^;D(AJYv4Pw&CG+6sf}@2Lyd zV2qJ>*zz_&uhK*Gm&5iVtUOU?l z)bR!zw;WZZn;;1jcU4^}<`=Kldbsjw2E4ibyKMmY61+-*9K$@}12+AaL;w6E{BSU3 zo=%Rdee+Q=V+4dv&{E@OC~$g4D4K!};9(a6tyngE84WSh8&3Om-ss^^S zRA?1(k=?3Mbo-=}T{9N!rYF2kaIex_Rtb2U=#heO&LwPMOL)4#g;Pt$bDHEuUgcbl zZArg2?(!Hy0vDszyyH}U5ved`x}*?(Tj=AMK%7HJ$GvuG$MD=l``>FSz%>JWJqBKT z@MFHohQH^Xbng?QCLH{ji4MY=ZaJ2pdq)VT31&^YOPQQ}q3kiP_f=Q%jCtYw6Uhc9 z45($!G_b?5TnH$;PT#wHV8T5nXi6MTMoHOV&?jctgJJa8l|y{wAyZ6ku*f79_#GT5 z_3l;HN&uSJ=`+=Ya`+?T%>$c>hD0uPDaRA3>_aK_?yTee_;Jez1n{*CWx_54G9 z-@)E9K?YW&lkx;oNsoQ%nq+=|MT&6M_$o*Ky8x90?7x`W@$%r#QH9FK)Dhf)t%RSb z$zS3UK8{%?!yVYw?>ngWjAK-Z4$9GnghC*vhVo(;+FfRkiq-Dxi3`N)lB4a+F{Vol6`cC^agiXCk=zt=hObalWNyVM`Z#(xaM zS_O2F8{BhK#zy6G>80ONw;$0<2Eml9HbYk));Nz5lrxQ}hFOy_7vQmyoMbx+yt52_9lkUv*^4nCtOG zRF_$ZPftax3Q6gzf9m?25jlU-jN203aHwScFX(C6qzN`Fwhj;5>~K~Av}D5_mu>u4Bwc-VJ=Xw6n#urf zzo7vNKwoC%nLcCwynF$mjI!Lq%D;HTghzcSHUGoIdu1@3=7pb*Chk)VQxqM$C|0{W z>>Tmv9jlM(A>enAVYrhp+_dYSiG43X0j0X(2P^bRoh=t`9Ix;%j zTU??I4ue33nLRtABQ(MguRAez+%-eWw974hV{b9;x;3yS+thHoveuRNWfvm1AqqmA zQP~#yL1x^J;ghLd_GyT?z?uSmv^OviRksB%G8FK?d zcWvB1w)x{Y?OY*rMb}TFiROkoa6zAG;_!dX5v5BZ0Q>)`YPu zfUucH@1^;#)BpCBOf4R7p+UOpHb6u{{an7q+ds7D{jJyYHCJ>?-Tq)JhYK7)+-@VV zEE5n6V^W|AR);9t(9r-jN5g_yt~%zN6+HdMv*9X~bwv1na>)Rvg1)V>WOZut_Q*l> z+)7PU)#1?N=6jx0$fKBQF4vVqrqIzj1@*4={y69|+G6bXuG2=jNZF8OmFy%|b=A?4 z|CF@&c{b~p19X0_9hR-D>uYp2nok4(lyZyt^A1#J6{%2Rx&mN=ssh)LXFt;s!aJ-` z*wcWX`)4CHfC=x703~?uB|0SGRfO8C&37`a9g)c$6=y)t8GoyvLssx>T67tp;T-!4 z01nPvrmO6}=BlFH*%2+h1z+2N6VeF=FN}i2GmfOPet_f>MQo=ct?0ukV>> zLbrtawLzashjE@Z)~#jOR|NFv;F{*$tcjBNO*zWoI9vF=VH0%pq;Z99Qswjc4h5i1 zP|9< z4hrlFnxz6l<`?!eSsA5dumFNg3~PX*9KiK$3)9a-$DEXJdRO)?*4ddZyZBq&%heeg&ouqKMjzb1 zTat7^NA zR!6SZiGG#H$nR7-4an>`H;Fl1&gJQJ6-1C{7Ozq#jAyQj&Jh(pk3WkyA`bWAL9@HvZul}m}eiP(|NEJyd?NWIb zL4M+7V1JKu=~%ZXx4rfxSLw|0ocpk*2gx+ZHfJ5iy9lx{eYs>~dr&k65LHSSn0denJSE?seCzS3gPdpsV zPPDvg;ni#ZX$9~})8c_6;yY-}6-7Lq2+3oLZd_OB1JpK@>qnzlQ~)yAW)vF!jrzdK z>LovHJ812y#JjuyPMp36r#!J>3UduHH@PV9eD9msQ*VjqUkGNIVaL19GYD}_D4muy z*SZ{#Y6xjoYV+#vFfged&AhCTmq_G@sL0o?uZ8k9(GIGk{gG^NDgiJq zJIjPURYRfsr%{a%ADEb`qZELKxpjAtHE6wV#fL?A*OIu?=LOT3drem#Zm-RuRm@S-qfq6cWJEDD`ZWXw<=-mw-BmK5u zJ3NAh!X}O;k2cHBeasc9AFbO}Th$>Jz18BO6lm__P_bNkXRK$r z*K6s&y~wxFX6n2+0CDlKJDQ~e6rQMJR^jR#iH}*X!Nk6zIoj%ACPCCVc zv9EmYdRt>Zeek(IP590GeO5z*}8d{^g;rGXGce6J9OchwPWcC*?h; z*}ak98r_fNM*Q33)18P(CIb^|0v3P#@Aza!DyPYPb-7zMJPxP9&1`>RE}dXI!t9pk z=5m~!Pimx~s;#YPD{l+)3bJ7%vL4lbT8vK#^Q1b974@B>{nNXC;?f zG4^HY9CqR0=T!c?1C7bH$-gE5KmRiTq6`&JM+B5} zn#mq%P&edZnf@rF7*5$6^?0|CU}%c$KGV%F)}M``L;jHa{(j(;H z0haRd`%8xuupD6d(TN88ZvpD(vHywxjrzfbe7l235ofx7gLwRWh+KUJo82RvaN)gS z!9*5}+jCb<9@EWY^Yg`D-AO@yAjKv~h&Dx`-nUD~iRZ+%y9I1Z5YDokK_qu*{E{=tRh1$OgS;Uyu`ijVf&xY4Rv;?rm~JCVq`1~C9OV`Gu2%Qrz>RWJ}d6VR9N#U2J_LvCYq+~}cTn0l0D}H%0@k#PR~F_{Mq23_JE>0t+Ou)R zxo6d`^oslH@Nq-Cipcim7@DxMSd3kaW=gb1GajrpdVazVe`Tt00{EYCi45yF4Ve8OU!97|-pGr^Z-2@7sX`rm*m2T?w zwpcqE6mzjGOPXvXhjE778dKE)$;TI7Xhhk3MvCNAL?UFyhi<f42)aKsjs|u(PP`7AG1?;81Ivs-~N z=47aPm?fra6R{dlviE_p#5(Wm7yE1WbGD8xR<4(l5EBd&jNOymEKRac-x0f-dgN{D zp$UlSf{yk*t#kJEBq-BJ-ZX5c>`GC)bO->hv{<&sUE%7*?Cw6>ZTyb(pH3CCOw7#! z(moUHpS&P2f&vR>*o?#Bp~*y+=B8D|fejJWdcz0|W2$WiVy3{rS56T_d1yyuYsVbT zK|;)h*6WVX@XnjCOvpv+?jHqjZB}Jm8Pf<*WW~{5jLNgr3g>yl1GNH}o!I%&TT1g_ zK!3r@#+~1-ze~2W_-$uleqcr-t|(|-XNfJL9t794p6th0BY)~FB0G;Y$G@2!WAg%pDQAiHd-9DAbo`Lo8K z>WnvS&w;N4;4d7w9&3-=qiCRe5f)f$J+OAN<8p<_;#k^baceeMim5uM!Z~#^Q5Yg% zmWfviw)oT@^YG%w!RWTtp1$fE6~Np@h}rGkasa74Ht)ui+?8I3zmW;QQ+!2dGJIv< zyS2PsD;LPob>Y9wV|oaKhF~b`7t&1`Ba$TGx*!-oe>yvp*GY2sQ+WfW3_GJ%N|WMH zCyVeAz+Cd5><*&z?GCXb%eI{)dg}33$czyK_)2qdt{-KuFGf$grq;GenUst@SVE7Ze7xZBQ$+!0M$*EJOET&i265hf~tyrmNDU3Ii_VB`*RSf1jJwv01a6VH_$T zO8yIF@!h)hG8k+h?qmSR?cQGq)0!dPD%r<^&;Y#0G`=X)WKY?N%W5yyW$W zrqINdXcui3j;<`W*kliX4NMrrMerv}Ni#829Y;ss@4Fff4U+rvf5XB8UJthikc{m4 z53=Y-{$dirAS?*Dy))*WYlcrd*$Z_Oz(_Ae&}k}GOGY>kOmka06_7rPdlR4-bgss# z+b&-Jq7}e&KtsJA&`i)ZgKkT*O$l#B;`67zZsOoK)F-r&RNnGgZBKH8&*s5ac9w49 zD>2){F^6a!lG{roS0q0!0~3T*@RyXqGY5pF;1<&tU7FO7rLCKymuhFyxBd^c27U$x zrp;ejf<;tiuKb6tfgY9g9x%w=wAm;~F=d3lTtOY#RWYLr`go>3-q96dy+$g1y#-|v zglkf`bbG}1d@yu`uFFoIHIZQG$%=kC&-(#3v0oStuPw)HQN;PY$>APC#kUL<0JVZI zS7&Z;)}|$K<)BH{0>PKBV?9GcS)Y+aLzA_gBW;EILfxX0 zw+k?Ri^HY+uV{X$@nxW5wv%3>KNbGnnn~>2d%J@*_NDApl!`getYjN+SRf$ftz4Sr zWM%J~B0Zs#ux_!k)$`VQ4aUmYYLU0*d0^r^y!*5fpNUH%&0M9|dth+@7<=~wFxkzD zFT&#{o;Z7)7|G=7zFe*k8mb5y?>YTX5e_U}e)dRbEC^~%_+c==#8Z0IOkfbsf-vh79H6j{ASa;&%4T?U}O% zugY&jrdnuC|r|F|*fpvtGUVM(BL2|#`9+RIbIoIodGu#do1c2*1T611V z>nIpOXUtU--3E@4%vR(V?-SQ?Mg1ax>3DxU${)q*EiYo?W z(`*$EdB)Y^JK8)uu>+jrClzLn@JIoWcXq~2o!OoPYX|kF5@^5%Jt@jbcaSqd^<3$) z!kUoFM?e1mKg(ozupEO|-3Rl3Cz0PTMwehP1fVrBD&coOBZs)MQU+v;xrP*lhFCkW z4rFuqj9Vx!)^tDg@o-xN{^Y^!^K(p@n9aU^eO}&i4E#AhEp`^VfhDn&NwBUZ;t`Oc ztIsnBIozqQEPCo!FYQG_Np6LsaOY?v(owR5? zAHk=7Z{2ft;&Lmr$L)>qmQ%%*hv$I=D+$3k&s#R=i8jdjn$22c@yTjFar?PJ59iJ+ z#9sn7UTW*pZ>W^#d1vr8djn3iUzwVlYUQDhm9^*3+vmt?COTK-yZRtcvhA%d{V$Xc zxh=FeKj&Uc-|unx=gnr(XE^nN8~7Z2O^R2M`}saTR$@|jt`E>x_Z-vYfI~Ow2t}{8 zv$@(-r9eKNzJX;|zvwu-#9o?Wpq&CmOEFR-Vd?D7i{@wQ&n$DuKBW*O(?#rI2KW|cRpqUKbI%fm(Z1Exo zi;Mtin?3?|qOV4+8DtfT|MyrovA=Idp)Cy|O&A3)pP| zR)v7_`U67iPFdSfqZAudezbT{MhpIbw&UFMf^9~<9v|eVII|@Hx(9L<$|h>nl*1rH zmh_f6w&F;}&8QCNzpC|k2suIns>LtzmR9xqrN&OBwyeg{-g@UseHGZp&~)o&ZsH$v zXN_Zwh{uV#pkim~;^%LuHJOZR0GrmfHl-V$b`xj}bw1fI#IbRP$Ylc~P!ejZkrpXr znY$XtUF1k{rH-)M$=NShgWzZ*Sw1q{i6kpf@qVSTthH=snX^C|gj2VxII;XP8x;vk zC6|38kt22}Jm{Atlle;^KNsghlO(|Vl)HXh|Kni)ymGAlk~AT`)WFQ#!>R-Dv>{V$ z*8cBBgP;3t<(B>F34kaL#_y;+g8ZRZNW4Vk1(q7vY4|E&()ahumD(g9kdfQf3P4v1 zq(`c9g7~4_V}gxOoz0QusYOrJpzCP81qFwtRClT(jL=KVY3FP`rWFLXa7gbqx*p*P zP<9`d5o79S9anpnYu26PR#2v|{`qa(Cl+8VlIb}!7Pw>HGAtqeW+_tHu}W~yV>70J zQZ)0iy0bfS)`B6gQdXNE>-;&3)?oUk%TU9HEz@!f3mjt!vN)}&*GvrNM#PD<)ZJ^( zk7DKclDIm2UZ28sigBb0P9QIP`|345G)7W2la%koUg8ti3hH^8ZfB=7-nD~ZmIvo9 zIfP!GPNw>BxoF)XRbF`+>qu`05=HJ$%zIiay3CU~6HcJebho547=ZBnKGDK2gO4zz zeU>f$6MO-13LfBVIcf-*C19G;4Vi>i$p+!yS250|`q&O@e(APH^fg&H|Y=mF}AS>EET;2(IY2P;YUUdRgBv2EKX}-~= z$oI6&JzO2zgj^H?=1zCFR=4XmJCW2;AA4-IY=*0y%C|S)L1xRk!48Gva^~Hl<4W+h z69inRP4Z3BiQfDf`J_P?NOPHG#@ftg1<^@ivTxt_G!0LK4*6;@4$=6%&EUWd_)p3y zrq3V*falkjJ~&xEgDfuC$P<=?L8<`1)zT^4r^F{NozPU%ZGeG>j^y4F%F#{Vd{uXJ zqxJ}s`ZRet|6w;GYixboe0oWBI?_Im5mcDw{WDY|pv`i2{X%`8Ird2P<(!Y|c-!Dm zE!20S-WhAaZU9t{yp%2-d}tk+feVUwK1z)I* z@Lz)xC$mwlL#0yho#vZ-F_Ww+@j#f$htSY5|Ih&SpqPA-jSaRV(gRqEGWtQ4p3!FN zhFG3|SA!L>X-HmRk9qGkg}u{#*Q`U_{728D0B|I(oY|s)D|b}&^?smJ922$Zoa0KC zk6n6svu1RQuVT=a&xp}_&7AbForoVz9P6`8;Fn4I2Zs6O7KXnX4wYUzIeiV2Rp8%~ zCD7^66@A;jiUlP6mcSD98uRAJF*&}6tf7){-x8h zIRKOOpgTdpfjDPsNZ_Kb$6P&GN5&dau{fkjvOEoi>pTVo+FVzKHdS19cE7s{ty7$g zv+whzX`?;Q>}*r6TQN;JFdg33C$AQvEuKGwwWli(MteFxsL{D zy^S+>8Fk%nc2d>1GkHUfJ7iJuz?JK|r&J#`47GV}0c~2by#9Qij1$GH(rYN)M?MzJ z%i51SQ{yJATT;)N6jwPvpVtrasL<2dk%YvzgATT|`b^eJ{sFyA4}l6`%7-^;WzvBl z6!>LPt}CKdKl@X3I)EaX?q_@D*j+w1^o+sxOnU^ZhSSCQ{T{|XIu|@Tu{Q6ao_ZjX zdRu8?uDl7CnNl(lsmL)5D?YPX^D_OxlkXwPlBwTGS681fXG*j05Y?XokPD9un9_=6 zfz4#{&81unc{!2XiSF{69^?etSuU1iff63GQgg1ykB|%X%*{6K>mGh9nj&{drs@D^ z{655R&SYrgpL4qFTDAGTEymXnlEC z>K4`AVz8|Z>ESytX{hV@lU#7$ zb}%hN;^M(taSt5QofBOo>P4377yJL1CxH_4oA`BbPXVo7E<}oSv{nI*s`n3-Z`6h zPo=E%+(Gp{rgOBtXi}Jp*417`Fc!S|)1z6R13{K#6O8t43scB9@#(Cez%ULJ8kKkMh%l*vLQ(~uX^(kG0n2Txm7<}JK})0f zr~Y1|0-vEewaD4JxGW< zm0}^+w*F+V@wY+OO51iM{}CE*T}lL9dP}k(3RpnZ_R*oXpti>37tR9ER9^Pd&j68tiq>QQ@P_Kt{%TPMLJz9`A%11y0xemZ~nhg&X#nPPD8vt zY(pnv`27O!ywvvd!K|Z&(B(pl&C-pUw4&6|YE+lFFrpt!O$x|kaTOtPADV(tg)Y?D zXIjX=RnP@l`Hsb}z`Zbio2r3d^i55YL_oycNF!@fi9$&)4dP$hi|&y~vjK{QI_P8d z90T1thg-ePFq*%QVQtie z{2#2?I@?L1WfK{j`(iBXElSVNRNR$lPd5_Iu&p9)}UhN8{*kR0MtcN-9j%E*r zVCE-Z%Q^$199fRxq820G5s{L~Y$2Gb7am(^LctcZ2zsgCy*g`$^5vIT%Az?W$V#c> zaFF<;S21D2jIIv}oU)pNT~#b~AwWao<+d(ArJ#FbuARnmWkUyC5rBT@6f## z;^A@E8d<3+)A4+`k`CoUDa~WLv1(5I-N&A)_r(UyBH{gwt?xI_T3=^^_Qy*FoWlZ) zlqNjtHwPqLtHD~TO~o2pgz{eT`XU%PTm8FJ|Hq2)E)eVf9v%koQInrF(vKXLo>|~6 z*@i^ao{A!fa_vxcmn|cZQRH^gaO|q5qwtqw4)amKa3;3i0-)6ars>heEuJ4*5lg|n z?Z99`pta@)=6fYK49ut$u9+;+x*Bo)qlfO?w z3$z%`amr%qOmMfJQF!81DotwILEyKyNH5^hM+K#Ysonm6s^9>|VM-0cwo2Zc3o-dI z++E2k%-G`Xv5_h;6(T0f%|UJ|d=A&Lg>DzVnTo++$tp%p}Ni}sw=A#9>j8P-a`OXq?ldPG#wXQQHo%tb;jgR`upw7vS9TIRAG_lw84 z>5&0kaKqc*&I**shw&A_2F~i)p2|Offs%BCSvQkk=BL$2(=4`z4E&&5AI_L#yB)=M zsX#a4X<*uT=vTpqvBUZ`R){$7`|nJNLJSiqV1xh>*98GgX+f(>veqCDk5{S+k-(MP zwg9fAlf`}yxvz>3u}H^j4%8X$yxw~$8X({P{1Ml360SbqBbN6h%dzhLA^p0zrs1Xp zT3CbRD8;S?XL-~S7~K5w@qwu*Ky~lpz`!>gQh4rP{|e{|SqS3sn)Lw~hLNUw1A8xO zgasy#v2sSSz>4fdCY{Qo7AKW6Vy7`x1{`iQLQq2|HKuU`~hKrQ^H5=x{J;hCls^l2*np)reW7q%uz|Y{Z#H1!1Vgs8;V5lkLjHnoyeE`84?_R`BJLt;E zI{|l{#?_YPVPhi8Q7Sk2;jSl{$XA6GPa|5#00?*~j|0`|nQamX$vz|ydhc&(+HqB2 zI=9=u*V{{?aduY-W^QD$^5Uk=1s8?w`b28MPg<#e96|^h;yt4ic^uTS+uJ7Si|X~5C43=(!Rlp(%B<4YjV4fuaP zC|i-4B9fEo1=j=NxXEg$28UF!V!W=iW9jjlz!5Mqf|^ zFY`PQKCK>RVxqk}sx87o_o5WEzdPE0S?)_f!y)4|_V=|9Oc~1pO@z5s5*Gf>V@B#` zolhAM)%bRAY|DAm!!_+tRmCM|(?=TfQNEmb)MhFfx+;nqN%gIe{LeR@d<4+x?ICYE zXZilOC!eGDm;)~O#fg1g8!t2FwQ8~EpQaPW5n{E4lL?yxw4Z~zixY;~IOVcsIr>s~ zx<_*O%YlVW1R#8cQfh-v&~N-i`q$Tn!mzL);+C=H|LD33s3^Ct4a1NkogyJ9A(B!` z%qSq;(lH<+AT3hTlG5ER4U*CbBdK%>NOyP7F!;ZC+pSK4Qy@)E9g!{MehPT$VQ6+(MVqWI5fXvBD z?N$#xd#yfw39mqOq?k?J6Qaupld~04h3T(v{s8=Mra?>6{<0&4pLdLh=!+m80&Kan zURxf#&@n);`JaT5genPbw;K;FAlKUE->$1n5G-pX-Ui$u_VB%)w2Z@m25k$F4t#X| zqMR7Brwg~Q$F}31Nx7N{Jpo|K=$dL%Lcu=sB)}{?3)V^yV^244lht~B;&!Q3b2T@j z4TtbSIPt1cffA+vCgvc0_r;sk*N;9HnvMjshTo*bJ?(Oxe~O&8{<3Y&sb@-6qI;nb zU*;Qcw=DMceczYAizUB!t~h4+c(5EAS7^8IzaO2!y9ifefQW~lv&G+1ipcJm4naXY zx9ACWx7xY8fCk7B4iNH?b^>?V9hnYVpB&ZT1s7CLzOf01zK9laM;z5rU)H9516ZY4 z4cpW>pprZFo^zAx#J;+DV(As(ViP$J41f;tA1m$oUvzDlzH z=Y~2H#@7zzeCXX~tC_9;uPeA+Mu*;kO+a<8GkNn6&D{{U>{M+&1i!%1 zDAxznsiX7HIB@q;}NlM?=QQTL~xKuZ5igpkflR{y#c;r)&X%J8|{!k5I3SS{xA9iM5; zXb4dz+Zte}Mj%JlR=dyv8_E}_J~n{q3X{ooN5XjlxlBO84b-5H?~ZYQ*Qm}~o0&?E zemeet4Gou!xR3*_eH759RX(#EvfPtmP-*e2*dVhp7jVQ>c0_VZd|U7Md-;B0x*cUtnhA zBMua7VJ|*9WDr*EP>SBaPekz2jE@ zp=#ydW_#u5G73Az2*CdZ6Kg?GUvlDoNoDV3ZV4)_UIdtk!;onwI6rf$10vA+z>9&E zVsJL5A2SU7Et7;2G0N^99|;vZdWoQ~3(ks0>bA{hnhGqmb!P*`-2I-$Q-a?uq?!5p%*ARPw$R;4hSV?=->;>WUkZ{GZ;7Ode z8GSG5qO}3_06GvWu}hNj*5g?sT(x&}fy*oOHS4e@u-ZRc1piuUsFz(F2G$sr+2F5^ z z9)Xc$gvZ^JQw=O$;{@omk82eab};J%BVZQ_=rC$UJ@eX<>)%+Ls2=MAA)`UnTq^C4cDL{1PaBerZSr!X|LMj`3afUq1l`Xl6?K zYSt1PwJB)%-V`4(hjl@>fY`D7eSGDlq|wKo4!F584^vyZU$xk`FMf0BRg;?Vt8XICoTYj%GBPZ#tb=MBLEr%IMXJMEt!`PZXbN-!{92UCr!wgFeu*JXLypZL`u zx_}vjSTWtty7z9iv9GGX%(&iHpt{vrtgj#BjrFpFKWKj3#1W!rdfI@+i19?xYm*x# zc6p_F==3gFuKvuf#;SHki-L;S5TS^m_zql}Gk9_t6OiNP>t(FZSgTMhrA1BWNgejf z)Vp7wstgIH+SEX+MRrb5{L@gF?hp+csI>(MF&PnChJ3`0E~Nmy!#Q6#sJNL4TVRx+YW>x4E%EM~Ca1i{Zcpg(RbqRA;cKrkx)tE z%hzcGVVk6`K!cACSh|6T{?w|)Ht^0{Bd<_r5o=wo1%!3;`EP;-;1J}K(TvZq@n7+L z$^Nw=@G9PU_u}qlwYzGt%>|pi$7c%7g@77Yfm%lLwy)&U80>m(qf~&8j;&Qp)+g=DY&o~|F)|k>U-7d}chesK&8TH}?G>l!%}Dp3F1bDI zV>A08lS7Ya4%rcZtD!qUy^VQ`J^4>U^v9EX^AHlQy(J% z_O@zv?4A(00kxv#L@)NTxmeCZd6x~S1 zXNxz|vL;qYKI7DtH^A+JI%gjK&H;P7GujTcd(DZVSN@}nlmPNp-2=`Byuw(?gTCQFl$Jj!#3*S3JVe7i4Vce&LvRXs$ezc6epV8(e-428 zZ$3O8gJv8K^!c($p8jgO&~UKqXDNwWc5+72!?VV*J#QpII6{m&bH<*&wEK(99`U{u z?9aX@E&;dA5fWQc<0aI6qc|wrQy0hPqh`Bg_t*DRoLl&YnIoER3RZ9Jk}5&G_O7)$ zye+d1@vpb%P=ibX<$@|b#srgu2T#P0MsZrS>8P%mws~o7{3l%fNjktEf~kTgrf^_Yzaw*+7)gm{V^I%1$>urL1=v*AiOLzu$ z3Bfun&1jwp`Fffo6E-Iurin~`&YJ2O>f$ku>$nxB8;2tXc(p6#8~Ol!N|>YjMbGa$ z3+4*T+dKoj?v z;b&3z(KuJ0|JgG26G!|=|0IZ>oWt6G%C0E}Px0HnIRSq34upj}*K)&TxYpYXC#zOY zA|n5h3X!}-77qCNVE&qta-7$-Wa0ck#$d2hj5-LUG@3Z^lg6v_;E zmcp@|B(Yw4tqAp5t_Y>Da5qg|AA14Jo`$K4l|5Ua%j&a5|zH^qbd!0>}WpEP1uJN&uad`;V9RI|let0hySpQ(@Uyzb7Muyg5^LO8niO96a}e ztiUO?pswDM2sC%O!li(rbVLSTu(_m$OP5FeE*auEp880Mjv>Vuo$ z^_!PG2)$!l6eNUeI%M_d!vc>CK(UA@=|Un#2%B%V5kPU_RLWy{7e5P(c~ntNCbL(Q z08Irek9<`-DXQ$TTkb{qKEvx`V7-)KlR%Yg2tB1wIq42!f>_F(;@T=?qdrWzxK~|} z39uSu!0aTxD>;sfR>=OxcARi_bR%5!0oT=2*XUh&#Ng7e=d<4JU(aR2}x_g?uC_aCVw<=vvs3TU%QLd?vFM){) z^zSow4UeUO0a`l$>s;pg!0~+~=(N(pqPI1Y{<5Z!bZ8n0&aUx6JfqZ`6#rpR7{{;- zbaz{oDl+s3gJ0D8?G+8DL_ql)ux%2|E>DMJ?}^L!SKMc(*VGzSHrw4(?jmINOS@NX zhc-%@G~8_rn%N=k?bRRCy)=;G=Srupx)#0sey=ElJda4@PKXB{EQr{NX5s|SKP=L9 zrYI0WFm1rQyqYvNn0GG}e(O?z@wQah6`;AZvxet??T$AAaRiBU;o4_k18UC&9eF4? z`(}$=9oZ9A-OB?tx11U)P_zVE4h|b~KCS{5DIlvY<^IY5l8X468VbuUE?lsMRDRFZ zIIgUKsXs@qzW3nk@8H_py{v)s%ix&A3RIt4J&UbYaT!!HONN!G%L(EPg!iNh3*Jbx zSM9S{Xz*ISIVSjIj1bRzc=t{EMa#V1s3;$uoL$mYf7Eq$5LjV4w_KshZi&iT_@f%xOLszk>_t8c@9q4i+YUBqiRA} zN6vCR9NIN5DIsREkas_OI;w6_uU0zbwimn;T~wiv=Yx{qtj&g<&^4PYn-l& zJVm0EPHUHk&Y+U4r|J{e+uvR{jLdfRofK`rDo6hSfuA%LH$8z#Kt^5+zO-UB zulr$kn0TYfnswI5S!3h7giNXR{CCr3Is7q-F{&bv&gYNT2Ys`@gqdFqXyc9{Cj8C$ zL#IENK7kb^PnFX}yUw$7l;CfQs^}!e>L}%vB;f|^=(1Um`JiEut{SG{0Q$!e- zidcMt(~V$nYB6r>A+%H0lfGVLIS={zxH$?~e48jqR1V_lmI&m$_3?ZKOJexsH%^ej zcB%eqYn65+n)`Cf1N*Fb>Y%8*c;0U~OB1pl(x(l6XQWzoAVXKdX|v+_mw)-|uWJP8 z9W$m>Tc3^o{ugTk=xOT(nf&W^T!Q<+>W3ZrZWr8RRu)8O3y?1NHD-(a^1*oGptCvw zew!$xS)*Crbx9Fb>Helv$YC#|_~{^_psYYN=MfJp(p(*Ynopyok@CmyD{$3nFUp85 z>sU~F4erw{8RVn&dzZ41x1a<~j2g?~ThCq5dtyTpL0w7x^A`v%OuTacwUgHzd+4 zM(Th}nvD4lNxKZHQ_F$U3M5V{ja(f0&~y`0$0o5e0#OirE%{P9@G(?lA{?PqKfsk4 z6MYw%nZRQ$gOoeYJSsSdhrKkNUbOb<7WA&i0iu6Q?wZ=-}`m=ETj+i%X;L4h}aWuC=ED>ygTsfz%a!Z%&ed+q{lxB)C=^l(~FedBOO4t6y(LY*(5m#vBZB zpA)C8rX$!0PWZ0LE8mEyNUz@YTjs<08D<*32sCZb$dq2H3K&gX@^e|<4Pbdszn}QQUB%s z>+zF^seb{DzoOos00qhe>e+vqg{D<3)XD3;ek%jF_^ZPk_dPotunS`qSy@-mGQj<9EA%7FM5r3)&gV zk$ay>ptr~hKIr-Qzfc#Sl9yc~HM5rp(S(RwycvjjK*aks_d^FkSTwoZ2*W2bhCrU; z21^4G?1-ngpw`ReOc%3C8}Z9u%3H7}y6lx3E9Se#s&PjYpkok64ReWI4OC0alk*XM z*s2ZaD6hafFl^WZ*6$aSz#KykbG*^HvrjyiVlD~!uoKYhvQd$Zy6v7L|BYS}5`$De zh<*&NUT{eC-n0fW6FvLUQJt4ieQI%}X0Y9_X*CS4UEyvdl$MD*eaq|{i^CI4$ra`R zC61`q`d_dn=?-^mcHmE3#+>NI{O(|@@M6fVVrPq|ct6^l30VT6QC(R`{r*gh)u7qd zFURF8VMDy`v2<%}z%~L$`m16W)3_?W3`+Gh!xjJ%8FxV&g3qH%r%+oJ`-O$2ht!1V zC4R5@A#AAmSqd5CV6pII=|)0*cWUle3(|Sk)66{9l*&ZnE@xF=3;I(0 zc4hSPyGJf_gNKORYEL5R^<5m+-dFU?sJizrN$O46!cD7oup#e@n71!}|H%v`K_R`) zxz8mzPk>0k3iToY`(!(AXlSLun#&r`BvC_du~?Aq*TjY`=#kU0-!_xl6{ib5H^#aZ zrs{`_#2h9b+KeZNMPWQ=m_U*)Ywy$D$vp=hd5VT51Bssufds?J8&OHcOF##c|PTdEVF$vAsG z&s`1i*&n=52pB|~raI(=T%FHW`($+>#2wvlRNUyB{9SA|Ov2^XG{%IRH-V;3x^S~G z3SOO7ur+EXTx^Eiw_KzLY}_dN>d?Z`K&(m(71$E=a%Bh>%R2R0k!3;yf!jgxblpnT zEoJKdU4uME4tGCCH0Ml%hb(UOb2N+Sa&%woMi5g%f^kP=7^W*UyF4VTch%G7rfubj z!;S06^Wr~<>C9)-1}J+JF6G6dfP4r@O~laCs#uvbkH-7}ye+J4(hDTOmsJngI3ymC z`|vUtKdc{1a_(PFeep8{x~(=nx;4;$CR0~GMerB4`_-*_3lJINzwrM|M_3>7<24!e zYj%==II?$|w-w13bruc(|R*a&&zQ~j6g$dw!oZl1FdRJVhHZj>$$R-VOgvJo3FmA9>H zLX-51;a{4A@$-d?NMyb$IZ{q8JWTe&4S51DLWOqp7bs-hLr?pBg{ZIxDLzjHE8yTY zB>>P+%16lzotd{km{%^jdR=N5eAF#NWa-Vl= zXg`cgJsvB!Y>y80+|Xz&ZeD2na>?V3dply=b68AJy+aNq)0+LJK8e1|V+x|WmkfPc zr||Y#mq$Shy>&2PBu69}T+gc@)!0_D>t9BkoLLQIdhh&D z(W+(0cnV1Qt}BI;voIfo<9_t=+F`LwY-~y_BuS&pz0_JaA$*w@bD`CXbq8WO9qu!v zsN+0C(X{Z;+{(cf@DN?38O;*wgqTd9Tx0rk63Ij`#yLleqJbMuIe-XC*SRHO$T=X1 zNTrGOe5?AMYb>wUJEB%spBQvhfcHL(vc&s$ArM-F)8nj~tIN_;#eanW%$XVdLJR~f z-}uc#SVg!L<(JaG5e?ja`ZCY@ZkOFgeEnfdFk*R`9TNM0CYs&^#@JQzYD$_!s>R3MtNVsw{mkT^eZ!2$zyWCMf8ZPuuh?&jhHB8Uj z3TXbh{;D96hMaM6`tOeO+CY~3-6Kw~xgIy;B53|n(ulGZsYJTF&}fs%8uAD|0!i!# z{lxTb+7XC7zco_*JMfg)Q`pAFg#D+he9|xkob8PrF3xK9k##%XWn{BY7q(ab8U z4Q$`#V=pH(vOVrbQagw}Y|;9$^A8;WfofGemvJL}L`)?9>3_oMZc;;nemt-4`}>hs zd-yR5S}6Sp(;=mvpD2w|gU_H@WQhBql8hzdE``bTQq*yDmdwl3SA??Zx!oOA&uAs? z=`%*0*f@+OyRz`dbn=`kY{1$+@O8Z}tPk#FNIpQc4;J2Si;RM)J6KwX$=@|4M+vRC zDZaPe(HyX~dXr zm}|5cD##hj$#rn0&TG3-kmnC9g+R&METMk)9s5=}7-Fbw%)r&psTn!#?W}+s9d|YL zmyWY>uSG8>N?lW*Kjrn1%zPWEl4YSRXD1koM0xgYYQ0PXU7odH=z^dnUAD#h0VSpi zwVDec!ypXOm2@_s`*cxEhE09tZiR67+zMLEavB>OJ4VG2KZ+wObX9&>iiAzhIH^I{ z5SeoUT)63R)<{tGy!frbi~-zk(i6G<{eG{^u)Y*7+|3;{Lt{KsAed0 z5<1WBbUIu!xu;uw@5TQM^*6L};$uisdmM@3i*mO9WFn0&N<1nB>!;?XjR3CEwRjH*< zc=zUyiQX=kW?+nctD^8W8Mgb?SXinKG-8*sB>Hn%sz$_dsV8j-AcB&B$>}?jvd^`{PBGiI2#JZ-F0NQ0Tk!IxkfQI;dAsg0L({SVWW#tXgz zY2M)wn_^=9!OlHNh$VA^$_djCbBaFK!Z{YF>1%#N*HI2(0Mmtk=D&Bi%i{XU8S%_}|B+Vy;h(5Lw2)BI< zrr^*~M2~ROTHc+q?@v`nYYU}+8#S9ed$F=tRqD=B(iM=!=m909h93+8GBneNY z(=dEqaRjL1;{do=MvtW*v`|0OBw#e<88dY>|;kAyrjBG{0Xp)Hw zew9c>5$E_B~OOcd7U$pelVJQsRr_Wx^=?wIwXdnH44I0k_bLnAP~Iq+-xo-a!q zE)O$2?BPa!zQ~s@kK(d`A$kmj;<;1Qt|z1snBt9h)u8z_L41AZYMA*}8?}j%meuaj zuo7W$en1r6`AwIl$jVN5D}C~Ikl?tX&mJH)eRr#FHLbD;qoR-j&kbMb6ENHvuhhAn zy+tkUAOp$~_H9Yhg zY@$!$UpP1NR%?cqY1!{3vvg9BngyZl+Q<*&V&y>C&U23EQfbtlerZUbpBBE(GFuFL zBQLa{Oon*-bWo`X!LN-SZvs%w)8LP9Dc$Xd-3_%58Yi8rnkz0WaGV1fSLIqw%Z+@*YR3Gk+vTX4BGK#=uM6XFi@Xw)x7)*NZ#*7@LGF&vH_!Xi^j#*$csf=1?M*w) z(t2DTrUw{@J@99(^ikBcs!|s^8W3On)-1mh8RwZTn@{O8RXh?i)Xz<|lWTOU$m=rE4mMPw00o~PSNi2*yBr_t~IqLdbZ+4!x1oOCqx%n zf#N!Tl23CyYgyU5u;JMq$8pVmr3V*wu|V0n3HyQ260GL$`xfe$ar8wtpjgMb@4;%D zm8Z70GO4fAjEkvW9XAEAuz#4B85_E2O<5&C%P1%}K5U8clvAnGVIH6cH4>V}USzM0 zB|bSS$VX&~sUWZ-4T3zrS5wk>HW6ksXtkwwj5;aJw$K91l_AI-D8yutwT=Tf7A+eF zY~EBgSEt_@>j%F9t{%mK=yv&Z$jb^Y1H_bq`YzBz>dOEJW&S^EdTV#lJ48AvK!y}l z)&bErNj1THI68=+Fg!^(QQglchWlP z-4j&kgi=194iiJH?MJH zFrET{x9ppaMY~LEC**RUXJ%9rMoaKuDPn@p@lS9BEmhxUXuU7fpX0G(V28}vrd+86 z%zi*0PS zEV2}SAL0$);9iWjVtir>KiWZ!(#J76BUY}(2lsmscDa1-i-TScelq<0b}gVct`0}m zrnsatW1z+F+9G6xI^z~$^}I>2esu%`LCd!Wd?M0xkqQ5Jbh?=9Dwo2jqhZanc;rS= zDILoYo%5Wx7M4#p5a5Gcco;rD{~EQXl=@BIX*waV8Y8!4ssA~>j|QRGnW1IChJ?OW zsPefA3syC&xGwhnJxSWY_nz>fCo#k-G1>SVp9|C`n*bSa|NH$S_fu?Ja4U65rO+<_ zD9GfecrnwKyVkLN?A!Y|)mU-Y=Hq=nfFQd8YBZJ|3_pDu=2AN9x<2AXHK07xnaB_q zm2P?5j}cYFnQ#~q`PNWd*IajuTk{K$nZ|h`>|^xTRwp@+@^n&z#l8W%#Z3$D_>j?q zOpQTznXUX6?EPz_wL$a`7v^lYv_Da1Z^q7wo=%Z|(=rat$B;^~D5=ZU99e7LuAY37 z+`gidi8fwSjR8`*Fn;wiB746Nh`sq2odWf&-yI(1|LFI;Jj@pf1TjQauYc3Exo&9_ z@;LJ+T1;YKz{H-2VXN87@-SIkUDg!{uh<{vwbKPmL_VVkobXh#TuESe=%%qPrAy7u z%gKNO!QClsNeG$B(CQcNbH$f8*g#>=qOU%aGA%k;ZB>N^7#Y@W3xzO%o|&Q zQ71PYMpWi6k-Om-()wn?$Fr}86&BlyHE84~Kc257cYf9JbI4REd5l9>!> zs8JsE#v@33dMSU2niFkG^Y3^VFI=)a+`}fz%~_#69mYj-oOl?gpM24|gK@dh5@~&V z=)~DJ<3;;|qgMqjA09k@A~Wm%u~1wE$@Sr+^w0MTQABGG^N{s;AJ3imV6O)VWFM`# z@I2q7DZS`!7`@x)*VoC~H!@4$1+v~!rHnz#!5^16|NZ%fgRzJcjL^iR*?ky=8x24Y2`iAX z8n+47gWK3iOh+0kOrD3kE~kgaikYY)n;Kn}#vTY5z( zJ5-f^tzcCxH@ytJlrFlcEURlW=akeZg$bR%xA&|yqwa@JHoG%aQ5ay zs5O+o$i{-yYRGW{ayDKQLD4aR6bZi3avvkdlI(|a+c0C;RO;XB@hcSx2PXl-%wL{bqfe0;#vA{F8vxXoK-P2jJf~A% zd7SE$`Z+Iu(HcN#7eje>Di;reiSl*4BRqV<&!0weJD9cxj||$?i$n1^SyNumQ#1Ne zk^KOy*QwT=t*6nG0V>2N4NGe1L%U>S!KNy*f6pZxJ;lkHtW|u z;0qa1CMN&76n_y+985sbKQ5I>v-1}ZlHm?cXxq$Ll1j)C^4@ysp?cIhd*zlyy;aC@ z3W!m9X^$cqH`5X7vq|e+kssWvQccuk+20eLy!3oO0C33M-VbpGxXMZB{XL!*fduLt z9(oeV#-gj!mj>x{QafGMd;k67P6Z@v5{>pS0{*$DxrtE6W3dl&0n?c6!2+qmOHmfcNd|6i$Yvg10SI)cX?eh$*Y;dmwyvI;Lk}6i>L9tqFt<~@BXr* z-uT{6KNC8A+k0inCgt_+De#1;*0Z{tES*3bT!I9Mm4LC8RY3ZIvOw33SKWPwOd*$_ zELtdHSyB9TlFO_iLN%({)GHTZs?}GdP69k1yw*CNd$aYvc+m%e^9$&{?boO~gL?!V zCQwmfDXaIxzZ%Tn&nZvnEd;bC`mH~hw=Vd9M=5_7oz0EgJTDe$+bu2gTy+xW)NuAf zIu@CIWSSFR|5ezNG{FFbuN5-XnJxQP1S5!Qh>uO={WwLGdVW4^($BsE5U`bMZ&ugP z&eZA1MCfV!0NWVszl+0$Auz#Jd4~tf56Xry)c!H*V*pxV$~Ni2BK90ybSDonhNps;5!-w z+AnvlAF~F;8uEyAX5)X^@baLyS-`Kw|II@Jhq5Ezv@jo$1rcq2D&VQ}`boi=OJt=F z-Na%epCl?pQ$;nrjlH#4da7_ixwY`%l~{O+P2E8P+3}`>byv2gsO2A<^Ew?`I2B)? zvokRN3oEKzw1k*I}tFy5nF_&?i`Zq$!FxNfF zORaq8;TvkKm*pFO(4@J*tk|A}EB5hZ{%Emy{QHO#?y@xX-e+w35O-cwMJ#&Ww(M)y z-*kW(a-(cfF5?s%c2j}kC`Vv$8GK7;od!{-S41jWlpvv`BBi{ zJV%Xl_b&doDW_R?qxYc#I-IyZQYk)0SfkAV!Q^|zwbiZeNkI@QT4jchv}`^d$FcWg zU-kX)X~$d9Txu5#arLr0sJ#G=pJ<7*xjSOiNSSK>XA;5s7%JQ>G*Y!` z=UloZ%CEi^T>(psEvxVQe0RPmvfJbSQ~kKZ`&a*`gb2D%AWqLI35_{Sy714`>se?IUZmm;@*4T-?x0!1 zI5zx?Px~uEYyiOF#WDno?*vE?_5h~xlL6kHSDwH~I?5SG<3HM8AjQ z{D|O`t@v4=QxjgTW_N!iTi3H0 z{!QOp00oCopPMu`jdofe%~Wa^!-StvO>XGf_ww^t0vQ~B^I1FBf?-c?vh|J1$Hcqk z;qI1NwYHfRVJBBP-!wMjJ=K)qL$Y)N;eM8MS#R=fFjNS)m?hQjA z`+7k2rxw_^#W*5XOx}V14cAGVW_Nkpy7e;9F0krOG{+stDf(!-=1waB zDA(O}+}HaCf12J^P9+EZbp zBVZpAWYgJ9piPlU8|-YIU0#bqNY3Ax@OMeHPd?vR(F|-qW6l>>_TeG7DwTxRGJIOh z^HK$h?qOHk_lpYZx$UHR2StZ{|k?oQLzI<#(YK%DS8Hl4HK*#c!VmCuC1mE;jl zsr$A?>s7{10J+$+J()1=XO_Gb{;+utlPeG}aO4n}YX}trsQcDfO#|v!(Cm-P(*Vt; zgV^~r*B#thU_@98vD8iCnLU3&U%<5ts^IR2-0#Ebr1y0D(!Kd-LT_|a`8zwG{?+0kY$#RFIx0yq4bt--04hbkW%|7a&_O3=o8)3nz8O z5doHfFXFMO9v0omf4^||O*&#E??ty{IUy$WK)z=A(F5Cl(bEuf9OF?48Oq@DAU?px zZIPz@p2p3KHWo4Ba2Yw#@fj*ViL=TDhN$?lD<(1)TLmP6zGt6~)H?&F<&iAT0(7n2 zGE{?71RxU@T_o1`0d8LY>67P}aEEW&EOCInO}6v>nfU{5opf#@AQ5YxR&H0@;rpk9 zI9}L8ne@Bo>v~7BTtCc(G##o`cEbvurBAI3o9HhK!+!c9?+x6tsOIXkW$H=Cd6zzl zNY-3#N8X?a%KlQ`e(o8;At&Og1DfYf-n*=x2 zC~AOy=g}1RvkDO<<&(*^c@A%25`luiyGi0oy!k;XdfMm={4Zc{Un_qV>$WG-O4K8? zPDzw@;~i(fAYJ3y+|_rsiBz`bm=5{#@i_YTqEH~)zQ z?}CopbsS<%17gt64J>5WC_mtjlUe;E)A|eGFW!Q?E#doQStTSqH);48Tg%%i<`sZB zi(75HW@iBifJLDN#x2%hl5LgLZxo*GI<#nE?qx?S2C?tCpLfFuiZ4%rVV`_%K+k@- zte0t;%Zn+0O36pn>n0g~Ru1J8obhhNWFQr^=gQ#EefFL3rGQurZ8joFagAf+$yp#4 zarUG`ASK|l!Eg^gw=HA(V4oq`jxM)&n1{<$wR5bx#=D?*mL^l|Tv&c5fYHMRnk^+m z7U);E?8063rVh?j!rweS1LpkFkP}i5OgK(jsXx_2gU9lQyS- zB4;^gC%21BN3oDh>)Qw3gQ5MI_N7!Zq^fI9tVWXhYx&`*^sv|(t3Pag2WTQiYlk#d zOd^Fd>eDX=^>&nfp&|aj*n~*;!c?Y67nrAbOE`!q45vFp{ zvdc%_1-uOX*Qh-k0Lj~EkqzrSfcX{%i!Z8T(fl#NZ_bwbg;EENkB%{K4 za8+EYZo+@|-VE?6tC-d8`T)!`yk>~Cr1W4pW-sluk;QG>0(NcEvQj~-{`LBn2E3@G z{$-Cn0149dL&Lo7`SAn)b@)ZCE6vwhj6p@RyP3{>~uk(wU{^XF_RB4M_sxEI90Y^ zZUw9j{Hz^ZeIPg$u36NB{*G|mL{3)95Mra*tae4m*)NIE8Km>791wyzAd;&hMnk6@ ze#aHZitkeE-=!bRwMyCLx#XQ^UtTq!{)B>bx8Q<8#`tcXPLj7mOyjCkEW3sJnIt+_ z-%4uJsp5o48_iJr{mTb9$Q|r5mg8Sz)oiAbCbl9&<^L$^Lq;uE%ttPuRfL5#3pV6b-cg)%->&F-QLS&A|U8Y zb5L0?%!K)0Ca-Xm)~zHpxtIlLbyS%mIxM-BW$pPu%lu!jB3vqdn{h&hBoEOebfvdZ z>0nIJ10y*3E!v`OT3a|edvQk|-?&`dy?0oGCiyp^Mnt0lytfNdb1Uv7} zi>efzn8YHFvP5Qds%u`C%nRsuru2_MIYs;I!*Aq$GVSCC5}o{LG+BxeYQvfW`mM1S zF}%tM@kc(F1H1+Re~yn#%RKGkJ#p6E90D@IjoWa+kjp^DW_Ts8AN)0bW8Xn+O~?F$ zC%DQl#+ww+zyJRgy4Rts6K9;>uplWdm(S zK#Pb)O7Vc9-FJa%;g06@M_C^U9RH*wpftF9m#xB>0{ya6Z{l)0eU0xe-eE8)eOjPo z&75glv+nNL_>uYjk=>MQeIEm>)3+V+Z{4tUq#r4u5Kn3r2fqS56C57FVLW`gt>kix zE&(m{4dfYH)`|F_l(7mt{&issNX?85ic^?+EJ&Bf9lS=PY65?40%zW}=xX0H=q|ah zinT=318`LTF$m@X0+3stW)|Anj3?5%fa22+80AVw>8m})p~W)d)q+W&hV+Ef%;PBp}M+($pIUQ=J6k!YQ@BmBc|ZI zA*8<1y4_TwZyNUXHgm0Nj#vzhP@HGO%dPM7jOYkIyW{JsWb#&2d7ZD~_qj;og(yf9 zus~C@#b#|Ah#T}~t_^j&!$1?C0c@-ksP54i1>uc(XsB?zOKCX2Tx1Rd_^=FG@c*%O zm0?kCYnu`UK~O+IQjwuUMH&V~K)R%)L_k2Ky9SVwR!ONrx>G`0QWyd0?(UkQ<6GlC zXP+J4bP8lKLsl$&yR_VQKmW&;UGC*A)x=FQh)2MPQS}}Q=dzY zyK?+n-8mgJU9ksY2im05<7(?|N+}=lPR|@wc*T0boafiN0_QGyfH9mD*`f?p+4IL~ z<-b3WPoc$PmPtVlXK=t=HQo#|O#qvLzt{;aue|80d*y_1*+TaiK2;`r7Fco$`4qxV z?}1N)4zWP|L)gQrf%{avx6|9V=Zn#wht~i8AvTC}q*w$x%v`o5wD!AN!n?ySA2c-c zM=ruRs*CRi6fFDZ^|w-Y`;g7)ln~F!MdS6St}(NCjcRNW&wP~Hh~xSQK1?AA5A-aHcs!#$edGjrX3r48nH);rSs(y;xygKPXyYtiRj)0B*9whd zW{3Q`jfI?Su0@!TVGAsMj$AG}ysY>-ibBH>GgH)uo!-T=@R|V`Htf>ig1Kz4weoE|&qemCbO)OP8<<;qXuNNY zt+3JoN$zvlO;plJT4m#A-ZyWpe?Hp7CM-WrHhU1?xP#+zgNM;$BlnWkNUfoCH{ner zVpYl8Ii%2XS1v^3g?Yy4;6>E|@M*E&tOPzV@jHz(^?y4dDgXH)^)L=38skuhDhl(P zX@*~qJ%>6j(%wq7NVF31@4YmELqWcS4<@Bv=lsJ!meP;Yhq*B@xhjRq7S>sj0rZyd zX--1=s#bb2%-T+wgX#l=0 zs-;Oa*>|;Bh)IOvd|VA4Ks`lFU50GBE`b2>nlRVpQ9HC72IcE@>XZ_oYd-Hf@+JUk zk$)l`Xej9oNDBcL>^#F&G^30D0Q)u1%^23r4z8XhlP_<2#Tqtp_pNncuSO6Wp6>qh z(_EoQe5L(&TqIEW)K`dSB@sqRzV}R+t;hc;K?stW<59m~rpuJ5WmiL2Slt|7Z$NJ6 zjEGd;_1Ud=Hk>2)H#49b5_2PHV4V=NZaJna%=zVs&1jBLkJ7Z9-l}uwp3Pau{lMXU3?p!PIqE;fkXr*Q-ka4ZqP$7% zMHS@8BqG?yBaUADqY}hD)N8OQgaKD(Q)Qwv?-{M{+$}3K-Qyzz5i`iSC@pl_6P^`d zNW(kM_qxOBJfztFT*Y8)3M_^zsONz>^T)bZbe0XV`enUg79*#hM019nwWi<&@hi<) zDB=oG>5?1c7)_5L0BGJT6*KeMRY-Qok{C|2?c?f0^2LI>f!Geds zkEj(2AAOQ%u94F6Jg!(dMod*N9aUl)xmWK-I_q-R$fn7Zr1if^nTg7Gc}5)h#6BPPt`<>6zhS4zI7A|h)G%E zfjN{ti{SmMl>hgQ2Su{Ji0wES)*+|=u?>-rVK3i^0}WXrehViO4yqfaUE(Uu*aq9=GKPgV5~r@g$i zJW+$EE!5Nw=cwU8Lf#R200arLJ-q<3PC9f$Ib`5ysoK0lZmE6)jhrSEumM(~atf~QJL z%@Xt-is#;e7TxeaxB&HlUkYttbOD+miuwyw(V?tr4s%Ck;!&F)=q8&- zP2n_58bI90C%ssM76fPP<*+{Vy$^J=r_Uyi4i|-{kGpMV{mF5=B{Yu6m)xiJlWpdL zm|pvOyu}J5Ns&M#7?&B;KeUL}?W=YB_Kn^R^u1vQ{8z63RQ$R?lf{O5EFz{~Oc}00 zf>~$?*;SRS7hbH74|cAgxtmh2=#-GF0MEcP`nJE8#r^had)1vKtIr_gOgO~(c6AhpcvnFMgT!K-2h>04tax~O zE(`sP>bJe4Q9<1fC&1FR$k)#!d+SHT z-<~wE=fqK}9~X5>p=kn4NkQBM7#M9I_;^T*;YQ}8M#gVIev7qauH^ng%X!kVS;_09 z|ERYC&B(1g~`Dl)!)CgnHs5Z)M;W*pX*)O6UW-I_#_qrjWv$)$vD}jPLggn{Z4X}16yDEz z2)q#~MBl$|zRdR(6DFBgB%5(2Ny6MGteLr_Yf;Pf(q(D)EP57wH2tHW{ss4TC?*Zr7OdZ0>Q17vs@-kNj%}gfs8&2% zb^VZH?>D8M{DH4p7uXMu?Y@ud$>)$vCrEaxNv-gxm;2`L{VY2^I0sNajnky#@Zvke zHva2wrq^$?zjxC^NK(vyC%}GWdHhUjb|bpT!-QJwXic->&H(`V7k?$XZ+)8uGgxi_ zhCpY;6z$p zG`x$(5Q(PGx15cpnCH<-xmlWR7wK9`CxBqb;qp$naJzz9X_)PHEL}?!<=inhS>xTmVAlz3Z9C6nKm+=Vpg{ z;Kk-K8}_0iij&;35&IL8=c-e(3_=`P`383kOwwU?2t|adhl00EP{h129}L}k0k5j4 z;`SosP3kh9Nk>w7-VHU_2)NVS#!{t!d8T~9gV|BRN{Vi_Wfyzu? z%U;{Hwxa2hrhuHlesFGlg<~A@XNLX?+zr!jb66En4rsmKHb^(~IdPU~SrwPsWO2%n z-?mGcH^7w4Xi^oI-M#CET0QzF|Mn3h$4d!4gkK)RRU(2 zq+SceqNUpfu*wWRzPBztbHD8#r46{;Lg}V14l3JSn0IG^5Igc({-;1s1V`o1H^!f~ z02Hd`GK$c@%{6v;Qgj*2J3g)tD3}sm&Zju7!Wx~YP2}~&R||lsvSMfY@_f2vBQ0Ez zmG$N)&httJ*^oE@G@sl0i|l=oXCXEdqq!WeKPb0NS-8UGBK{m>;7wp4jg!{T1k6xE zz6ox$@;exta@fGjGrwI!$C7Nlr$gd6+{JbC3Fe3Iv9Z8gp4*P5^jH2$yi z`CN-{-Ia~it!2hYB|iEX;o--MJC=Rkz^!Q-5gOkGF&W1@4%%>cRX5_WB_2)Jp!R;?|pTK$%ZLlx4(;# zpV~0^hg^Inv~_IBip;dbHEA&0(mr4NzN5Q*+`V=!RNB5i{#GY|_=f3FpB~bqO_TrJ zb#dpu(}i3!b* zminAEo%mFc@6kS(R=Z;-X$rQd6pwzhYNESAN`7p#Q{vIvf&jN3*3clG019%`b~0vi zy_|NDT3^S4;6)XTU;=0 z?tUq5n`uIGdf@^~aoNuTRcV*Vid-aX`IS}Y4T7rUs$TN<=4VQep$ppXf+2(M0Kx6z zq#d*h#($c7D0mb8dVc2p7O^g?L4(F~i3Sn_#d zV5VOwGfA_S5ZGubkmC5<5X3*5Tp!+B{KpHC&EtS30Rp{qWLKMF<_eC?uSvnZ9}L(v z1*sFx03^q8GCBw6*qu!WzTw)*r+iZ`^E*p?4FYKFUM;4KH{@H%*bHC&ZrDoujf@Qp z#GEX3A0*s^c%iQDCc`l)1A_`>ybP zF^=O{MpL`&*pg6(LamLk*4`gFZm&b>7B*QCan9pT@|IoGGg@6li=rAU5?ZVu2M@LI zhz)0G6qGonqDIgZk9L=u2IhTHrq}{I6^OB%fkwGRPy{#O$fIjMHh1R(Q)$Py&@6Q* zHr89lpD|5l5$KnLAS=;M_Kb;vd=r=IEq4O1;6o_8W```SWpU0J@iL~ySX+h@pHmob zQ<8y<%0w*4iu>(1vbHs^w<13uCK4DIf4xtB_md?#Yye*1uk!S4yy1L&2gnvg3RB+S zW*f&$Q%5K~nbca1P{-2#tla69E-g;HsiA>ujHgJ-Dj+@ALFoGqt@BN(!x@05^Lhq^ zt2rmbo6PS5nF~dXm|#is!VTZuqmHnocteD3^%Ran#9?@E0dXPu({0A{4Hz(`Gk>ah zmiUz#0Apy2wSiu<ZV&F+WIC^7QBn~5Q$7~P2-$yn(=?7hWaN#}PcLU3D zYIyCk*tKZIS(oR=E}F9Ix;0Zzl{ua1_0iDd`4&>|N&`~M>X>i5?;uvnptI@`u7(@a z%ZCB;5P$nle*OT9`|7v@mBx`MTkU_gbKF)~><^tlZ{xyZtagIu6dJx=joKrdb z&<>q;8EwhmuAs{uqDx}(+vC8!6+7Pb*qTqpA=J&&xrBM*QSxr+QCPrKKN(RGR@HK{ zyObwaU-2EfkhCu!j=m6iGAq-%DX`Ns8KT7MAB?sgfkVb*pssT(Gx*b8LF2C>(O9E* zd3Czcl5A&nNlIJbBnYo*`E-ar*Sv2r1pBT$UEE}q(;KTWdc#JFA@IP=w)F8PS~ud- zeK6&|eeKAx$X!numwg<&jF1ES8pdm=S=8BWb}iz^oX2s}h7pS_9f6kS$HSuWpS63v zHvo@*JDh*+eU$>IFGfBr@B^+FCj|FWz}A;^((Gy^^@1(3S31z^V%aO4{HCloGgs-u z6dV(wu@bz=>F-ZL`nbafvxWy)>Cbz@ae68tz| za>*4r5rwBz#LU6Z8HMKumkmuL$YPeK5kFXdBYn-L!BFbv)Kwl_2Iry7qK=a= z)z_MRYm^B4+qle_vUiJDov(BnP0#}E4PWe&l#2W{e9eFBtpXUmlLx)d+P(Y$YtFmI zwz%k9O81WTrhOgrNJ$rGO>AjAE!*mD7~5)jLUvOb@Aa==hFf!Cmq?{cQspMVnNP}% z4w4#*^E=sO=prMTW4C#4+cMm^UTGdUj5dyiuP$z8b{^7AfBEY3V7=CEgfb=?yLbBu zTHVmj6D9p3eYmjH&zzd*u!=F9Pt@bp1(Z7Nhn)W2C)EzLot2XuzDH>$U86>hpRy3x zHG{!~`%71udFv(}v=1BZQnkC>G`~=0$puSiN<8}ws>EXK8J6B)QCQSyO<;JqBb zD{(SDs>$Z&-Ig?ha7y_U`KFxfpdUK|CHo7kw$ zNy7#V$mvmclh^Y22Nc+ueHnSQGi(5-TS7R1-749ahU!!Z8saeTB$)UQ1Hn(HzMw8i zTg{u>nI|tQIhPBo(7fMRcddBV1t_JdeJ9eaTC(RpRi{w>@(b-ej~(PNw$@ed)8AkeU_xVBLunDh)&Z z%p;j@_IR`}Qcw@!NOLYcQD^3Ew-jiNagB!Ha!3!#hybST3Y_P|@z^Wtxd5Lm`&4-il#Z2`{HR$AGK z!9@IF)E4L`^Q%6L12NNWvfYt_Ck7ZjDF)T)lFqMWXu|0LT^n<6aw0RK|vlJ;}z=JsyHqm}g5@aM*W zlO1EKzT2Zq<(@FF5*_flr^5e1*(s8HBx2_a*Y_maw6g$aD_)|E=5E#l7KInXZ>jro zx})-{auWD+EB9K6TA$vh(llXC~P!36riQMrN7uh102ZnIEMFmoYiUrhnb72!cPH1Nj zn+~dU(h5u006CmPxiGwS`=VAHJ?h^85iX$Eybv?VR{P$ImnfVg zQi;Bo22IH)QV1Ztm^hnQO&lcpx)-^(<6_U}3ACzhIM(7;P<2uL_;m3H*M z)Knr|$M#itcBi>Cw` zf^F7liCUZ8sDcyFVXIK8xEgQj*W`e!_8e_zn8!8^>}b|!@M_s&3Q;R>8~S!B=K9g7 zUR~v#v28V4&-SgRQ#FuxGwx3NJS{EvO1GKk0sF}a&??Pc`^8B1_)a4=DKw1Fw(-TU zlntZ%B3p~y8a0}=`WNZjfEgjT`a_&{uZs2YNB@n>j-6R>^_le77*>T_O;Siz#6VU+*{7TH7L*suiCVx$cO@bHPk!d!3$bVL(KfpWfy+ zrYBKQ#cdH${6aLR1mbpIAp=irhdNCDL;wDHGiv0L+ETW;ifwjCU5-+2t8_L;9=fsRSg9K|qj&8bzQ zwbDa0CZeDr`~urxm($glj`RS4@B8CY*?S=a=HGeh3C-t$>6Wfrc>gDGsWCl$Ez4B1 zN`^6u)Fi?##!Bh-2BI9FCpas-s%hUF2e^7fm}x7(TmiJ$1SpO3{qj7xu? zP$fAR9TZ7kd82_xseJSgd5V}VmRmqNKDk|F$i&F#JnPx3%!}3gn7D)#JBAo01I{R< z4hDtr^n;v+GvvlLxSO)T@IPZx;(ny4>UB&E7za{;U!GX2`)rQuHLR#+y|#u^hY2i_ zY%rPGK%`v_9WBvc6eGW3mhdtPb3gg;yib!ximu7sU*lT7pcja>`$v0c32EfJIqhC& ztn5lXQ)aG&;o@D?M*qH2kK*V8n=_4VK*1qT=Da7|i#zp9 zbFsYnv2x|kRBOu_$bp;_qU`B~1)MPJcH#?enP_&TThW5lN=TCTR)6(aj%n-$Qoz3^ zv|(;YT@x6C=jj_g8_S$Yd-wTz6wj4$2V`5^q;qPBepG~3ta0hWVrit=x7Psvqb2xO7SDbBusCp^0f#LBEMQ+)-@EDg?>O~XF5vOWDRIkE1T^}Zx1=m9wk_)6!wcFT)lXtzSj zIS!9uYS7KMlnSXxAuF%zdd=ocjgIUxlY8r>NuurM(hc+A3e?By5huLY_anjsEu90Y zmsj2UQ=M<2Mn_BtP#m`mc00dDbl9vc_K z7p}YOxt06Cj>s&P62AQLBm?2!dF17RUsPw|Aiu%;O%&>ad;zC^XNrq7$qlKIBpe% z^Q%V{xEY#G3f?CAP4Rw6O8?N540(u4laK;xCLf;7HbZI62!7o#2%XDRI1fMdH|f6} zq;p>}o(l~i0hWwMTi3F%&c5IMIak7o**jgf7pP!4mRje#YYAnxMlXG#`GvlH)6-{N z(@`&2vcI!xs7CQ?e$rNi2-9(EFJ3t|Z4RbhCN<&$H3}DR<)Yx^TiUh_N^aF26WIIa z$jYgnE~G-@8fDoENV)fydx&bMSM&nO!oTt11{G@RP5aIr;`dAiPDC(Y$%QfF+*`FT zA19tjV(d5_o$4p@^?fD zBoTv%QUUizlE{#sPm`nYtl5Mx_#5#U#pnWlY^gqSjU7DQpm!eJok@Ra;vAF&f-)`d zeQ+jvP?8_CW2n(eIUsr>OgYz?WV-CXeEd=4;w|3$KDx{t4w%s_$yu6xvycl}>bjy6 zmkF-F=_U`1H$V79fC&7CSv(uBA&xst%G1w>Xrt=4PJ}LpP$FB@~ zimZ=yx!3RAXk+zZ#x514-+o-R|Fq|c2pt-BfRKQ*K?B0lykH&m1c(4#Qi$MK_Ie(^ zPUo#ogr^&g=}*^qF;^^wi4fl?oWTgg*q}UzMV(Dj1>%{r;y2v5hLU1+bYDV;zwVrU zxk2IQOg-iJRj+t0Z9Uk+7SzHULY}xc)@d_175^QbpbYh)6lMO8fHhKF%-3&ZHN}zA z`Hu6$>?*Q9ihr>q24h<>?kb*L*)OL2+*jKj`>_5f>_8q0Zwb1^y( z9I1)bYwI3>oeE%3UKWZx$$97`xTSi3(x$*EmmTv$*IOGT6jl0l?IrDIfhtm}e+x_ubV@g?eK4&E46pyrOi4rQ`Kg-*h*Xer* z!%{tIUaECgaEXoXGa!qcbo^>?s7B*zy8nDyb)Xw|om3{4!#+NuHSY~>&O6^ixXO8M zVH(=2lpWx`wfn{n(c@pAYvyHeU;hqMLrcfuVv-SB$omT_M`@$Gt)?rTi;c{IM`4zH z!)@bmdu@0A{O+o~?Q>@`lr8PQ3bZ5EBX!SWM;gEN-zCiCn(^Bv>5wt@^&jecyOKt8jPBww$iWVP>n2xs^Cq|642nL+K)8(7f zYq}=~$_*dKqCU|AX$^cw-_Z5q!Cutaz=NSo3kHiUSrR{!2>0WRWBCqq+eZp1WjIv^ z+8l+PGVLWR3!-jc%7`jt>~S&S^-0MzXj< zHc9Z5BFeTn59Az+9g-oxB(W4G8%9MFV1(c4RG@shG>@qE0!vj&oAWEi-uP_#ls}}~ zvb}a*5OoXWw0y5h;@VbKy++z@J5a!3vKHT4&hw@Y87c<{7x5irRjkggR6)eQe4Zeg z*n>zL(}tK49TPe5L=5|pwvaV9Hx-k}7Fk1#tdQDYo|BKpt(kM;kCVoXkHzE4b0mg} ze%Qu`2fdEE;GU0npp9nC8!!u*&V`sr7ew|)1bP=fscSL>@F1V?~leXZA#DbG~d9MJsWx?N4bUykU!y%U>&gJ6dX%&E{470dORU z6&we`=euvydEK@$dYb;bkZF~_1pd}_0g=re%zEnq_EZmIX%}77}GOJ?qnM`Smjs$^Z~w+YRK9Q z8uvzHVr4pIQ;U^ttt3hlv5WNYrwJ#G`3fSXFxPh1UEFe?iqz2eX!XeRrI?5G6>YDt zF3#_`;jMURI*RgdkDBVDcdR=$72r2Dv_fb88{qOE`Gik5@zHw2g#{ai)6&~-sU%9? zJ{V82u}aHez+LK))^hisYQYo%Ecrf})d;)^*e4hhGDOoP4sz8bvJ!GNr{*b1H^`IS z+we>6OXuB<=0wH9_kbAH67p*R^+vL}LHCN|f#)cUStKT5pTqgh;lA}gYQ6e1I?U7? z<_#5CiMYXxy>~+swdU3VRePltdAg;ITk!3`oA(%&016 zK6?p604a2x9I#eSgaI|Zd=rCeQn_OqliAe?rS|67P@U zA2ozJiV&3si83ql3UhT`l3Dy+=Xr;*v2s9O2m1Os*Xa2V(jbYGd%M9cEb*H9wQnz0 zZ6~+hdaf~Z>EQlbMH#1oC4cKdFUUI{Vb*a_p%8m{%1?T!_4sw4-}XF`>RZr(2@`Qy zd(JCGUiNK%s)it5V8x~g5Su(t*)rXD{-F51R|AhA$DH&txOuG#~cC$*IBcVvB0Bsiy@WO~y{Jysji@X$7- z!e97*eju%G-QDk&QV>95(JZlYLu+L(HYaSo~m5;SY8rje=`HngIK6-e=)DZ zPdwrTJ?J#x|@#53?BmIS)6RD{Pcs^bfxPvl2LG18`&dsPm9{ z54qO(-@=+RsiZ%r4LlZ!_^%7V8&CL&x3xkADtdLTG7I7{IyEfo6_Gg2S?#inaNo`= zMRh=8-C!J$CjVtR73s0%4GHkbE^FHQSWt! z1h(I`5-|qOSPMP8zC+0WLah>tUV?B{}HGvc^}fT zoEh3rNY0Knzu^JGaS(B@g{3RETJy8`m{cZfn`qRL`IIy#a6V296e$M{&5QFr$z$SP zk)$#0OIUEQd9|=JdwLUHw0KPZE1mM}29{Su<4o#$rpv=LSQB!!A%-6u@@7~yA92X} zttVB#L`P<7ULEN49&wzoFj-$Y8oV`=$WWTsRW~+)B)F%Uj9Kdb?W%v)>fYSaF-lJ2 zOontU->LG8ryHjl`8UV&1HmnFJ`lQ0yIezs`b!Fa@9b%# zy&+pan1r}vX%9itc^`j{SM(O^zU+EJbx4djkYs=-ws3RiIIxrsWT450KyL5NRg6Z| zf0WTt;4wYzkB=vkof|v|{W2k_D&Kl6%=p_sL}nv{CsgDTcb{h6D90 zT}Ov+{Sql1%Y)p{`Cs2yu+ zNBt>o%udGWIsLp=M#CX}(d7iRTmEPxD~$g}!Z?@LF`{@aH!lE0ht*l%{;h6p7zUGt zNalgGs{tP^!yWP1@6vqc4D^2iKqD%Iw&1e)js7LMb4)5_6Z?zhe6%KfgFY2kJ4h$6 zrqCl~;BnD;j)6Yy-nvQF>L#xbC&|pb>#e8@{;N5Ca3-@F5B2ITaM3%?G}TR7G*>OM zs_EwCw?=c#{nPAj2dfv|V_zB8vtTLtrLVLhtd(!8Rpw$>otd-bJz6bKs3V%v3d~2H z$eXEgb<6M3c}Y<&uX*pJdR$QWii4~R;_Oo<+>>^k%uw%>?-hJHhpXu$mNO#Zrfwgd zvWAVJe6A*7`c|$&z2x4`sYJ`4p>)jh-iHyCv?+Ivm=jAg-vbq%6F*vm*+ykejaDex zY?QPIp!q&_-7ydrNd+Q6W3z>iK$HUS+ltZGvjRP?iqlB6e5Cmbxh}Dy=3KG8Hq=a1D60ymc5qTQhZgP1Cf5;v7) zeTr$aif6E1kCvM3xq{^%8bIvBXq$Wv_j2qlF{F|2{6lG}M} z0`1oJE2x|jd&53Ym6j~7SiC|=l6GhH{YH4us$SUEy3!ZL}JG-!?e8_<$4MdXJVHx8|0`ir1}y6V^G4vFL%14Y5Y!7-Ax zGPiI|wCiv{khk?VLO>i$V)E2x2?)~sT_AQmp22TA5`UrNcvC9t=enA0v<6k!CUs1t zTOMV{ES!XBm_~lOThQ-0g`g!3+(CM392@uB>T2}Wc4l&rKsO()*;E$L%_T(~ifr1X z7rZM`X`#w$gmZ*Ekc;IE8)mY9@UtU@c*S|ZHkl?C46{;7u{?K-Y(cSS$tO4DOoBTl z8a*7HFJ0|FN?Xc6cB)mJm3-8{@WQiPiZx|W4HS8k?21}j9U#;8&FU&#>$E@1%wf{SL)LuL*u@QW zf@EqF99K=5_$f$vOpFT{!~xRf?iJ@DZPV5&*YVR^KI&IhkZCvgrakQWXS}hS(CMy- z`6F~j>Iit3Qs-t`+a=< zDeB@VMrt=ANd5OS=TY#k*FLvTGI^BzSTz9x= zBH7|IHIq&==t!mbd)$L2O8ae&I~1E$o%O5EK;sW@lbbXb&GCXd>ud4S&HbKWlEMNQ zsuyV(Ev#(;s99*$remXTx+61`The@mBflB-KIY@&9<@jHF(@SZ3e zqg42R+sYf5M(FfETBwP;to)jlKtEPU+p^xF=o zEZ5Ov$q6*IhNR4u+3@te&QY@<;W@a|7wGVp>|ArxIx<08&DrHfooha)wsxhS^~h|Z zXME1*n_*!ouf*tCN8Vox5I4Bb?@f;{PVm%H{2B$4u;>~t>4#aO_bM$EN5;y;$}L2s zPd_*RdWs`{MS{5ci^<#WUtuT5DD}YUt(swN_3|~>+~hmS)3R@-Evkkc)CY6RH709l z|Fyr2%mH4{hQ7Pm=AzyxF3qiI>Uik9P@AfEU1}Nn zJ(|o1f)#3cXZq;s;uo5p{gU;5&mwndK*C%=aF%sR z{+CmHiKU325duj4r>OTO_jivN$>5RYeZ#1PzJ8ZIZi8jS1T}x#3rd_lpKDY&Lt1D5 zb(r6l6ze&Yd^{GWM}S#aR39#Exxb;6iR=?hFS8U}w0a>mmRWCz`eI$5-C#+{c${$5 z;^c6=tSY4`6!;8Ek^13oXBAjColD_ue{y}T$+_E=*mv}kZWUQo*hNpAE2;7k);+j2>m6bAx4C=OM&B+2!jdxCHV=*ikNIMkKv zKoDqQSv`n{FX}%|oypTj<(|m--q*G3U>vlxvZo@mHvcD43E@`5VwCcFWO*Xc^(yUM z(U%`9T$5GrFWH?Hm##cr@h?0n51W8+W%G75VEnX71(;1J9`}LS*{J%X(YM=zW^%EL z^49&+>J7RPcm$GS!w1TXma2-04CbNlOIwucRu&ylN{p8%gG+TPti5Ql`WwvBv}45k zY?fwt*18LY{7cneeU7;;hq$B`k^N_^4CF;s&xZ(i|BB@J4Bq^e7#D7yJTlNx(jRCh zp*HS(lTO=8b-Lo1FUP)LSZ}x~bMZ!!H7~ArW!q0bwRLK?I#ua4?% zyA7{t_qb`GfG)F!Gop24UH`E9#U)z6n1^;Z43Z@8_*E;yRFBY>iF5lVbYOUa@9vD= zmR)Sx-sV5mq#&J+$fun0eAkX74I0j98H}vE7h~1<%u4{e%N!IU*jr@R$M1jo%wZD^ zrqWB(hd=&?{4$^FLpTp3Ks3ml{n~VJg>W-rsr~#2Cd>zTpLEP1*S$k&_`~kz$oKcz z=)fFo`#$b|nR9_Riwo`>hhn0l8wKaZym%mYZ6DO;Kx0VGStI!Wm~{8wQw@~Y5kIjy z&YCxv$Iut@ma5lWFZQ6E>(Qv1i~WYu6N&@0YXj9oZg}K;ycjzSyU1IY;;U9Ur*s?Z ze7DbLa4g%ZgkbOxtBFz z!VPDL2Hi{K_NJ9H5cv+*=%WUUVCB`{39Xcek5`qvs(HCRfH~yXiCR-GqVL~!SP!gP zib18g-6zVxa8%@)cqg?NUVrjXQGwGD4sN!6w7)8DIgQ7NUua21dcnIRPv3w}5$+vo zdf(*fGnrtEJ&SJoIv(o%#lB4L?o7*`8@DguA-P!%3mfe#h2fhA zo(E;{f+Mk{1G{M;8ut-bXtnl9HUm!(;`h65-9^z_+?kc`1a@Ze}Zk4DZ@fK2Hg7nv(d@a*&m6ncBT{I{817LVO9f042J#!0mM zs{WnWC9ba6el>0LJG#Bx2BO4s&K^aCVt%wY z?6T)Ne?pJzbMj=So)YJV^OaDYzFSLdj5sG04^@-f<~S)4n|+qDJ%jC@A@y4Z$BZ3m zUo96$ObQn7yo>yULH;dId5ZP3>qgv$nJNXz=Ksgb>k-c|`oPA>b7FC8vfV=k*IEzM z4Q}=hE7va3`j#Tldy5ZCm@4MLdvwUG zafERVPBLMW60Gv0=gz~Q5MX`YHX=AeJE_VjhXt5==|8MVy*WtL$Pmaty2)U zZ}2-IxbUQo8Rh2E6FNWDuFUk+hHke=x@zoi7-Y0oWmEMBwaZ8~%*if7Tsz8!EJEr> z0sZg?%;-SE|4Zt5<(kc%_?H=stokG=JbH$!EQyhff(sHwd7`U}YeBXdj-Rs9yR{A~ zF7$@^vP2bo4=sCQ_y49fYaGKXe!!8|AmB-8thSYZb-w3Rv1DlK>t^>Ko1XiIPez^J zU|EQ1{DDAPDC~Xo3Sp9I;-f{?!z8iV80W5cb?e9!!Uu|mGMmnFOyuwoi%zCVy-@9k z*Mz|(ExUpjD@vTc&F76#68|jS*vC6yE`;w#-fM|F3sq~Qx857|y`}o*8(=OfslFEY`vKOR2;CGgvTMrcr%%S zZ`3^SzdZ-kl_P)BN~;aBzx$I-kfNG)!ctQFmvuPQNA%D%XY0A!FHS)GWPQzNLCLOL ztyy!5hRHQ6ICd*=4bzTU{rQ^zitw&K`Yu;5P&cza9FkYAt=S4c45LD7T8#ep{~+j# zug}BpK3j36I^>f(8wk^nv@)+oeyGyYC&+>|IYB<$dOvI&&ubD_SBy*%G1tAb)hE;c z4gMfoO~8Xp8P^Bc&o1@8^2vQR5*o|$(of~=P9Vf;&)nKmc=;J8<{>=(UEz|=K*4e3 z*v)1=SYMjr3w?RE-S)H4Lwtl1g%`V=c}oX;t<|M3HDu8;FYC-)=auJC!rF7#YP(IO z!D6ehistTr9#joOqrKBD?8Zq)Tm8qY-|O#ByGIzax*R!qqqJM zQ*f*uAx{@357Bd|EXw6IYf*AJD?{fkk;R_qEdJ>nkKole#+3*6m8V`Zh_V zleL`H4D3vDMdbi-ws>o165(ULvzXkN^Fp*zftS|UEv^Io6khT_PtNHMTnB|YF4?;+ zcI1<;u6(7}D9%@#RC2}()jGYX$Lcz-rpiu6_*=K*8aDJWB6RCR&B^k;gGXPpZk~T-30Pc~TSg%&eSO^seN^ZJk?o(}80mey z{G$}aH3r}yg~z#1@0Bekf01a9em9z_zsB4_xBNnHGgWw1F4Oh-a>Z&?0-MC5b$oI~ zB}c8yc~lnf(q6m6nE=fqmy9$xY2vR@FK`w^OxhC~sqwh!+r({idF{A1=i+j`3Wt9m z0yA&0Y%fU9H7!OopU{goq`_++2nTqPI}!|9ja4s^`y+d-j&Z7{<`Gpc=UnB^q1zPa zhC6|?@b=S;WSPo8hjG3n#~G@9{k=-X{yTFTU7MegF4Fo_p|FaS*9(|@a|3U?E5ERC zKG8|Q$D|?pGwePsT%X?tBU_A^gaT_@At_>+7rgg(f2qVDb0(_3buN-%ZgPAq>5!Zk53fFdImP+*U5J6 z^Y4aWU)W(09O?POT+O@U$80Eccq?9z~p{ zA55znV6?=(p*S!2Zo9qX52dlFjUBQ{p1As#X4&TgC#HI9Wx9WDMS}e2vcQUv#qH27 zB#~VFx}2&KGH;5fkS*Usg?CZKgB7Lr-NT2vS&RV3Hb-BRzrxTGc2ao3Rc^gH)vDXP zqc9%R#^RZ?+_d?>h%KMTH)jy@rkcbSYtg3wC+5C}dD~iob4DvJOUfsqjj&Ig{7)K^ z+Z}6cBsS60xAJYul73G5$S3~Qw`p($;`*a{WV;V0eYh(PSB8z}y2Ac||Hcu~Fe$E= z+qVj?;gRqMHG+TnPT>5f^cfiW_&DVM*m?`7rrY;_oKiqRP<*6ADG}Mo5duR|5XL}4 zL8U`L7>!7bcvKMS93`lfNOwp~>6VRhlyoC85C)9@dl-+O@9%#OXJ3ypJf0yo_Q%)H2lcX^kr?e_`=62eplz}vJ_)B=B=nlzpoPD3F)3SB|1`~a~3`t zGYv~nTzy_7H$GS_8ct2gYSzB+{gv|2Um^3{4JB8mcA=5x5HZ%nfg{-`14DuZuCJFv z2l)_5yy?(;39yiwKCSxqS}X-#%BG1$2OD_^ewkfUcyYW2fLRB(z_ zGU07twIf#|V#+-6imG_lEkBR~GxJskQ8HyN!t7__dkh9^5kznO@Q3cpon_}Pwga{t zOA?vI;GGca7DZOtOYWEJMY^ZwzE}^={@KSGYCQFZQz0xLeeY*f$IaZin$Wuu2jx_b zgUKevRJ6mf=6lG)52ZpT`f__uki;D9Up4|x-IH7X?m={m?_*@emwAw!k>Ipe&#eO7 zz&(c5N$zqPC;WAlu!CnzCj*oVvvIi6ImyDHK!uNnsKs*DM!4^TzSzO@6Y<$v4?e&Q zK!XZyDIJf%yzc5D8`Uxx*Nsf);)9Ea&KhC9OD`wuiHNOMLjQ!&)RVi`1zg)BKT}-O zuT%x@`F8XcWV_T2+aqlZ$k8*w68?$c)sR#CtR9v0?9&Kz`X3V828Ssh53GA?S z_XaDSdWnC>UN8)Ma{2k`|E7lo(9^CrpGIm^96jK01=A7FFfc5GiD3g^L@F%&Zm(g- z>VsBH59oCkUed9Dr2o77wqXIsPfY1g7c{5kFI!iCLDMerS&2&YbsiUoa7c&EMt_d3 z`<5l+`!xDT2uo`UU8Zjj25KCZDyoRSR4nYS-dq-~hzG>1y_>i8`2XRydWd7gyA`(h zUIFcZThydmX5BJXAADeM#Dq9s0k_cSqjJ+7IS9pJ$gS^J91FP02LM|DhBy;wKn7@m zr8U@}l5ng$X=aJXt|(3_)?D$9k+(Gl!lfb=idwzVndVC|I%ir;!WXm}tIMQH+|vHI zPhkIy8ben@M(4OMwm2dh*bY`_{N8Pu6I$qTcaO;rnyc6KPPMTCX_ru3p) zCblA4E~x|D9udBotI%{S8_$9wz}ztMV)A;#EXAf8oudl(>{$fclWP!u{nUuLxdYB}w)~_UfU&uxx@X|8l1Lxnnm(85JmbDM*sP`qn zQ=@%=zdX-6zHI{WpMNE%F_mYgUofz122=GoJ2XL&uVr1c6c;F6&FLr>gXk#k>;IQ~ zNHV?j7V3VW`LhO3Ys&6YGy`9^-KgHtp)j=2HL>3Kl;XXp9c}y}eKX3{5RXcgUhaJa5P~t9*8(grWVaiQo;{cLv zRm@(t!o4qt^_Z|RcXrP%stanRl{5YA&FWrX$2QZcwoG=k(S{?7D|H# zdDZXUrsJ>v58et*H3xeGA+!7porlltO}Tc)-&7i^ngNaUC=9x<-*^Yl7D1wr&PJVl z=*A#Rsxlm1a9*D z6MXYSpG!qBHOY*=?E-!vWS2~NxMMUbial$7j%D#fLty>$yLs90vIu1; zXIBcyS18h*#Tz&k|HohRp=-xY$WrF3opbNq^l#ow0AH_)lEsBm!7#3J!gBO$>RJ?r z3?nS`n`!U*?uat4edWvjlkHJa)ZUKnAqErFJL-WdI+E$_k zcJ>kCguc~W!NbdUtu@uP+-GNSbiJ+|9KY`>ByEwj*UE4)fd|~%Z^PI&gbPkiPd6e^ zOO!_t+b-X6>H5L-Vd(#q_}(Owx$aUDpGJ?Qi_Hh_(2A6H{#BWdn3O;Lg6R zqC>gX!%v*^+iW7c^FYk>R2%c5ROy=fo`2#wDW-3-xoO+CcJ@kuY0T?g^Ro$|#99@j z&-Y!~fv4NT_okef;2x?C=@mAN?VAvU*C4l0ORxy9m0vEz)E>wd#ZsmtLx6(GQ7dR? zcTZ4Kme44t+2S9qUip9K_797Y`;j|U)*2-Pj|%+-%#lP#_0?_uPK~8*iu5A0JF65V zeydA+s_gA~dWsRclYOKoPYR>}Ux99mV7|(6z=6fhDam(Qtp{zS{j5r$H)(A|1gc}X zHbMtnA%d{!1xMZcnN~CS%av}2p})ZIXmOf#Yz+rV>~qYt{!|-|!9t0+0qcD(yAxPZ z{73;%!PDM)O5DXa7o{ zAF+tAopFRY!bSwX8_YUj1vYVs=mJ{PTB+w2>r)Hj=k;nVyV9#J;-R2xb4Y0`amUvoE9qUr~nG48OYuK|JpKShiEK z{X%jSf_&UrSZtTPX+`t0|C7p#16wbR+-vjDQa-fj6{(~z`X|!0--J@8uhjHBqWv89 z^PWvmAfIE~$VSo@wr?EF@JECa58heBt%s05SDo<6e=s{*w>&+=cyOMh>TV zs;v()XT={sOp&J_;YdjC0mC>#2&KF}2dxiE}3}6psHPaN0q+c8k+nj&tN84K_ zBwO=ait}gqy>Q8ihcXG&d=Xn$EzZ|%);Tr0dqCIOpV^(PG}3b8A)7_^8p?+6HiudB ztQ+3EA>ZdcBBmZbj#viCPU}`W#t_=ER(zK&K38mxU;<5{Pxkodhva^GkX0)D_Q`sD zkRM{pF*WEQqcD_MlL}ugO9ATWKV1xvsCQA$DKdXCyxJkq&~r;eP;+uY-|)(a8xC_> zFrL3*C$Y^ASYwd`1uoe#|Ahrdr1NV!N|chnbBN#bX&Lt=y)jF|g)j;+-<16=3);Lf zmA?Y%fWwPUXo^oU3x~3kYXDUeEk$cnicc!GRz}!rn*{bF>us2TBQMNx*w5XF33eZW zLMh~Rt!Ub$E<(lWdkeS6M6M)C#4g;yvf&561ikHTRuuc1UH>HF%%BY~T7uI8TKDZ+ zF1?*=ZAw%hH|DRqKK_Q%mREab*|j0fx!1T}HjI`LeD;b+mawLH#S2p3B*2VD)S;O8 z%1)GyTb^9?`|kkrs#^OnY(?>Uen@k7i+mkWQ;Ax3HVvmpqfmW#rN(Ws=1p= zEiR9l_G86QS|ZgAhKk+hF5B`#ox7X?09H+KsAHbOfa<}6ke*@&hnW9e>MJrkteP*y zQ+-Z11=2?XaT-h5wiHjSHV?btCJfS?A2pUjpz>fn1t|}K=iBgI_KQUpzR0TiSq?X% zVyR3r1lqf)<;@aVTjDPY4=xV1J)b{!(vR?&%B=48gONRsQm9j-8e~9f9$!&Wp6R~r zom@K-TL-TjnK3d?u3UA*Zm&K4>01vkp|>)Y6H9Jh$|Em3^RlE}iyOB2^43*V&maxm zy(?H3PAYtF%ZBx{`>l&<1MA?~QEZPJeTTuNXG%!^NX6ei>gTgW*PoTyPYmSz;UyFj z%xU55PFkNR8+m9Gf$h~S9o07XyU^(jCY5tL!-@xs&la;({w2KPaS2OaYQ#PXCjI9x z5o8vUU=2%r>(e@wR*C1#w+OhjG@+3+tL{X;N^l>40=k4&w7@M)WF#XZ&Li1H)&j9N zf~n~K*gk+6Osyy@sL*^}KU3l8!u1+xNSpdlgbEv3bnfP}6rAkV*Hxnj%7$F(s#-pS)_%`-`@7_xRYIJ0Y zHsALz38zx+AA?H5sR(TJ;su>7cLD>%FKO`C_@1G29h|h`{bXiPbDm`9g-dAIS>&Xn znu*%Nl73S7hRjJn7UIUcrsIRv!uPHGQ-Yh(j^a7fiSo`ivz|p)2lgSXVx{9Ea-wy^ zbVPAE1?KQ@JY_3;Kh`#(BHKARFwW7Q6&S;AZirv*^aQH?H;0gM*Ljvr=~jj(j9#VZ z=Z(qE7|fgFS^RDo*#0V8{@US1ZWMVVXA*Y5dyz9Uv%`WGc7BF5Dg5Zt=V9kL*oBNh zQm49Jh}lmkPR^?^ckj6E#<9@45M?Y6SQ~de6ijTs$LSs~-G@Rx8P88{%)w}!p35Al zukdjjThHISA?4!qJJB$ur>YG!ZoKct9VqanBCRi8<=R||;(SH->#^b0>CudN!NByi ziVTM(w{pR?_L9mr0!yP(e$0JdRnFNMy(CLEEtxE-y>#RL;Emn%5U)+%0biEV3D$=5 zw+%EsX{7gDZMF5S?*Abak=zP;m?|Mi)0|A^NvmP~@##Y!(6+VZWn{`1xt#o0EFAv6 z0G=IPvvZfaECn1f4>&F8&QrlX-7qQx+EN`gltK zVb(RbObrxU4SxqGzd{tLb2bufwy=l1r+>6lsnf)+Y_uByK_?@0zqBtd_6700*NPza zkt_~vEErV(>ZP7Z<`7>p;R~dvscb=O`K4`+$#%=GAu+``Daa|<_$o9$ig5P=!S=yb zUV?5NOE^$`&hfilhbh58BwqjpD)qvgvG6|)m;`C^xii$heG=beW(I!-4Jl(}=INPk zrTlCsr%=vsrv3>}rk)#xlNlpa{@rex`amz<6GKn+(enX-1vTZOVj043Q_Li#(T8@6 z?>6ohx=DC)S$E|Byhs0j9X=5d?Jg#lCv^XzOIvYE%X?5q^k%HzV^%+i0m7_MLW=I{ zSoS?uSakK)K!x9zc*vs$i)+aT{oGj0&I|@qC7CClG!+NB{~^dX$05hqHnF(&siK9v zRaAUSvz)o{R64IoQmU}<+$V1LSZIq{L+(L>G|FW03OH^gD|<-KqC+V%taPDw);Wc=N$BAI#p);h>{ldu3crf=nBzSTb~gy8Pui{J?*BjI z90Wu~$XPp@?=5hD=ldFDg{VK@T2HPjc_z=KC=h(&!-P@bW!U1@IUMVK)uVH`H3%p~ zY4NF4oH1>8bVc049sLieKnrgiCTq~{@k3o7doWQzq1n#x#uZTfTbaf3IM%I9ua3ga zcOYkVeR*eXX*kMQw-ys+BsY@PT9b5OCnGO`jESC(SEhy|Szi1?D6!wEC^rSE$=tAuWt;j`{bkRz z@haQt_*C9FtK)9D5V^5a!T^v`R{tJ>#uTY4G>mKH{ zThU!^ub+(OK;Aw&o3t(CokaL*dKl`|KE4%ASgS|^ZijEakeyjVfRC71pSaR3F ztVi&m3r4dZvnI-f3c8AJC`7qsC|axnmMvVD!EI#amWk>w&N<`GEt&(`Q^#b*$)x?d zPV)kka?Y}~1Zc!Y$|_$^{g}~<6fa+P6%bfowBTV0JK-_*h!jkSnoxxDv+QqVLT7NT zhJ&sw2OFiZHX-K3t-33t;(vuyzfQ6&qiOdmXO19_<)^Ha>L7%OmUp>ep6|s06V@9Y zqM}Q~znjvoe0NHE4c|$$)>p#?L{A3f%okEai0sp4H5xy!RwVzMqlFd1l#b+i%V)xY zRx)jTW=p}J4SUiJ%2;W{v!mV$`1X*SHTPzM9RMgRT87GUC1}R#Q_^c=uC{N5%Za6r zF03X0Pw;T;+%{>8{94&i%Haw+tN`L5dy;=G4)bW|ip-3`5Y`&_r}9D}PM|OIJLm`A zT;@loU>m6Ks&{(*Iv%1t0g8eDiZg0thMq)KZAkHS)LdpkKjWP*%gG8W(<_-I6;q3@ zQrHuhh5cYi+Al#e)BPLMtr5mAccH@jUrG!9-#-jPV{;S^9_uV-=CM%Pp;!kyJC@mk zb&u9tp@epsFGu+7(*HqMeYxrHaXW^EY{}~GP)3T|V%F=+?a6Z&H9Yw}Kf3AND+q^A zB@wE#N+63NTgNM}XZoVT%&rJOlP=9lJ$I0?`f$+n;IJ|sF1{ci64)@hao~< zSJanhYCn4D_3*(%?+T&H+#VpEk}Ox7zpis=(-C5HVa?Y<;rBzbT?1JryFbZ z&cdh9k4tj93m10ie54T;snA}OW$nB(&VuUHeefjNxF8%;l?9OTCTVDa?Jt~wK`91 ze?9!48OqPsB6|w)Q}RH6=-OvRlb*{?Idyt|3%YJAY9GFfpW?GW4#$7H)HJFy2RgRH zOcO1oB8cI8Kp`(7&_9$q`7v0{5o-&xt7&lAsID?hPETD5cb?HsaNE0kkImtgYGIDg zteVx;Yjdh#I4=lc(tlB*Mp7gmKxL%i3Z1T0s*#-f@GWA2oDP@nbE~qJM|XPqs6!E= zMSVk5{8lWL&wb|zkeeJx>ll$Sn{@lN)2x)A@3TAZCa)!4#Lb9nm?EfHT!DpZ?VmIL zXc3D3q7@{)H00Yd_WxjHMFWayiv}MtNUl9Ascao0|LPIx{0;Mib4u}50>)#9&x-N# zqJsL_lK+Yq0q*DCAIUzaeb-=E+QYPBOiJ(FL@s^Ac#MrJZcCLbu-&``waT7U+!ldL zBg1bR^ZOD;GMGEVN8~o16>~KbI$Hs(&c$Pzy1?Z?s*RS(&Nb%7OWaW4u4y;6M?| ziSudw7U68k&B4CmGaK16~-*RJZq0BJmIw!mRtGN9f9tVMfM<`J$<;3!%Tg> z(x*(V4229vjZ(X?G~UwJ;UefRZ0GAHue3&R7n(qvzN-#Z)+R{Le}8D4<68h~{nNN0 zo&H}bh7N6!;t+H(he7;QKU4{J1(_3#yuV{EGc(*fK;(;7FnN*rKX@;o64+(I5Z|X) z`8$puR5j(Mn|c?(V;!mJdNW<=pvmlc^pf-occ~`)f?%Cwim))s#7kLC>?OP72UIr% zQFW_PX=-O+(Fd~KAY3PuHr>k4V*xF2g>+ZPac`f|Zi zmkghcvk}lIv@sE&*Q9z%G+|`3GSfY7EWT1CKX#%J0VBGwhFF&mkBj%mZyg#tzN0R>_eB6T8R z5Jb%Q@p{H(=f<+&7CV)>_L%)Dv3<2`fma%GQxuUEf87%=$?Wd$jLW>UI1*>0ddw}c zbhWQHMg6EMrNU&tW)XVz`-tN0j%Ks}Jaj@5nQIj#arahNfA;6?KXun4?F@JZ2R-YF z$COvpk=I30j_<4RFU;AvKBU4s%@hTvm7M$i=I0=j=d(RcgG1%|w32CCs|5T^#^9bZ z6Am;*)tvOiD&`-boROs);PeJNlYQL&7sL-h71Q()6+RE~_@2I(t|HL4uRptxG<5CI z?*TX%v1=cLQt6gGN*Nh!F34TYXMWCfoB~DAlp#g#KsjsU3r{SSm3(i6`ncR|#XUpb zK8z{Z;WGnTK>wXeQN;iPr`TyP$5RmuBT|31wY{oR^5aFnkq`YgJTJA+vFkKVTGyBE zpjhw`^)4CV)y4`NA0P8-Mu*9Sl3CFt%q8j?ATqdFHUKwwm=w}Z#W8Ovbpr7ydu?=kTP=eJJMm| z9ASK*CCj^IWB-=FC#5%Pk;uRAzVfz>hp{@#@7)aXjTT4Wr)PC-{i2abwf_OPnbC6W zHV+gj83DzYUx!qZN*yTf+F(BkBts^KGahWIfRJ_)XDqQIMC*bBSI@7q^Yq#*!MZtV z*D~}xMoKE{*sC4B`2oEm+`3~L9nrKEb!dr4IgC4Y4(^ex{O8Lwn5s<0_ObV`2gF9} zwkp+4ax&8o?kn#vC`-~b_12%>Kpfh^k|~piPvVkBVrK5Y_0x$@8S&*iEc&k?8j#6o z+B2I*X58`B<=UUmsJO&K`6=s$SeKzoDMQ6wup#kN-GikeG<+v+EHqN3H9Vev@+ zM4Y}ylj352eqrWeA5h>r4)6@P$j47`bs+)(bpic;KF>)=mPLsV359%jJ^Fi7FjINj zZiFvBePGqmZ+~#90pOb4BcWo#jZf7t!W};V6WjMK{ZzhpJNml1M|h7;r)c?B-urg* z_bCi`ea=)lr@dGy{yGnE<3@4V8OvMjw)NI!^#le8MR^8|&>2(*is7-vxy$$M7%h`c z7PYByVv6%atZLZvU^O|Dy_mkVhD|~7{|TV}90^^<$i9)u$w}?)m7yY!Mnz4@Gou5F zJ=)4ozV&h0*9d;KWO?uBGj<(o_=0vtDgd+bh~CR?fs)xy4O-(?gMlr5y~S$eZl7Gg z?E9n~1z@O@)PYXa%+zG_km0BW_x>=Uw721hcZ2lf7$ShNFDLYIANDPMtY7~40S2nh zr~z#7pblW%6o@MM;IOIp>VPTakMob-g@8Vgrf1))tl!QT?jVp5?MuEtKAIpHBOe_A zQnLn!dud0W+i$29F2h}b(Rn|2s3|#Bg0Srtz4F_I1Q-xc?nT+X&KMCDH74%_c8EDd zXC#!ebtQ3)zFlnjm6a>lVx|?@1?89wBDW)Z8p*}5+2Vx%p%#KD`pd!pxB1aj1V&Tn zjiXwYt!53*1-TkePC}s&E!bB6A-)|#~tNpAwLkK$4PRHOC%vycR&KS zj%ZHJDR>aT;PZws=y*6t;8`O<;DNc~b-6WPyB+S0Eoyqyv<&Sf9E*%p63vUK`gQ+l;TwBW`dX71sbNfF`IPYaR_B34xBgfnJw{ zDE&#&2Lp{Q(Uzv{}lTq z%#)m5*By@Jh^)ve9yco2pMQD%QuLF9W|M6( zzZ$s6={)`}KbsW=L9Ja^Ivro|7jyZKspD0Y_Ae1$ z2IIKtDXe(f>SrEs?&sooLy(pxyu=jIeN}Nfd}`}J#uU-b1e%HOX4FPaM`Betg~CA8|1#1Kplha(ORUVP#tlGMM;wJ%OC zh>H85DY@>TIy*m+2BvSryq20S?g^RK@BWBJ)n*VoCJKp5g@d*eJBW#UFrp8jkC9^{ zE*_P^slm9XKA8)sb}YrD9atKzk;qK!?3*#4es67HccWEwW6YxI$kWYud?7T0Woo94 z=+!mKg5X{E1|!x4#L&{DtHh6RpvHw;D;*hdM?aw5vEbVG;TrS!?e7078n8tBhmHcl zd`ZK3hH-aLpQz|s;@p8)xZ;#Khwti8$^6+cDq7c>&HgBpqnLR8=nGr+4~P(}3{Q*X z|G?n}XUJ0*OFgdzej42eRJL(#XP%Qb>jy2LDW>v`F+k{Kdg-*u?v2A+DWv)C<)65i zM2w$K?b@39&i&{#A-WvKG58XG>~2gC*copnEs?bBx8JKOd?18P?1n7?oG{1yLMRlk z4AncFH6dCl*zPw16s4s~zx~Qwk_L-<0YA75)Nru51n8^e)^D&k6cf+!(30lZ`NaAH z-J10F^_WzA#*#8O;g0+KwTWuYp)?s;y_CQ5q8XYt|6sp6w>jBe7WFA! zHGmE}PB1z{agRTF%5pcW?l%6bw>mu~ZOq`^R01wD+pRroN61KgDW{;AfB%g5shQW{ zeeJ2hvcIAjAzD#=*>BBZL_4#N7SoqMQs(8ey+)_G^ra~q>!fQZ>HjgIh{eJL_`*<) z!)v5lej5ut#11qwn;0THD}7+w&0ybRqU|^R+NpoGnJM>NOdbRK{cNLRig8A+hT0+O zvmr}6;b0sX#ZJ9KQGA8nZ z*e+fflzmwTkaaxhu`t1>s1|rb-fm|CAXNjuM5?Me5BD+G*Ry9@P=-FC!hQGWSe+csio^V_(&&vDCYwE}& zbt*Q1XYsGIBnk_Wx!@c}_u$GVcHlXTK z%MP};q&$;y}b|iyA5RKo*J0*8N6$(i(JazPN_2a6Px{fGtKygplbKVFd-?9 z>+n^~O7ft7Lq1mrZvK#o0DXZaS`d2M7su1UvCLzemW!(Rai}vkvsx-sf?f?~h<61RWy=lLPO;($u?tpBOopvV4ps zvGGtsI`qS`dV|taz3SuU?9UpR=vnePMR7WyLmRzY?(oI75pktascNay_jY)B38*el zmbvRlj9_tQwYyixWl$nuF74)=qItfFm*iq{@&5$D~T(YR} zl_c*rD30$e8_wY@x;#8cR#aZ`?@a+umd#tOLtE#A#JIvy*v(e?+KjnuA9DY*X&ix` zwken%fIQ|rld=>Ud*dpGadN^bj+8Pnw5adl0$L-fv%>A7-{pdj*}XJrEqwCo~uR05(i zes!c*;*}g$x|_rLJ{a{&RLa%NQ6UzWV996ZbA;>Q*=6Bk1wmyR_6v_Tn9#P35c+xI zLq6t8+r632xFG)7x;6Jg+?zO!^?HX3%U1s5 za*o37FsU5kgGRh`B}oZ0wS|Q-e7Bj2bk^)|_d!>3CWjO6Sh>Z;R3OwQ ztGDwHS7)Q!G)D(@vo)nE{L{EA{ug1C3~E}{GZ)EuLrFhSZBtcQ1RTjT?l4Z};;Q-}rA4?v$as-oeCcLGN4RregKf=x z`JFO}fK}RE_~;Iu(Z7EkSoB@0B*=W3Q1;Q_T^Z0x6h&l(ovZet_DFf+Nou5ssDr(l zV^}mjKOqdD*WxQ%d~37-pV@g)bf@Lw^fimLaLK$~y*a%3s0+=<%wKM`h@B#jefoO7 zSf2Z=f%C`3%wM{ZmoM^ALUz*LSsVsAzu8$`FLkVJuZfg0bQ9w^TxfE_S7yd7HA6Ch zCaA$9e72XoaP#f`DpB}hp7yM{XJ1-8C-$AmtU(g`Je4FgW6iB6p`}|SZwCW9Hu>d? zB&_(SMuw?#hQASK$D!WvbAL!w{BE-nR%o)2;D?I5BkEoEStQX zJx%(K=>)9}RP`L!Cak4uLBGZllrIcI)x$l-z*ZLP@od!RtUU&K)K3OP1?@;!ISIAb z3;1GIcToOwFBosnBV9vACQ4^gf$HB1FL{e4JK5TFqDR_)7ryC9DoeT{E0*_1!vlez z4dCI1YA$DQukt}0I8nb=(t zE&D{-8xLu`j**rcg(#M+MSA5OTS7{K=VGemwLSH^$#_dtXGFz-KA%OJDNY!fg1|fD8f@<)_G*?cjyW{Y z*nAgRmFj)8%9Q2ORel#2E*q3MKeGcFc(itF&>h|q(A2trcW>^h6m>?}rfOnHZ*x?( zR;}Bz20-Zix!pymo>t5qt-j@tiTQ7Fq&;zp0?zJ=bHZd95`fKjFM}6gOgan~e@Y-e zcTzArs=1y)fDQ_<&YCZ%Dn#q1SP%V7Ba1jFELaQH53o@ZjCyP(!bWnPZl(DiKxXhZ3!kY#|r zkc8HuS>Wogl?CQ>Buy>4CN-$(Z@?yJo&W}*(^BfSQg7k5!e`sUCky1oOp+T!@*Ofr z2qnvp*{%Y^{oO?enq#^V4 zHtv6#FlY>o>98QBEMKy{{e|}8vu;yOv?uIx!em%aoHLF#<2g+4=3tTS5X&AC$jE?v zz8&*6tzk{j{8#C3|E3Z+WKdp%1Yk><_ZRoc36&LecDDmwD|F<%Lv1x1>)_2a>O-$=$ zknwZ1r56StU{CF{>!sC}B5Z9F$|MW;x9lThL__fs%9o)0PdX>mAdk5z&WgOW4%RrN>tqH>?}T7z!b+TO8vY z&I;!b)g6Ufe|M`z{~=G+g^;DZsweSCe_9=QeNWp~q^a`cHl$?k+wjcfOX%435u0?# zSU=II9HmxEcW7BVROx^?*WzREc1!sNLBr_3>~8rY`t)Z-5HeA1+|8GeY006`EgT$^wF2&a zcxGo6V(&-aT-8sw$nTXgk|S~0^{Laao0y-vUW25&`ko7;+~YPHjT$$aZ}pzrO^ z7BYh;%C0%MXzL&BnL^lpdnPnh)oZ$CMC=6RoSO&-eOp~ob z3RyyDfm#qIty#+|Glynu-)W%Ds?&vpc_#)RVr`LHgrVEucbGlwh_{X|0r zIKrj&IGjbhQ6=bhv@ryKdf0xl)lFm^-VKNyKQKKo=jny8I4{VyN{Cr2RVCkxAG+Z! zTgl;4_L_LB5p7fm?!WL8N)&J?fOfsY_3rF+(!oNOZI_jKikn`+tURq#k5k*l`E*>P zQp+<>EjnH4Nl9a8b*(K%$00b6OH7*Krs~)85t_OW9{RjKjy8@%!T>7C?69qog`rxZ zY*dz-bpXrg3>~00(^A*@zJ5U#nUKgc3}rr(RV<*jN0 z&^NER_{RP2Rpp_#Q-f?hVRp8LAUcL-*sFoa$$G24z-alLaGav`*MYeLJV$a_gK}(C zyzHZ0G=sm`M$L)<#YVs#7wjF);Bk<<`baFcv%Hkgf$24T^UK^heI3Z`3_fW}ckF4t zWO=as!Y!jX>_6F0Zq_L}Kh{l3d)l0X#KiH{3BgzOWwpTsi~GB-wwUW5dRUm3U-`P3 zv-r`qimw3Z%waG^=m~KO3yXij`^9QzLz@0#1}ZN0A2v?Z$uI};Q`_^SMAtAaz|8Ct z#hd=V0NmXRr2eZIZK^Y1=JUsd8VOQNb+0H*~EjkuByM@jy<^#m!pt`B?IMI&rGO0{Z7hsmWoD-j^5tcS`ZC@YViB>9fvKz{*^>Yy#ql#O>mDuPhf zS?g>Lj@MRcz-7w3;16aFST+8+H5hmKLU^!c5!@+7>6@FoL(v%xQDrO&~h%VGETGytXlt`_PE0&V1v3?HazE{OnMJ zQ!I4C;VFglija`e^2-XT>l(^_x3dX%9b)0WOW)O()%97mwVZ*qy9FnIUL0Ne%GkJD zot=bHpIX%I?|gxr&~%v!iv*D*(JS%+(gT-I@pFtsY zHALl#jAq=;Yfn~&qvU=hhf?s%I@T~moVT8NizwUZN|aE|H(xm?VL;!!@}A<@kD5|b zN$y;7zmtDZjh7S|h*+#XxV8|V3q092M$@%RK_+H<{C!a-a@AGuc@>&7w=*`>CsZEi zEq;8W`_SM=M}$UWb(Nm#B!Vzm`ztgSy%4aQ*Hoxb0!hrWRfOJSH!7g*dQG;a5MF`Q z*`cfUh}>)~T5GQU;oh?vvh@T9fXx39%#rD#48mK~?Bqwu^}fQ&s$MbjRvt&>*0k#@O=_uOMspKxlPX|Ia9d`v zDa>8{vwPNOXi_ph>U{o|SjE+qmb?K|zDBW(GsVg7)7BW7H3ruF_64mUt;6sPhtOHf zQQm-OEV-ad&%EBy#nt!_!vR%D@NsPp$akG@P8;IWlS3lpjEIsAczz4c>IzvP>;Yyh zcb#?8p6Y#JVW^w>J|-R;8}#&Ju8GaYC8yi1hDoj$bGD)b0s3!pPb9+am-J~T9}ndP zzO6k>(I8dlO8G+sP)guQFLo=G*x|8eX{)=M{;BF-#NOOD>+x6E11UoCR0gk5UD?&< z^dMb>P5YHyKdFKidNdlp;l%m7umGZ03WYJ12`tV|>J`F+{Gz&M{s2-jCjJqAW@o zRD#kr#UiuhNvS~NxtG1E3)%534qL5~<>6H<6Rm~fBt~i0#W&VsC5vyY)I)rmXD$0- zhtF3BVM=!A=L(ds4f<9ui!cwV>hA?P6n6fMaY-W!XS=B^Oa2u~qf`I>{#CQWEeh(-nX2;Cn<1H>%{AqH>-M8k zj~hNKrvoS%{P$fxjQyVPYamT7!h-6DiQKU*y^zE2tgY|4*bw^BKx10W6no}&bAdEkc1@@M*R)q(kb!+$T{et`wtq;8bL6gqjt7f8fxqcD)jD}i+iwWrjs$*o zn(6ZEYqNM8BT&7NW}Vr0<-h>hoGP*Th{v6VR-=OIKVD(V>dgpw;;n1YA&M=B?z)i! zZI6{F)yFa?EWqgX{eu(e;)NQ+U%Lfx1`9lIPPhc&WZU~}U_>$Io~!=O+EZX!n3m3L z*^BgW!i)NY=!Hpr834T~Oy52QWs5eua{-HWv8cLdILvL(!6OMV8afvZzIADS$>yOw zHxj_G(n$mPM}Q&~gcT;qbz3K`BMJ()J1xvWBA@|7e4Y{|Iq{dXa)QLp_cN$yN2AkE^DrJRs9MRd@%gG9s|UaYGCQh{OBr_#1~z|hXk*jDrYS%D1itR>&uX988a_5RRJzm@mujlXpXuw{j)D~j((L-@i zxW1?PzgIV63iT>8YC)Xkr8hqdjFDm8gHlShc{uQKXrL53FlN-L&jIThV{>yeX|hZ5 z6;dRmYbJpd)G|?X7fBphv>a_lughl++u{@4j5SPNSIV*G5#4E~}D0R^rOWnh~npz&1rm3VtEkVj?=cu8ZTbgzsI zLa>S}xYy2jr{z&-oS5cx&Q6wIYRf&q7eQNuuL^OhO*uzj8F6u?h;mQUse0tJk^0Ja zEE}MA#(}L7b~RINj{(G(k= znatzWXxx8w2y``-fnHXi@AEn*DC!@v2hZnq3&X8Sa?^qan7^0NFntGFY*R~g3O-j~ zm5E0C#W-~dL%Smza=QuyRPVuwRc3sC1%^fuEEF!LksTocPxB!IXQy4QBF?s|j3i1} zFFo5d%UM=3Io-c4s&eRhS;A##FD3(tShP*rRQuqUWBjnXBU$~JCjpQ*y|Vf`cT;qI z*s^!u;sI%%I+9}w=%y_?IsC$6jkDEjDRdHubQtOkM?r*davG-LR0bx_+{=-DY9Ngy zl5KaXTzYfxvO}5JSViAuBV~5-u9!;FfJwc2sOSSBj@6)tl%sY+J4kUESTp* zcuH3sSOMddqX_~n4JOv77T&DP@rfhha)H53l8=;?h?oBXt0kozU{K+f4{O}`(m~F@ zvh#hIKMH3dt5B-LYT|PxUu)E0)xBB(pbAK((*01+9E^oq$pJNwZ-x|J0+Q{crI(hT zWTrh#x5(mSq+hN%Fj^ehd1e>%>~L8idn4`e?!oOA0*k=7Lmd^~eyH}1me#bvEyd5Z zf_>$v81)FxK3(IC&(Tde2arTquEcS4_>GE1;^ZF1H(lblqt$_`?1T2w-o7EFCz_uF z&nm~ymvm^};MUBn;RiCZFP(@^^{wp5EIFMz$vmIxrVF?PxZn+4Tji@lxs zSIwEB2bOs0tm26s5w1$Af}EVf+^Y4V7JoX!?TP#{F$cPD5!A8f2EKi~97Zc1IcWzA zk9p~#Dgi$2>oe^!?rbXlhFa16xtmtQ??rX2T4eU(?vdtLB%qCffp3Wiva}pi{OKfX zXDzy4{8N1OzRA#rez9#RW+%({Sg@1Rnw0v)r%%|w|KPtHze1f zi0PrHuM69YU3uTk*S28Xe1N=PHEkWkF$7T^f?ko{xZ@M+WN;}Y^Yt}c?gqd5!A4~2 z6fhPf4wNI=ZYXbyf|t#WGJMo1EEt6UB&wH^tRm%N4<| z<$J}*{7#YUy@`4l-sT(+TZ0YR-~bL#ce>M+^a6|}&C5@(LoMR&G5ZdrVfF);DY5~a zveO6QLqmB)$gE49nN7#G$J3|7)q5(5vZ^N27hkgs&8SLeeG&@XfMxIOlFTVZvyduK z(rE&?&@HWiu!Sk(`>uERz`V}zbYJAVrb2hzx&38GGMbS} z@=SBFFhMP15_H6faAC!s{UUQEr!HS66_V*G-<>AC#u~1}yr;6ZVYndtYyp&bZ8{{S z<&xUeY($9~_iID0X0;D_4?G!&p`Kwct%5^)!vzlYtQj;TK2~K@Re>3H%D%5v(B_4c zF_ibuPU&z4W%Dx%e%XX!!~t?T8}mpU0qcZAv8Vvkjpj7`n~UgB26UZo+J`not;6Oo zwA^EP5N5WLz4bhlI82ZsO$XozH{P(94L%NH-iUXU}XtUtW`M)7(6 zM?H!f6#w*YQLtYQ$LQLl|BtGx4vTX8x`3#plyoT)B1jC~cu|m+5JVb8KqQ8t8xfF} z?k*7+Lb??gnxTiTff*X<#_t`y_x|qpJoESuJ}~cl&e>6C z-&vP)c4liJ;W{RAup%$~2GTtO4p$xIxBamSW9+O}j8Ls0T1VWxfV1E`JhpX|ig!t{ z!7a35BkrnNvmdol=Hk4;C_yoqfQ4AXz9A5h`={K~wu5i=m+ZiB>qSC;G0e$~8qg~8 z7>d&&Z&kfEII{((=5uC|~)wl&D46=%x~^wNNenYbs@# z1#Q`}p7Huw)Ry>{a%GN$LRG#_;vO=dy{s)7yZywC=mluH-(WEmzX}47LyPSCizik# zUZnhIHVGg^qdN2rB}NLi3bq=uvPuxmi-(nAMXrZdH@VxK74a%qCj(R7^Vx z64f z!=X4Ge@-b9#E_%XTb`QG=6`F&I(2FK?*bp7Vy9x2qukDKYkZZGr1ay7PmjTDfwiqg zQGZ&0r%*hb;YhHfbuxcHstkyNZ$6F{1Of8aPg_T#t-Xj6PtEnL-L+PNzkiPOde2Gx$RsXs66WE!FH9kE6V@r+8}L2D|vJvzSK+A`kw&-49>sz@i;%%pj#xAkgW73rlnZftl%=RWL%#X6kosG>&yT4yaaVU6-VyY+&db8A^ z_i<@C&EYTU%7RKVuRG;ZL-~2oQ65za;?wuKI~8x!j#6gAj_?sTjgB}#q5UgSAZcGS zufekY$N%f6{BHWFESa{I%xUfm#($6U2`+1#Dz39+xr;%w*EV*F9P z-xz|hN~&wNALP@+H~Ahraw@ZZj*JF3#WANlZ5h1to!f!nTS<0DaPdROeC>!2!c4hW z);32j^0`XRZvKV_UN|iXm}OeA|K8XwKQU0`?;I?U%)~LZVMKI4exty{+AXzG=|D}x z5^7t)sCq4E{FScHjF5aw&h{!RglTc+tjH_(kKap=eCq@43GYJ_Wq`k_fKPWWKR}{{t z0K{BU$rn?aj;qTInI8pI_{+)rhtq0_PO}&CKS2h-1Y7QQUv(zl;kV1r^6GQw2~oY} z?=Q%V!<{y%ov;^q`TE5`VSb-Iekzry;?XBQ`RAbhQN)C%)H(^LJ8C1#Kq??tWfWps z&6$`G);FFH;zp~=BG^FZ^>&oOD@}d(ULcjH1=cAt&+}7Z6p=gyU8mAMS0xoh*wKew zb~HE!gp&F&2k~Fv(O2N2_oF@e)z!Kdwd?O=3&2`%vp}{BSZtK!|2kv9mqU`7w8S&@ z2MAHOGo!)C>JGM40CCmb$u(CEkgH@0waKB;X<9a;-QR4m+&11YyIQd(1ek5BjE|a& zh~w-LD)%D&8`K)!K&;i0L$MGt!7zVvUj~VSceR^_!v1b^wxbjS%{9@0q8_F`xXI4f zt)I8++l}r#^=QqLV#}0u zn^-$lQ?o)9P|J?%k8+Fc3jgJzCAE>`^Zkm(4kJ2E`nC2V4W+2Ol{?d90L=JQ-FmX| zX*0xtVO>jNhc<0xY!ah zIMYtwyV>c^$#!qT?BI9@%~oq1zwGoY(x$6O5t3r@hpZ7=b-)n{zWD1T2(=DO7R6Dv zk14+be*_bB310S=9e$g>IMOw){Y?nILr8?(3LM#Sb1Fw3FRRFV1 z`l(8v7kjW3`poD((Gty#s^rL4*0fO#qaJD78CG@cGc)$0`rscLHTjCX8*w$?{~?|O zAoRswerLBJlB|$Bw3QEyz$uAwan_7L+W>15mk~^ybrdjiH(^lNS$ww~mqsK3RhO}{ ze9jda#V>svZSz9y8qm0v`cW?$f1i_aRS{|4Fmn7XG#iE;LDIN-xWjL^0N zM2eU?0r}K~e7EJT1UXr>Q>gQk(I3nns@D81Pu0cWlfPdcDZ|x_Y$1ObAUiEsol6YT z4!`9eP6#46loCJEn~pjcX+})1Ic$UEJNLehWuk0~3k6l36E{xA!;op0(`(vw$~EgJ zwoym_<{nV?&jQQH`o8-}N~z$oL5oa4y4SDX*24;#2BWAE@aM<3+A_B#?~(!JHpZqc z`fK`&-k)<(Uxm^uze`Vv#J5hWxDV?*UYqT#Q@sRVX7d<4f`CVQ^5bFE@xuvJed<@% zqPlm)>yliAVZN#+zwjKG9XcvZo)dULp94Th<2VGqF5F25+;|YyHg8wHt{^}j;-AHA zCmVXB&qkgQ=CfSIpQ}nwboEJ&M2^~^f3$0{WbpDxrXrl&_qDxNhH!HMz!&cDFuOLq z(0ge5F^=whAm6(YwRAoq=V#-FK!9g)KQrf?((c~Xebkeii5jOq>z2U0Ks-tBbEg5? zTvOT`pp5Wpd;#YL!(Zok4pIUlw7uYM2;cSdcaot?YwFJN!rG~3V(*m0=|?gnoxOjE z&E20tRsNlM0`pn@0!j8i(?)&pd>BJh+a-Moq@fTuoRnSu!th{J8X(#V2ui-kp*zXx zkBANrW}@hlpjT#oQ0KD+WzRY&N)pXbkXh$9SY{SihelkG7D~V70ZyfSlZ?yMJE@C9 zhPXW9ePz`0^S;iLkYd80%s_W(@yzB%8xf7dpN)3NF(kOutNJ4-oW7FVqgm0NU)|XE zHTv@z#4@^vEme4(`tmM&y0FRnpfH&Lcfeq4HPmU=57(ru3^08mdBC`gDaHidZ}X<= znl@Exe#=RN)BBLLXu$FDHJje&9(DYE*B+Y*9jY7^n8zI=9B|Q-tgaML!o+K7?B;&D z9FA}f(5=}Cnd8a4>yi2U>97*f0JQlY&ejmuX_Yc6$hhA1R%bmW=Tp_hMek=#yUN zW=oZDIk?w$$#s>(Ebm)?VbAo~g$sC5O89GBo(N#_d6PT58RP3xp%=D20@9M&?+Q4i ziS)@J{0I@R3go>RlM$&5x>feOSSFi*bu#$kfD47Sx0sbD)`be#b;?2A-;e|4+Y zbxd`MD_O^ILai4_#n~+0x914%VpGHzHXgnyTvDADcuM=#wYyRqMe4ZbLab+`uV?*Q zmqrJuQ9_?uhI);0CfekVu(}q>rOS;M9_s@Wbt-cAgtoL^rAm%g^;VmQnaoG zFBnhv4m+mdm&9V7Tr4e_dJoA_1pou_z6&>5%$0kZ?6h*Kg7^8gVObh7v?@(^jCi_v zrvME*&onshvA^F=?){6ei6gUcvDW2=mCv@1IKz(bNa=3SWiA@$P#h_poBch31HuXa z&M=MmJgh|bmH~)d z-_v9z0e%^oF`H}go)x>ebr~FhtHYtZQu~FE+3YW8@}UoZRhA|y5BF4m6O6o0+!$2k zMeGz!BV35ZY|n73k{`6UJXAdR2FXAtQIFS`oezAgwu=YgvNmp#1cQW2iS*d5mLJp+ zI8$qxBg66;LhmSSO~Z5NXmD9X_y@C9q`H1~<=T63+bHQkXE3Fxi+6ZkeLS#0_Njyd z$u*UAn42keS9Z-Fw@8anr)p2PGbW>I8^RS-1EIiA%@b%QXvCeacvla#FnzeA*4 z%jKg7#~@M8!>;y0ENyOGuO8BN{aE9egEISz%JWfKF|zoIzdhudoBlW2vh&&F^gFZj zWA&TXFNbn%tO0Wa=opd6i~Yn_gSo|9B9o%ObXx#;c#dejhMl=bcY&8XLUKV#v%Y{$ zMyZ|e#SC-jswB0s7udN^f4&mKU3F30#}F(xMwEF=cbp0((qmDd+<~{L>X=dLRGgq9 zOE_CaQa=E_k2KR$^_d<6>PW^Zt=eTpgCmz~wwqPi?E?OCw|od@m4CIQknU~yb z2WpS(l7YtuYyyHD(f}(d=L0vM?$aGigD{g)t2WhgY<;wFGoa?DkQVI?GtzFHZ+-k~ zc4nfJFL@waZa*0!OwUr671c*Cxp#7Q4!v2jF8skJT3uJnB+Z|-X4#iiXF%F5{4l`? zU5x^ymR*00PwdVMZx=DltHQ#DrMUg{>JFq$1qyf(Sa3}g*03xGkSM-s9@a@5#RPdU zcL)5t=)+c&wRx?%>T1`8v3cOKpdBeJ>c3PpZ$M?}z^@0*!2YbvW4!n)(4czR`jfDg zZ|$rOAo}Wh18l~ry8g`bqRg*G$CZgMG4wQ%M@WG1q?Hp-{rIEDoFGd&Bh>28=qInDzXamE}zCKj{71d#hhmZq zdyY#E(eo;o{~jjtrO8ClH`u1e6}G&OseK%Esu0pL!5%|bXeJ7)9$&(UULTQ~v=n(0 zMPL&MNCQN53T!D~ci6@~wIK1(*!PdR_Atg};lAO4V1_r`tlWwXhP3{oYF6*9=r}(8 zKfSq!4;=Usdnk+}7gvWd9LLZyY=BH!Tk84q`lPkBVZ7AtWxDV6*|@~Q+2BT)@uitF zz|EU5?ra3CQ@`?hEfJ2dCK?+t|E8mz-h4t8Ez15`r6U(f=&=j1y1x?g)MZ_hGdo|; zEnLk-6Yx&^EZ+9X@(FpCz6&tmi}e@IClt0DR$`PlE=3xKBfd5Qi+l7|0OnJ5bc6X+ z_K0y|kFi?%YPrqy^hEvFq58NKKqlJYRCiZu#$s171YG)l8DL0m#Vr6YzjneKNx$%F z4medsSC?l1fW_u^~oS!!#M&uOsl>2o;K?*_F09O)1te}*CW zn*#ll`9o`I67Wv+0SRRxRaJbUU|!1_M=pI{%Let4Xl#aPW{jfVC~o@zM<6tiH&J0_;c2#Td87(ndDYu4k8Y6ssIt^-3pV!`@~5e<1_% zmWSy*ROjZze-8uwU8nQ`W^bI^o>IXL7`D%aentojUJ@fC^ZWcybIne*Wz z>Qlb{i~7uq`tvaF-}JrRvy9y$XXAqu#Ms$cz}`YT^Yzb-WVqK`X8<>D>_z!{9VOH@ z<$GLh8XME^Nu7Qtfg0_F>S{;1_nUuZ{@GEy}x&p z*X@Zjh9udrX}J5k86aR+V=t0f{Fzx2#R2Ckq{=)eqJMsXf}=_vG_EZ)b?m1dt)L&Z zONU=fI)vWNA7To(3IgQ~%Zpjq2)ri}lTWS~_&xKEdgiysO@|Be>A6SRbMOvf{$A-h zk`q88TH(!mg#L7mBVHGGgT4=cPSp) zD?tA0oYa=KfoNJfZuB%RQUXk*G@7#i-2O{VJp;un0{=Qova4AI0$xD$4Sk9$coO&I zj(J0dBL5w!A0y1I7SQjwfCze%toc1npNZa6p-h^ZiK94JGyMB$R>gf{n@idMH;}KQ zzGz}cA>dP)NUcMI^xn|Q{M!1#6?na^9!rGAuhfMBO@%1kGM}U3P2QXPI3Z;#Qb$GV zrLCYR^W-Ohibb8rNNk4o+}WTjv^A#-esH~_e*NZ%t97622!6Bma+SAi{;)uVk@9F3 zEnU0Z=hIYjJ`^miZ5rpc*GY{qe$a$*By2t^wJ#;cGCjj^k0(|w5DL=!(Hra{^@@fl z43%yJ+u6rAq>Wct^)Z?s{E;FR+Ibr@9x=kr3DtMzj(KN>zW zEH@x5)SY8^+_^4)vx#4*$)-wAV})w+ruHg-L#5%B({{q*!Y@DX$JQv7=dp=;FT!DdhiouAS~urX7h<(Oa7zX6Al}1p;>ue0-CejEK))`vKM!4$hk?) z+dpt?B;a}qhR81kKMwtZtt?o9o125($ikl^U+#C)@c09py!E3_&w6S-J+5CD?=KDJ z82gL*+ZwJq|s*l}WdExzAQ{T{1o{~DVX_<2Wz`S961IU2T&2x2Cx|WX6Vb3td zz6pt&Enme;r>d++6Uy3(h+05>i6$r0mz|T308OjZuIx=pVsA5Ohpy3?+15_uE>kWV z^_Hm#R6EV}=+?DX#(rF$UoiPBlKI!^Va}(ylNT*=*1RczJ90I~$9NPwY$d z;b2gLmyt#~pD-MdHPF2&aZ%AeKV5AzN+{5~1bLaZC9zXy9^;j8dWwwVy#u&V--U$< z(Hfa?mj00W=}(eprKaY6M|nNk>pW4?W}-wfQ${Ck|J&vwMn@p8YS&DuS zPXURJh>tL@F>WvQ$A0Tm+$4i%tJ0ijLTAekr*SFk{zh)XTGN+G!^`<7b>}xKqe1j^ zamAHrg)SjpI_yEt>$Z=b4vYt0WZRO)pE{1q<;1sF^^_!V_lZ!OIWm@PT{y+ys-lGT z>lkH}?Z;J6P<(yryTm?ze`&Hs1IvN0__;QEF*f22eMYyqn5GVChMs1A?agYom85o% zbS$oT6qFc9W=yG;HsDCMm;|wfsF~zSoO|1GBiy4;bj*2?WiYDjT4~7##+&vqy0Q;wG+J^fTuY745!(0ElR`dfdeX*7IAkb%AFsQ6E z1IP<*rZ%YBsL5O%R?*U@_Vtu6(OY1=)0#invCAB0ldv^cNG^v93_kY&WL%tH>@^^o zd2GoO3zLI0>y38rVqx!LXz6Zia?B}lOBh)ees2&poA}+sp-Rcq#emwMW;szy*jdn} zl?EU&NOYMnFMM)`2er!U>;Apak25pz1;~BM%jfhXh1$c6>%&6ov~sToUHR?r@Q?KE zmfmLud={Ae`|;#3AKj|y69xCKD+?%(WAoA+vOx*G(~w#@eT&R}K{(Tk}?MmxiWu+|wH(UCNtO^_~+p6+X&omG-!Tk_P^_5u<-DA_~$0+;cd3C47 zOHvy;X~ax-F9L~%d2fIb0`QYh*zZ3G7CR)6QMq^VW};L-@$Ss=g6}Wf&jF5y!wS@Z zqKx6__obWTtU-*|Z>|C5rFu7j{I1L3Jf_i!$w!@VeugYCU#%&0&MaIXJ2|!hDS}du z#vzx00t5IdcqLk)0DhG73Gn)sh+~@}hjlup&de5wuEuGFVt|HXvc2*fhYn=h&WAMF zU?vYMO{SG^ZD~$+MBBFmECQDML=4NkB>dUV zS=BSRkEGVH4kxZ-W;wq6%Vz%b_rl>;TCp=taefK#Mx~A2bNLkisTS{~>DvC&-X|akK_lXmltB<|=;Q z2Jn_v1;WUaQr+zbMa<5Ko@ILc-StSRpGPh^oV*R;CoMU@0b%jOD1!}-Sm1loF z5`TfoIKqIvSX*~IZ9^#RfUHj$n8D>q=nAdG5LXD-vx zm%}5SmVKpbjCXV=Ch5#$__WMo(U)c18R@(0`|BdC%=c>Z=DErHHhB-HwioZcR=JFm zf1FivdD!%KCBd)9jjvu-^zk&_eRw;^Qoi16?ig5eE}#9_%)`I1Ka0Jwzl)_2ZqCw& z_Htk92*G|XDCqveghRF^KODtD7w-k4p2VcLnsj&QQsTElsa3D{1KfqUsLmQx;VdzKg{fb*&|Hlzq=_M3IA%Uc@fku;#a+pNGOd<3r8^ zw7J8s_|mNb71^)_&7Cj;U!(pM+A=~u{|Ug|-~?QPO2%pz&8cARm6VL)>MNGifI0CS ztdhw%wL_?M3e*Yw`x)JQ+RfaO;z9Q`(f-qR);QnDhn&^K((J#{!PE!MBl7EIU@z-? z+UWu!@VUwT3kGeV+Rf(?Mj=v@2g@>L5;> znT-}g8yoG2tkq)zY8V=1!TdM$scIy(XSFEm_QDM4nBAXn)xwlu!xlmfC-2febJI&C z#7AHJJB2u8kMnSR?$`B+I)NXs#vEaV(nm##i6M*^(1wk#c@K`J;6637z;X>@G=Hjw z>2@iA{jnSV66;N(Os_aP4KgCR56Fdqr=R@90L<8oF=eCum}Lw?nEno!qz)=KlC*4D z@G6J;du~-Id~ogO?_v#n+H0mOb67SR@)>@a;kDnh{;l0>hYF(Ry!Q6LD z-d0GFRw>Leo9L*R_~A{*?9U3kOJCGi9Y5?zSvO;8(PE~)*@xJ$ABQTxi{{`tC#~%d zlQv|Ek{-){2K_QysCbvqNci{v5yajR^X8-b<)}Eb&v#b5%#P1tt6!jW|3xG2;KB7^ zT7b-7>+lyON6zMMN%o5MKhq(KDl`7xuHO|GYWt@JExHfn(AXTZNzn7ZL=zF<$(-<@d?qVFh_3oX3&A}S~>QX42jXtL$mAUcKbkf@X zV!Kvd$rCUH{-x&ysP8CA;3y@A!8a>==pcN;D&~Vn7@0UX4AlH7^vbCx>6_W4#^+_)1VK;RdE#jI`$09`={Ts< z8v*lMSg(TR462G^eGF3c`cj2{eL2<;UI<66jD%2bB3aRS-*6TmhGHg9k`l1YNy4PU zCF-Iphj%;otno=?GO3#KW`pAzc`4GDv|LCqx5!gJ&*i3_u<*wEKtra2y?Q%0qgy23 zBu&bUn#JSOcS%Ldu1D3TKdfSBT<`gnG2&qYz=-e*#wLDlA<% z!@NVc`SfuI?|0aBy@piARKhm;L2tNWdPGngm|H<=`L9R&mLtfDBlL1^9nF-KCESL?4&@@NU=?=1Vc2xxI{berCLtO;l z3HzTmhBldrv>WDb=E!g}126jDrkZ*GP4%362cIMDg(MK%)s_4Kc_Yu-H0g?XRM)rwvnj)qm+APV4TIUj>H7XdMzkvxYlsuK^D542GgG#iUym0pIF&z?ym zi#^7v@>OYte`R%lidhCWgKnBTDw&`qnN`MOn91e1J*&1;?W5OnlIxwV&~D2B{ybB} zcWD|FlnZA#_*Ol!KE;ropMLQCnmL*B>k^B-JHH2m(y&Ch^&TH~M`-hU1a+-94RD}OD8l^=$MzKymo{c2bE6T0o!APX9q`*$|$$aRe`2JvKAnBttzTQtH zoR(wEC2Azk;p^5lbDgqC9Gkx_wf002o3AEhj=6c;rdC&bV)i0;VU>=HB zDmapxrpPN2S0>?JX5`!a4riWc@sGr8807|;yPnP_NpdqYb123Vq_sy4iVd=Ht}gN% zz_Z<`$(MC~F=X zg3|DckAwfLq-Wk-o3U%OJ#UxF+_bpk0fE_k2BWy0@{%5P|i^#{l6T-5_YSz+le z)7E`&vB$n&Y8;Ko{NMrVz1oifHkKClGF%Lu_CZ(@=ri)v0N>zoCMC)nXU~i33;=o$ z)ufKbv|FYhpOptomDHUshpm=A<^?44aBeD}`(dOs6;M(DveS~ZJr5X=eS6&)MTw=b z?~;d!1}abeOp{YHSqa65$%8XbZw37fnMQTAKgfDk{$;YTlb3bx_Z5Y?$%L^{+?sfA z_+=sKHJhbc(kF6Mw87BlpSUxDd>imf9~xSU61 z3^yS?rGiIE1ORtbah;<6a?I?{ho)*E=<}oDZ>4dU`RP^*ocX0sxFUZ7trK`!*iWmF zZy07!3KoJKu3WzwSb*ZxN56W{d~zE`G9gSW+SbRDNU#<&{|k)W%;U@VBq+n~2lg{hhU^I0Ro zMpdZ(efPDLXMexJ^wV#-VH3)w>QX2%$5|Tq(bfhZQ|J7tiEyP_y9$rlIZmjDRmX+BN4 z&5VLHxwR|ZCuNhJ4$)j}GL~J>cTIdRs#BkTEm;!T3Z$zSD4@tkha-;A@JSx;uv9i& z$o+BHPn*9#FinLP0T9)M_zU@HHkDq^?t+Lx%Z=pmwbv0bo5JUile$yAnKAU`YxB=$ z#@cIIG)-^$m!*_xbN4qcIq4tt{Scgw>=XH_)|D(vi9N^_k*K*f`YM#whn5J@eD{30 zF6T%UMos^}_k*FtJ;2VOEspzTwR?mxUMPNh5Wdt3Kiv3Shw055lADl@nK5hXgz*9M z`ZX4MY3uuHYz zLsz0YbqUl|{^3a?#Dw$qJ^o?iRcI>oYrl3huuJaX3I_3^GdtJBElN5bCpX6PBzmA$ z^(NoC<2f{m9e2-eFwlXB)6uHO_veG_d>}H;WI9xZo-e{}Ds3$+iXM*@RR^vmfbN)v znK((V^_3vstz<*+LqdG&Iq)+DAFtbr=0Sfz;9xx%6gi9>LG2x^ zAwF+}6O4psgSi7&S?FPmoQL;m16C3}qc3(_{$97gTQx&?C06{VR>ZEJK$=SxfJMj# zjwYzRj8S^fx{||N$qgejiZpV2@?I?Ro97=@BaAFecC*LP#u%P7b^oB>!yEf7c1#C< zZR9b&lD^sICEM%w+$eI2C;+RgascvBce+icSN>*{#`;|mqF%ez7kCav>#Nt?wiC{5 zw3*04FnlE8S-*b4EOos0G{Y!WG`i_LZ}UN!aj1}H-LRXwTh+=a*iwyE3oQ+LL<~-< z6#Uv=Nsl9bY&%{6aY)O%E*PmAPqgk^&G;Dqp$f{$d7-MmL%P47!Au#viaJNU;``WA z7lA<@|M@(t_z^%)ai!Of{&iIqQn~AOsc)B%;UZ|A!VR|=mjJ~&d`+!UYbY2Q>SOQw zs$g4J)Iktpk^S)|c{r6C9tV-H3N;eH>p8pnX%PX5;+CMl4!%RwoA1iJ-+O1j%`50{ z`w{-*M1Paj0i)>HW|qW@BJe+018vK4l7*S>v{^o^|W|O#WS32nl{9nxi z$L=mR3UT&LPs^g10o9S#_0q+_>!oyHF_SXI7nUrDMFLs?N?B?-7VS>AkA{DHxm`+t zI8vH2sGXdZvSEDbEYW0$la`y7X|YN8k+-)w^Pu`M#V*@0W0`L~TUWkKaVd|h2adSQ zXyS{ESO~>x%j~|w{f#kk<4Xz+2E^(F4Yy2nC*PU*s|?{NE_<**_F>rGGjsL0!2B8K-3s*L6YY&W;z?q9;k>gDm)lQA-U4V@06RlwMzDXXj~gMccPeOVTeDz60{LFnHP6X7~(FRNMcySt7v-krjboRkGekh2v>2Cqh8TmfZo4mM~LCUd!OAe z^aC^yRQsd=o#oG?Zs=`rdR!5f9f!lQo;jM*#zMN7Vs-I44o7yzkt{O!f-l7{?(&6b^y<83fnSd23%!uE zgV($)(&vuMV7TIz!!eFtHvZTH!#>#2~sk06}>zWgx`T1xelv|7lP`wmd5^%<2;M~kB= z*?p6_mE{_>b+l}^8W{Ko5!7|HkD!~v^(~$EvNpdA%*V|!JIUPVX@K~?G)Ub}uX+!A zHQ+BasC37Bt49xL5Oub!!AD(}5NG4h3}Yv;JC0! zHq`X2(g?bgw=5lOEmvb$(93+swss6SdsFxhywR16+-}#h_g`cZqB`Fof-3F?KIIOo zCzrVK&b94X^DpOtsiS0BecZaB#v}@UDnU~STlA~9`r#-&XxzDDJ)RN=gG5Ggs-Spg zUtZiTHgmc_#TsYTj-!ujK{tGWl%6Qw4_Z_(Q)VNFl=7*MuFpI-_u3gmntRgT=#9I? zv#~MwbmY29Ra*yPMt;6z3S*Zb|Jm_eEupb-Y>#gIEG@Xx;dMe+Vqip{0FN{;MJupQ ze{N#n-?n86lMJhyZJjcMMZriW0yD4p)DNGELo)>>d3lA#Fs82otZjcKP)$Zx66ns} zlR(R-%68#awQ){?k0M@Dqi^07@0EFEm*Gwyu=D{49JhdJ;;^`FOpn6Fe`Lq+;mvnV z8EdRbnOvrI&S@Y_; zn*Xxvp^<<6J6L?e736sitIMtviV6iatam;9^`%bIKE29$q{9)6x#{}l{oLUjaTP{fM&WHc_`)P~Wwe46SM*fcI$tqO`(66xg~j#IoNE zN@J^+w4AT4$!1+V!eV>=Otwq@l(GGbaf;_<;L|Bo8DzXs&ya#HwHiEm#J!>BV%Bkc_Xd}3B((m0n#Do z5f$t0gq#bp>cxccotk!SpRZmA>PD{NQQI#){^OjNMD2g5GWv8?UE+8xB=js;OMM1p z9ZD+{{SH)8(*LUEPs{N=ZdKOc4@w=v%@~4FoXN^PurH>(DGP9cl^%KFX+jpoK?UE_ zFV=VKeu}%hVFRH{j-#zlb_?ir$7*6X;%q>^ABc7p2`n&@K5MhBnPP8lVi32l1uQ^J zD)2kNvGur0Y#;Aya7XJejSiVGQ9rhL9`Yc_rX?{r{*3bh2}eJQ9jAo%IauC?=yll8 zbh~O12PEX`O}%^}>JsTSn&z73tNnWUy^6^fT)}yzdfvz`18Q%9hPT$#wv3iEGBmM* z(yK`F)pfXV+_TpaV!5o=3Y}eblD3D)8h(EZ_ADEVZldK^oVBc1k05Yzsy=P87=E6H zzGV*Luyv^uf9OBw4MQYYk#C_syhKYva^}nRKm7TLtN^YX-`gijVH+CqB4Mxds{g>s zS6(=DyKEN|=2QgLuS82FP=&t8UfTHiuy}6?xu9Vtl-|Q>Q~suhqY;&n;lgtfOiawlMc0%9VO~Eg9)F`Y~VS8u+V#cR5r7q;R4gNGwQkLQ7lEhM%Zk z$z2jrSLp!@B#Yv-QD=(uWjZ0=`1^VJu<`g#UM;=7OC2BnC&e+j#-+)kO|3Y0y?l6b zslP?_cifWo#+gN**OB&q$e=n}8Dwh)!bc{%n)_$rcJ{WgXIm*kHX~vmKuiSw=LQal zW@J;-_4#IOzTGr?psWl3Z`AJ3j-Ms^gYE&SL4)sp8ozaP#5u>5jE9l`RyNd|Bk9GS^R$T3MX0O z!w%-OGZtTGJ9Il=28yI}@RG=|PxzWYzbbDyvn(=?X|k{$=lFfwPsJva%UpxA zA}zZQPZDyfdYEvrP9kf1as{ejc+z(SK(s1A*@WxSQO`i#2Z{OIY5M_3MAk*{3aen9 zh@z?dCdGMscHG5_tYNQD;(6RKFgwW!s8g$URwE&>i*HsqmW*_v$8lV})aI3Dm!VIO z;txfeQDSVUdI`!ShM_i_e$boqxD!ku0v6sT9THklTaQir%KsvUymwZd>%9JzV%l?j9xp{~7e?3d;;Xg5F~|{a4@riU4P>2T$Lt6ocpMv7!yja+O5@mU^A>~jF>*-MB_ZMM=j>?*avoWecMcgYa z^r%2gZkOQm&Q*^nJ83L)nYd*b(D&u+5h%m&Q$sAsS5?1y4>PBNqOB1y?O_ zZm)(n<||+p`z^lJWkh~LV5eInrr(|7X5>Jt&%DI$_TJD;KsqC{1Z$E=IS~-|;(LLk zGF?veX|j$FT5jNxBQg}Q{r2DdCyso`>x*DzfUFeXgPAI)XU@(;rO_{c;v0DfB?|8) zo7}bieBf45nriT~X6sgh5_4w@fDl==2VYSD<1@=cXbworI> z#JFLm>(3D{%d|As(%y~ha2ojt+x;bmUUbB_k3}WYpU2%E-;_DK07az-k)c1p9s}_4 zj_qYi!LPw@qPXXRI-edPsMsUx9Uqt*08=3rJ9_Qi_W$0=fJ1G=FZf6(dv8+ybvCLQ zsJ}|sire^bsl4)T6JOow1#Y?5tZg@T^J+R#byokWIQjGA9!fqEV?R?F_Aq^*XzHkt z94xX_rsZ|_TjQWXmjpkVMd1E zz04Uq71k?Kn`6V?xv3$>?ux8U&X66Kayuut@9n8b9dnfA=z(b2x`fEmBR;~EdO(4H zb*er(fPB6TAJv2Q? z-==aQjX8Y$vPe$TZsQ4mB)?yJwlmLajMG^f?@4R{*A0#c;~34u2ZO(l;!{~<@}+kI zUF$iEsgRBSdH;+prs4GaW0ipgPvDZ|UM|Pne*WxuKVSSv`Vg-V3VP-mZl6QU^WR4W zjJ}lo>UrfI4I~gHdf(Xac_Z^lyLtSZUeH<`{$NuYLGwl&xSg4c?~_62@DacSg3;@P zdJ`AE!$BU>N`64THfyr$i2Y{k8hf}6yr31gsvw%{q+;g@4LOi~jFJy|yH8(^t{|H- z#x$b_j}-USMq64|O=sU`u(Qe3)3*1ij3Pe_MuTY4kUD;)#4a)(IeMTz9H-y)Y8|7~ zreZ>dXLT8C@KL&mgs!$;#C)T-vdXz#TS4#K8rj<$YFhVCDFOfxC2O$Ot$VY+z*Eyx zuP1p4@Z2k&1-P_!kpG#i{&_{gSiD9Es9`EMFMA#)I$7{02^HEG3W+2rNgT#J1!C6S z=W}PuZ}S=1IIFphEUKrd6K!#9$z;hsLh4k$tM^jdj&D0!a@^~b<~-;r6C%U(wCS+t z!WuUcC#Z#7Lt=!V%Ip;U(0oF1$rq_y&->_Nz%zqIn!?GVuc0jNBIixos2C~Rs)BE_ zXM$jC)q?e(KJ^A!(uvepBd%0sN;C>QC`-JraPp;G17L+bR7N<}kNrD9BUuKe2>^l# zA3oocaABS}RU{Mp&D!bz4dwwyIUyDcj)RHFR5)I0cLHU(cc9@JiocJi>BAdlQ;eSz zyH%40K%tNU>(6vu8vCPek#G7rm?%ckKAoHfAh@#IOprJ&;II$b~W=a_)sjipOu z$A>`Z>miEzcXd|ZO2)=yq5@-j!?dqg)~2GSr{uqh7BK93MHZNsI&36(Y8%WsJt~fn zuK!1p`4es(eCjY(R=R6AO{FyAdW~KBym(eMkIN?P2dnuNL73x&TW?BrxW<`6XAv`U zq@Ql#Df76FGDSC_Jz4PsUPmDWw`#@gL8w2^Rt^|~QQQCT=Ul;$x~*8ZOf6x;+EO}h z->qn_+En4Mt@ESJ*yRY9TThSOrT&6{vfUdYTr$vUP8%HUF^+F3b>V$o|M1{DY0uI9 z+8;~rM*%8P;g(sTCbPLQbPIvzFKE@C17X^n;d}iG|3}tYheg?KZ5$ATP*jkRR63+Y zKw6QGk&$&25Cv9OIkn>kfBQ&g%O4p7&^Z_`09Dj`L4@9yv&*B*?X_G z@3rpViY^P^#|YM@jo1daiAVU{+>NF@$g25X%}*tC?|Z)o=vhzN+$`bu)2ek`wYrS^q z@c#Mo*HS^v;#f3pZqI|Y0p5Mc?!;(hUvJQ+0D!66J@?($^U{L{qwHVuizTpEY>Y& z(IpS6kS10PsAu{$@YruG_l4SppU|pq8~m*<34Qk1x%q^)^K$#n>@viGBO8d1ue^2K zoMw+&Bj8&4$W0NO6MSJgT?Dw*&fAe0+hG53&~j?--H=JY`|@4Hakx*od;fFgf~CXg zmKyEdj%N)_3KPi5S6V>2R$JVxo=oF-_JcKkLdx;(jdwM4fnN))UGz>Xpxr~)EPof@ zf5$4`fK<{Su+cP}BFAVClL2Ycj^*n7frw6ha$_N`Z@NN|DVI_kpdkNC{r~-Qa6Q2= zAl*h+D<`U6D)fKDzYGj<=EA#_%p$y5b9xeoDliOnW{}&}NyudtT%4DQEWE$qk*(!Wxsf zw8SLfcdi~LcEzTWf4@NAjespQzc{{&7W?+kSl^rKPt7KP#17M1TzZ9u#(lIJDADviotfvKOn4gL+0!4l++ewWk$;fciDH;*;9qowH_-Bd{@??M{~<1+uR1Vx2GpGvHagNK!b0JP z)!L?1jN&X;W-j$KMkFeAl{1TIztOTA7gF-(ayQV0TIJ4=PKc_U^e2UDx}(ldU0I$VUcpTq9GR;>%1eir-qd3 zJU(zkn{wbyESabw)Fp?mJoo{bYN7;X&xfP?6%lUz-=BS81gCCnj&<3p1G@%r~m$i&#_vM^62=mFAoP_UTL!0 zBb%Dl{61>nI=zaG9B~>#zg-l$kYl)acwLZ3wdh~4PLO@U#zR@&``BhpK=GuRuhItv zjR(tEyPrHxiSp~GpK=H}5m~E+&Bt-?m}*kD0l)v>bRK)Z0|o<0S#TG>0OvPB)ScT* z&Nd_fZqn~LhOSw}9>7cLv_bL+;>pI`z%iEn;u$15-1frA;}aS|!9g;D3i4YD*PPHX zJL|JFVd`mV$Cv54!;UhYY~`s%e`C|O0sm=W(_=Jq zR}F6>b>*$xJLP$XFc=^08uw*qJ!N=*VYP0v=7sATPrfnDNQ-N0ppXB4VBSl3UD{9f z`oGv-Ep%f%%kC8M!reU|CzV~YB*J(370sk**HLR4o5REDKEThSLcjdQ_V3>Y+Pg9i zRCJ}*b$P(ff2Ajv7OyVZIa9O5v?a&__9lowU4nuJ?bYdBkhAEex$?5~$O)p6gK?4e z#~7g-$PO!{CW1AByrbnsP9N$S;Tw_4-OA=WR;9~ zi(y^H$|PJj17x%=gkk!Gd*S*qDE2A~ZPS{~If3mbbGPb?`tLfA#O$c(q2G&rxwEvd z{Qc~##4f{U`1R+0^hG92oHa{{--?!G+sV>R5s)S#jnU+&vY0ASTg;8H&bLGURc$~V zNygz(ds1wa!sYzPZ6Wp{0}{uO%1F#+vYJnKUX6_{`~Ua zh=!`~t95Vu;Tl{zhuXel?R%jJ}h>X!Ty1(^93W z!5E)}JKFc-OAJG;0tNb#lTw>Upn@Qk_LC>Nqwi}mTYPsM{~Xo45WGpmlld{6kw3k- z(oE?WMLTZom?jD?f2O6Hf)88hkU}AYlGqpU`y5C}!P#|m`_W=oHRYzjE+u*{gK{*^ zG_{R26x$bWJKissM)=e{O-7JSNTIY?S+VB%8!E1c->c&@9@2+$^hbHAsd$@?J*B5& z9WSd$Dl4=ZoxkQXW_VSqlt+7qx*66@iPXuUJ&wL9SNCIHJhsi^YhDIHnf6 z<2+Z|f-L2Pi@*{K8VgkCPql)CKk`8k?Qggt(lva_K%g$>?I;7_`??KtvlD9RrzD)5 z8c?i-|F;e`7@G79iu!_$2|0}Ntxtv9 zECVa8QB-+Wor&Dg17_z5Is5y_5uZRWX7mlhE$NzEUZeR?-|*}|FT@1z`1TK&wR1b8 zUwnTZ!Sj%{NqUr%`&NE7)D-qMb0B_>o}$e_0CLw_XZk5J%yg5KrD~kPFT5o}jmBy& za)!M>#091o$5*tLpt@FHRKHRtI1keb0!6JR2Z92R2aeSjd++>HHU~l}OH+%|cY2d8 zhTsCH!Im5p>z(q%GV?!4iUjUA`q|tUAEoI<@iRR7;vT#Md-!t401NxMGV6D1?S>zj zi-Rd{t2cjTUfm&zP1f-{h9Z5`KHE>D_53>SJnfwIZYZSNTxp?vhi2#yF@;*e8w+=8 zC-<-Y@XhGmL3VGf)y;@J#J+Zn$M`NIdgpK7gn&wu^LEbb+IkMWWa~tGrFpAuES@?8}w zzD6WBr<|iJ)EN1GVp?#p5T}NK!B%hp`d1=KXu5s9&qBMb*vF!XO^JH$8%-HfQeoT? z%HB{MKKXkhVL-cs1*EqEkE0*v+X|q>U#S4@D@GG2PH+H{z6_juT7A& zl<0OG`vDC7VTPt%<~h4LCE7eT2n~Oh%_dLFXRrpUmb?Q|gpfBUZ{OmyD-rt2bM4Y> zq}9BLC+w@Cxp6G;tx_{P?BBC(z{Ud;+a?X3PC9=n*ciHNO@-Rb=h1R9IMGEVEo6>M z6x~tOX36cQ?hP(;vW(#2EYB~odh0UWAhKN&Z+pH;BrEw}@i#sJhxEIIitpPmOAqkg z)+MbY;;t?0%?^GxJR_GqxwIZb-w9GtRjF%Ny;?2{xww(c8s&OYVXR1y{@|v1GhdUD zS;FG26a9=}%dI8@X(Wr5lQ3XYshD`;VG>W5_N8O!zKSaGd zx#}!=zj1&lw7c%$oiMF!+Qe$*-#7k`)D5<%6c}~(Z3cp~yZXLb{68s~2_7Rs*I}Tc z_cadpw^^jn8A9qC#^iDdXEg<`bGT$h-wJeNxk7JxbLfP^45!Y$rTN4hJhOl1P90a% zUz@g!a#1kH9E61@U!u}@ zT(FFvZap$CS?;?*F7^Pwy9Gh|vq^M4&k}f`Tcb|z_-ZbwG)8cv<_Fdu z1*J+4;dxO~As7gO|uyHLNKvH1tOOX1JBwR*TcGSp8iI&|9%O7*8}<9h){e zDf&lg`R|M6IHSq=rk&ARi~q_=pTVWx-d4=Kp-Gm5$c&W_9SCbsz-S4ye8xWZXtK4( zwG(a=wnT7-EaDI|eB{;sABgMp{O+itnNd}v)y6OKI zz$OjpbhzCC%UScOEKtcwoi(A_UWeij{OT9rnj-u8Op7701&X;2+0p~tZ6@q{3fG_h zD}(sYOYZ}Wm_cn1|MtI<%HQ9E;?F8ImhG{$cnE&5{!nE%Ugn}wIROO=$_0WW!%KGi zv51rvUUofb-XKd7v{eix4L7uUqQghyihq&B0twsqWC!q6>dggb(R#E ze8sN-?W>|k-DU#4DcRv_Fek9+d}${66K5(nV)_CCR( zX@28G{aYyd=Oi@&isE@QimBnwf!gcY@LH?5qA=g;E2KJ)@K!uEE3N5gRze>cyj5?? z)5;&P^JddH0#*ejBn-MZZXzo_^cqSmb^l=+;ZCHy(Dh>VwU<#g)a`9hY&dHYyX4Rt z6O+cYThef|ZQ=dT;4=DnFTJ-P2ziu5l@d}(nAnx6yeS!{l$4*5;$I=IC<|3WDC0 zk3b?xZ@#VPQz^3yW9+F|Vuehn`O}8-&!yum>!6hJw?t1Dh?G-CnerI~$!y-U{+0#V zGjJW&YBlm&)iUr}{n!dtYA#b;j10xeMk8pQ8xW8@-R~tb!nbKW-qD}M2>!?=9vHso z{ailn;rk@ZaThGdrL>*Si>wY@PrE|_?vyhWIK>Kpv-q4HIU47`b_#ER-K8&x7q)D4 zi-8MFPU24fk)UnSsSP5HQ%*5(gQ|CS^(G%V@r258IGGgd%s2&r` z&tOX(YPtd*UMQoJ(g8`1^!*q+a1b;V;Ai~^wpdNrFbp*;kDqe!ZF|aoXw`kQW3oR7 z?4R=GY&*fY9!_zMpE2WSPM*Bx4hmFEJGP?sKR;fA)WaLt*|7HBS%7_z15Vrx`o*+^ zt70d6sIXXwM9m&BZt8k(gc)kw+4@Z4$ZqMDpNz~bnf1rw}&H=={ZER+%*BL^XppY~!U_hV;2 zllZPzXR)6QX+Z9r%-f&jJQ+_2IiG`_B;FbwVKPM!6IwY7oG(frWv@r$NSOX+b= zs1%v-)@;V-Q(>9*2`8tUoyb|;;Cv-;#7q#X?9hF+9d4^U9!=b-WhcT#R& zVCQAYmL;P$l}nPRkE78r|95@n_o}-`ux=KGew%ZdB&!SxUXp1RJ;k((DB_(M`!C1j z(!Dv^&G5RAAm`VX&RpmkIPROhI^L3aeg<@v+XV|BJik~Lj@`JLnct^+r=q{3G18YU z5H(LXI(%$$9VERvhY?8M>0$fpFI4q|anQPh&TkyImpr9aQFiES7lUxc>PfYou6(53qbe>h|A72b*rpVc{cKcM=P<(NYXWr7&i{JLS$} zw4JAYgZ}8pMGX<($Md2`>n&l7k-`~2Vw>ToSP+p%uh+E3Xsgu z939MJ;|r?FTI#_)63ZZTl!7}?;FLEoV@jfzZOHP-KFGKk2 znm5}`iA4p7BJ$M=Z&Q!=A7`x4Cj$oY)mc-$xmMqFkG+~+AMRG4)5GsPSl~vs+Yi|{ z(HGU+_Px-e-*v^7)S?vYC>p@vbG9`nL>&|KE7siq1P8zDcZbE_RV!6|=?jII80sE$ ztBfDXpZ&RvcT+QvR7l{Q`KWAQ1AmoK{FOpKl}ZGvQ`K9C7)j5DSC-Q^Ok@&yZrjv~ zWcl(qt;6JgJ4`=kh@vI&=iC_%5QIp|k?a-LjKf`PT!CG zSul4mPEw)@UULue#9%F#R?87KK3QCE*e#c2Ht+OLO#G9c)G{}fKPqm(ZNj^NbYok7 z$G|Er9QKWp+&og^U?Hm-i+(BI`&QKuCvh!2hK5H_hs?C9k=A{ZFWXYPfrzKRWE52U zHZtjt*;Z5K9u1pyV4a`vD8oy|pLWK7V(sV#SWmD*9bO6b%v;M;U9nXw$b>o%iLrNL z!g)sGP;ab}IrPx>NuSrMwXwd~4_p#?(etU!J@*$|aYYHr4(SO4+3+Ri*dgovAZW81-Cs@q}Biv*e8!G=|KQ}I0K(-SVD zxrx!GZcIOE%n;4P`P#V2!;Nno0(3~cCqN3F z3g2N;<92-$-wGo~$R7@-om6=2cN(YWWnZ4KZ}d6nBEh8=w2`+$9(*neV8dW>QU4f4 zO*kwtBvA9|<282*?D;py>Nj)kv~&r9LjcSWe^%U|qznxGn;TI+AK2yjmMu_9ym8~` z!vPb)Adx^bFOnY0Hr*3+Y5n5E6N8XBn0_d%+Li0+hfoy7jRRr&p9uXqYq^K3#PTBl zG%HXOq>tv(9y5Wc5}a{=my!wToB6H4Cmfpwg2u01gddwa=BfDizBRSAO3=xuNip!R z3mN!CLGu3FQ8!p>#CW$_OA@-d-Lde3u+xL$)7rPU85G@D+S*mUs~V42!@L0KU=h;< zEaJ+9?f~+{I!rMKOF5K$wctS9R6zbQzsV#-poOwa}r}- zfRDhE4Yhn|bo24_c?yGQIG!3!__0ZQ3n@g5&u241#+SV0hkphNWFN5-u&$ohY=jbr zW4+S@jY>S91o#wh>;t*-e_w$ezQ@k|Ed$UBOzr25E2tv9U67A>p;j8r%T7M@T6@}R zy!0#V@el~~;*r^KT-}#_N{_aUlQa404Kn?1o|jl^J^@%O6+*@WQWG`0-)SaL&wMl# zePdp2cRTR>Xn6B$MXvhy9g!3_!YjsSrEVVM6Ej$u9pyd6@~<2(z>T0_%dzZ}@A1Jb zAc%N5HsM~JEZF5rNEg_B6Xngcinmu9<-^6^PV2ijvk1Mg2~Gq1fDF2MkgIz+Vx4=z zG*0q*&rEkesRaezh7;_+;vw|4_h_LG#ajAW&FK$Ew>nb5&bZ@~lOS`5__P9Tm8RQ_ zo+UY5CkZ>WJ(=VQ4T1)VsIiFw%cTZY6#%!XPbhSvy-0xJENdks#(0{PAKtU zSkjZ5>xU1uf-N*cuc}}83A^jAac`y!y79`CqPxVLG;Lnyb=1i_csu?4%ozS37QoTt zN7wQz54o}_zNzW(yaCxQ)>_9KAakqfSQ$kdvk~uQ^PPD2rNJ1Fga(_B#EQ*zU&<3*Z**^WsBtC`3GA*7 zQbY5mmwa{>j6e7pPcw4C?ZVG}JNQPg(#S;IL|8{6t)u&Zk%e7tQA1QxX?SI~==OcePlhn)5>`BC-!blU!Day0K zL*>D(Xtaq!O=d=VGt)bNC*unBrC280*dR&zj7uTV;HIL>$FGt8>%Vuj#8t`{W%TAx zi&lbXjR$u_C1^;hKL1%WCVTj^BC>Dm?7*2j5c*haGHWJt>4V0%3Q%2Miv2SfML;3l z`n;52tbU^X)vmmw_5i#5a4RHpFBXrtMWSQG@^jU^UCQX$`iswhpVXgkK%RorD_^Fi z=<~(vjk{wHek63P=&|}w>5TMS?95j#rH4?&ZOF5>el};@zm<4nsYg;T8^&G%k#y+-PcpQk8#=KE1mqexE;>YH1bdzlqFJi z6*_8JdPiTmn(mcxB*m(D+SdT?ihYh{tG0?RqATJXQR0FIDRd(c(F<;Vt$@`QlobtNIJI zM5{x-A`%1E0_H5pLmNBN(ARn{*7-e-XLE|1X0MqjLrcXuj;xZGmF%hP3s*)EnKN$x zxs*vsg6XU9*LH!rm5_8BpW@Y$vdu*+U61cAPkinF_3Y^}hOErmC}kcuu&V=)EE00Pnqvi!Q&ONW3i89a-YJcVt1`sGtR z3eFcIL4MJt_nD!LQDU1<=gzLtJ(8zKvIj$tbmTBlz2g12Vo( zt+`dr7L@mJL;~KLOUW2>lq7{Y?851icB9_W!AK+b8o=)48se&D)pZ)KsoP+yK>LxF zKd_*s)@+24o`FlhOe=7x8_;+zykqA%0i?Hk*`hzArO)z{;m>P6#(T{agl^R+5|<%` z`&fx&Bb&#E{kU`wAci1%QA|^GUMp(eAa&=86rK%%r=?->E9^kUh~lB!|0+igWQLc6 zo=l=QeL=>~8I4VJ)W?oY6D2t{MzC>j78r)@90A)v^rL$)tsC3PVdH6?Fg9)~ao1#> z2|b4ZJH#%&e0>OV%1*>A9oKr3bp~9J-PMiU__G?lKg?mH`hgh|8@l09J6N)hl65)A zQSCPXv67jc>3Wan#rd$wd%}Sv)To^I7>|fQmC@NZ)$OuF}Z9K#x1tzDL>xn$*)tRme zXKOfKwWq)$9AmFzFI#Wnvvh}}?K!;g9e^UporUsv>5l7Oos)E1^y)IF)9_w@Nv}-k zH?*Jh{V0c_R~l@itPr`$54ov&^*@<;*dGrtk}RL+XWKOO;Ax^zcpTr-_Y{8om^-T* zE`75L3$3N`+3Ii22lw|Zo3BN|6)y$VMsP3EL?4_kLi}p6*9If8-I`t%>kn9DSXc{* zq}WGe=sULeWmvM%ES8nJ0U!rl><6C?r+#s(imq1R%{?MxQ6t1&8g*%$yZ98|*@HH6 zt10*i_U5P*`i4yvtO@%&Mu1V6dy*c#2f&8HrI3g;OfMnPCqrs-Gv1iYrQSjK<*Tqn zwYhsY5F@(ZxR;QFbK8hkM4VQ9(Q?^)9((;1x45~^iD`>OmQHm)=FaieGjPCP)l>ST{9Mr|INwt~cUr66^_H=#qqV7LI^eogJ zNJ}kWh*jK{+F#eMk!x)gQ_0QO->LS*ILU)<1|MdJ?t3F-K+4c2gioZdzhWf=K*#p5 z#*^j7>48b(qgjg9RtjT-WN>k|>ZXGVxP6nh!irJG8(X=OpFDb;Q)9_iPZpw1*s^7= z*#TG&5B7dQ9_2NO8t|kO-FCDsiJivE7!{ce$QORQ?tX}+hl^LG>D(Bz?i5_hz-dop zwsY%}eivnUEK4=6OpB$G4b(Ej)S6JFpI>1Skjl1=?c!^1+bc-_r?WgK86H3sO&;&G3s4u#GG0l&)aiFT z;&-QEBf>tbm$v-MCRPSLFM1Ixjj4u@sy&{6QdO%JYKi^=qaGJQ(Bj;Cuh>KVYS}&w zhr(8%9w(ZrHp%xrNetA7`bJLmQR3I*tj-oEKZKRpRDVjoP;czxA4? z2>b+2g#ZXY$Nn{zR*ByH_}zjTDpf@vwb}U^IVTu_>!7pFvNPQg(LIiHPK9o!I$~xs zS2`VY#o5lO*4<@5lOM-~+XY6%{>0`@&mwVly;c`kvn9JPC+0&dW2)CsMZQmVk|J`Q z|5N((C$LeGvNmzGND+Wb6uao}c1eIl{uZeRI6d{#dE3!;*&#dN@bgkajGF!eI94n5 zSB%ky%a7NjO;_J%MslQXgj)o@^7PJs~ySwsx_A$kS~J1_+wu zHi+*|mScIJ1l)FB6B8^jK!UmP%Ns3PwX)f+>R-TU;$cUhdn6}9wy=hk6khuQ{YEiu zD_dnnGvuAJ#JzCS5pg#)_|7-%nP*=Rip4J$?kRfAZZdd%Tm$z5QCS zDT?+oL~P>=JYouu9pMUzgG^x&PUN!icOMlBP4jwSisNM&LEYuIio7-mT(-^3zB1L2 zyNb`p_|BPX0XKtO>)hP1N)o#B7(Pgm`C4%GHMR(!;iWb(ysDw=hpGQv&p33uM#S`+ylI)axx(;tfB~p{krkKBsDx|&T|#^A-Mx0s5qF9pPps2` z3;o)S;}LsqQU5_xf$g{b&ik}en#r-bcgCs_9P2GC&d48K%oB19yJPH#lEf*YWA{eldSqF?yQPgY4? zvHcE+s$&)mgp=e+%!#Rw3%4jvPaUf0JN|*Z|7+{y`km5<*vi*Q=Ppo95yo!lI*Y{gVWM4dC-oKF z`ChS~N;M>nlj@Pp_Q24w$I3yZ2`%)PjM;&pvW3*S#w{nNKwd*d;j`w|J ztUCI-@TA@%JSqEf4=ALOMwijYw7PH}epH#9`_Fa+Fq?{}q)?Yw{)0??N7t4_Macbs zigOs&_42f1uGJ^OcsCV2DBa$pG%BzaoR~D%&lC;ofvxX?=@J`8KeJk?$(w7}gVc3j zLG3bg1H|uOPyx37l`%>=Z3JrQ2ofTBDEwG#;Qk9zpQE2#ct}m?W#U{NEisTzs7JXm{$eHe&gR&BhU3M=4h)Btdm~^Sn zE@TLU&HdOlvHNnK$ggA{efre)OH{-D-0M)Ca!e{|w!0ZNlUj)Jx0CRkk)En%b_P+c zSAVXU5E8eayNf@2M{~_!;I)|YNm+HRX(lp3u*hyI>r9aVuDoP_xK_r`2eFR&rkm}h z-kXWZ=5H=NYtYH*YOlYcM6+n4jDr`>2ySOHxz;T;UuUU_%toQS2Eki(%1a_z-fO!b z>xeDo-F+U_Ktg@hkCq3MUtp$04_)0Qs{g1*I^kLRF2tlLYHxIlbA=e z|G7&(W%hTvnTDAwuj9w!p7iXcts-27Bp^x&WMgD)md^X642m|O45_dFyal4?)|hGV z?dlkNbVZ&y$c|Mu;1FSJq5@25s*K(@8h4OLxkSxTrFqfKV6#&V{r8z&>5tm!UvKLg zZ@(0LeC62WqV&hKJ#3A%OieS(ryyWZVzS~Qv>mI&eT;Sc?0@g#fwH#|xgvZkNia52 z?7)I=W-B%Y2igOKjG=if%iW6g;ik#s)K|UQjR(WC;oYJPsh&$8^d%??>c2L6wtc@r zzyG7o?C~8T+w|ToTxT#(AhIjhkYwiYVv^{tUpcp+9S+Hauj8GOIO>TW!a{?7$cK+YdcEN=Z3`_b-o-nnnBd03a?ebN;|Evx)yv`kUNPDxIO<%_Y^k}dC6wmB zkLNxfyp4D6sdrFMKOmGUq}SN;o6QsX-E}3JZ4OlY&0oJ+akj3BnoF$=xe#2EGxOr>X|-?SNi81+x7FB?V-J@rKW@;0Ymbykz$? zhX7xAuZfA;%_TG9%qwJ9OOtoGLOQOPN{kItr5|>3ZMRwq)i9gVPVHqftA1;T#%J~! z8TGt*zY?5O0-go+32szIG1>7}?_NVi|LQ+!XMhY#Q@yC(!Oi39iA&3a-b}82e=5x- zU+|Zv5z9H}pjzX3>#!O7O!1@sOrg%dlOvBWos=K1MPUhlE(}t>n;GpWuzn|hU zQ(s%RlO_bE^qxZqkSOz-^EbnvhoORi4*-{OY=mx($f2T~qkzKzmKfgK`2}Q}z+37= z-1+fVm4()5-@(F;o07j5p(1~cx-s3iT`x%0kb%DOC5O~ac498`bWmDMl&%0vJ_qdRtr8LBx#+M5XdCNs)ok81aaA71Ae$2Fwd0KTudK z=2>+(_f1!(Bi!_*+cF@^TAR(zv*nIhT`t_^mz# z$(Dy${t3$%mh|&)ipD=jlys>#u0=pMv`l~IhcZsK0ng^bWHgu3-CreYMBy8?ab5#k zTRijKnD#wbxx+Drkyq6b=#(P%^CM;>Vq|zX{j%4uyDykaTs>A4g*VisusoLWccy9; z<=Qobhd;)*m{HuF$*Wg+r4dYa73=jPd;N@vFd#8lNG+-}0d&&O`yw@%-5y!I&o?1B3AO?LD-V^!C1!Jd@_+y zBZZUl%1ORR+0w2YDm?W8QtkaWP}3nrAjK;`l-gBa%T-@vR9<^ODx^@OYK<2r>%aUk zZ4QXzwpHpER%WN$_abLW<15omvQltAJ-ydeUibkc+bboMbA0;OmI=)rZ|_2i7wv1$nNe<7YeAT za}`5sYhj?Ge&iUr+bTZ>sXK6-I-DLuWTMW_zNVu~B}#X9Ij72hrpgWwP4%Kj7)L+pZnwmWq$%W>ewrEgj=(m?x^0Dya5jHrjRKY(rPN)9>4IP-PbF~ z(w7ZKytmMHiP4}rE*$;ft*}uMCPPF*IG?i~he53BXE|R$4(M)rpE)|B>BEg}b{E5C zzW}zlxEa8HEe?JvXS0YX-fknCG~cq)^kLBvJ*P)i!Yz|KSpPKMIsPy<)tU)od3$Nb zd)S+><9>mQS`X^IL$`x?d&==b*zb#%)G2|3Bo(ygVk^)Jm`wjkwOVec!|J|eOvF4^ z8> z#33)L*KXPF>LdNfslYE(bhhD-Go|mtmzs((Z!l?wCkeene2JZlF!|jF&84X3I6qB8 z__Z6Litta@T_0V*Q4P7|ZdjQ^U__X<2=GZ_MOGI(O(~i}qnK;tQvfh6 zBLotz^eFraw$B+4DqclltW7gId>3o2%ewN)hTIhfI>Z&*T%4@?nR|0RvbrtsUAQ&b zcVuK5I~3iXgLCTLRjEZgaCzzAZy&ULQKx<(i*5z$oq>n#EAxIs`b*z`88oOmPf}Bs zHcvjd^|#JRqoJyC){Qr^Vu|2XqHl4S=t`6FHO`iSQJ&x|>q7ORv@*HM6XU%jJfBbHg52(3nQdH7x zQmD{kpf{uVpH4`VNc`ECt6ZXv?}@tdw_>=E3q6BB9XUG7B=LnITX?usg|lSj#~VjW z5c|m(dH?0oZuG5`b1_4AcG*6-i2Onha9l3ydT$@jr#?Hl$Jp3Le{}ZxcsX&~Vn9O# zYv*2eFMj#Un<$-$pqk0H2|~jrmgEk6?UjHU3eFS z!zP;BdY<&hPUxAMMpcoy4rZ9GvdI&?4I}M2I#<5}Bx#GKZ6@GwVmA>nmHv|-A&6EjwfYed^XEBSAbz1w5e`PE1$~l-$3LtdXJaD`z2ZWaBzTg!^5^D*m7vq94ne}2S~JEiyI93tGx*?nTwbsxN@)oXp?3eG`CCo(q#?VXj(mkaq?P?{6kfoer7_hshTFvmgy&~=ReNlHQ@EQKDOQw6Uz%z) zC}1cH_{c|F?%9RShDHDCBy94&#Ctzw?)=TzM+M{*05e5D$FqEoh+Qw?HGi#@yH;%( zKVWzje{K_E8k=RCS%}s5)2Z;ny~ZP=mi0u>-QK9O zr8LeWme+r*Six^HF*L!M<}3@kAZSnxlxu0@{Sc(D?Ucsh0J@>VJRbe7OyW`C!^m5L zX}cX!T~kn~hz%dXPv>vK#VJEmtv-%?Q7^opc-`0f5+c7mYMg|5tfev2LdU>BMuIPg z#=dD^08*U0x9@HBF?d!e@N29eHso~MRZ`WQm%#j!4;d?#2HwgN+_vFSYdS%bFz@L! zg=4sWJS&ODu}_@q_{5_`7UV+ctzA`495YUK0jDuXMdC?Eltn?PZ%rpiyK*KL zH?gEL-V2YQQf~NXFdTJ;k*C1$p243!KAVC&!6NCi&$e&nF3F^5WSGq9s@Juj473=> zsQf|lT&q+mQo_JQh*Qzm>nuV}aLFjZR_fF@OU9_oax2xe_9%_w z{bD@EGY;AnBJT2~Ov1tKvGb9?wS8dN!*gHwU8jBixvk6d-jB%2&Pwx5`sp4Q%3`zdclHHJD>z6(&Jiu)~Fidbf=mm=7+Ao5mFA zhqS86jNZ)@fs*XUGgW*6YWNA%CQ1CBEXeG6sjnPWn`|l?wo7F+ZRd2m>++ZuWm{dx z>_(t?VTw{Hg~97au(x5zzGMCXD}n`I;U(YPLlfoLk@ljMv#BW6?Ite8Z$uG%DYH)r zP_k^Dkv7E81WQoP zUpeBFLi(!jElW>K2FId%@V1JgVS5bJG?X(Ix=4VPrxL=On06mcojXfBc`fj2NBS;;JmZG8P_hgE z{=JvRfH_fnFIcNaYhOHg_*x0TV?29_V;*&yIDSDFJGuOK4|oZf!K%4-jwcS*01tU$UKSuq6g`AYz#VnNul3`I_7)M)b8DxKc zaefX_MlX8cnEXcK8%E@2Q57d%*E!xyI6;luJ$Ht}`71GeSE`Dr=UR##TKO8=?@%8P zy*UKm8)ytydtrtYU-Q-zSaLKxvAL!c2H(YgP@EU>1r1a>VJwW40(!=BD_iG>HVRk5=N37_ zQGk@9FwX_nHXESJxIl6sPVMVu%Wrz^m+4f!brXxItdfWClKa|h4dyKHghiPe?tb3B zoF+8}%PldON1M^{G!9%&i}V5xK-TE*r!K}_P-SiMk>EqKQ>(f_m1}*+8Mbe$5EyUu zd|(QYSlZf#$6}FiP53#CP4ov(C^c(L{{!44&%Cz*-f>Fmu_;+cLA&%aD)2B1=ld`wkzaMkONj={(P3-&aQhIgcO%y|8YVjOT ztm^>r>%fWU;0Q@yGJ5YYb#ZRwnDm2JM@E@rV=ePzHy$fS8-rpOJ2cvp*z$zz*lF4G zjU)oKF&D{8m^(|U+!{&A}s>vhM z@HlI4wkd z<3mcx)wncMN~J+fMAzy~n3tD-bb!w>S>pt%H550%qWiQ1VpuobrDV zP#ON^6a=QQh2`k?emWn^DX>tZI-?jH%-Nuv^McbCF2Mu1O57y*o7$RlCLsI6z1oL) zTu(B;Hd!nNNMRxPOOxh#Ys}DH+UA^lwSICRobLzCqzZVC3*P@D&I~b^G+!S4n~>{!DDll8!U_ZL8#`&y(7=ZI`Z(z+P8=Cvvb>PW{T;;%*i zD_2H;!bYA*jt=v#t&KbpkB#r7d6c#uM}ya^aJaYy60E(O?au-go-4;2$2go#;z@~( zf!YoxeD!#(FL=;Fah&xzO_E}13<7=h79HgXT**5kT|f$vz;`K@M#fK(U|h1O?p^54 z6)O6ugm)8#@?oe29h(c>Vb&R|Q1pjp7b7{UDZZ56RhG!0&fmKt;UcbEtf5uLtk0TL zFH7i{x@=04b&3Gls6}lK@_AnJyDgqy_FL|&k=wTSXI8L(2TZRBr|6Gi^1B+oanl&U zVI-mK;&$M91k8|z5E<(Yg_d;{o-c0L@=b&=4e%5g5k_nd8lOEc9T&U}UtT5Sx9-o^ z05THub+Yh&?q-+z?rn4g(tmPyYiznG4i_t9y5%p20f$JG+jeX-JzAZfcmyu-J)ia@ z!F?1sDkESw(HX|(M!GuT`gD40i_rwnuH2$ZyG78#M%)OrMq(x0%SR5h_k1KDpR&AJ zvYryV>@p>j(3E~@LE^o&AVO>7)#3fvISi4tHn@$HnJkHr?@g)(xMzh%McmAv{ z$r=Co$DIJsnAHfO44hE`9G0sn?~`%A4Mw$J1&TIHi8sDLqeTo?e&8L2J3({mFhoQm zdE7Fa>Z2evQ{LJcj?=PImorKz17LBn1*yz_^F}zt-P(deriZmpg&qs!lfpbQiwRmtV3J0Coc0-TVo%{8m zM^^O$D>%Rb#{zI*dqX_e9+;7FW3z=Pdd=ih4k_jn0_)-NwHarGDa9~w@npHQK==N> zC+*nIz2QfV25npTn}-+86PlSin>VW$^&K$DiRCg>QzB6yu>}{^KPN7>>tO1JZvy3L zLQ~oG5VW5Q&pDDJziTe zW_T$tXVmcn&b!{Q-@yMob)KYNO>7;<@zcgbHNGL4%YGU888k)TwWKeTkX=<9(W-9> zGrv)+rG0hX;QWPXrKdsHj?De)oxJ@#SaS2~YU}dq@(xq3jX{j!+Vji!^ZCpuO3Zo_zCT$dOt#Ayw}@#v!mL>)l&x}W&Qc+&lB0u&Jm{Y zu;X9ZbiQ&Xbv5DJ-Acb7S?YYfWwhH_XEN)x-Bd59;TGC7b|`bz+!;8;Fd6;U7=6dGRt8>`<=ohV*5Kz0o50EzSC;z^q1v=bsV*sD{2{U{1ZPO4S z&jq|m*86;!EGsyif6Rq)n7KF4`$nz@U6AZOkteVZ%5$0Pc`ncyJ`&pE68^?nv$sxX zECqm-7NBT7&s+1l1sMewZTn~_hn6(N(+k;}IG@`&m5r^ONHH~ubL`tTQ#7n7iHQ$`0WPg6DFnojk}+R-l<5R2u-? z`eIN`7V25b=*Cg<;)ho17)xoObk92E>z>k2IYgHIBC zLC{mn{acINc>UY`ORRM30|8f_CO%4%PHhS9@-%sSPuAeNK+|?ydhN*pZSD>(gCbD@ z_PdR$7K*n3H(8u#y33D@YuBntd}`$8CW%$_J9J8^xjghsb4SxR9M&E;ZP&|uzveS; zOE(GD`3ovZ58p)zrP~z1**nKXdoQ3sbP9#qp#%LkbM}>d@#8H$2ru^Wlw0AHS-;z( zy$7p~(Jp2+Xn7@?(2=cKvl3nkPok=Nmbk_i^WLxvcBiA>ZHDGggoQ_#iA zayC8r{sJ`xcsw?rt^Yc=%l*1orne%Fx3}4V>fjL6A+VQQ!Ov3Fc6-a6LLZ1N;~w54 zewD~#t^465`XxB12r9Q^8LwtO@>W>^T|@twYp+5p;c&G%MB<5&=7Y&Zuz!8FRi&Of zI*`LCp)}StV2kS{HRCcWYo%8!W78}W$(?IJmsCo&C-ljE2M|o5noi153=ME*cD&sH z)Xq9zW93afW6Zoe>?^P8B3xuOf5x<=KhgCHrcu3Ayu&aZ^H4mjuw`n)P8Py(jWkwf zpF7MI#O%OtO(y0|gVV$*XS0b&&!Li4L>Pm|=fn6cj*p2zM)B2U-T zY|%;2BIM|tPLFn1!+qeDh75jQ5%+o2wBdz>b6)k#H&hK^2QExMv+=`?aLKUX z$e@#?x@E^NuK>YrRE2d6ak+SGG7yiY7TmWz~&#e3;68ft{!! zMo}sL`nx=VZ+LA}cVwgDP>P&kw6x*PVYyI=y0?yP&}bG+5^*~9(`8_^OB`J8*Tf-_^=Cu(WNJY z@5{J~=6hPGg1`mU0tGlK?DFF}Gs+-Hom*IoBHt7aM>xrzfe5@XXA!jS8I_qaPfPe$ zWKf-|-xlMATD}^zupx%vBVClng$3p(ufSROk`H66bj!g&@=hYT=Pl;7Jkcn`O%3BTiOSjQ{ShQ(=-bSB@jV31wTf3q^9Vf;VZ0tg&?fc}Bfk~w%qi|i^ zud3`L4bv{k??Z?bYR-hej>s&soVNTgTLt&Cx5~pfupx)@9c?G#LH11Xd2|?k+4u#D zgZX6*#$VrrA{J+I--zQO?fNtndCp+JYEG3_Qz!l!>IGf@roi50Naxj{WkS8rP9ED264X%_AiDN66Zv;Zb8g>~olERM_WVQ!gh@(;XIWjY2 z!CKOAq15iY4ejk1wn+Zu2EOgiJ#Muho6avxY7TETS5xqf>o=6Lsw0i~#DAOXj^yO9 zg_y5A1>!lY$SiV)b>o}udcvhmn0N_+c0_*n`-NEMLl@a{;1o2clGcn4-UbcWE*1?% z!!2Zl2+o7`fLUjH+oV?fzj*9_69yUIbjY`{Qaz(#;3ciSfO z3w6DL);m9Z`Xg_A`2#k+CVR4z}NFrZxBQPu;p|oWqju)FFAkhZgkT1K5TB+OR*EVKYO)f3Sr3hHXL_ zIy^d#o=v34f4vqcD@)NkMN~i$RCST`H2XQV*bRK^|ebU^VnaPaIFqcO3oaS&Ovai zpeu!UZtgPdvhLoR9+ro|dU-dnkGWVqVh5p8C%~SMc$CAy9@(H4X9b#4UmyfN!W;0*c&&o}(Nu(~v&>XQB6q&A8b}XXUglnk)D9JR z*)Aevg5ZLx7+@92>5X1#zhJNCV)u>Rq7BhWi6}6Js29BT(J}h>oXtG>0IalpyXPj`O4d{4Y#sO1Fe>f zxPuSq-jNU_8HRg=rQ^YO`~AW~<;eM4#YAFD)ST^^`pGU>2ph4zo)fxKLrvpS zGsG?Mn_B>4v<3UG>fFeF*x}|Nk#UVnH4*(;;!NCFXZbuoe43p_9?~VXP-xUz+ckAi z7S`p!MW{{exVP{lz^fGmb|V<(K$M&I&3FOxw4rAY^miX>i%-`qN7rRZ2o?@9YA;K~ zDH|FbX0fHOM6Nnbfn$dcRgujAmo0Rs({-!%ta?%um$Bu4M3lJyBj#(bk?qvBrB$=pVb|93NHwSVuF$q#d3pNMeXb-S#@q|H3Ru}=c%UC?9CG0#5brfB=EGRuvuSa z!>Yb*TJsFX&GO{NNi)!SHOE3smUOJif?@AscnG6OLXbycr5%qq(#z>P-5~FslqdgS zYDkT$%Mg6kbUF9+u8Ir|OV+(@TFUVYSAE_So?h%hgvIZH*x|&I(jRUgansSq+628{ zdzF5TmO3fNRawP-&vJ5Z-o30oQi(RAy8C{OycE-yl5M&Y zSy^k?=5PWMvJjn_ty&DuWK|=5H#-odn9SE z^4O5kL_EA=;IGoI2J%Q{nydKMM{+i8+oIgd5RwIhEnJsPp|zavBO}jKp%ARi6CuvM z88zlYAFyairTbA5LtoENPQ0XL+=Vqq%3mXoKikrF$i@EDT(q*E27!Ok?WJDNq~kJ~ zJB5gKX5nOqSY=CIh4ecLdp^_GWMicDn>n@`? zT$ADQ)|{f$Up@d#E!r?61vSSS(%yUi)@m%R{3(dwZ-0f8^^)LYDxcZv?ksQI-wK%j z%-O*i=_{oyx116@#Sun55;fR&`pDFmV%LcY_n>~^BnW$BulLNVsBmL4?2Yzi0JPcG zzqxsCqN5GOvx=Gd&M9f1QV0T6^rjiskXl>YsyCR(7H0H7W!j-;8CuE(6Q#R_=^DQ* zQ3(pS-iQAgeyHSg}&$GVPnTg@M_B|LrETZ{K>nPQkA;Zh^q z0V3F^XH5^%j2W@*!?|O;n9FzY)X8ctxmYT`D|*OnxsN}jVlZZBzw#TTWy+bcb?NOLTX~38Mjvq4p5!i<(B>jEoll}CWwcM_ zT{4d@B2iXJ?sf}kP{M-1xcsiwWmxz<5YgcgTWjt1y{v-K@vo)Ilta+lt-Pn&V;Xmc zd#pgTqbav#4QsJ9*9#c}MmM{T_RdtFg~T$aZF=o?FK)e8{|T z-csWC@=hDJ?w%eg5J;plfsA|&Z5!CIEQ*7FGxL~jU~LIO%=8?HYNdlSAt zd@1G;M#kU6Duv+0eB6hr)V#+$20E`}qOe@D!l>@;{l8+b4?NGW)|K zx=lX7JjBMUhuOCN3GLk-TuZ6*Vz)`TrF?~8v@lEYq}b1K$<;z(7Vpd@@6Q6b#e%h5 zZ1-aG)mF4b$vepEQRK$`g_lDy#JFEw5JFVjm=l$A>|Wx6*6P;?%^SYa=MpxaM9xm7 zkfhv!U9DP-*A^jBDkM<9+W&VJKx7;8&tzF^+~3pTPcyGK8d`<0p?Xq#q+u0F2mjD1 z{FVF=9(3Xj61a!$NL4PhxJqu^CC;7)+6$?R^PLvVQ9SR%`H=$X?t*marv=`Nc7wW? zE29rFrf-`dFHLm7vw*mjA@n@zHrf6HJw?%C!%wGs03KQrAY5>06fIKw)ibp%Dwwh- z`@7&9S_tHk+t}~Dq08Et?EeUGdyHx&!xyQ5#CMxEw$mnAIklisBi;D@*#H9*}O zUf15DGPAR-XjRRKl!t`FtDS>L{jjmlkQhxw~MDQ>9dF-G)Dn~@2CfD`b_fDGp=B; zy;%>ab8lbqLR*INi6QFE_1DDJWoRMu$@rbR#9{kZ^Vx#R-661WQzhO@ehqZjtmQf+ zz(_|J|7bh=G)?0rCW>a zLwUO#65OYQx%_luUw58Dx)6==&o)7TVF)gpXxM2ODo$n4PxWIgm-R+gS#Dx}!+L3x zEL`#Wmu^4OYF|t&-grbwIy_CWyX+bA3!|2p$H;3|W(JCD_}n(PmU1IMmm+g_;*%-9 z87)iMT$S%G73?dgrw|)>mwYSb#gTvZ(FNf;V0ZHe5qzS&MeFo*txk9N$<2urU|IiY ziCjU}n=HkJ=G^42rubdK_S?6AJu9)}M=eI5bE-GoN?$m*UnCcqJ;8{Fp6<{3R!WeZ zZ)pVyjDAKj#?NeVd=V!ws{OnU^@L$W?Wunx$-g~3G!bc{URSWKq5ib4g1Iz6lXcDA z=p3#<(I7--Y%TJ1E6l{Y2NPX*!q~`--{IE(fFD~{vE_Tc12GA-X&+c1hyq!C7m?*pD!;`pMZJTm$m*x&(eCqY!w{oKXh3mCP zuRaMSjn{6J#~etG<-Q=25HR4z-pZO?et(oARQen?+R!7)B_8ehggN{u2;;LiMVX7t ztd;#R;3nqJL$Ec8Ng8vLO$7fp%O^tLJg#D`Tx*72^tPz{(#KHYnfGiC2K*&bFtV z8E~t_XcmUp0Q+Cck7_dP!>DRZ){wz{8p%-7W`%BJ?Y#i=gexvgAxN(xW|)0KnAml< z(x_wG2f8N1KQdClk)ERm^$;H~u$jTJp?oz{1*_u=YQHy6t=)2Q)%H~p9LwOyaZ)hi z8Ph15;ndoqHNv(o0vpThB6O1dEuntSp^_UPW5B!LW`_bc?E=WHP)i^zQoZCe z67EF(d;kPSSb{uCRdM^}R1dQwHBuE{=~vT7V>RmWzA!5x=~@sL2I0n;(=BIx_Mz!F zP}|*iVt$3(Dd9TX*gPwFd>!7GB-&n5zmY^ODgb0*%<^jod1xl>lZ|)?`++e% z>P_o>&rsXKQc1VsoUSk?Jx>c9kya8D$6yqR9t8AvG(Qvwhu!6xW~s&xF?Uoil7bn{ z!BavPQqiEb3v+ai@Cf?$+R~r0D$3NV%TwJvu3}c?n8(`X{pW~z=FPAwUrL9$%M1o3 zH@OM>*ZTa5=KzE24&evi4bwkCuDKsP+cpc4icl6KXTXfh{2Ad`7vZp|m|WKTv0^ZC z3PVi*XBzV<#;l-yd$dmBgC46S2xd*%xeba_$@G%dT%uZ}P~B7Z1vgXb54QY&9ofzw z>G|v5B;VcjeQ`FF(D*0?c-h*qz`i&SES2_f9IpQ3+eoMHm1(=hmcZ?g^HLf~<4mqd z5Wz>ond+(y2k}aAX($j2c>B}}fwQll$BdbS6o62Wzm)I!>9SeWcOW9u1I46?TXVkEi+n*H+30ZzuaGi zdTV$M#audhgxvhNukQG>=awlj->v@e%}YmmO-jfOZgxV>;N9!o8VD8b6=RS<3kOC< zN|5ECBgH5WnW6Q>B^MYwestRc!Cv>c@k)%UdAP4)tSA0Hq5OV^eb_QCjBBlzgcFN3 zRgvCePc?~1^3Wen;1-PybgMoaE=i!?=hlaH<+fNRuVG}=zqWQ4bl+tXFNK+IaxH%5 zD}Xvr_8YWH4kfZe7^=6>eRuL?t5sEy+DF5lvpHg4Y_b9%B=^DgnINuEV} z{HPw!{WBUfaNiACkt@lZ@L?J zEzEuZcGsO;LZ-$88yr;+bkY8P@e1RM8L!cAyepTsb80eO(od>8lrpU$b!xN*L*h`j zg}ZxP8y2=j^Q==9BA12{_u~z>wIs$lIaE-j`f0k+BDz6VRd1739hh`)uK(=1sB|dj z_^7oMDP-8+cN6-n7DRNUQl4po0D`0NBjT;Yr&%qOGbRox!z=_&k==erYCo%Ht|}2Pps1a{=1r`HO|BBieX3ww-w-YdG$zt}nk__WBP5R&WO>E95ieFW5 z93h-i^6j9rp^Wqz$+|T2ZNKT)anprM{cEG$lbfC5v-@tGSjSg+iP4%}&J* zbew8<(-Rx*U>zg*E5R@5eemKZyjwNrLRiyWx}>K~~I`T*Aw*tT|n9P!xLn#|ZC=B-M>B%LWPo}Ev@*Q;aR*GEOHVY9|NRNT zN1yMho{|1P-HD9D#&{MA;kyBTr#%XyI~c8)KU z_}tsd@9rdIq3cC-^9+}>BvZYp(jsLsUfSThPjW<9d zeUAmzG@h&emW>Qd$8Y2BUd}#>26r}Na&W!5eCDuA<@MY?K?j4*zUOEk(H$X5YRzpR zMY9M96C)_BqTIIY|vC(9Baa2@k_pOHSQi9%^ydRV6)68!@I#78vAD$1ZA zf&BF_aU*YDreU(!ibQ$n23UCANS%JH9{qudUstzvjWiN>%=-r+37+MZtNsgS&YtjI z8Vv^2AGk2TA&mvu>sVaC&0{97cQ=pRTFuLa1dA}YwN@4^LRH?4a0UpQi=fhLX*;%P zx0?y%53lEW4xv34!rO6S#q$+C{KiJ!Y^Q)J{7RiX5ElgI zEojZ~f5wm}-&gv&vQMm0b1c_*zf-d9T136N^r{;pEcxvE7X(JSVodYtJcl@JjF7Fm zS=$|5)tX)Gmn~!9a7o*IOy_5hwnzJ&;#H|uePVHImAS~He07xo#y!2x#1EHYyZE%0 zw|ff9XwATIJ}V5iG^jv}=pXv>;ozy<*wVoQzcalRzsn(xkhgRKc9eR=Db}SWYVJ?RsuOcya)0;b`x}&0c=9&;q=BTDZftAJ_sbw#rMdtSr zAr`=R*-?u&r_MX!vjIFUhT;;*CmWn=`SmOV#8((pq9^Tnumj@d7$X|-YNvIS z+J@BDh(f2R{@rDf0ir4)lQ9LcA61eOz0+;uqV&d9*IzaDLJhnxhl>?|&F^%VSAOla zE!LF(Q!yFS*Sqf7a@cB#^H%Zik1HEX&^bLJB(TJ-(MFLwX>z*5K%z2H?XdzX? zOj6*CEauEO|499tmD^tb5~V9nv(x(h4F7^Z}vGAM&=$q z)($*QvMVVFc`oWb0r~R&WSj?S1N*Bz{Co5Dk+m61!QenULo4*TOsM@Xjdrr0HL~UC zX}(KU75Rfiv8bT-7_-hffFLiqma$p!t~ZEzyj1KUKV1y}>w(pkFCFzWELq_mL%Q#4 zxO^$k3=-tl>Q*liuH8Q!?U-ofA02d~3-{|-8jMIXdw#??{*scQz)@r?iN_@&5Z=(j zUer9+X>7yEu^8yz-8f^-JnSK{Jv_v5q@Iaj|d$Ulq@bnU=As_s%?SF<4K$A zVqQaRO%CbK8@Wb!g5%24S?$+6tGq#2iFirZGn5!ZPPqN{u{{zBHB$e&75i?x_Gzuzj=S7RrAY6tk9ZhDc%ECi77biH z0YJf;5J=XY)g{rLW|MXNg=;A-_)MxB6^cb+c1BqK&mzNtUIhc9||Gg>=S?sxl;NUwNtMns8Hxir9F2IL|w0M?Y?6j^m3i z!Hy}fBXLTgFpJ~uHfN|7`Nx}@3kLcqw`wzV7H{UDG|h21I@;+?y^1<3ls5pO8e)=i z43CCLoS%tRd}v&P@~;6M?z&{u6V;ZQjpZbN-N0uFwmMEcX3b7 zdJvrJFr~x$Y)f8n6P5s)qp2h4n~#H5|CQT<;j&WsC->pKE=l8_$2pix<>h(_5;B*B z_HnXMhhd@Uwk|$Hst$AkLeHhR2u&lq++U+}iRKP2n^0Tw-JVz=N)~?!73v~YawEvqqfhn55oDnC6qaai~ zD%6Ppt?=KQ#y_0MlmF9mE9JBM(X7f4Ua~E*Q6f*K5e^%-!f>HUJcSn7{dMmvJ)K*9 z6kjmcqJvM-8m|$ahLae(u>r=sg74xyw>m`Df|pX9r{6`fGn#PC4L=vFzSHG@eeEk_9ZWt$cJt!==Gt3 z2ZJ?PkIueqZ?H+JNnR2we7!CwEy*r;$&fS6xjfNijBFU5l^fX!FYd3;J|c>CUtV>A zAQp3Zp-l0|U0J-m15;*0-)C4v2wPo|cj(M_0CAhv1PN(kgu zUm{l>R%qCMbn_fp{rh|Xc&YnjNW#ZdH_tU5|1Qhdzs*@Z>iNW&)t6nkkZ`Fy3rjSy z_(^Fnzqlo3^VwIEMKmjELeuvNc`~cku)Y{HX3iR!cpY|^34gcG_TyK_1|yhs2)4I6 zcr^L6yZ>WJQ$-C)bjtLCGyg^5gm~PYwGi7~P zpRj;0v_-xMTNCTwDwh#Tmx66vUf9)IO3x0H+I6DG85E96o)|Bwv6}ZS#Hg%EJ!i7+ zj1A>aV@dW1z ziH}(M+2HiDSYP2rou2DQr_|5yUqDC6#&*~vUi;qul>V0FiZ^f6}>2R{|5Ex%ScND-blR=#LKxc%R036 zmJ~A9In}+6Xobe!vb*Aq@+@6-lUml;j*ysDIrPqPxRLL{V+$&AUn|)W{c*>)VYWZS zz`p3)(foTqgN_S69Epcd=!1l{h}xW|K@h+QI9odt?5G+ZU9}R{-iIKL5LlOVU9l!E zwf*PVv5g5PjODIB*C@HAQ9Vt99P)kJ3lUV&%9qZR=B&tk?jew3&1;qT&ieI0vRmvdLliY@Df=0rwq9iq(@7ooCE z+}`dwXu3~31ZTEC7p8J(M&en$*MGZK?#%PAD&;T9)=UPk5)lH3*R0&+2d=PaQnf(5 zWHtkC(h9sVhzie*QeAFDyI@Y2xM}XtMVcWg{C-h}MEj7QLgp&7gI5>6kkn^>7q=Wu zh;F4e{$N{dN5M`u$!$&6C68`1B(PDmx%fr=oIGnq-0dem<7q_?$<`RP72?;vXxheNhWTrmf{F+VW1W~QJ z>#ywf+eav*{gHh0xLZ@dzM;b|FRNiZN6;v=Q^;+23_9+b=GOB*rbIo-a0F7SzfoE^ z%sZrosa=Y{sW^4?)PN*}1AO7^(vZ+O-9w%RUWbI1ZFey($E90_V`lBQ@~9xiDF&SN z(8t#=3U&xAQB5TKADwnV|5}n&C!e)Qb4gSSA_;O_G0X-?9u&|ug4#K zwFyGSw?FWnf4V1K37@J{_B*4E!`H%fg}1EfgSOU#7+^~Ac7He$$Ic&%tbG%nymhVf z0k<2x5l$gKu+JE_yJHi+iDH9ksWogaqSs}Eej+vqL!FVG4mQZ}wsrXpjx^tn_vw+~ zP|F2oKgJc(alpj^2Vn&I8|krX!-rbwxmbItCc_c90Ygwb`MrS{rdi^chJI)m)?rzT z^OuEfQ2Vk;_etw^qDW)0S}++$nf)84T9PcM;(rw_Jn_ZB5Tz@ju=%1Bgo&Y4&V8KP zz%>BxvQUMz00#EFz)5*%<*RUzTb4Y7-L3XJu`-2Mk}D=^4H-`9I}0N`JS&N+l$mBX z6b)#|PR&9bqSstZi#Lnc+CGQKbttR6M zb3}nH6bWu_JjvCM5R>zB$wl$-cqn(3`rnpHzF>9N>O;pWdirZfji%5BgB7H?^sKqT z2JEeOzGuInH@!2ck@)v1vX{aCq>*!f!1%>96Yi;5x$SR#rBOOo%~JNDpF%A&Q-fL)p`Hc{S5~ zbqF8VAzUHDm+2{go!!UrWM?>Wg_(LbxuZr?OTyx7Z;9~aVjLLGHR5K}dN((JIun#x zRZ1I8xDVz>Dj8nv$}ItrJ+#x|n9swCw)*^rX_x7K zfvAdK%YA-agq9Aus(vp1qX(4J!z-7E9e(iT3~tt573s&o=4rZ|X55+-&A<4BZk|#X z(Ae7-m5@z`Ix$Dr(?o_C|4ytrpJ3vi0zo?h+rFg05M|~G4VgKsN?2$Q=y ztm8xhQDfZo>PdBoK4H`8iuI}*@P5fn#x%ykLAbOI{V12MlP~k#sb!R^=i=6OXVoPN z$RWFxthI$yKMov*1iq}xU9Gg*dPpOP%E`&l8my=4lm0G3p~F)wU_k-C!O?e{fm=%{ zM4#quc3@0$T(72IR1W!s8i0$XcFW#TgVKt-IjAV)P|bt57!!h>0-XP;$CGLHibVKU zim({S_U8354DnOi{{bIZ%k6}lmBY=V{~Q1$@me3%FfWO#8%QfyrDV@@bU`3u z`#K$L7P_598}S@vN<^+dZkO91eCC5lQ|fUd-yHBc8?RE>XOj!}DjSHswa!W}Z?KnU zj3K1c46xJ1F04)TKO7d&O9QUlz4Oh*TJg;1cf+?I z-_d1+qimjUITUaFMm+K@Lm%@FJ~#eD<%Vz^9+c4#tU?uGZeUO$5{IATrv$O)1sp*) zC-JFBn?4s7>C8PwlOof4oZ1{LIF6`Ak~fq_Bl5y(dw&E;uf)YA5EIq&m_;yQfqlu&4BZD-L2SIB34q zKfHA2nKHjE2=a6@pahe>n`?PNIg6a_A^be-YbQ!{f08}^D$&T5x@39dv3K4t%bqYw z;%rS;d4i+3V=cP!_A3iBL`(O0eN;japKZ|(G>Jok4c1!FpE&<47--nlF|ADU&0Y-j zr7zB#ZWRr-?Y*zCU$s7*vn}GzD)}pV=QZN{Wm8K?QJrAp61JlBwP7k5U|9a=B79*d zkQU%uu%He8U0|E%Z+vOVXJb=x_L4(IX4M>&_R$!Uo_JvV-;D;4p z80ISG^`Y2ZGh#~8j~ax^tRDpDygtnTA(cz&8-pk)L8KMtS!01tO!FcAlL@@e4zlf) zWpA>3HYx)0m%HvS^$V+CLTa^icYv8k#(MtR4}WdxsO(K;$xl9xCTpj%5m+X$)OIh# ze5}+w-}WZm$tS<$zcUPyNI3dTwaP;UOS%xkNduzpERq`ZikO1;3svk** z=t-c?*X!9Xvpp_QWTv`Wn_F(5q({fC!=j~q|GxYU@i|iw6W^*kHKxTy>mp4=w4ub@ z?ars~npxT}WK%M{s(kB95T?sDV<8k)WI#zEnozCmFRW?S4vyLGG2(Km4^(;{gUPD6GfG|KfypGu?%bxMySP>UboK2>(U;6ur;X3kR9wdg6H(@No^k9 z%CLL6en&j)HENsC$K+H-!||8*wJ70lYokR8vG28rvrE-lpM4)s#xEduB+1^aw$A>X zgCc~u8#%y=Oc9MH!zN88<3D>#Hxc#Z-BaeBrH$9vH$2m|IhpfT`SFh=#hQp0_uals zyDTUDW5)U2TiZ#N^P;xPBvq7MQf9Tr?!{vA3-t84Py69bMRXwmbOsoYU?kd zccF%kEbkdW;4&X4QDB?sC&vbnkj zag0F5kL&*0Qe@0JzkdvB1%A%MY&(44@y^tWJqddk0{Q@4^Fc&C>!JkIT5h*#cYQoh zh{n2ksV5;a=L&7-$;wmpFSH*e{m%lOek_eYZBRHK&&1dCdzGW_RXkA|5Uk7L^pL4g5h9g>aMj|RKwb)hQq8{0_vm&d8?3go$q!``ngmA4%nNr z#GLz7`X*Y%hLyP!R*^Ddp)UXGNVvYmV<4y)(Sdc<0p2F3gVg8qaNe@VM4rt9RHqY8 zjw)NXBf7FO_opA>N$D0g-i>oX4AJthgWU`%3FwloS7NfW zklHJM+z`{Sx-yInmDVtn@{tC)u^%Lg)WhTt4vN zMYSjM%T#@wrz%weO;P8+0I_XD`jH0{%9?;bO zI$dnvn{qTirx>cs?2h=WyzCwTd%M3!#kcXNXW^|he72d)9)ymy+{mK#ed>1Te*JRmb@EChM# zz<6o?9cV<_50}%;HJ5QZDua(+Zsp5qFLDn`eSX*_bCI3R(=>7MHy+H-UW$z($2H#K6w8fniI$qdwQ9hAGsu3$R?<2~a0rY>580d|@y0lO;jLLjU`|CFcTU4GQ< z*Gn2hyZeTVPKVB>=!%bGI^R>7s&y-p-7ND~%~P4WqDj=NIeKxC()slGq1D^lX?n>h z;+13i1*>12AT|1*yYjC$xYEN5@s9He`617}Avu6hH~MnaA>ODZeY&clk@E}Sfz`GF zeMLE$i3_h0BU-9_`8wd0`h)^;)K)uSIi#MjnSeR_sxcX&hy~%ZSDDTSWFd7q4LEcP zB~If2AJI~z0{Gnhm!))m%lzo8+deCVab#lLp6g88#--a((C@b%N-b<&@oeJzt>V^Bg{5 z5lr=K&+(TYdRca#8(5uQwn7;D=OQU(L9 ze-5z~aBe@-hMBg2baFgM`c1$KI=|$Tv(=<;cl|#@!r$N;_V4lz&u$3H^Y{ z5}O)!pIM`Tr!AfZxZG1ue^%ev?<8G$t6he~0{ zQwOcWYd?09mVI>F4#7CxlRNrgK{=xl8I{yHA!IK_cZ=c|U* z@0?2&aoEmJc`oU{PL}fm^K0{8{c&s0bkj>8e!4sEsNEXr^WCbTHyzNw+nLXP2Iu$& zMJtgH)pjLZxIuZfU>(R<9ypy#DR4^C^3A?WKAQmH6GQOW#M=u1k>1XO2)5bDMGPVCPL-2VD&~6)kI5--g;DDUA|4=6@Ihs>my>_vBThC)UTCTgQNUA}F zJ&(!-yNw+`yn$F$g*kKVh-Zg<*8>pL&rQ@ve{cUvp}wK%W2U&r&NXpN-T9KmMMU#5 z&dF|zbziDm66I00x3(M|b%tqK+9~E0MCmG=9xx56@+(~oi(ucYrS_4)BO z#QB)XZ*kqNxW<`P)xiA|M_oEv6i8=!dS4;;^0I5o;zM@iZ zLQav$f!`rw%51$Pte_;6qZWTN2GSFo^LD(fc805l%^}`c+l6%5Pf}jn?p4EU@*d9| zal+e@W{AZ@9_fVEq3g_KojT;aDzCz7xWcloNId%=p2p6#c8U_j`lX{QV;DdN)CLLet68yv!y-9KYgNOeGYHI5s8^qWxi9GGHS1?=$X^^yR zFVLE?mux|$Wa9P(n(Vaw?FGzN=1wEj9H(^&7oB5w2OW3MQpy3LaFCUz^zy)GZQ|Zx z^?ChNHr=POwK-Ql(-v)srTI?2`|0^T!<(dC^o?S#`Fh>ry~|mLoGR$@h|2tgo?;C@ zyA;_HWMncjlDSAI0~w+IPxR#9D=%1olW!;uB=K5)OU$wO^yk1afZEUUl{9R3KZ-kN z)dFpu{?Cp^VD-;x$3Zve+bA^D;9ScxfC0w8@nZHk*Ego(6?^NlD~?P*A9lB&>%)wf z-Gw~)>(mYI({AJQP)9EvZQezfR%>GG`>}Yq>X)}22i7?RRGS_RHp|*b9EFimSG6M; zu`RLLfDH+j-FbH*{hsX4X2Rj+P-PdK&U>ko;W}=|Io>_>8q{BE5N6UU+OPD7Jka&F z#%^*|z$V*eieRX}?=U$UQl-Po7ihST-Fzt0j`_tBv>KwKnPDyVYuEj5^_E#HsNB=O z^a%2X_g;qut|Ml!Z&lnDPnTv(7n6N0rl-Q{h@RT8`N0CwMlMw!ZT*g+nWwyUGe>uK z`nsFC$flFJo0()D7r}OyCGYN9(d!TokU{fd$zof-eU*Oj^uluJ+4TXc@$QU6B{eI< ztd~ojT>^CLOw+nsJTI1SXs=x7WMD;$BdfEuaV`9@$aV{Y#Mv2_(t zQEuOt5>OCC8fm0sDCrad0cnsf5fSMangK*X8fhek?v`#rx}=q6=#s9X{@>v9J>UEN z*K)DI<&ycnd+)jD?6c3_$r#Q(nPEdJ3~uTJla-buIecA-4yQ?+XSTpdgu-6tGJc2W zaQvWy$ii*qr$OPP3GGNdba!pJp5X1z`{63tN{;4)n*MN3Y!ShquWRdh!fyh&7W%;^ zpXD^rn~F|iXORC1W%#b#-nHtG@zgkMqu^gi1cCR-vq)Na3JK8cIz5KF73^^`?x)T@ z?Z#kcU528^n#mMJjwDLhVsb*H&B&O?p5C93&agC;H;8=a@`W>}>?)AL+@^OB&{5l9 z5w@SzJ|D{oCbmdP$9>uj>ftyHuG&a-Q;KxRpU(G#(JyjEy<@(%-(Ms-+G`jb$ZmVlhHCZm~|FYR-gNagD+rOmc85~#EyW+gO<}{IcqC^HkzON{?019UwnDM z%y#vh7;~IOnK+AvreuDUzv*UXK~%Plq2>|zcL@E3V*p>D`5_JJnF%J#gD>ozy7nUT&po z*q0Yhirq%BvPP}1p_62U$#z=(JR$UIef8_viLo^mKVE686CJ}hOOy{_`^&vikLTH9 z=17$nFK=1Z3{rbi!etXzonqZOAUO(Xb-gd!5<9Fnid`e_trp_)iBLr$gW-`PQQYHxjr1PL!EZL^{jysA4YQ znYmxu1cI<@TaDB`lLle;QbC%`Tx&vs^1f;~Puv|H?BRiWbsv$u3@4BrWi&Q=N-)Od z%R)x22gvT*fwon zuhKlQU(Sk8GEKRKDN9V4&+hC2W1Q*tKX^cf8goTn&HVJF47_6D2Nwqw;EN6H!3YkZ zou%c+dCC72s0`oCQF`;QmVd0u)c*YM3f>5gwR#q>b;^i5IP~JQs#*r{hQ`|y0iP$X zh4Qa88y_k3DC;W^9m^$DU7L5azxMNie};44?@dy3Y16WB!#7A+fIp@w>ZuYOkTqlM z371#4j!{TJh$|5x90AGzM!-QGU~m624_Dqt4U@cbhU_{$lFI9)u5|)>i5sh*V_{Ru ziN}s@-_#Fhk7veLm`ev5KcUS;C(Y7kCn1UflFWlVC&4duPI+53nqKN81Vt4_`B!{? z%Aea;Uc=Ygvri!WOLYc_GVhQ!$wEC*TqEUAwg1(Jx&c$#8Cp=7aC0X=#3lA$HIZ1$ zXIRl^i8s4tgUoHYHUnF~66sl~{%|EQsX?!$%u*q|oUO@yvigDfUg~w8cz!s*zqEZb zo8HIWb`-;k!Oh6juoX|@|3ad8DbwxMc#qO(=-~bft_D0ifH@-Pppg!#o7Y9^lZ^1e4;aHQ;-|$6sh0N^$)QwH=Lg*WXPu&Al*BV_J(fa{ z$Z|JR@y+=QLC~ko;`-w64!!Inbu!p-=Rk7kST^=tuUpS;`*H!-o9TIp;SIi}U=bjw z9cNr*3{ZEOnISz>h^!OgQT^TPWQKJ!=Cg6*B2eKoQv#)%ts2|eL5MqGyhen$zRO0G zgstz3v*K>j^m?{&vtDoj)8M~xtzX{|1tZ}OF)1u1?v1FCyngHK9?_%|NXG(SIQ|UA zcS@Fx!~8Z;cDTdshZ0Iq{IE9G9zPEBDQ0gjzgR0 zm&*1RT=M>T4Yz)40n{*+NGoVVqmwmUCc7WtA2~}5jPPQc-VNSC)XR&wo>^AwwQgkQ z#)sTafC5)~i9#)5o@4AFO|AE#TB2yW|~77Vdc zfy={@VA136rpzKjDaRE#gwU(QoG3Lz?71Tsl1%_ z0EB|2NKs9e0283ZR}$d}%+sy!YMF`+imi7U{ z(`l!GpR;bn6#z!r*-uT)R=L7QpeeHiuFEl6L|R5(VEZvmt*q6)#^cEX0ic2iMzq$= z+BD|pAQiFUC)Tzedb(J@W~by&0Tc74@7C2bL5MowKPV?S`161e>zFk#d~4vIv4A0d z`sJ@2ar zh=%T2+7GsF=~50lY8=uv?8rFYzGFW0Myz`(r`d2J5Zn{kl`0ch6 zH=nk{px0;6k%Oy1#uROl6|Qi(PynP24wk|Z%yb3Nc5TFv?X9a-pcq$>Kp2hE-)cc1 zhwTQhNbId8g zF(X!XZN-n>k49Z?1OUzOTef&l&TtO^>vs-7gt78AHk9*KzYmfZ+P;Pe>E##j?@%4sETvG3D{l_l~+V`;HI+Z|xPV+K&fzKi8^j zVaXRG0{EO?<@4AHF#6o|ohI}h z72(*RJvF2YAXutgAmFQ}tD_QPM7*{=T@3XEpk##4?k2j3%l=lF=awl|;w*31)U^ut zQdE;U)XNzGNLptjorc37zeBn8Qv zIq(>AJ%3ch8y#nRErU`T+xi@W10TSA&%l01p}pa>UJb8@pc;f8|dUz zHh0`cN8;Oco;t&I*+N3)gS)Mxx0a>Yto{9FaWE65A8p`o5Jv|0y;MDa`;Q#Q@5~&G zxtnK5aCsWNgFrwIf6U-|tx2t`5M1B@&qvU_T{XpVneb70r3tvQSdSae9P7QW4xl5B zX9`B};kS#fiF1G*y+$fEN&mQ>t${v(TQR>bD5{dTNmgpd5-btFp=}NpjeYS&2 zb=smBIc?W=7ii7UEK|9!rL_Z{;$XlbUF`iVyz`*+%CmblV8kXBF%=?abeP3bPZGXlLVCPX|d;CU! zBz}U?Y^A(PQTW*%z0we(lmp{T0Fi8R({hG+zNp1fsbb~PH-f1$Y;wFAku6m_evmoc zsV8y7pu)UdmMvJcNyspYTMuNhqmT{1$MYh^2^-`E&6Xm|o)eo0Q&#~ zI$I8V6*;0su8R-mv=Qwl4hXPBpk+TlbC+&zm!ONOK_I*hzDTsLl7mt}fE>Wc6KGbl zBgUTNA2#lV*8xaC@fM(V-6L;Os z*%z+D)KI#gZ+)-1iPW`sH-Z+X`q>0_6;~`C7>b|m$2B*6vm`_0FvN)VSdKwxM^8o+ zj@pYQ%|y@eb3|vw)J%x(-wC&`&AoXYdJA2ABJ+aU3CIG5<({}{yuJ*iwQeRm~As*6j@q=&C5yGzuff!QtUA>kQc)2wZSv0+2a*i zN0i+@y!>Z<`|JBYe&iHUw>C=Xr07O&#_;C~UC$O|3<70O7=Ez2k|4b(G(9@Nq~C{s zr}6r72XwVSb(J))^ZNUgH?8JwQQSHD#HMd7hVIyWozc6~b{an54L$ZVzf!}Zg0Qh3po9SN| zUii~HK1-}^iMXnETS2vB{dWBM@=7WhB}vEEGD7a*jks)z;g88Ed{}^yYa@Bx1h!)! zrx1Ut8%J#9X)km;X%|Q*ETab6Sc%Wj52T3=qPP4g4^v+NgRsxAp1cNb%yy8Wnt%QL zTdW}9HoayBM5@#L>|4KGJz2n$%>jCdM-w>ot#*iE`^*u|g_8FIq<8KT)t#Cf0=wyr)T;?pf?%U!Ed{Oux zCf*n0zT6V-TU09_rEM7alDXAUK#-fOv}B9- zoJR%DCMO_~>-{X&sa8FrEH2u9@ai$_(A+eoK92_~aCO=#ZiuJ>6rfnoMI$;Jmv&+C zpcEp%H??`JLT_|kBu=a%Bz5jf>DhoCGp@pKg~&ZqzvF|@5XmP1Uf5eo#?z~lX7Aye zC#VMqg2V7VL~a>`xFFXnLmSi_04*102$0`q^G~LAVZ3hQ>Yuv!={`|U1k@&n!kB&U-f619N79 zw*2DoCha;}k{$M3lgxRPQvW8MGLRLc6ipyTr}JeZGiAR-IDcc9yg*pHkNGY_&-%=X zjDY52WM*L}ki@O)eII?`KX-LpJtDnZXfDdi%)7t60bKOrG41VO#Enh`@Gv6Z(n66| z*nRX#e%IE&chbzm7M}ahgP0snRPE_lFu(S{p-j3VRizf7XiiQU!=iuk=_ZejLf2_xm4b= z!nJWOCW`&YM~9<5*ihUR^)H#MJR{GMo)}q?20t{=d3fGuE~f5z+D4^>#m{tYH)f*g(ey=K$?NRNM3BDH+|3-gHn5!!h%Th8*E8<2;@E#e1})rEo%`_ znE4dF{~9FkGjxH#0Dw}zI`cwSfFj~O0l}iH0_9wSjKxnxcLVmG$U>*_+jD8R9RxfM zmi>bBt-(Ir;VkTt?WxB>;#?BU-W!Gwefp^=Q`$5J{yEogC>_UUGIhdu|JCDJ4otd@ z?&^Y%bEh5rk2?5OAOVm8N!rQqg8(#H)ivG*w<{!GO?~i-) zKc2~x*&igA>R8(-KYqJPm7-c9*UuAn5w)~wia@#n*;SGe7@49WM#lzsyYTzfb;cvh zfo~1I(c6njcy6UcxwU?7QjQo4eEoP44n_^DzyP%P8}_v*Z38;ZPRB6t)kmT0r^((A zHw(4*pWIfl#jq;4GCRuBFBBsz1ST@2Kj6G75Xezfc zdg%R?v4kX-Flpy3+^6aRVl9PbM@q(4 z5AHiPOM0hWmmU{DzUngiSE{gCmW(vWFa?9(uz_EY8?%!Hit|_a!s+Dbs?Q-xJ&=EA z95*D>_`Ap{M?<&;+zpt;mf1XW+AwP`rk%Y&gFb*dk~n+VnTP~JNU;GbNlLbP!Uugk zQTO$5BKfcOX5XD#|x6qJ@O>IC{96pqJda{=&E*%1+fL=5%QtZ_HfZ+!P zG_;4i?>RS1Qmjehx~oW@cS8s7A27dQXY*c@qLQ~|^#tz!2r=e9b6=IhH(=&d<8d;xe7T6qWzph|iw-1V~xeboM*9VlcZu)3z!Iuv|1q92E(E zjaonUHur+8ncJ}Wf~I4r1o*#wW|bi}Hj zo)90>&$xWcsUcq9^pb@rSp-bRbq-C8&aGLx3#S&dhtWgG{5?b)KSneu8JxCa8+l1G zm%bs7mzmM`|5qoWLYlgHZs0O1cJqRP?mKU@No9hwo92;Dg*&$URX}^HD+x|;Nz^NC z%C-%zty6{#3CcEjN2&WWGG&TM&Slnc>u6qT<@qG1VYzk}AeDvnu$DYyzA54XEi9wG zx5N$9UafJ;JL%*B!&3+)iT>I2&bK%~1Rbeo%P%yjuM#+0ii(q89{JMH9j?5*vF)kw zkzZIl;%xBxVi4uM6agCZCcCYh!d^sxNTwQdxH8u2(F^>o#JP)Ca2#GY6)K{s8vu_n z$?h==w{tdti&A@W$4eVV6Iai=Z$_g_KfP+~c{`L9XZT4>=b6q8wg70MaaZ~(Geq&D zFG~m@)JED(0w`j`mKA*)=RI{r}!{W>bpwEmut4XBXOjE3$f?d*pbKWO3fT)lK4+IP-8(N_lg%Cso*S#I zm^K*~cL9PlygSRo0sD}5upjAU23VXv3N{~P^I|PO@YYSF?a0eLDxO&B#NB5txgcu+ zNu7Dt`tzfe;{;QjsDJ7C4B}q>bk1lUaF7^cj6jiOi|iq^Tq#zcj0}UT4hDN4tnBD| z?ZfLJf**$fa{=GmSSB`KDS(F{|bG~R%N=>6g*xZ zA*)FNct8oQI%s=R5Gk&xGL|&lJx)YAXeqKWJUnfdV|;g$<)+tm&SKa^ro1N-+x9@9 zZ0#hpI+#CzcR8G658ZvoNPu9tM$uCCcx9TF`SSvuO-1{&<+z*+SFCw}_Zky@t>RF> z^2u`8r-RXXqEIJF3>y5HqQ^P29u~yPmxpoPwPfwZhPm)c6`fv?6j1G@rNEN=N&x^l zSx4!!`q6e-DW1o~W|BniD$g_oJwL_qADBEb!vIceCQxb+;zs zYcdFGgUMQ_5^z-f7!YmZo9T2Tv8{t-RLQPqjWXtiZq}F-RTxAXENHdA{J1&cG?myl z2pppC{cVXa#x%r6OKsgOn!Y{w&TG!;t&w0qmD`X|V5g2wbS>aGnhI}sd+dneFkKh! zp|44?$|W-EH~$AvRlCjiB@C?2CIU7Xrdl7=X1>)!aSm}}g(L~7qi;4B;?wQP zE^g8dmrnha6W2+m{=5XwjodA0Uc9z74zP++PIMc|W!R^R?UZm0L%>tMg0yzi3OJy! zGnfTkHRdJ%VNHcnOPm!(0sxY-O3wjcK|444f)0qO^qT{auvn{^M^9sBKs;%Y-#;Sz za6cK_0?0>5~=r_s~Q2*%G2RgJgQRY#Jh^l5utC?{K#1u z4DZ>Mg8>Z=m(SJAERcx4A9v-2`)fGH3xdJ0q#DD;qrPGXbxVEmk0i>U=3Y;Kf&p(j zzwvKaRkq0P*MXRGtz0`+RZEJXm5$yu{Y*T?-*#|o5Ix^J`X{lMaAI)@F!>gT8QuRt zZJ=kfTRX(M4ck39KDtvwYdnv)S<|{lMY*5wlX!%7;xVS37~n`wGk{0r^26;-0jP4K z?&oqJRY4gEGiv!jpP-N61k#CIsmw-7pg=L*MW*JdzyyZdUZ)M)+3pxoG;&!qLNbq& z3azJO?|XSuX90WLGTA%#)F2U~uWX&4+BS%MmjevxEHe1-+Dju6u~a{_%!qbihGkd8?K|i?LP34s_s?HrqZJ`7V@N`7%RWh`AI@mN# zd!bvr@ctQ!c#sm4xF#t9IqZVDnnaUgVq(|i4#i?eFr95YzVK4%g+o}|a5V*Vg7HLI z?doCY$`R8+l(?5ckXs#PW|FMPNw{{g2kEf=Tsur~759A$H&5=BTT;(MBc0YI8JUd5 z+pCIDCX^24TdZP~_%qp7D23MA$fb#^9hLt`7vAJIOf){+GR3(x z?P!#CaGM^!yh@#9TkMm@qT*&ioBbu4KeIJ<5EX%J5knxA(^hJY$+D6In;Vbp)7z|2 zS#)OjT^1J=ax)wrzcAn*HF2e!(P-JxhMY-RsM-YGQDRPC3Ga!dja+#lw@)}%UaQ@y zMaXZ_uul;iK3quy2BnMnvzZRb{1|YgL!Ob68&a}8ZUZ!5t+5LnMplm)#NdKKK1^s~ zjyCJ~ejytrn>vwj^`8ilByPHiM|XU$G`AIAM*H1f7j4hvhJdUz>_os|csj!kHIPEqB*!ptM6R8da?_=c*dFIga>?cp^lMuG zq$(eg*~0|wsv1kmRr15ek`eSH=1hT;Mgt*luZTJ#7&vt^y$G((aH;P_VI~0pR3FK)*M! znk+kF$P1~enhPXcWq-T)u&-fLBmQAk|Kz^Vs`ZSx3XWN7Ll#6vTW@)JyQ=y7lA-(r zio-a-NEx_$;_Hq@Ph%;CMnSw^|>SlP|G-^7Ps zS;eofKrJ+`7FTxKPAd`jEcw=K_VBl#7AueX%QvcuZ0fQ%W;MhV$LqTrKD{wY_WV>> zy`QhgvHpA73}7kxAwhG^0K=5w`08;>zo@$w(`u(}p_RW5C<)Xjx+wkY%!#?m_$aUW z(>%E##T4O=9gvx9E3xp_wlACOK2a4wj0N2~;e_)R5MK7=R^fMm(+;xEnP-eD)nD*C zW|h1+)8M>RyR@G)rCMa0KqhMYPB4c>GB@*PaOB=ZcXxp*EY%gFD1{nzb<&>;8_L}K-D_$Y@iNu$Q} zB>R4FS}yg&dv`uia}P#HyoyZp@bixGP6^xTtULetbHq!@1^QELr+9Ks7ihTTRG&<5 zZqkJ(fxXU;x3hAL^1WMo>Ds2)!;xf+$sVMK+^uiKLyAfkc8N7acOyZ=xm5o$8YET% z#d?+0rScJkwvSC*E4}D1vRCm#=56pdrv^Mu6qy!G5@b4Kv=E=ae()W`W7;tnC~f;r z`meu7rUPblU2C;xk+$-DWpQDK zl0i&DG0jkXw~M(!T7#$9h9k+dp}9#CB@5-!wN2mYsK{!Oh1%_ySTB@G1q)mHo|-bt z0HpYK=$&yb7B0-poxAI6b8w+~VBWGv^fQDdey_Z5S~kVCLpcT&wV{JydES&{cHlps z|E~}I+P|gB3}>&uyejs;Pw7S5qELA#U1_K-wF+<51z}~FKwX=ewmx?X+5&Ha?>2G~ zBlpj!JfydIbs zYuMd6nv;_!W_HBhEb)-=oq{>7N`FZIp!wn$u|{(evbCmI@DiDDw5+gW*uEyl%IgQW zS5(UT{=d&Jp?W2iX3IB&|M#-oJXXt7yq43qaWA-A?+y0aou(OK;@pmELTU{*cg=Ne z_MzODt6>A4U?oICUrSq0+jgbA!cZ#8+cUx~4u6#3jR7Uf>V~6rq z`|zbMcDP$fHdp$d$c?#GU!o$G z;ym@#&dY0<{=t7X2(XJs710+`=_;f;FG@;Wa*)9-nt4)mt4adl zMw;@IMO9K3pN~D?1(5kI)9*eqeQf$<@#}d}J|jPl;2`6b2B#yxIDVhNTdMqP%)32N zlC<3Lm=NT&%{ArG+k>v+ePrqbu^q-T<%N|UsLN?!e5qhV);9j+md zTBU@0Ld1!mT^_z~nI`55D2Dc!vMuE8(9%hE+%d&0*+VNS+bVmgp&0u6`2Mpb#?LUL zPbB$}$Vcex4gR%{vfmOus_hNZ|QC~D{;0*KN zy!N!k`Qp|a5307EGj!MOIgOH}UUYaZ>z#$;3gOT`5HvzxV)ZxWPs+G*9dH7la{c5z zCTXs~Dk&=z$o*>{{3j;S;C<&iZaaxEzS-Y@JPbA3OO>>^qM;Vt6yhg|Fjrr~yXUk! z2Vb)fQL0BUNqp1p8HavEMzO@(j94QdrzDkq%^ryE%@iBY8)cD#&@s-E#B|1kFdITt>7ul zE@ElCe8=Cv_f{ShcXSJ@vEtJPR-K}UzfB3St41WLi-ozr`@&XB(YtpgWgvyDAE5(L zlMSEpZ6BXmD3f0Frx__NSoCxO0ZF_=dB7qmElx;Ob{nj|v6?{3SHkIWnqPuiyjdt! zTko>NA?l^&l~jV-I@sUNr9Ht{`oSEO+)Vqvn&;2m{`+=N2|zOMv?k}?{^P|X&N}(M zZ&*dAkvxx%6QzXXEc^`bAG{5Xd*+^(Svq*DH+6#-adLd!(OypHfaQRXtmtT0NKr_q zt3aD9b-pr-ek2=mv6it3SNlF~KTsL&6_0&!h)WUzE!HPbtQEkX@l?k675PzfUy5>7g;f& zB;CCP^aVVRZ7Tmb_YCiS4Ap}69k?HD(cv`bDS_NG?2X`&uhq-!>u-*g7cKQ7so}hA z3iu7qrW&u1u>r7!5?^KxnNw{D+meF2yPkM~aV|gGZawceI_C*VBoSBX_*QoptWm;} zAqDq8p4tHM-P5n+51|u1;4_+OTXb#WMt^72*&s1j2Pl{;C?}J!a-+ZOEA(hk$pH(c z+@?(8{8Z>L|Ik$8_0kuOa>dTicf@=D_lT>@_&8=XBMB@suGRgq=Zw)YcWI?S1O;nC zhEGCo>lzNvo4KaN3A%q&KiYSGW^MN?8mP-qleKn{$aFv)buzT|((S24HnC@I$S}VV zuR}>bd=pRGr7BtVsV(1{{UEDKm2PdK$HRz9RQ%ssTRPk>`{BmYd^T#ghy+hoc4kEr^4 zpK;zr6TpXK`HZ+eFV6aNg#wX4w)r9Oh{emdrA(pcFJ04!LEpNR%Ta5FZ?@QNn$)8^ zs6?b?rD0zN+q-ia$xZ?p2W*5eb8UONl)apV*yC*!Nm=2MREx4Q-if^aF#Sl6$f0|I zO{A+@JJ*gMLZvz!$KpGhCZxSeN5K6`w%6y8bhy24T+qtIqU5N2{asWGQkI^#J(Fc42+7W}Q z`jE5b_AeMz4OyrMH{K=f>^4Ii&BL{x>3yiUB6h4GjzNJ(Ez8$%MY8>K zj+$#Sm|Xow!@ZA^M)F7Z&7K{Bf9(ZF83}*2kFA64)CQI~{&Iy}Dx_z4Rf$A^N*g?< z0sCrY*r@Y@mpAt1JY$A253nF;$~%NmEPdC7_w5LZ5)2sanhAmc+*tJ_ zmsz*=V8g?7b{&cXV6eNsW|`eq$GGA&P}wsb;iaLH)1*uuM~`S{2lEVyd=`0{bBBaD z#d*>KUe1rCk~C8uTV%oSGVW*d%5o+)Bny^{iUeM;<{R% zIMwJL>P3!Ike&q6Rwpu`aQncTFe!V)`p%25pys@>F=Lo0byZrVv((p4~wyt1wHFqeM!9{NMky> z-0#!==fD;EOAwo{fmeqPt-fiL;0%>5scLoR3v4unwh0ClfQ50ESE1ezi``>@wpCq- zLu^fgIt$QES#6v@C&nC&O7mfOU?cAU{8K{jeMWCJa*zHAWPmFcM})A>2zoHHCexn$ z(<$dI^i~UXg#XoK$)&DEZjN)hH{e}DXavj>2Q7w+HxV==NHn3RElW|*i2r6QEtDdNW`gKm~E#6K*~bUY@`NcEa`N3uUS0%drwjWee057TA8*#?B8$M|Jkp z9^e#Yh>Y&vYAf+vX1(FV4u9VRj!JkWA>xw-Ay`UTsxSlvwHkC4YJTo7RJcQ?jJWnN2kXRkI?YyA=F;NC>d-`GTp}2qj5Z5B#%b6am&1(-n zIOCqSnCW{*tY|m!eM%|3JcwHg{-EsO2Xh)f3DFFQWvT5?5$`ccapFE<3@@5*i=IY( z{;|6uyJXei{QCe^NK(6Gzh=oGUoThwB1zABtD9<_rg$W{J3VDs zHo@s5vslkpsm%>7;+%?a`aint758%$ysbGB52|s34|RZ<6j-_;MKe>l^xl#|j+DNE zr@>4%k=6`1pHxAj|<}5}WA|LNLcbLr|vv;wOgF*?6DHD@i znjn4nID_E5!K$hot@pk%ij{=~c(EIbJnK?KGgU|A^b0Nr<#B01jM277r|_(5 zl3nZVS9=kYz%H?n82765r)} zf*3>J1AbM}v$i-~W%0{H7(h2vvh8#UJl%Z-Q6tU~j4NkX* zo+{RoJRsSZ49=JvHow$tmNWrjnc~Kg!MUoovO|4RXk-lXM$Vp^(~+iITvE@jrV~`! zv?f6v#EC;BDNjpKUEp637mG6Mx-d6--x1U%!j8j$CUx)Gb>4YC^|A#h>%bG6F1V=x zE=YB=LG|%+h5g3ceLLe9JN2tI%Qo|^Crbj=K(71BWWoQ33%RBd8N#dS~HLqx)R+?LL^@^;f$@G-xfJJw-~svDUza?E3Aq z`gq>GhuT6HE>e!5IvGs^CSe+Q*?;FBEg^WuwhVq4s*Rk!^5}r5R5F)-jN-@_eSgo7 zt@t3_gh0Aqd2gx<+RMI9vN(8ckz=87-JDO*YF5R}Azo_pFmxGM(S;P|bD>b?+>omd zhdDv=`y6s!X*jn#n9P~SlcJ$hb?O09bM_%pBG(PLvweYZA*uziK>uL0qEQ-z1O=v( zv7H_4Gd?fPK3}_N#)PRU5L2tn_SjKqPh^ori&*v(S7&&wjcI4`4qWl_htD8+(lOYY z*;89_t9COR)auLgWqILi*FIDGmN{IQQ{baHvHl;Ty#Z;RX#c>6OEA<l|h{nBQFTlLjn9EZvePFx#do zWql5G#|Cflco&<3BoervW=fB8k)ho;T$*g+waFKM)yOqrJ#-MXXCVyx9`0Vqb$IKH zgaLP{k2xsZLS2lVCsfO|&^>tL(Xs1-B@j6zc+e{NR$nX06!&(;$*F6_p1kQ#DXILm zl>h;^Qk<|2Mjzh<(5WcG+RK~e1RzMBiuwBhGB*9e!01kO&CIrq)-(_(*%5p@8&L`z zW0LuWQ}?-}VwcAKUXP249bkH)YS4Rg7q);G?)qf!VpY8Pdy|(3LOLa89Wh)$j38nH zG53;y%V^&NV3X*I02Ezb;K_!_dT1<7&+Bv=flVMNtTVrSjTvDcDa zxA4Rt^buoS&*NSh_nwAV?T)nZ*Z=760io_6Nj1S;fcY7#v6zx;ldae0e@Ds?T;y?A z4wgeTOtMD9`|I4Mp6nz>p*6g(l!-y(Bb;%Yf8`;Z9B36QlFMd?{MBF;psM;Q_1s*8 zyEtcg64?>iXhTKVgBQA3i}MNru0q*A^>B6oAmSxbT*mI% zxYKV7akRhK>)cVt;Lg5Q&@G%p1Ix)+2d2F$NB-6X&O$s6K!THmc` zqn4lSb^N0paA7B)=lTc8a)0a-Ou*qF4bJOTNYY?vOv&f&wHoQQGw_iDpRo%mAWbvf z+#p}^<{Nt9jEiD>k;)Bj=NeGrs%iSy$uJHS($Qi)vew?`h`Fi5l-WuFHO=KR5BLyjfx>^te z{W&w2kKV;@1K;k*;m!5yh0?WXw@H(^wS(C$qju`XK^uQ;ODhhPX1jj$waMh+ zJ0za_X4gY3M~ak}3VIKC=o2T5pWv1uze;fuZ;j4bl8 zy$qs~i`jk%+Mjpu1yM8|eS*?M&U%duS5yH?1!84xz&=1JRlj0hN3T7Y)>QdkTh~^s zlmzj_)( z%!0|fzlr1*+H~GJd$}ZQ1o)gAUn~22rFW2=Si?5qj@OkA zw`Gi7(c*1iIu%wDv5>B6)LY^q;r|@3-7#;X?A7m2p@Gmuob?Z20)D(db)T&J!O|>h z2I~ZN+-OtJ9u0SmylFN4*nhxWqxsKOoOI81nM$erW-~N^Ah_BOnl_^ zOEtd<4*dbrcTC%DE-fZIqd29gEv1ehdx{n+CE)0S^Q98D|OB? zi(;;B<2?0lbQ0p7#!mW0EfG8H;#qyC?!JKKY@z>bF0)b5sml?^-SM?lqB-9ATQ}fSbp#?=GMK9xepqGN zb9CzKEHueWgT8dM`aD&^kFrIl4nUJ%^v^G5i;QeP>~(&R`UrhJ;x+7UxicKvA^;et zy|B_0Txn!Mm2B^~#z7+{{o#wXy);W6u1dWC791xR0qN&5dfXc$+z_-kP=4juYbH`b zfq$sifQ?R`mr=H24g8i9k%FTb{v4ufM=LgnSqub*FNKlhP zhYRd71X_4WtKy=nN-SG+bFqX{)HiMNADN?y3FZV#8%lgA$#sn@!a$*%(`MRO0po(C0Ht3YS|`HsZPukV^t1p7noxTyLc#G|7K0uTSz z;2QIFLw6Uhu6gRHcc}hbyzlyG+QfSyZ7iKlK4)b;_@IoKe31^eoXe?dTaf6{OHgJB zYWefR0_$i!vwOMFyT=KheWI3*o7uJL%qVpM!=BN@`KQunJpM~VgwWyq8WzEI;`W$3 zSASHsi0#7g0qJ~a!<#P2*iV0X3@=rEeiKpNk-LqzPqY_BR+V^a<*6G?=8aVsvwvN9 zZZUFiy;mL{4!jt86n+10J7r?ixBkIS9H@j;c}~yjWqpW?>Ylt0H(BDE_F5kp`UPnF z%lI6{9K{lMy5A-~bMm6BgLuW{JW{kw4!OW=0|}`gYJyhl#n5dCK_F;kb)cYC8(04bC&%PW@d^A z@dLEt&AggvY0Pi7%R^(NPs;TLLe+)3311mei8`#X9`f1-reo}UMq>K$ufsy(`vy&* zv!owM<-v*m5s~%E$SFyf=O=#qf)3U}UMO!!%FA^52h4~cn}Hwovl!hnH7zo5Jk=X{ zliPkB6~I?_NB}oQp@OO44qf#o6*31XV!7+=hFst86xIzVx!bW`}H_7Gl~&|esDf=V#Y%@ z;CO7-#rUb`h0wuYfkYhb@G&_{o7j+|c7dQrR~IO2PoKM)afgWgu@4tb1_tS+QQJ8B^<(RlHGAF0jvDTJM)Q_R0i()me@)Bn^RUuM#?Ttu{g>mB-s z0I0uiW5pp(N&+36zC_y;RyC$q>^$;}ep*1G(tgVk=9S!bRZRj2j#d;NVOVyG(MU23wM{8!z`8QO$C`RRo}K0*rfJ< zQs~FMkH6#L-{diWdSH&-6;DG3FWadE_De^4h-@lZ#%0{rec&Shu7MWsvJ2+8@_Xkx z{=N&hGGMrY#GB(3Ad%|$n$M8sQwlaP>45{$efO|@Yn@`p=#YwsGAP!x92$?z#4$xN`iyTfK%J0 zP{pi4jO>_}xIg{eA}8c9_3*OJZuG~JefgbPugp+l$GSqjCyzSLbHfjKZ*B@-d%Bij zTGF#X`3_ximrk<0xHA>QXga_9{y^u5ni^Fc`|2yt^DVfnPh&pzMfw-$N$I8O4&pgq zhx;58f|<>7QO%kO+akeUgRM!-i$WVRSA`ayG%A*Q6K3Nk5gn>k*eHSOUI`~>kEuOA z*9y$!+_rr^pb+l73j3HPoqtGGGP1=k7!G{Co2?kr;M7lPKo&ICqEWsJ+a?Qowo<|pLJgu8BTb;>Lfr7j5 zki0%zO?DK_-PMpL{wsvi$x@RYtuKeYoCJk5+-B7b2djc$W}}R6P@7{?ldWHL1Y%#% zCQl{2F2s$nXQmVH$j83!&^4OqMFhoCY?R5w(X6qiyFDrF{`7ig)-S@N$Ttdr+#@BzUog}|& zkBU2GahuvNwEer}0)7oZ?i47nG3CiuT=x9568me}TevaV9J$J#2`Y|c#(lUG3?%W? z8kf?9C14yfw=udNT4m2q454vhuQ^f#>ffNfE8{4mUEYVfzT}qiAs!g373%mus=hm( z>c9OTNeba(9Argh<{wj;WcJgtAT6DC(uS^@L-y`>7qE{mC6!nA`F}7 z4);)1yF+rG7BIMZ7Gh2#lB^n5znJAYrqTI54Ty}NA9Wre$oAG6KP@C)iaY(Vme~T>DPGgpcGFt7?ziF zez){N?Eb(H>aN=G=esIaH)yg%IyNqcgKqXlDGt!`KRc-_A2i?s4&nT5&D}hXz}*mC zV5mIQer_4s-}$1Zeqkz-4%ym%@{}aX?w{l}?t&a@eQ~0mw5bWww(H25D5vB9ly!ch zSK8Y(3?IyQs$E;d)oWLZ?=v{Ue^Q$;g(B9MkgtPilcb42%g=Udo)mqG5a?AF)||X` zjI5l){VkAgST!>xzzk3+t^tlY=tpxjLX<3WPb)=5X89p$}Hy%aF?Q#{Z zd`1s9^jW@i0G)D(Mv7QF8ne*`Pbv%!rTTafoT zac(KN`2yTkbN82M-WuG<0YI>5h+`-b13F6&XXb>Jzh^}XKaXu%;iGVSv!LL!>_1!z z%#R4@UiOgItdIRk2!JNtbVy4wadA zv$+q~CF(5e^XKy0))QQeJ8zWC7>O-cKs>cGf~10Pgg$-z{yw{XU-0M~EpUe{ z`O0m<_-CUPS!eFeBiYkL%{70Fu=zF4e0v+M= zV+CV8%Zxpx`XdTKmvzkdHcUOC7JFS8Hm|cbpl3Lmc&1ffkgslvKyxGY)3tby`njnG zk}zFC;}Y5uZHo2kMFI|*tS$c;?c|(^Cz@Se5KPoQv19VJqaILtP0b2DpvNs7ZBY+o zh9tD2l?)c0cjP;A*HXGgxeq=7dmZ$r~fIrW_ir=8?Yr=A1pH zWJ+VpeYwkdqg(5FkNdArEW-GsiT|s<3s95VSPy=qFyl=`SBwn~l9C-zggp z=t{eC_^gxk_k4$`tf|_?266$?;kI&a&(`r!19-qEa&pdh_pB2w3L|LPc=nQ(y@%%Q4>_Ep9X*QF1dzrjX$=!weCz98GQ1}tJbCKV&Tm)9+zZL?& z&t(AD@PIU6VZS3&a6MUdp*;K4I*s{<-mdd^*96Rk)q6dE>3$w2;#4lMB;-hkc{PV7 z2~3ZjxMdeI?bvYMRky=yB~bu;4|@viTVGn`IW@Yq44A+!^s5;=T@N#(^K;2p3m=Pu zz+LhO{%a=hEYVFmaQdj#BU@r@9FNTN=WT)>>biGL(~Vv($^Nb5X2iO3DIV6}AGaud z|NBPL`np8KlR0;Ec7i&eh~LdR)^brnYOHsae%H}lUBqE@D_ zvN#6aXeo#mKt#yCXPEmJz2jmG(sb=}B^vjs{?*4m)4y1*o>kpc%rZ=oh7iTkQK z<`HG*tb?<#zI>}zj05Xd)l7npknh<#n0RhjIcpJL-j(A_NGP#lflZ*oB@bla|Q zc^Egepw~X;Rw$q#p*GgM=mkejj}aZ2PazcYs99bAUVD&ey%-ur?Uoh?s zaud!j)L(ZS!%vC#iG<5uB!Sp+7I_rg`$!k5%qW_>ODYG(3BRBZY4iP#6xB~7CQMt~ ztBg@AIqH$kO!0+v(6!?1&`@zA3q|%VzIj|u(b1E;ARS{SB~~k>N6w0@(oZJhlL@UB z8N+K%8>L9MYKgJv6=%rj|IN>%q_1`NH3;+;7GfAVQpkcKhM;SA0FXBr{ng4fBlh;0 z!%zT6O7m@y=XUoHPgA?=SWMpzd^%N-fjimCCF{eN_1Yh#*$uyaA18zjp=^J=W;<|6 zQj#N8keAspUQ>eE`3=3gVb!B#QQxtL4KGflUbzL#O7)AN)y&ZD`kg*pQ8l~h{vPv3 z`EFsLjBpTbjVR4O8TLeD zKQ7Qx7D?F5h3>o`t}={T+T|tNzQkYU`bljrf?I7uBVLUb@A_tbofvJ0eWtGFkJG%4P@B~G-eBoGiiAC-CCKJ|**isMpL2G`pF%foA?duY@Bg{zgnA2!R}9n)u| zZUK{17?DN41o#UOst?OO%wPi+tG=;|uD#!oq}1+>e|!@ju{Zo|f@O4qh^EFA-S_U1 zcre+eVpJJPnj!+nTqe!MJpuP~5|8~dfq>S$(HqKfs#+P~>Ktl@heWHa{wFG#=Rr-- z>_g!d^&dGGsh(y2$5%nivAhQANve6Zv|LO$pha!erh0Pfz)WWAXZhaQL!BOZ8X9Jm zaV9)}4?X})JUm2T?_Uk+sj%jR0ti5;@=H3y=+b$fc3*srx|oh!SjxDraT>pJKaFO% zhr+tD9Z%U67NNGfayL1GIyyr6N2a;N@}aH#{hGpxuUqL!HR;0s(LvoAk6v06pD4+J z9899pB&Ebn+pBA!9(qV;nO^EAAtx_lWaM3kS&!fV2e&MObho_IfVLBKt(6nt6t6fMh3C=c{&7Kol!4$V#)&Rf|*5iZ~`Z?npvh z(X%s+T#9t6;2NPh7@}}rM7-_zlaqzSZZJ&M@ISW3eY0>F%Eq?;4N&Y*Z6E2hs{B_d__4voCz9+0Xt|6xWdhjXcp$ZN~i6EkVu7nk;0*jE3!)~_?$g{o{4s$nH!E60=>pE3E3NM0}CYC7+DU^YPw1f-_ZCMUn)5UU{&@~HHxW__Pk9VEaNj$fA z@#kNk>Z7@-$k8wvwqyiBv-Z6^5dVh=kG&!+Lp&8}yXReU5mnk@1CSW#Y z_meCC=owyrbD~{?Brm~=`?L(-kS6UXhyp?hiUGX4nV3=HdP-b#*>Bb6m12;=BG2pa z&_j_Nn7tJXoLG$3w~C99C}f-)#KvQxZ|<6LpYe|n(XCJd3weRvPl~ysWU;OVrk`+4 z!G73un!S!0Y8r60Ym+Ng9Jc@OKVA*np18@lCMREmBT1XmGf*GQ=vsE zZh1M)L~6ySr5$;m5YS`X0;5pdg9ZQ#gaVhMWTQ_BwG}pqh7yWumEXiP-s7YiuZ|3} zV_@LnhiN+vYO6D3xenT8G+&H*$mPgyJ}3jHsFkxYHDjGcxU^(i=3jW31a$YDx8qTh z0RqA$J@3#jgwJ$=E8EPGI9E(%aw9!*B!?eVI5hRmCGfK4QDRq=bGGIEC7kbv3SCPD zg92Y^NhIse#)Mgt zXL}bv`E6+UYvG5ANBSbDd4uf*gUo1nJ|_E^vsrE}RET|pU1sZD*CBZq z*B+!S-ukNO9g855G-rxT%=1mvkLWC3Y?@T&W_3;9EGc=YoUz>y@+mN5NA+}S#qP30 zFP&bO!Gy13KGZQXLP7GO!nA1-RSTErv>LGNS=kHR5Exu{yXq;|BAB;@&|5j<1 zxfU(PPc-THIPa*+`suN`-8#N#2@;)hQ1#!(A}chjLNjYN+EdxOlQ7462Y?T=-u3t7 zghC`OrGcwWzR?$`+^1fK@-6`dWN(r6_RqoVYF7w1@27<8$$m0MbhHjX1_m<0yTmaT zxV15^)-QZALkYE)Wd}n?*t=&@U}}0WfT5X7SKA%xzBd#UMhpCR zF-$}lS;iy9E@N|XDWV>p6N5&ZeQ^({aVoVw5W-EjxyljZk(*X!bx71bKEiX!Nk=VQ zJ-mSMwLI{59fTwnmV}s+-f_GTTEP1teCCp(Z#S-Y|GhP&#kKvyo71AP!t}avg^3iK z)Wm%!!)X057qNh%8{n?~opoh7cQWZIW~h~l{ALjFldUWH9MfbWvw~HW%TzRW6Ig{W zF%5$C>g-$EV@FW#m!AbLhBsB8JN zo$Q6wyj6SOvx6i%$sQKOZ1|w5>H4xU!sT0yGJB=t<9jHTSfin%Ejt|Z6`~W^BJ?l$+*uC}uqzpi$CwO{@nNr0{wE$>`%(B8JtWD91YyC8t8sHXKopfJq;T& z?`Z#{LB@EXl)aJZy#DX{t`8B@RJeoDo6C0p#i21|gokvKMy~-&UWCipf2HJKw?K^eJL^i>!ISSa%N?rkM+iZ- z+r*KOhGB=}1T_H4Vkw19JATOW#D2k!I&=bPM~W&iN=NJ(I&&BCsynT=h?tBGqy9G% z4L|nr+%WVasU9@kI} zz~r>uuls+I{86*E?gSjQ7)qi^_s4mzhdFQ$bul*W%pdgS8!#a9DQc zAs8xFoE1#syHTFG;-YSE__pA<`T&ODQLMDf+WmrEkWmoUY<=3Wej?F;;%tm*hllvK z9Vj}jE@D6Oz)t>Ek6j{{fM~6vUieKt@bPSf5L|;Mtjw0&#BUOrPxogC%jjVe?9dzzk!w5l{${;c+fNxQZerLqB28x zcHf_A$zca0`>j{W2azLcuwG)xb5~}0@NW$*d*pMV?A{n@-D}ICbs~l{eCtNK!bUrk zOi_>5=l};M(ZVn=o9xuAmf17vJQ_pvC4q8z$)(G8T03>XYdRygSRP3~HK@yrk;tpbnZf4qnGT-0gT#_3O|XG3h^S zRllU}&zZqoE}Id&o+_{puEiYhzqrCgaav>R4nndzvr6&RwZM%u1eXk2tOrDSmWqR`C{-0JuJN+;+1u%ytE9*=~cOHdGz66h9IY4w+V2?@%a9 zw`QEuERhQ~Z8iXAH`7->s>6*@^>MqT)t)fkIr`L2VC$ly#BLeZceazlxJEtuM|9&+f z=OiK8__F!_ZRUJFN>bNVzH;DsD28J{`$M(>+j{&cqrHrh#RZG2hxbx2#J zLl33H#c=o|Xd&<874(P!c9T<%ar)6z5=*2;awO8LKS?@6Pja%qQ;^E95*hmNkO3i^ z>K439r-a326cz8Er2PEaBU_d205=I=parjaj@fF2Ew?ZcLs6a2tM_>ffF9}J<92z< z01*!B+Yu6FtM2fAk+;Dcvm^? z0K8MXoaQHDT4a|$z`y^lu9De`s8>>vszVtH%nA2u)aPS|A!{!gpM)hDg;pQL&yRCzPg1GB;{3ZCuAy!yb!V zZ(p9MnErC`{T`s_vhI8C%~qo>@XL@mCU5_-#1PLxk)*6@!2*V6Cp^=s$wEXSX77R1 z6sD1u+H5QAl+qQosYnY(F)0Dy;42OyhQ4`9iJv}SkMq9)#m;k!;dkYv_pjNyZ~5XD zhVBhEv{hXXpuaeCo1ACzR>;RmgSe7hU*I)c zk@p|tqowYL0%-sRMf;v>TFJbZ@Cj%aDz$A^j1M;kjmMReQ_ zj9i@uY|Gi`@hohzdch}5P1w^=ZD>rO$2)(oD$E-Z-gjKZgNX!>dN43{-6qTh`S8<~~)clyaVr?Qb zq~^aQCXw~06{JDevY_ymFZL+t{=o~OjPA0cr>HS_D1oN}7{#gRFiuGn=QuUw;-?9uH=^k}S7dESH0Z!yy{oPr=?hUL|Ml2X0Tn(QQR-~V_>jw{okwrY63 zoD;pK>@G-Z>B!S%Y$bk^I8KHv$)cuh(Xi23SzN~36?Jf{b$Pe=|MN;PbEYSb?|Is_?Tw>ZIv8z%esAetNI(VbRg-?|Qx3Rj4Cx(q6)Hq$%;Dss|Vl z^{HHu6C1?mRie{EA-Pf`=A=h}4OqqzpfPDJAamVa+)b3H36#EQ4i?56&wd6&Nxkto zat(JrY+11G#q5&31M(OsyNU>2WPc$1IUmRd^H?{Ud)#^nDE)f@nGq0IEm~gT%kA!m z#DbMh-35)+N{e1nvQix1iKuO0>fy3BuYQFVAzAr_@c3YQ$FyR)a9C3V&&_!%^1EEq z{8Om-LGqJ%L~2uLu_e?dH_hY4Y))vC=356T`$b8!>1d|2-hu-ADJ}6in;9jpJ|lo3 z5sPO?n88Q|+m+;lMDis%Ef@^;_a3(u;IRls;xfIdYG>=%1W(ccABL;JUw=ESl{r4m zITY6{^wb=#)sh}E1#%uU=73R`u{VLsLQZo@`r8H70E7m#KUU zNW`li!P9?#q4dmdWdDU1@fp0u5a4V^@FErA4ib=`%uQ#BCW%SRLjW!e{$0B)jgM^5 zmBMQ#F3$S@)Sj)e2R>#w5mdu_ots=N{sMfhwV5bB2NfMJ_(gpb{2%QdBlnQlg_+W2^9~Cb*1}Th3>KD?!-QCy38R|MUCG|RxoLT|d)y!kzKdZQ0?f`Ge^O!d( z9?Zjt+UDh^9R9vrTDFzzcumM}QbF;rnH_x7?^Uu+4JDQF53aGF+N{uABXFEG;QR=X zcy;}L@(i;z5zPmSlpd&IL4FB!DhoX4J=H?ZI{w%Uk*g7)9;=B!qj%hw%Py!6bZ4S- zpwr#M$!kqWM0e!orRe^@?bA6&aY)F!3{a&-M?KM;1(I^J+`fF-1K zCD6nXl61X{5ILj$1;iQ-@2-2CDF~iGMpBx6uLWY7spf$k1-I7AYYixUdiW6Ks2#kn zWCt%^z7*zNB>ZCgv%?lGdmS(0s*(VLQ@Z!9lTa@KtyOIr;8MMS&XV4Q`M5Np26RRIo~yiqaw#Z#y7DJlLK*|PjvfiC&& zLX>RHE4;O823klI({)pOdSR$MVj8E*5pkxQ#r0v5Q6!*=v?`Cu zg9p~@QiPlymyFawrLl35g^EG|DVN)SDpK^ViQu55byUS#Ldl0A<-zczrf2`M`Y#b` zjGQ=Waiw8*0})({OT`(VA&xbtbk~Q!dW#RD!7B->Z{G>^i)D^)v6fypXl(}6(%4@Y zTfe%Bu3Yrao%N1IAAg9xz)Si-VXiHl!GOPbo$JplZ2Fff%7&xBlKDs1ShSh0dI7)5 z4G8SSBS}2*%9Cyn2cV?^fl-~R!PY?TGA^1>1JGXYuD^eUade7^kA&b$W^jZ$1fFL+ z+F!;*xZRv35Ct^X@f|fool)arzp`aXWLj(cUo(%RyWqLs1fT`|^mP^4b%r>v4$8>x-SOY#pI%W(gUD^O@bf&$Qttx zTJP$ldJPyi85z7FXYicz+|yN(ey~BQ)2|9IypRolD#K&~L**%@ z%g$b~$@X^l|8eqx;L`^7`ehqNmifH!eZOq0zzm*hD~usI4l)BgS!9pV;d2*2T=T*4 zVe`$0`8^TMS5SKq^@~9=*Ca3E{{ZVG?qPhiH@HE>72mpmb$x@r9)WZ5p1Z9;+3vvl zv7MR4zD#b@HzQUG_PO~Zm({%5>e#rZ&}u{z6 zDA^rg>s`qF+J3?1sS<4ahO$f>Pd)nU)W~qnI7j12+mdk-nRch;NLI0cK?ne9YZHsS zebj1qvs<&*{b%_abGxLl8{?T~fQd7)$T-d>VO=-&SBEm#fz#w}zcJb9wNW-Y-3Bdv zkuK+BgE27CKACZPxSnxhe$UT8yC2-bi@KFp6{FPQ98gq-P$C=A{*@)JY!4lVYV30_ zuZ}83pe5{~COjOTq)}uwV0uX@s;fM+?=RPPwbq)++0$8KjqOFMCcB+#6xvRE)G)FT z3Mfa5a7t!$#4clo0c|3ABj4fwXHxpev3;xG^EqO6YBo9;6mTp$iH|3$J$-bH*SA2O z@zGM4_Rm|OLP_m&aQvZ;F^I&6H@TVMTHU?E5C;>lY7aq@v%xSg?Ta2TKucE&Xki1a z|3lUd_39UZ!=}$hgt3?G^-t|^uC)Zmj+y9pv`#C<>R}unoFaK&V2pP&*l0}=AWd&)MY&1&%DdI zE%zt7dqQ&ea01*DN(F3t$5FG>o_j{HjO}^teYHuNx2uJ+ewz z3a}c2^wyKEND9vR7Lfn`IV_0y#AnhY6m|bbgxF|jXWWViG!T|(_k2tk0%!8qQki+q zXx_QDwk5&YDDl~x0lZ*2($1`=4_;Z(z~GJ|Oc6=wNXXfPM7$3qc-)&`K0W_`s!`%sR`ve1GeoQ=I=6M7k=S=R3@MM zC)1ETm9Pvu;jdA5 zf=q}IHQj5X_Ds4R3eI)Gackf{05m$fIh`MU0EGZTgn;Rjpr?;46S~@6cOp4i>qg)L z?`->?BI5#hu4OrKgw(pkb){3bW~`n>MqRxW$3)2cNJ3?-Te2huJ4Ii&(7j|sM_X*7 zO|QSd_EvJqMaquUnzF!PU?syZ-x0LV+@0zrOcLp9_R4NV)R9<>j1gpfe& z!<3x@9|4J!@UUoRO;(gk=8(4@iER^Oi|Tgf+57nQTrfbscvy zz9J$5Tx}2SrWe~ZF}EJ-#+kOk`YR5G_x!LN0Fe0i9H;=pMO6LeVyx#nA>$q{+j6qB zH)lo*PUig2`WQ#QOx$wo52ZHWj<&7r2Qn9i^!9Ds&q#cQsW5wc6H~YoMqeKG#+bG4 z;*ZB!eRhDELT76)(oxF)0t@*ICpMl@o0(qhc|VNvcAt6tZ;>FQW|+Di=~pWi1)?k< z8*j~k$FF_9S~wpjK)YsR!W>QUX85t$V5_Tum*IUE`w*aow{hTF?s#iDk zC7FupNJ20AdJ>Jf>q;ccQ907qy+!fp18<@IEHJ#S%lUZa$=o`99i`@!JyS+KFex;u zJ8*53QcER1t|L8S*zq5|EGaI^T z&26%v%|1W>dQ1@&dXeYWE-ZPagW(y;J4m~tG+wfk4g zs(<8GgDMAqY)xhuUI?A%QV@^BW=NJK=H&*jj;{Tx-+3s@v<`^-K)_f9$LN$<+WKVbb?Q z$(h(}KAGR;a*V`Xa)O~dIc5C^fEFf2k_fX?-!AEVZ=JtPwMZ(yax4z`Q7`-Q0AQ2g zaLfAyW*G<%RsAavzz<|63P0EKie@1q>nVB7rA<6088Qg(R8?O{?$*awk#N3F4%Lh*c!xrI0$meq7T1C5U==WaHaM$HTWiqF zc{j9pm&V?ZGAe4gA4hi}9yA`8CH+hwzv{yjtzc?Bz86^4F+GW@ugFTisU$|%9`o$n zA`zto6h1^s13d$^FrZv%nxVl2YoU%UjNwsI^yWd*cY9tS>(x)sO;DcSUnpSWR#Y06RNP_b$83O!@VaI zDX;FK5~N{u0~~A=`F7aD$*s%iRAcNh>M#fUP|DXZ8y1*XbV`0YFngm~~cKDEY+w)~5UmeW{h8FcGf)&d7Tq zoUbkyH-bb93TC(BU;fXx3=nTu35Et{GlbT<1fMLOMILCpMUShUU?KZA;#Cbbr`lZ0wo z7O?uX$wS#$RSEo{z@!AMmWl^2XF`Oc3C>6v?%eX&+-7kwTuqx{6!+^tj$td;2*(J< z#5G+v`F_%6m8T{fl1JOUQeF(q`(A$+o@nWvT~2F2t}H3J!nhefwf0Xz_Xr^u6RPub zh7i~_VYfzX#9sgO6h8&w28yi6Nk{c2ELb>-AR$3R*PKkr(hE*hQZQG(Np)kcUm0)3 zzuD}>F{R_8j&`zyQCTqT@KH^bj!TqoOw8r&BE~{^cyvMZGxIl5<$o3fOq`91T2d_v z^>Ai42{v`6W|@bI((c(eS9e|*IblDdgZ1>`gX5|@9h_@gL|Qv_ZDR8s1%f@Clbb5#STkdRTKF%ZmrClvwCs17-?6R_h&!#RpPY9d>uI{(Ts1g(k7F4i&3 z&D$N);5Tzu%GBFL*$l|-di_It2gSt(h3NM)b${swd2o<81_LL3aoD=&69k;F@Fm964TE-^%EOQrm z`ZYaU9jc^DPRrrX4Dm3jTVlwPl+HlfM+rc-b;`mdx?V6(UhWNIEU=LKGD8xkj3uMg z1BOf?M;}{@$7~kg0P3t|Vz0;gp$~2^1fR$E1tRW`xq;7fG$-@-KGidQc!dc?MBOsN zPr^2(2LazxD8?){(1j7y3=fFn+7;T3^+us~@bSpA1uy+$)!Be0mo$Dr+HvWcm`_RA zXPxLlxDE9_|D;=Rbf8!FN>x@}#;EB%JXQYFJvXRJO^@BmZcV;B2!)%RQy5cFKImTk z=Mt%^qB6@@(%5_>F!h?CGclDGJ&OS7NhjE7*F=C9)lG@CSFsZ~K(xv&G zLh_*Pwo!+66d&w3zcP%)T`!+fBerD5=czanHXcO?u`fJt^VMRGj?*gyw)h2yLN`Yo z+T<52BgYzje$fiSu4tFu|5P7D6GFlAxFWlj0W6~jZVeI}d{n7gcMp{Tfmh|KTa4Km zic8-bHm4eJre?_;nO7ZTx~D=xkN^Jid5bP7TlIL#Rz!6FX^NTn9ishidlNN*U!I$M zf%%o)!bBm|A0mqW5e*j|yxpMfs3{SBk4tqAo;rL78sQPQlFdIT60(@u;vRm*A0e5f zv7c)ask5{450_yE2nlFlgos!;Q#{F|hqrkY@1ZbhOgH$#Li3h3J}dYfq;`wSd&-@s zK&yCrKS7uFN}=7L8)ut=yv2laLYwd87-#Up%Iu_Zx8RFBMZFW+bjfXksr~<^Aj)oz1y@UjC zXt#TPYXSuzZ#87EsD1{ggV zlMy)0-ax6n{DNxRz121!KIE9{&=b^;R58q#q{3z=V1YKEdN{np2=q%D+`jGr={Tim|<=S@Raz78Jr~s3wKT4 z${-q2tMh{R*M~xLehXfJS3jk7@4){BNN6b9>Om=A^ z2uSlF-+B)0e0$6{ZVlIQT7*ld^w~3z1Sd~1&07CK%;K>TfMd0)kfr534qQ_}TZJNZ z*#-P`{d(^MJ|Zb(#4hagn;iWDdu9)&TC|opq)j}6Q^BlEdsN`yLf1iUU0&louE4v( zqZ+HuB3(a$=f^?3Vzg`Q@g1#HZJ8?Mpvj0As3@@a<kBWvW_*#91<7Z zS2`juJecrmwCeVdRzTPBkAAO;a@o;@?DJfgu>O>$GdkRy@g>PxtcB8pym|U1>GI^#IgYLK&>VbFL+PFv@LINTO~9 zG?!EZQsbj1%|{~)bfXNqu^l*CE)&f)`56VB{>UZlH(T_Y#I@Y$D?F{m#A=&Mz61+F zLbM^LE$s;Lu(29Cd#axq$tQrp|G21J!ZzFEeOVA#HiEUWJ|TFozD`uwbgSK2-nto`HvM=QS(EsN{H}I zst{Ct?2gQRuIMn})1B)82kRLiMG9vSH?3JUnzhy*Ano5IP}`t=F!Q)H) zyU~7>GF2mZ3o!Jbqp&~RfV-3XW|RX!q`?7^wLr!swz&BwUnpcPyAC@}XVzaGLctTL ziSJ34{Cxz9okXI_;gU2aY7PAT(-uI%b7ACx^clj8_)bhW{GQO!7rjmoFB*z47Rh{y zYSL7SD`f7!ec;Ae^`!8CMLJ_O7Y3k748Az(frRkBTL1{7l1s)XFe)!#&{=krcC6^E zoNh_ej2#xa9~@uHd&(pQ=G*7cp{-78UA(nM5=lBRen5lc-Wq=4 zd^V@>=vRH(c3Df)Ph!z@Y>2^n*^HlM`D&|9C>IvcUi@Aua;nXjz`}aVl#F%=Q^4EY zxc@3=FJma~|1hx3Xs8>vfBgpdE2@4{livG-%Am`&InXTi8YzxPj(i$!Mc zpRev@z2*s`Kegf_R*i}1Wa%-DX=_%zPS{Q0!#~M})we_oTdXJVJ+6-*+s3#C5*d2Or5Bf&(M0B35CisbTghr7;&OE$``>Ei zW8Eo`zQl0HQj&_i?=EGdmCseko*OX1bd^`Q8**RwcpI=p-DcX0RCD=JTC(I{H=kECZEQAvy`t`g#$s#ZQMWcw9GpjFshMr#ekP z(Vy(Uqk+8Jzse^7P0QtlnGbJKQB~wIploLSZMW!pJDm_vBU%cbePKF-#rP$wx>fcR znM{zH+x$9+Z{0nh#?oBMPHlY0Eg-zG=up6*KUk=|qZOqmWfyqFY!aEZfG$AhU8i@0 z@mMiu3O8XM;7?GEvQKrGv~^;*>Fs?eXzuQB%kMSPq!dD^$&G)jlAaiZ_nH}pZArEk~GWr5?9uPhiH#ksc|4ZvtknuAyoMb;r9Wn0@qW8%75;#al_fOXlzbw)XmG2d00JkED-Z{KS*XG#Lm!pDKub z%5xycn`|rd)9)6cdhIG);+TozRz8|XyV>vA<~EGJe()FfdWQCa2?u@E)j7TgNmq2F zNqEgvQ;_d(NZBg>Wu!DH$tCq-z&^nUWfq6!iCs)>G8Kv9 z6Bg(TTnu~^v>Rf+I|lh8lG+`)E&^nK^baa|&B#Pl{SNjRm9XXbRT0o8G0fSJe%jlWdU1{euW5B;}n^(%6*;9w|3S?*T$DO&{lc%ik9*y1G zmclm|uU0Yded((czA;kElSq4|^Zy2K0A9I$vVYF7pDSx|WBq@giyuml*KHxm$oD3G zceF}VnO^spXlB*LJKkQEa{of3AyslT(JrIAdyV#uy?(fyy;0w7rQG0trd8N#`46B9z`6yRP>n1)axycPc@O%Pw~+GnZEx zrOeC9vXiPwk7|v+UU;k(<*i0@Z0v@!0`^RyKCq<6oZtnL5dVtJk=oCNMuyF~*+b>` z`%k)Wv)3Mcl?VHNvB#i4^LavYhAs?8i}Ds#Gyt}O{xzcgM;^DbL?5j`GI9M;o(lEM z%;zJYDtiP@R6HR6qDca{EM*4x940cDH=>1gN!``qhYIDt1`-H*ckGw{yv#5Hd~^UM z<_GJb4oXc*#@!C$lYqk7_m4CVl%s+41r{{W=-Vj&}&(vuCe@uOKT$Ahn zH>Kba2~p{m5@CRJBN755L^>4_28?bPaZr$!9571h4#^RslyoyGL3)ht=DCNS@9+7S zmoj$W*Y(MFn8e5RP#CC}?yCK_dM&2FCEfzulpC809RY92`&d1Fm8?dpZ|UmxzMWwAYN~<)R*J>< zn-iUjrsroZ3sfNW|9jpK5=ONvUM2?X4N)t!nuJC^lcl7HQEdL_-l+^pm|se6sG4}= zU|=*zuAN(bM>BPJTnn9W>l|A%@DNam(De%T*HwkU)fQ~Ho(WOKB)QL z*FlmX_`{3esrnjxqHO!Iy!zVTAc3O^NQ%!01T_R94db*WAg2`6Gtgd-T5(%7MqSpcruR1Aq{r=BvwK(36pdbt}`uO8EUwIu|u3 z#ME@|`8)$)cC&*O^f!tOwMcL32PD60=Gz*vTB(}=Hj>h#u9ZHFyR5Nt-~Tkv-!K&` zypM(bRT8^K1EP*48Mmc3>lVVIeHU48cmps7|C6PbPN_{N%S2!LZL9!%rI|$93L>_; zb?qN)=MqY^N=96=JNM!~hnGpQgMQwS?HNN@W^N=N#<CP-$4uZ11DTKsuq)+KHK*SPfn3uNg#)vDq#faaPLeX_1VA!Zcz&ly*TlLl zFUeR-0v^?m(hrKQ60SzKffD0O9iZ23JbhSG%VY1FMVsKUlZ}$TdC728r?eA5D4BVV zH&=GKR>tzE889|~fJH)w(*k*}UFLE@Cb9Bb=lI(ld(E}I>K1IYwDiMjPj{Ey+8z0N zE&PvlAE*O_!%ema9>#|*5c?iZSW#k&%-XfV8|jwIs^_0B0BKkDEq^_ehyh^QsN0Se zmxumkKg-_7TY7fr4?md~%F}t@RrH7#Jf&M(n4UO5fy*;5mJ%2EU0~O8`0*oEul~KR zIvqEn?e#1@vrvH@huT! zqfk5Y#qUR+I(S!G(m~2XzXcyF=b;LJ+Z4l+JHk3t<0<}dgn*m0v_5R!pm~QFg%J$Z3;kaTB5!6Q~!r4|Yviu2yeyOPsmcn#$`2NDo;&4Y=q4Y~^3* zhDzN$E63t4vqKoXSI1{aynXLz%B+NcV<3^2ia0+0N5Ds@dD>%4M-RGW@W^Cu<=@-9 zoR*6NGQrzpvdzRF&gKYZ^mDzi;?Z=z+uv@OsdiM5e%Idy0P1!-fL4CBCm>GSJRqZ! zdGsH=g>b67KwaD?13*Vtqfb(pqYj}K0Y2Kg;c!+UC@Yhh_kT!VEp#+VU1rEJQG^S+ zV-#d*6|`(vO#%K_cn?}6;UK|5=L8rex5SfuaIrT#c$?L$Kk6Cr>J3C6SueVl?|WKU zwr6q&m-nkzt|!Thm^Fhk-G8-f+XTQb`{yhyob0lcq7l_t+Fe8 z7a^h?s&22F`mUuukct6b%8F0Gd*dmS!NPq$+6Vfy6J>&K-fxSkw+G|l#gO!I3<>Ur z(8S7|Hml71#vxEE=w2AXj9wk=z_blt|0&>}2Gk22waab@y;zVPFA$r7w7ciZa1kpS zYR{Vl5a0OL4`|><%F;Q^D{vV#uCgSrAx5Gh7+`;p*`QxHIR-6;>eOFK6(*?%j}cj>gT&1#Av)~;fW=~q!j!5?^^jES{W>S zg~bz+Ig-}*Axd#Sx|1t1$pw51Daz{U;F?A550gVbbJGv2jkQNQEkZPi^>$o_!Z(U+ zwRw5L;eX7Mm=rQG7i=yXX;#{%_s#CZ$0vS!6C6LMYI<`++Y^AINE2Dp4Y8!}uypFL z{F`y*8??*vt_OCBh!Sn&Jqv|5zgY+E{AgeK9%l5CvGe-jE(ZiG4+uktQl^7zy8K66 zt5+by_EKfSRnMCdg{`K~j#sQ{<@>?kHE&WnePyGDQKVnQ|MhL!55jSIyz&}$pFNQ! zOI8ae_EE^eUftH?^vn0ExC3PmTtJxL3Pq1s;jnbf`^L>{w0%gW9l$XFT?1&sP%18mFOa)*W$sVxJ)`+Ok-CHBB9h2Fq^D{f@Vs3YC=cvgR{L8sq8o+z94*b^=2 z#?Uf$lQX>!65k%e;x*m!NE(Y+#I|kOtQW3#3Ibj;J&s9|E z7Y&k>yNy2rx+&NhOmei4FFu}RHMf7G&f9;)p_E`1(enlxgDon|v&aYOCTbC zJhh|}MO{#C6ncyGUlOJ2FT8_$t_rAo02Tt4T;RoqM7IQyliuoqLE-D=47M%7JPQy? z%pfRr-nkv5BQD=vuw_V$GttX~SsU)S^Rv(!wwO>9_vVqtZ_911No3>Xs%D7J-V2HN z=yKIeN ztWxkFi)r3nIw3mUPLryXy3IWDxG(x{b=GvB=zZATBVs68dEO} zh$(;yGNR?mY6~1tT3?8qty$|rgc1ml2S3Flc&yEgC4VP+3#OSjf5HdA_FwjC&SVTd5MUr4{Vls6 zF8w6zVDU}DY+FJeS5sMYw&cda62`g@*n3ce4+?!*M0hNKTEp}3c?c~^1EO|#c<`D(oR)KVCTiXs%I z7S1h+950w7Z?sQXIOtvCsVK&I7QV ze*3=I{0JfC)b+FoV4U`#c_ngs7>A5`($_*4%94=eWWh6uW9h9H#g^V$rMCG8V2sIg zuc2Uf#~GsIjM)1}2h))cjuqG6`xLU;Uu%*96D%cJjD@ed_Gb&JpKb7&%#x zo0nH1Wsz2h^bucTND7%u`{c8D{1`?9QdqkS`lpi7L}msz`coMLNvG%;Ga8-oW+zo0 zGLLCffX@APKW#xYpDgWe7R5!KNTY~}zX=4Vs-t1uHCT-AlLn^>+k#TQ3??HC#>)*<;^>?9THi@G}%W+WTJIW%woLJ#{4t<7Og+`ak;h zmnBL`-b5=VCWFOoo75+ zo(8UdLE-So2$bfQ#M6Ne+Fyh}P9Na{Hx~S=UqvhVTGLxfZD**OE&o*C)OmUf(#vmO zufAPVz6vuKW}o^K{=x#rMV7zU_*-k9?Qp#CSMj?gncF3zWx8iWDs6~*Vft^(czv!A zU@2;NrqZ370;YrSbKE&>%5`s_W&jR!Vjzw!?>(=$#|^pr`HZc)OInE#5j^pqiI=J6 zn|E?ez_9pPQqn~z8$zN-r9#21K+2QcBqdJ354psL@2mT1IK zXI+0neCH{fjFPPZjRsVuo8Cs=#JdZWzSh`JqJTO9{;1WbvM!wNV$>#Tv@~V6Atxic zN9@I+JSMZqhb3+RJbSWeD+2fY#tq*aZ%V&*%$QOO^WEiFZs0IeIR4GuW+k`^{{O!< z4c@V4mNn zuKLR2|3j}vv?LL&tF_-c0j%T3)&S$ULVKrhqtS~qV7{s=wHp&%3~)1RUt$?FMOcTU z(;rIGFC<=c(t8-Q-sH&eb#_}$w$cR(ZR@N%x8jdt;f>ki0Ntr*z<2q{7}j=fZQ$yB zz-X_SWe028cK|qtlp%rPotZ=uByINU@~8+q(;0itkrvM`_E)Ll&+GmO#4kryRQLqkjYa@iks%NVn9^ z#TZ$0j!5hf@01IoLyLdg`661^BICx|+ElZo`K6yo?KbY_j^N<;m#7Dz3e+Xku*t(V zqGg}l9Q@vy5&R9LHTX%E{@!Bx%!A?cHp#d%uUC%B8RQU|(v@dz|Nj}LqQE^5r>BeO z5~cZiX8-oVCTO8_*^F|5@i+y(1Er@m6osy+_|++Efpp7iGfEK;QnFY*g4 z$8CO)5h}C$gS99G81;>Xa-^@~2l$j`-}vu-DVo!g5B7{65Auw~1d|NoCUK(rx1-M$ zb3&C6!pVXojbwvD43$^RAP}NURnuVgBXgu&-M<~5I_4Y6Tx>;f14T82>GPG(3Y>UL zE28Na=$l;UA=FXV^*8`G?u%r*upn7hI1xe_YZInl@l&?4^;msq%w7k78Jhg;cB6DK zSd8lSfUw%j;JWcA^Cr$l_3D5EwSufP%LY7kAv{-`eepu`PS6!qpaE;9n;7cWrjsuN zJy8VV4I$FSU1(vHbdbUbgK={RUpr&Wy3R;HQcgW*nkwSd+lDAm^VMfQgOUc||mAJ-u{$sRgTb zB~7I)22eb&nozj1#AN`sV-09p^P5k{%EouemFR}_H?b?IQ;smKP(bbE^dh@R0d_5- zo0saWa)51Gv7{JGt2Z6ZV&*K?J8TVfwF@fZklqO$^IA#MWBdo0Mw#hLE%QegkmC3O z{OrQ7p2}x{!(HCk26BOZOP^8vyoFF&b82t4gpNfLU?=iD`Tx?6Ih1I-B@E|zRAaO$ zh<1NBPPB4gQZiohK4Z;h-IW~yUVTW3_v)i&YbBv{Kw-U&vSjKcQWkJ}R6JY!E&W?M z?C35g&{a!OY5KZK0NAw0xjQ#%--r%?k~Lhi{muJxK8;8|_SfHG|Mjx}-u@)*kBl(P zyK$o7MDTZeWvPlZlS))AGm%8}VLcmNm9sn8+s=aK=-=!7#V3qs$mk-&T`WX3tK1yB z2u+JiLcCWPsm2*ey?U4ii(+3a{tp9Oo<=lhw0@bmM{qjuO^jp?2z(hujLtjNQuA2) z>l)?FA6Wl48>ekI+g9Y5gf#ne}mCg75mk>WU$ZDBWl z2i_;U@UM;1*`28KV?$uj%Ah}kR`)1$UYfUV+D#}c#@bAdk#48hVR>z*+yUn+X#We8 z{}^H?L~@bYZHgwiGh1M(O>>6zA16x?;oF++JAHgbQ< z2)+^7LPn4@a0~Xq=QEFES=+wObRKNd!3enO!i<5(&bpvNq1kHsZKHeCeXyLR{Ir4$ zGT1v#c%qRFC9IHhB%0Yz1JYDfN#Sn1cQ*_JKi4$T-g&ttR7x;$@YB&u=thU4wqa=c zIXgCXwB)j8R!6V!aq;WrHYfP<_EkXAp`krjb1;{vZF*-vxa-`X>be*&unr!&JBXy@ zu^2k9@);6%&^KFqr#LUh`5;eth(QdTVM7g(81aj^uYq~KqdYR=-7u_u#y?+tfjX+` z;PjHErt@3i_T2J2YQRt${_y>z@vgOVz;X`I8eK{lvy$FRUc05?`4ncpH7HM7GV*_DzBGZCoV7qr`@`eNldT6N zJcVHFn7%N}&((Yjuv{uFYR1~3`A2Rq#%YwMU#wf9#Mg2i;x%Xwh8L3ueBU0RHT;qD zEfx>OdbF%2{#Ub5MFlW?_z@VSlpL!)lY}<0yt6go>^xsLU$)Jp()ipoB(5RVv|*Hs zwMD9!Fcs|Ey#azuvH0Lf2?o1pBzC~A5{;>+9{(HA@ft#144+;=GCssKql|GxuJ5`E z*+(;4GfCA4ATFyJ;oj(a_!@X)_`K*+qyb!vyKzXhFFG7{+sAiS_KGBAgUiSedPaCBC7wtpRvI>+_Vf>(K7HEilv-~Ro` zdO}(-%kG~Qy_Lj0i|hjqVcZ^nNe|;ch`YQI-3pUgT50`dCQiF<6K(sPq$N*yEY^0K zsk)>$;M)Qscwz)pAc^xXFhMzL&&R}8Qnh#DzQs7=VfTM1!gH*;!I5{!Re3uXa`kh#O|vbkRNTj?KK@qq#OgOH z_qlG$8&{WGSIyp7+fEa}^Xg#WvU%3=LUKW~tANp!^v6{&e zLVPsk@HH%he>pE)jK~W}(b{jofbglcHE^qVXg=|C$IZg&Ml~b9^R(Piu6Gu| z$L`mFs&$H~eKwnI>4&|L&G+$nUi4V1{1-CwUoEvd$_t3Gsmfp03~ed!jg%*dy2G3E z$GBi;gWGlq^0F3!@bTOQWS7-=_4_PVp0zhqy1zaOj$7Y2i4F@t8snb*cqsZ;vmFqf z5zo%`sH@O36#0GZQT$|gPhY+!twokf=v81e)0KvEH)2;^N8?PP-h&7XS+&gQhWKhV z@bAHEc*c{LneAzU1C~=hrFz{~*;7Sj>zxXSTBcwh0|#C)I!+wt18b+8$OEh9?2xPn zXh7e<;{ZrSF(9pL`vF^~Z3L`Z{i_rGcQppk zKqkv=g2p?hhGngky^Ye1Q(O^VLdzrEz9i>ixnv^~&um!5G8m9Kh<7-9!3@IVwDF61 zUPH;_K>?-U*~w%8-p9BPM#C@_P2|?{nHUA7-tezu_mjfQcT(@ge6b_Q4Gb{dcb-+M zACTmWOn;nrzt=?4<-Qd|I4+E`D}ANfnx_2wt}5HlZoK2ECvGQc6UdwKL0p4C(^Ao{yxjynl1Lt1EsiOmWsdS{vpB6SC^2Y8Ns? zNz^W@)^?#%FFRY@!kxOznd{qbPAMo4fT}J-vp6_;dDC`zUexaWhf1Of!@=CmK2WBk zC!EpO$kHhA)J{)gq~%c2^8Ljs~sg0KsvAkDi$=wW-FrLQf)Pt%E(W zFR+bHfi`Cru-4317e{ALqR#;(9;2h^+iU;^E$dOUE|rsJ3igs9-UqO|9XzV72ZGCp zr&{=Kl6!@5*L+2~u!)hOHZ{=qn@16?wkL=-hb+>e*uA1)ZU_-Z+~uufO;C5EBRi1d zbBN~7vM)8B|C*krcI(aJk6N#xrq%#@`A1QIaH?-ji!V|!t>6GhbyPn^F?mjSx9zQ z1$iRCRdAEESFim6FYT- zBINH8j$j}F%8P+w?DBDpaA$w79(MZkrTycqHug#Q54v@Uoi>W2^M{Uxoj{Gs$au+( zJ1opCCri5g(m*_;%`?6tdgpFE&%j`n{uY&Gd?MAak!Y3$=Zv zn_N^QWp%#Bk7#u4nDfGRdV{~eAsCNr`54k#e6a>w-D9eBJKcndP4;Td~rv}kc`;pR3{FUy7Q@rU+X^=1iV@g}I+{m_0b zB81SXkw6{2gm<`XAcF2}08T*I^7R%j^zeY}DV@x7XPZ`e`Z%oCk9R(`moGYXPj&g1 z6+nOqz_Bn|XeMQG0&~V3Xx3@a*UXp_2I)?7!R?6aoG+8m*Tn4XawTh%gJj0|l^z<3PMKju|96u&wRp)9eA zu`RKEY_3Z2!i=2i1^uPos1xbkI$p6-*E*-!;lL&P>yo2}0C&8;%MbVCd#bw*4eV^f z+B`+s8*IWk7u?mfMg3K&g6Xyzp?!nCJ&%FL{5o1Hn!|{!7s}v#^Q&y(N@=?#WlN2Uu1(-{YQd-W~x_pzxkh+Aw@uZ1{unC;3D5ub>PL`hNd$xNP zhI=N)LlMpdEi}W%Iw{(=8RK#$KBC#8?pnHyRSW+TW&dvzxJ2Wvk`U#Y?I;@$51dcb(al2xanqa{?W?ZOR zz^Fr>i$RebCEw0jmIcMm76T>mkPdqVUQlC<%98K%ahkvxU>f*>l3^Vz^j5pnjbqFs z0lklA$#nWu7RNW68Hd#-($lEj;kHTc@Yyua@GZYDWT_!!oIzD*&mT3d6jB1=ClFLW zO~gAy>c+OafkR!)x=S7CYqk_|ikTf5o))K}PpRD?G;YN7Pvj?8Bmosq+kvD2LG=fb zd6sSodU~5!w~XTZxFz@&{tA4jq^_DfKbEwuFSHFuRXLh<%M}ZaVV^SC+KdY^1f56G zNI}yVxzImnpoQo$Z6I)N94F{KP87}U2pk!hcPe@^RJ?X@@t`yk>4^oq+80Sq-2%W> z;^W10lJCnk#BF-Blds+EWVA4h9iI*2KWtJS56Tf_K((Wfx_JP)sW~T}_m63guj6CV zjT$F4Gb9At?XH}~Rh@=k1nmtPUKk*U1)w4fc)hpiwnvy{x=^M*8=eWhRoSCguTz}| zqj!q%{b9bK4a>y#Km*Hxd(I*far^r1aNVb>&G{RrCdH9>K30I?>n9x z@7`D>rP}^YrZQ+`KI9U=+0nm{5efEljt2OKk;-^~9qC@Md&GL_|>xK~y_QQA#ZOphy-6eQAR z7&QJ%Uj*AaO|WDV_QJ$7g&HHjl%8d3GQO-G&|S#OUWw2yd9S68Qr^f>>7I6{-A4VE zZpsYX0Z%u;O8xJk41n`QU%39P`Nq6LRCV&l7b$uQ&y(NZ(l)><~MfMFonE&ic-NyX7s(zX0E|> zW5ZeZ$zRxSI7k>M3W{PrD)1?+Nm)RU5ZT;8NwaREpR?N2Fb0ZBtEyOm$4IsKB4V!}0EWi~pGCbXmURH_BEB2HX4^`A6-q1nrqrd|1!)~* z2W%6~sqlJ5yz4&^S|0Zn6ngUi8P#(-d6VMN<^klW zl^gmaZU;Tx>~Dav*%Fa83C}>qRIHEDQSBS2#}C78)mM!M*e9OMR^K=++~>?vT_`Z& zZR&ry9b((+tMb9@oi2R@`6Auc@cow#6u=9w0mS2}$MH*Uh>5Lzh81%3jgsP)97ss? z$xspX|5Hq;rE82TGux=)RlxDBW`Hk;+E$l7p$l_*Q?jJ;f5jJ{6oI?GVA9Y3P1-x2 zna?1kp^hwh{`|aNkENChX1#G@7R2aJvX!uz9SBF$8Qyi@dKO+riOZ~0QTkjPg}Ae{ zH1w0vNmp$zXkqay%8;Ga%It2klO6slusTVayD2pe3Vy8 zwW?}#;VT51462>i+6)g}7t#oj2sxeO|A-w#nfno0r*BL-_n~{A#I|#(R@NgeoD z3jZufi>Yy;jw&gi`to)^C`k#7#s^E^rTttJz=NKL)_~pJ-U8DZqX?X%Uk8p5fKh26!ATHIf&X(1iCI~Ut+e|xLByxNIO1{eQaac-j1C^P^PrWA@O(Z! zsOAw#9x4B?he{F&Lt|SNBW^N#3go`RGgI@~i0e}gNZv@a|L&#WUvzb9woreUEk7>F zm1ZYMsE{7V)hZ~BXG}Og@G0BrTbfN0Wc7+BNgtEfeIwXD;d$l9gapF#k{8pQ%nAE_v{fktAS3d3#rE3c=)0E|!} zKxkA3SCTiEx9?9B(hUd@4zyFuq>2_m|5K1Ink}&QqTbj;U4hr z>7o|j#|AEy^YvAhH9eMaW-`=ZB^|1E_%v-nEUm+#yUeUo2p}gBq%7x%$~=Fd$1uaa zQ1c^)I`9<{U+r3e`4oe6lto7+$xR#RxL<^>xve1xFU6`(8unpluCH*X@cZq4n}KgD5*?7Sl%kAB~+?+~4<<!G^Q&su}?6DE%tDQQzvTRh!INr-ywcz^+rBpz7uUMBcZHNo>g*cPP(z zs+2Hrx*3n2@a*{3EBb9!uMyMtru#eJmPo%X9Vx4yc#|RhN~(&`j*PJDUbV%S?1PW< z=B!{vT#HHPqVyaK*d#zD?9o#kAbw_wgVe0q!$g=>a z2P^np^G=)A!{9Gg_6dNXKVEObx(Ja;n`D0z#nD>5sQ5?~*<1=)5@8700)dU84`SSS ziF%E2JmFgGM$!B;QG_$jcG=ZwnUCAn{)mP{704u@v&zWLfbAgRC(i3olTa|kppu)1 z!Uvml_T&S6Qkgo9D);%-xArqx*0MIv$sEI7h#bMpJE?u2#f2@MEOI>;M?cjGdxV86 zA@Yr?VEz8|c;9tE<@FC@+p{*2sfmArKG3J(+UG0I2oLWNg}70vm!VvfELDQO(b4RS?FA$+c)mEW-0~Ls{fHS8J6C zk>iF%<`EEOj;g14U(QScB|m@fvX5G98NQh!)t3?sZ#bBTRDBLyIgFaoMg2!VF@Ho< zE_`>=uR|Fc33~83;)-G+T(V8q#KQ5!YCfLJi@4dE_S^HioZX+&OP`@UeWayps(K8* zMki1%j!37j9=;BCh9Sl%_Ly(vlIDCis zKy1Kx^kIqE!OyZr}Y%F`@<2Qv!3KH3yjSd3?SJF+~WCDBcwyLOWv<^3a5T7^{4a8yB}YZ-XO@vJQu1flSi$6rQMa=|n^oD|WYxJ!!1?YXOiPG8Tx zcZg7`Z+L0+r?SPuZo-)cSLW_FhnqfLwPce7Mp8Yq&yVi&tDjZa{`0dt_rdsn$YH_P zHM4HiXjRwCBHR3%3Rj>gp^ILAzu7T8vzgV~vFg-|Lezm`%w$JKTH{Gk$WObe+m1#e z6u28RA^YmKg>~n6p79C1rGoRhUQ8`QLEFh4pIv{ct&9ss!3tFue(p0zDtJEfXg?dZ zxQ=BwMEet@6^a5Y9o0`LQO1_<^4`lL8MvA`HGB$oJ~>mf#k3*h2#Sp%Nj&(@4OSegWR|YB$uLSm~ zc3?mhMPq`_<3IrXQp(Zv@g#C&K#OcO$gfH=7L#j8LJY_iy=w>uuQ19w}$a5p%@zLJx+8Sn-Fp>8K zbz#E|{h$CeM}%`rM5_J(uFomaOAxz)7&?(-ful z1!Qt}iVv4an!`e>vUdclQrcp?$V5T0HK?8|(scx&xmZ~A8oI&YeVuk+0ObS(Pk6gb->g1gkT`tzSuGP-Ru>PsT zfD6AbCu?ii9})p6P@^v><>|+d1GAq~ld$j4ykCv${Hn>ZmCP{~RfI9l)cjt(1g6*J41muY)ABj< zLz(Q*FAs7c1@_8Kh*Twv$Nq%pC5X-fS)MA+P$hL}6>(VE{M0E;K^--j3Qplj%sF@a zja5f#>0tTTkMCObjB+L1I-@6J(uP-U#i`xJcfH=l0+cUNLx>ysMJkN#AHydLO@nMB zsr$dIM-+@Ea>V!9S^0DbsLCYNwkJKo)t~I)8`!*^R5f^aU#*PL@M}nHlR!6Lt1re`#f^46d%Tga%&^WfM7TD_)d29 zQCxan+3n|x$=qETL$9RE2VV)3=-(AF0nFBJSY*9hyizcB5pc&mj11xzy~vDS(>7cd zKSLw0?9Y|@<~81*q9kF;%fn!%dPldSHlXhf>OcgZcJ;15R?>8f+lln#PUec=Z+?XD#2=Oi`q!MBSKloF>^6?NP zjdZe{aP6ASu{raa*=Zv*?!mJ4)q5Sh`~Z9((~jjnPR5Pct>F&Dy7MVJy>}b{c+7;- zx#wuP!@+7>HsK>43thiL2SciaA$MLjMuJ?LF(d9%G=a>(a&|>|Hz=9*Q~Zi+L7)DP zvB@e+L1})tvz~a3iLeQp%`$${yw>iuJ^d0#fdA4h|Iwn7mB!-%qYYK|i@UUsxGK0S zxOj?sc=hyvEC{v~o>`b7)ulnF}_ih7E@>dHcr3jI(NA+A%YdX2Ry@8nb zw8nBMC8vo@xIGEXg|<^JfWIGqSAjFCKr{;I;EN%t&q@D_W=y3sv0il#>)+K*M31S% zpTGb~yy%^P-k(FS9yJ-8T7mRs4xB$>%9#L>3bsJM#Jfx zX1!5n`FcULvc{hEX1pm%FSRlD_w}|ATR}jH%I#gXLTCTI<44_`k`B!rK`PJ}o3vH; zlPf7wLPHS|vLv{XcYqpNU%Z3kVFq`ng?GG5&G6FtU~-4NtiFhojg1QTSf@aTZM~`? zurpR-vwc0OSQ6vJGmSa0a>{52fOkDO)0KX7!lZv)$N6R>27cJ6ya*rnfWy z<_S1;Ea%3IPnP-(dA!2mV0<|HqOq#6^t6UYYCc5rt-f&L^pl*+^bSHv2I6?>wZs{F z_}}Ukql9zy6r zHd=lP?fdtN5(B<+yI<}(%)9I~b?Do!a*O5>#7H&@bMx*TJbrw zhy@Y(7$q}9$Z8#Bd(6m3)j0L$H-wEo_VyKrf|B_ z2z-zoBc6o&y_&7h4V`~J7MRfiahNii^!|$}uFtu5C}~S-G;Q)CQ$_5x8z&Jn>>a5j zZu=WtM>gJHJACR@*?~6`@m_Yy*f>^E0=b?v0<1STiTI*!rJNni8^~OAz|1EjaWMBt z5AN%uJ}zh+lC+0a9^|B9mpRo`rQ|3DR3VpiM|GA-eWp;FAQ6LGcaS=Im`fvVP9}>r z(xD?D@up0Vj}T*Pz`vOMD(04DL^U$JcA_dl zW~IvVl#dNjQxj>Z);cRd9cal+*10w)UI(9CChL=_4EME8>#qf9CW`r5yDX{B!yzvp zMPILC8n(zcDVheKiycJ(K>1>RNxOj?B_i4_Dj)(^b4}>*X+iaiJ!TXmy;EPZqq5Vm zi<~9GfdS9SH~uh+|NHUC1(TR_*Yyq#d&DUTwm<2M+|ufA-!KE86?U(@lA;Z-DiwA* zG0s6(^(Xi0Ez9ZQN_J|p$CKyWy=&n08LCb>k~M9K zz*c^NmMkrREDe1TLy3XYop1p(Y{8|?=25A3aBV04Bou>0B7M5UQRYCEZ5g#D1a z?fC~=x^LQEL&#MBW+J`}SD1lj(}DlAgG$Kav2H{ zEbFKjTKilq_FzVH_~Pj$uK{}7n(-iZYWO*Y^bc%~0c3y(p>-a~D5B}D*!vku8oDDY z%Wvt0!{}hA)s49i_y@Q@YV0O87Hqn$z1Kqxkd7#Jq<64Mb_4Bl8q9eHAZxmeRijL& zCdoFxP`Qd~A_adLCQ$(O*iQfgB7SeFP8Sk6%wUU=dx8_Eh{G6P?P!p_atkI34LzLJ z=8t-eFr@meu@>H=L7DeY*H4vTc`ZF|pkXY=#obI0(`=lzE}M^S2b^QJhTU?CQh#I> zpKHN^eh8RhVJORf|142MeYR}xw66siwulcn(n&yvFrVO-aecvzJ_Mxd*D(kyiwHKq z=R@{90f1YnZ=(;)1*B!;r`slag>w3cRYW|V5zMn^ybcYI2L{C18VmdxZ#+-=s7>rI z;XIJ*M2rKhA*)Eb%A=kK+cF3kqL^D!!!tg$yb{6-IU}k+5jmaJO5BRw1qK$Q?_}G) zb}62U_ERsQMfH&DeYK>xk>SJXPcG}=LX1^^Z&@I)#1wmqQ8G%szP}e_A3@U~z541y z!BKn#h9&*ig&h4C^cg7Ss1u;BW91{kLpb_ZD3Rs3{BsH(FPNSBQD2Iz=~@4}8Pl99 z9HS)J4Rby8vL(8(nkV9G_Bj3$hH#$Ouk-pFj-L`Nk}38t%kFSyafgMFzAaf*IYdi``;N6Y=TI+3~6z7kcBy?A8cQbiY-W z_@Nr`xwy`=pM8u=oq;n^@6lGj*G%pUr#*Xucm5f0R6Gqza$M?0NqCsjuNfNbXo&9~ z?y~#8o0d)G!7kexN2i0kddbmV1jCXe9hRNOBX2kmNl_&8OpLb9+Dy@qGqO446JZsu zrYl!auCMyhE66Mq1stga>H5#0_r1Oe&VU5Dtr1oYan&zeT1y<%^LC`d@d``y`Y6LY zYhp`xnr|B(Cv5qGQEx&~O2T%&Du;__4F0pU(xDV9sL>~f^R;J`G;lg4wloM6HIl_Y z#iy9jps7C9W5^jh-F`Nix5OnQ307UGOfYrwTvz3lnq*nAdrb`%u1J-(T(q+=C`$Bj z6#ctg;?p2leE1kKKtW%Nt@F58newPiSw`TgC3GGqRr3WUrrxWTL`!J4TQxtnt@8v1 zpR;E`e8>jftcc_9TEse0$;U-ePSCXGECGxlyGM8_ald1Qthk%knlYE}Fw%*1VjC|~ zeaxH}hZIlwPe)wyHM~p0Ek>6FfNg1stx3pD0|~c*FpO!i617nrISy`QKP>(}C$Mu)M78Mf_R2%#SmUfIQ##)51U)eH$4doMYi6_430~9}SZT z(Deomk$qe+_^xVat>4eP_WAKyeot59PXr!+^iva1xeGMMY#*P&vZ*Rw8m{QI3PwcY zppq!v-rXHW_KP#(pHjgY=in1;`IB#Lh}z(&0=340>3lKcBXtVGpchl@4Q>m{4RtG1 z&2zPzfaxjgN0yd4o`Z@-&~GJ({YQ<2@H{uM<}@NcVIpAt59Poq03~#|ebDqSAxyh5 z)j~up%Ovs@QmT3qFrNnK*#U8NfIl{t@r!H)T+*&@yZRAfs}#>?^B%-&i1@n^`gO~bOnTu~Ag%p1B6N-DG} zagIX6a3vJ*dSM|m(`W7E@1wiP|G+}F6Zew|z>Le$fWD^l8u{Q;y%E5O2Bs8UM2==I zq-203L`HvUx8A?Dn=!l{LX>U~tb-zzWEZlkvS{LVa|AU++Esh1XWp`Ap5%r5fNCce zTAFAnaX0Xj{9_g*jqftn3IU6+Vg}q9{Gu|kt5joK{Y5r`jn2}p8VB*_|B6Q-z5HMe z7+444ELT*F15YrJsGL8rO0KKj1t~c2S=f#Bdgz(ML?*KSlJBi=Z)M74u6QC|)NLU= zdLnCB8#$wj6|^XFZHnrB)JB06K}V6f$r`~WPXNX_{=}7d|(1WnfW7?V0c^J z1Zypj9Hj9k^TIT$JB!B%I1_*sy2StE>Z{|L?Eb$2r(jT`f)WD(5s3-XF%YCnMd?yR zKtLL)fr!%GQbSU@yFrF@H_|yeMuX28-rnE)_q<-$U!t;|>zq%WcQ8~d(F{n?{M`j^ zFX`%zW??uvSIma=ArYT(DBltoY7sfirNdL_w~I%btM~;|h6uNz^m!Kve)z}Eg{;@W{}Z{2#l>ZpMY9i;V zstN$08jYN_3eq5C+^nN|gk0OQ)n1Y4{s-q^Hfnrm&~09`m?fk2r_I5a748}?kgzb!|u z)W}hT*z%ucX$N?XORl&U6GPqEP+Yfu89yR@Z+WuUx6{YNati2`=DBhT?qcuP`HxAx zpHn-?nkiQTqGI26%xd&*%Awqv#~|6xv&i!X<)JEM6J}pN9J%``^zdl`pMlJ+=1yxv z-LN$MuHwF{-8m{8sRUI^J#A}TIgz5`C>_<)HbvFnKhl+~bag^L1fMOP&5kb9LIt$t z_$-wL@w8M??=lhp}Y*$UN6m4lHR}=DMLUdLJ_N`lH@G4Vv)H`(C>d%;~b8XAV=xDTVF&yC# z=WER-zXtg2w*~!jQ$_!b%}Gf0bZ9rpCKwrN?;Q#Mv|JC^3gI*>q$YT1Cws!&-CcGg zu;zxq;d5C-!dR9jC#G=Fg@X>wOiXm&F2Dp%(u`f%^QJC=5ckjr00v^BZXwfUZl%=% z3$u#h`24Xv@1a2iC*Ig6oSnxb%9+8WJq>gdDko|@X0|_yLXPsF?;;nJV}0rP*>sQk z@0AF)S|$b_-H#mBgRUixmu96_wcTej`;bN7iWq(InzF6r?-b7?o*TJu&5 zaU#^u_6vm+s`Cwm=eApk)6@OLJs}@+PMY|b>&P^dE32I zDgH>_w3XIJnB8v(*<%U*F)>wb+n1;8dVt}lE*ptu|Dca2f4kWBKa=8)a0NI&RV;rm z6G)|5!YD^Ql+J+EVU56QUSkGUvCt_cm!)C6xbadnDm#ShX_KmY>B^sB-@k-qNxe~m zyl1zqq&-i+=#Vt%(<&@KBfcI}1M3lIvA?tA>djtEHxX8I)av&{Mu*4P`V z&%v&aSYoxoy)c?hKC*@5X>-T->Wh3^^>P=pR|8$VCF^))ji6*?y@=akqD@><{QgM} zh7rwRUJ+Or=9@qLg;VF~Iy}@l>$xHg>ea3TgGsy3zF8oc1@)8aoiray9tV~j+uPAOy# z!1+j(+HvXIfMqkaY8(*J8c0uzcNC7XWwtx9aMtdw)cFIG`JZ$E>d8lc0hsF1br{pg zFzIfCmX^TQ_YGxYvp#o7OR%yEdpz186n&FC&~B;Ziw#27MS$3Em^ckK2Ut0z;{Kio zcQs-8DM@U@Ut3A096y$R%2BujW9d*iFSk3Mh3$y`+*WXq2ppK(SRRNJY00$dkJ9dC1K(?=_nJeaYP(qRt*S4YmQed#pE6J4|jT ztCv07moKcu>6&&CD{s-3NT@%+tNG+)-d|*(4%U(9CdpveT7o z3p`z`-wEylf;K+-cozWlo#OBbe(#9%V$ zDEr-@qm4VN`~lhjjL=FhxGl{}-@?ajnc>&g9HrPR=d`M*SxdNfo)7h?2{x@OfKBRY z2G;R*0Bg}*4TQgPbTApJYsObZ2AsBAm{f`F>n>zqIhg}k?lc+@11&L3uG>r)+5f<(vg+LwST=k4T*x`8P#cbaRd+ii@&zEs#5}0M z`H9;ga+D;J;j;L_B~5X0(HZ&sWO2xmh?>Tx`!Pl*46(TZIBKd`@(qG-6JS_!b#@cU z2k@^b@lAh5v*;eSG7#YuAh&SqgvFab{S(dp4JW>L>+XreW~R)nH1Dh}wd%erG(w_U zyBc|^AN2Y5-uJ5~;kNP>tNa#os$LW9XM?zfh| zo;)0VCOZoIIY|;JlFFwgP-a{RP+eagXZ*Wqw{gTEL`XUKaCcglX<{n-^~tmpK3jLntp6y6_4V-=+`rQqAa#mFu< zK_WsIU~8Mbegk5zyj}R1@x4^=h=!&I-!zmm7pX(O5u{_B&>O#cB}k_#R_b0Ft2-{UrO3O@X6VhZdRFJmdHS9_U|(nChPo> zknyYAqpPbBTe_R=Ynj~)m7ZP7`*kTqgn^y)F8kQ$W{RPzK;l~~jZ7iG89K`X3Q&&k zDBDI<)Q9Wlol*;CrT}tVQtz0g`$WF0xHDIoe2mm!Q2TH5az0Nx$%2VwAD5I?XCI+e zYc;SL5JMxEvV*AwW~h@{#3z=U8$c~wyJ=dO=(rlokHnJC(RCMyv*LG?nO>R$&SU1I zE}=rW^iQSZce-0x3Es1JQy@qwpMqRrjRC2*>gZ^wHs`_KOxDia>{D9ZGk0vTKPOV! zwlb%^I9{(b#{hAHw6hVt-n?>PEAGp8IeHgY)MfS%XhaxcB|*%Y^|p*ri$J*&OT+C` zgS#&|bUEn=PgyDc@;7&$MSCkbzdm*9HP!lO0P~MN%NhX~v#40rDQk!7I7Ig*xsYmZ zNBh-Lk#_qQ_m=EDj6*0of0Y*^&+8FC=-pPXEE+G;X(G>N%Kor!x!LW?T7vn){}iU^ zknngIrz?RHb-m|9?9oh91-E0$Hnqz~xKr_#L!mv^C@awzk z1APce3;J5CO{!5Tl$HIrwsCJ`n*6&o=dR&E`#Pm%<#pe&w-8{J)1kZjh$%@q`XIw5 zBkeu7F#I>dYI@cIL5M|kqAz>K!Da1uT_}j#POs{e@P4n#PZfyFky#wM39xu=gksqNHkv9{_iHD}+Ygfp-zER@7J8CIiP zfE}AK<8Dj!^aKcv^r6YYfm^xo5vcaE8K@xU=%MMv(Di3o&l0*swvq@}ewQx2-Ohp` z(o}iRfN$<(mT+zT2~cDHCa^cGzf!j$BmxvMGtTC9&r(W2t`X5CUAHT(e92qAtYAXT z^hgA_rVj0$Tza$%lz1wSfWo#8Fo2raiN`dZVhc6hDJ3A*>}0yk>#$+pEU1Dt&ds<$ z#FxO1?2_aBnDC)`MH^!-Wc@R+12HPeQwVeH@y$SLJWN+Q&NLGCrsft|J~?*;I;JCa5nuR(q~qog zpc>KEwrsjNKu@#fRbfUY^!dO+ub1CxuL6n)-UPMBRy^`*4x!7*V!RT<4l}@8Es)_e zcNJEZw+>YkT|~ONnABhkI>oeXLA(s^<}25`z=1z1jZp$@&U3W`OmAdO(HXVD%To{D zs&9_Xf)me>89x@V>Mx-*$HUDkfvzXBnOFzxfkMDuJW8~B7upvW*5W2+df)>k-vnX` zx~G+0v3T>P$U`EpVZm;d>lV zoc<<2@c*+39=?KDKrjg!u?w=dx|YLu?&%g~T^3(U1>n6rTo4-^$cNf4R7v6#Gu#wZ@e55Kw+VgkN;hk?wSkG(Q35UgjXEl(h>&H2ZLW}$( zjjriy4`zxCJrxz*Xm+BS`;jIm3%s@J(9u~rr_$;*vwbdqB>EJq;9~xqKl>0X3N(P# z#;10e3%-d&%yrXt@{pfm#Y`}#8?LqRn~v+{i-Y($%S^Su+nE>+)u*wfcMKEVfGZCW znBqeERkdlaKaaB=R4@J&S$i^n^puvcr8xM)$5?{XR@Itjh?6|DK z(llR563NOANFW8-z;c!L_wzGVzqZL6frs3elqUmD3ZQqJ1z5qpa9n!SJ-tw3Hncp|-V~S8JcT{`B9`f2RBChSR<6oxVo?3Lx{M8`&a)ujn@1 zymrRg2+`O939;sk6RJBuyUR$d@dT%BS&nFPde~Ym5V<$s=Hz_30mM5T5f665bDa%r zLEAs%iMwHwCioI7SgVu9W^Ug|Iau4*zIeBGuQ)gTKsZ)O%0-Rm~>A&uT1T7d!&__pgCldg*$!@)-n_pn0_!UJTR=U zwZ}M`T;kg|aX)P*))$K z`u$8tNSP=3B!8!SqJ;{_8Pw+*aSjV#Z|e5X$~l|jNH}i=!uGcctFh(|8N`Ss@d2Qy zoq>%)13(?=Sv63|N+-tOKJIX?IK!3_m}Dm5bb-ZKYcz7_CBJigaPIGFAWD{tl}wVY z_B3&CKuKVJ_bZ*gK$HdLlJXR9fuhmQ7FdC7FrRlxW8OfrVIjb)uD~{BzAyFp*!4;y zviYL;s6htUBVvPpv6bzFxO)s$&ia9gB(u*3;cwp~w2G>$Zm==FqTr|aRA}R~B5}*b zSEq`{Qs+(d5nUaw?*Xx^@q3G*tLt8ZR(?{ooNSGje})vpIuNxX^*-AKV*X3LX`)&B zN6q;0hNZ#IHx60Nv|Qvh9TYX$@U%=0dABXS(p(5FcdZ#vaVKcHwZMaG!Jhr)twIe_ zu@#K_(aQOp+YaZtS|Py1}JPNx&%wc8r9d= zzq%`oqQLZcc_FAM%FvG!bstv#Cg86s0{HL=WNrrq5#;4Qi*Ixw01Ng}ICrnwacdNX zxvMyo>K!;F7Rdmqqp7-VmTVGDbIlLdmoJ0(UBdad!s`Q`ZJ+IhmWJ%=oR5S}-qV?A zLh&dEl@FuM#^Hw0H~S?`&0c3-Z^}+cm-jlC)qF(Le@8CRciMUlB|Zr_{4RK@_jT2a z_ifmL&HswJf3k-u#AQZ`GV04DYaAwJWB1+SW&HJ1WQ1ueRv*T(QQrDs28xuc0xBkG zBBqDwI;Q+n&4!dc+Ey)$wt_*}X`z5WKFvw;CJ4~KePEF7 zdC}H-l|vX%r}05vTuAdd_)L$n?hcxCL!IFg+lw|$i=h*<=jh{jp6|6VA+;j4x4osd zKRgRn=$9(buDVzGJ#agy;Aax&kK>5Y1-aSs?P$~Ih&wPgmJYyxpXXY>)k4`ul}Xh~ zRhTs_+4eWmg3@^S9%NT693d6i4sR%;f^3>$KD~r#O+OoK8|KhVSc$hniS-|j26{Wz z7TEl%EZdhlwJ0ve4j1%p6BwPoA?&!=Tb&vZD5U0YvTLBPb!1itbrp3ZoZv&=0mGzrg*&K6HGD-Hd-oWk}fS&6oar18D{V|3@Ye1nD|EdW)_!IIuv9(fLLB z8C%J~V^A~@f!`-Pnr(b%2eT_SokQ6GLpkzY;%-i(lg+0_MB-KUvV>d4tZp5)CvgS> zf$V2?kL2&aHsI&+N|Y#L(r0p6F#4&d4fj*{#%+WbO3iCU=z= zyK2J?peI8HZQOotc3?eYN1Hgl2^F~Ix*L)G80Y#-As2*FYn9PAm8hPJ@&F_5KZA$= z71@CP>J(5+g=1=x#iOafQ?8(<7#U%WEkHaEa>^H?QxgjvTS3dGkM2NjqY46rwtxCW z;xOv|Zd1`qAsxnn%rr(5E(KIRB!%>P=4{cah06B<-1Kf{>!Y`)`B^1coc9kC2K~CxnAEDL*ymXN~pAxntq9aX5dwKmw^L}05^-W_?;y6Nb zE&?GOEqrEIlGAEN?XzuUu-g-(++T?0aMzjR3get+y%TC_ghvvL# z@4h9bg&cq{PkmNk+VpWdLzU$PRKR&+0w(XCUQm8@xl`Ob|0;(LcMiQ&faQoxZqYXk zU?2V`Tw?3D9f^;IFhNtrVG2`$rra~k+Df>p&Pdc3PYp)c6cn|qu5S67Lue^jY3czSLF1*(m!9s_RDVeQLqm` zyWm5yISc49pN-}_YO=#DK~gpPb0Nh%k0jF+$C=s)cE|Ro`g2`A9NQ3*Q@2%o*yIgX zjz`Z$HZXoD9lZ6QqkjIC~&j(>JC1wO8@7EM~w8k+<>luugv7Hj}h z+!a2iBB`k{S&)YX#L?BAB$L}QcTq3xIG1Fx^ysi#UwMolWzU=}F3%je1IRdv+Bb0X z_EayDOIY$BawChWuP49FL9VCmp7mDGJH6u{`k8`d5%A(|S5BqNkN-ZY;?)Ttl=^6^ zV4~Yvyys1vo?_YMK2>CpO%;G z8aF2z%%PPm)m5mV&YlW5f0PB!&onj(1BXLx@_OOO--J8S9Nzsj;HauN?-(apl z?vTRF>h_;k{?HZjyfy?Q*3S?+5s`Xgk?mQaDl{ssa;GHZr6jxSyELl`K7&l5@X0Wy z68d6h1z+1H&M?=J<^Hdk;+RJ3<@{`Zb%z&|(}DY_mbu|u5g?pV2mH4$HtLpbuM zlE0eKp65dx5+uX4M>b?4TieY2O#mad}fK zawkYN9n;LC3#!t62JYaVrsOg%stP51><%=Vgt>U@mDMX=z+CyzX>vK}w21N3ktx6aSctElt^>z^p)KP3+|mnf~o(iiG+aVakHV3P)I?u}>qT9CY?U z+sP4u&2-VKjdt!zscET4{%dfO0T^tv_(P7b#-A?fpECz`8V;n$;U-e>!InX&j-5V5|8c(B%@ccSL$miQR z1PBPaE-ujYy(SZrmwx0d#^6m*p5sPnBKt~zZ$e2Dsh~) zc(wh*PvdT(s6Hv`HQF4jTV|0vlx}8#BzKE~)|VkoXtar|gS-_MKvVfM{8qVXg^et% zxND@TMD|vib4w4>r(jg?pIi?_GitsA*yBEh#m3(W&3Pbt`x%G$gZ+|tcN^{2J>g${ z?uoZ;i_6l5_}r(S1yp67BNe6~?YeWov7=c^qlfbv(c69QRMr zeKuth_!_+j_qq%+%idDmxyu#2aBm@*YZFJt`lYf9b|ysOAsV}4RiL|0*l0^>Z7lrd zkvYr5T9yR&ANhL)Sn@<=z%aWMvEntY6Y+&~ep`4~Xt1kn1;>^WJ}C_3fr@!8473(vfDa?d!1l;ncmo z7?T4m&<48&N^al|Hc_Ab6D5f=Zw$#r{|uMBfYCEa@sG1N zXn5kR-C>}V5}75y;Qk)mJf~v>Wz5ut-RzSTX&fUB*y5LTQ7i+ZA)KgKPcQg zcxyqNI-sScw>W&d=q-c`3Ia33ha<+v3*6cw9j+b4d`$$Vg z_UdhYeJbrck!+5KVHVOm54Y?$X*B!Z($c!7o14#+dC&UflQygQNg*bWOtFkX+phOz ziwXHe6sSd~_y8Nd;#ZQ{#~2gfEHrYQyLvYg zG51X6aK<=#!8RpLsF>-muKJIBhalb628y z@hYJFYGab)hV9iAlK|0_eJzcfz8k~%*ospGmEgdIjyAFIA&nx&%UT;Ji#DI)=&aHo zWrdV2XsH4r85yNOeT{c=RlZ#2R#C+K!p^_KCzmBf34R6|QW4Vmn{CSus*w-bt5%>+ zf{tRb=2&lmJhd0E=APWEcoO4I62tYUjv2mPPu`H5C6lS%u}v``8c6?TVezcIfgp!A z1{*J-&}>dZ^EK}&x}WQ zdg}Mja{AR#00oK5;?nzy>pH!s5Y-@RTRH$>ETJdaPf8!ZMgD&HenMMIer%M|-pufF z&1b3GG4{31Mfdgb(>r;}+1M5$sRb_D{7WPPTfmeOPbhbr9H(?Gqq6otbfO0~~47 z@%NV)-HELFnW5ucu8`r#CB}3G-*)nB_)I9 z=fc^hvTe&Tt3B4gY}c^gee?nGtPF!TxDRVjVXk#>h1`ulwI*&04v_N~mJOwHP(=0> zhx-E70ru#BfN=4mT#VL3-}l%n^Y=={58mtxB3FQB`h{5K87JnraLxN3DLim@QAiOX zovq>Q$~(PSSMwd@vh$exXCd`qkllyM#O>a49xL^R8Om_ssIknaQg8inVi>_(4XAl| za=msQq_&&&uk#`4t}0JN@4ZNJxq^}Vg8`1NYeB~K@HBCBtag^w?)#=PDhBt$f z%M42SuFB?Q!&YU!-A}ZVN-lg#{<`2P+E)*!-iy%^+H5}5VdJPfksqq$|u(r7#+CF&yIr$l6ppdi** znKGuO76gdf?#FQdQ?8nd2>GOY9nH*!&QjEt1lsbxoW^5Mwp<3QbB^Qq>d%8C30~HI zJE`we0sLlr0Vp_iTDE-=TC~FGK0af!il~TITa<^$F8!r;0O~{QGlZm zBqh546mH9}(ZLor!Gqsq)55Ikvi!1S*M~PH2@rTicMZS9@c|XK7@KF4_lOnR29(sF zDrG|F^<|4)4l|beu?EGJ zrhCMEK;`?`E@15!C5y%!M1Ras#U>JlPaoLX)Z(g8Jsjx(u1-a|O;m^kP7 zg-Dm&AMJIu8@PwY+OdumrdG?w@EHyKAAPnLCNIX74F*!E>hOK@#WwxI_(%O7y;a%< zfF;3+Q@^!(@Y%mtPfvgJ=P z=htab)OgBjAHKHHbsU-%)Np!b6BV_jN$XxpYG#ihpXEoGRW-S8_7N!PUWeStQ5E`p zgZQ4ztGU0ry^!lzU|1atmZab#M*#PA{>{DJD#Z@FbUlFenGwerh)0*CXBSz+;9nqNnw4?$sOT! zc`MP&Q_mfPWGoN*Il2#8fKt)Wm-+`eFtcQ8i;LROlpdmyxn7l?PO}{Goe-2u6)%JN zBlvnHX%q{drN5HF`q~R#2KWuiZ2aKnLlLxRtN2)npR-;YRpqtbqpveJFh7G#d%1RH zb1wpH8ky8bCs_vm?fd#tLTQ>WZ)Y?+7ZbD_yf?WMV?B_&_n;C>Gw{QH{q>3Zk++Ti ztz7C-X-S~>debNjcc^C>mq+u=ufW%v?FpuyuFPN5qDzbHtl76esATw46suixJF0(r zASR!~b&^mb{wbNQPc_k>(XC{0wMdagn`-Pu&P|phcM;k9ham|>lJf_gF_|VvnoZ@{ zJy>XBm!c*?Z~e%2yk@|0HsWZI;mw?g%4+M8XUfNl_qIhxDcYTaqvDSe8+UqJA#T|*tlA`C89IZ>r zkw@=ALsaKXI()nDF%ov|{XbgMAW`M91FcDcQIc6lBPQzQr5on5&0v-XZp9=iYSkWF zTJDB4tx3)2#a#TW1+f2kxa7~~0x)c~{h6+x^G9w)nCnMS>Qz@f!v)fa- zj$}VF`0_4C@3u0r3tH}`Yq2yy^B9V#ulK)tz*i%4M3%l3BaMGprg&w$u*Di&XYm@C z4Xi{zyY)m15zHDxq54bKB>RVVsOU!y^5%p$%iNtzJ!-rS1nt*wv!UZkSTer?yBX_! zF5PB+or<4*huX#pj=c&%W)roqCLB2H$LZb7L2fXT%JWP_r7~5GXxMkg_Ge3hzww*` z##oC-`|u6^AFjH`-8w3BjcaZ;HK|-3YY!a8W|ZqhhXGme9>mT&sW!Ul01_51@ zUL~m)af9ex6y~Wsad$;;KInh)N$QXTzW!lp#ZTf3uh=JziG2wkM?ii}>4v zj{>Hbnm;f|>!bzVceT?VBB7!y69eu?VeC217DpFKTIne*iA>mHR@bPF^Y4R`5&XHy3xYQ>52h9#T z#%tVy0|Z&*xWBB&J@tc7!uxp1_H$cs8M9ul-k$V0i|@=iMWf%F*x8sCOyrcyvGP6V zHH)a|N=y&vMJ(gYO^R1<<*UEEZDtlLKKk2wrt2l_bW&GGU3tq1fn#A{QF-_8g0xA9 z5|#56%P5x92+W&CYdCap4{~^9&Ja~7eHzpT~n&o&Aj~-y|qn)f`IgovFL*W6xhQm6Cd`yU- z*P)gE>|F%y)BuJZQ)5hI6k@y4#_r$jtYAF+r1jUV*}jwU9U#a3cQB_XNYY&atD&y8 zgVEy&&F(1SNIk)_4gU<|uj6gi`^Dyh(!4MB`;c$N_t1`z==InBqD**>3F)5SEn1Rg z6dmMOvVL6pyR?3FDzqP6yX{1u(oT`HSWGE!;zMq}5q!yx)PdLV9SuXN*L}4RG)d+e znVGCP{wKWDH^L_IrKB9m)2S?1u_`U>!deX4MKXPf{=Fq%`aV$4^S%@sU ze_U5D+Y3yOMUAcIY2Ryd(6s&_P#TFcC~e=XFr|OU4JnhdMat@Y?h{#n+*ehMdsr`0 z6wd8v+FMm%3b(7&j{mp5f2a*%#{$ehULd;fXvZrF!(JVuTUm0)GrwakWk$^bil~=) z8ldxXx+BzD&i zutjK091ULEB_NQfEb$TFpM)q=C*CXc9)Ie}68W z$GKs&?!WVOE@zhuwg=c(>A>4|DPEBup=c{U^T1Gs6@p?2Yy7vr7Y+oEn8uC`c# z`;?b3*Tc3=G062>_4tUL#zbr#|N&9A6^QLNz3rko1w_9ObBF`NkW_w0Ti!=W&d zA>Y)K^tFmr+2fFsTsE4%=s7hQk2nq9$?w&;%fl)!2hLPgVN*EwA88P{U5Id1=o4jp zMta&#$9BWByQ|%*J-vOz^yxEBg=09pr{M*a1q~-z5eeDi?H;Q8Qa55rp5|rN?w1^! zpGp$#|WYIdrV?~AmV(m{+=2>Kz`JVdR zZnc7iU>20Ncc@gI)w=8=G*o^QFzX^PVnk~A~@PMT0D?J4NWutze% z=GWls&9dp15$W6tRBb}$p))c9!^*_o3XdVtUq6b7oE8hqQOyRd`7B3&4L_Qoz!O5f z8xe55n!$J|IK7f-@cU&cf!9auQ-tT{v|irKGhwd}?jbYBsBtEpcNAhoTx&XFc)JEQ zOxT?Ee|=a3sQkj;`@heSW~IOEUcoQ0%)fHqnYv?=_3D11Uj3)1MEezup2Ve28`Zp7 zgQd%-Q`ui1yU7=xDNp{I-EBs*ULgnwXaW3PSL#DuLEI|*6v za|$~Jcr0grOg-^B_DGS7^sowwidpVW@g<}d=wE36N`{)q)jhq{p-B`ZHx-{n$jj?> zAN#FE3$II8crY$U+d5;`OI?=?s1SPNZ4AQ4D{Ia|+1E^a@%9Y4_tMSj<%HIiG;WU` zo@gu;40e@N`?aBrhu`)%P(lOSj)kCsYeDNqZq1xb7R=wIsOSa{g;45BQnP7WKIcmd zew7U?URk7{Za0TQ%W(&T=F&;T7aHm;BVW6+dP`zUft~sm!cbQo&pnNS`at;y!Q)W9 zhy@0(qw;9#b$`-D#KYe42g_1Pt*=k2A{U46^S_mb{=9}nGBf1~OTRZFb2Dybpx$x5 zZBE`Cta2ZT!>3QbFLpiqMyp|bZth9E0gLzS%|+foO%$_SOV?mx){_Jdh`mer-P7`@ z6fJ$pR22zYSBQO0e}UlvLTqY7H9D(A!RUqS6Hy5ENw`d)<$8+Q1JJ4;0ABk7Y?DRP zU$#Sp0SI6v@_}$H;jz2l&aJED?AN-Al~d*OW8POV^IZtI{peoS!v3g9?g)Ueyuzop z2|Ar7b|s*P3Og3ndv5?RP!aH)Nr}5ewU;+tSot$X!1i5~r@J^E)%9r{FT)NH7CLF{ zr*JC)zTi{nT{x~E&SI_QENVpEFwS73dO!XfvEW8kVSNuV!o6_jn$Sx3+npRmXFSXeQdyv zb1p(B!v0$Y!eQ7lkvY?^jn>~=2QsMeHCa3!@X;|+=j(UAiqT#;8-@=_J#l34No394 z`7mMrRP09|+3b#z0`}h>KsaWK^vl(HYl8LPrIyr_X=+R%_}BA8dFJ{oA7%$=phuW? z-6(BcUbbrNx!LXvCneXYgdAVOJX$x%G8CyIoXL+HYBDfdVKjTkR<29eBPA;muSTAx zo->H(&=AoXowBdDy~fEFABHR@9Pt*-zWihHb}xa@M8iv7{-q)ICuut z8jBmI(N9Erx4**%Qg!r|1;Nk2lP!+Jax4t;Y)OwKDO87Rkx+0GE2s<4u#syDz``nXg79MY^>x3Nx2=_Yxj2~J9j!;CIzp;ixYjzf` zJ47EDyt-B!ILxQ&-0HW8Fk46_2N<#DQob0pAv3q;U6Q1K0WiUA%`2cqqte`a*y9QF z(36zrRispqt@LTLk`WY*!yM@`pd=WASQABfo0H*W7TpG`HMKCR+Cot_nsdfpxjl8? z>!gEsGB(3Q#=>Y;T&dCVn}-B;f1@CqB%gyASOyNaErk>diMy?@QujegN!lyVO2 z)9vk(q`q=Zw+Xef@}%q~-P${=XgDfdGIHniQ+_*evmn zLiQ2Y%(&6)PP%)$lPEO5r^nAy){n)c4q?EN#@@de2btv>WlQPELYTZ1%xk#xz6o!A zKmV0gl%?!a-f!8fBm;eq2jo!YlldDbXMPwV#rwfsryYE# z7J=6-RT}wI{G#4FZbKnzy&izBTa1oD-*kHonb~PV2Et(q#4$xxugzF=u{&~G?Bh_S zGG>5S%P$s+MOqIk-L$A}3V(*B_B z#jaVx%E#0&V&0wbcnzU#UX5AJE}gi7B!th0DSu*9x%wj#-mNMPan7MYLOh;`QZKRK zA*I9;>s`CkN-pg`?(hYlj2RaWNjR?h9+9ZeuUm3QKo@_-L-8@>2`v)BSC88glPp#- zd{42^6fULdkoyo0X>Bzz4Y*@39LVU1l!$D)m(H%++1yq<03rX${3qx2N8?b#H^-Xz z7-svAxol6WGi{+OFGWyGamP`qS^M^u#Htygv`4LHVu@Xt^$*p1s%&F)Qi@NS&8Ev6 z%q>(Q{+Rk&7lZc0pvG**RNgH z6S@}?#G^}s{F2O2)$tX{H8MXWL-5GVg|ckwyNh@vXCjal%eQNOGus- zjmpW`n^1*SDUqY#`jPbsrwwjriCJ(2Op#+V7#agatnGAPEikquUIS6OxXG7X*-Y4J z@7ig76*TrjXv4H8lm`CDZ{U)7&EeZ);QnW-W@eEVXLp6=s3VGl-uVs%^5~28lmxl< z8**J|OZ!2ZhkBxFBg7aYkaklQV%{>v*@W0G_cpDU%@PR$!$q-B3y&X*zB?ojrD*Z% zgjO$Oz8Mf4_)^XqUw2n*OR9lO``rFk_mgPx)Cs2S zA1Aawzqi42lHGN;a9FP@kEMwxU0%ymX)!MShzYwa9g4Jet5e*=E2L7QG?;u-JED|b zWahF%3txHIb6px!%Ub1j84~@*TY&>`l~;*ZoJ&OF~h;)v-=JWo=R&YljR4%JiJ-)NeM$l$2n&ei7=Bk4TIXsND!7`K$tO%@I~ zn)=D>a$vioES65eX#OymXDXh zbJud=8FBhkYCIuD`|)oosJ?BZMw%OiJBO@tak=glYM^zyzGg979v0zI6Ggav^Pf6M z6rzE|*^uFQJ`sr$$W7B$pyDgup%h7slOAKoT3W1_mx@TrJ7h_qrhi}ADAy&Oprbd6KO z8)QG4UhVWWG65;@;qCx-IvG^gwhra(IIsW7ZEh;Gb(hDV zzY)sU%ra%M=%qNX4&o5g ztRainx-{dZC0*i(;yE_yHQ7R{2z z>4KjL)mvX*IEiMib0a}sM!oZp!DS(P{pIm6;}=IkoCfu0NF2pcvR>~$O%p4wNN}q? z-DQ$5+p|>+#}QxB-LO9qB_o}}2;^$-!IJsEhDj`%ag>zFGas_Hah~A{=Z^R14l+#z z%0biU@fmlbJl@1M+pP4pP7+kRLuYL^8_%8@QQ@Ajzuvy7sZm$nXq_G;&!U?Xk4V}j zdgK?W`5~$8&?J20>W1?Dt;xCJvOtP7n(HM~^_*Ud!kbZdJ^9vh$J&zuI5+fP{N5J- zxG33Mjz=x#OmxWU7wPzZS}psn-tp?A<1ah~C=+ANj{FSvzYF6wcyDUMP{T6W-s5q? z?v||0Kn9Zuw43HrX;+A1q4$bgoY)GP7t%MnbY3-5N8vEBT$Pz(4%^uA8L`vrzf*yYc3z?Zmp4mU1GRrJ5Hh`^ zP?CK_tgjc#Ihc}C9dG&5z5z$zi!<3&Oljids}K{j6!=F(Egfm0GZdji5o0`Rr`k8D^(a>9?>Y z>G>l|S2ua`ytThKg6xvc5#Q*ZF%%ZIxJCkFjN2zKlsR_Smk*gxzi=qsTp`99IvSdD zNT4_AWC-U!;GTqMNd<0EK2Cd7B1Ii*wUmGol6KXe=D4I?jpsVlEdoisDl^>E;$Y>A0?kJ=z%r+()?2q zlq0wfv>O6xsvG8@N5`Od&;S9|6wG$t>@$MJ%6Nh^qz7zaXDoGMC3)3y!m7B&9uUe6 zU_Z*AN3?_B;5aIp`}yN0=gU{T>xu%ImS>!isXNrvoKp|o>uf^aPzJFoQLhk%FDZH6 z`>5|tq~`a?e^&A_B!pl}K)x9+@cR|JzokxEXlv^7%u199k%MWXj;YPkM|oK%K~%N2 z1~mF{*GgG!&LbK**HbB-{Ew&jisPkIZKty8Sw!=^__*=pLZ0?6Pd6CN?sJHlPg2pV zzyu$-u@2R|)oZlAj*t+_e8aDs8r3=W){MzU`fuy~cPreH#pzlzvnLQ^fK7YR2M~}c zkTEG<)7B`n__hW*F*{IX52W~pGAWhB3pG-1li}&Vq$TcIGLX$o+DeL)K=y zC=WJUW6M!T!O`bORjig_%Ahl9-~WGPT?aVT`~SDfCS>odL`KA+;}Df>smyF8B4lKo zVIa$2?~CUjNVO-n#ex{`cwW`93|j`?%+PKcDye{aSCcRG$07 zMziVe$3bohv1=Ww0%Ee;lZK5DluZ$Birk_CG`8pAx6SxDr%+3lAr4s*#nrr<28Vi2 zt|dLG@mUsdohNh&=*`K{JwBsMq@FJAhmwSL7>rAcsCtn)qtv`HFu*oW#MpLs!l(AS+hYIM3YP`it@u69nK#OI zCrA5%vbtdA(k8Y`@;&Ss>1s~V>IK&G+HZDBmw_z>a>bkdceTDd zcgtBX<<)u*L%uK!1;9S6uGXGE||NS^u#Is&AH zajT&;4&X{;^IzF<%YQ~yI?MxO~objnJ4S=;OOM@XuN-as%f;C zc1h6%_;uqfo9K46pw+0&9?@qvP2mYPs&jlQBij7mwUt?0b;+1G=Y$>M7M5e}x-5Au zI>rR$QjV}!nh15J4G+%G7rLz7S_Ba=*JxoSgHW2}r*_Q*U4%(IK?3Q)1CyIhVcctQ zOE!B2^pXZ!2D?=@QAkOIKUd4wY4XFiYvj*P@05~7>7eV^l{Suo(z`dE3NM#vdYl44@?5~UZuira0@HS2DJy;qz zPxq8@`*jZY zpc5QTQ;AvzFCf(lZX0ds;-_rGi5i6e&z(^RYS-i>U|h16LEnJ%b7Lgx;z|4^E;P{V zJuug2cLdFPKET9Sb*0!OA8WTZQq{diIzGB_4Z!%K%K&|?0+YDsFK|z4*o<)0eL8dn zD}VZ>UJ$1VEl?A!mJB%tJ8nyX-J}l83W@ZK1au*kDDf6+uWs2T;mVVPr4?|mkZ6V& zJ^hmY1HRc=|CO4`Hyn15S#M4=IcynDX*v8jHPjC5(k%tpp}@Ry7@Lh~&E5w#Ji^c! zd3c>_)Ww)sWrV~qZ@7u!zpfQo*qS6-4!y1 zcO)kY%MQC60T|Y6R6Dan9`b0R{3WC>+AgDG@qP~BIp1mexqx{L(&v10i%;>k=ItP! zwWBmUU3Df2f|l2H&OunA-uTK!nNqv&g*yMdUR`oWo46?`;Pc>NOTa_2fRd+`fGu7rL}L?9_Nr{($5=dux; z^UsV}05*@=Dzo}Z`~K<^-H5%amCQS-1X|J8Hd=Y?!DvXCy>4Ct%V{GPs0Uz}xcahr ziTq1F6A!+e? z9U$(3q|E?XJ!5?f@}u7G+_f3thfO!9ud7_Gxdx%I*scclSkb_NUCew?%z+8m=a^51 zS^i$sc*!?DVGPi44GTnpk1wRI|G@-uuHg$URWt@4sQ|h8g4s^b$iAc}V*>bj z>p8l-9TS=aLMMZQ1{i3NsTY(|-v2ZlO+GO!F@5@jttjnx(f(DsU2YJ#hc(V6@@J+B zOXu-y^v7pw;dMu>J6mlV8Pwe3IE-$+nGmMz61h&>y_cVIdD>;BVeG7H1dN01xH43~ zThg9)Q^i+P5u!_@ye7No1no(oMiWWT(>OHyTE^G%M0hK4Y+5438O&6@i*Fi3MelG< znOw-#hT(ThcdDE#nI6UlTCyHqQejvkZCf9k+`plMh)m1XKW+nc1vgFj>2lY3ebQyt zN^0>tB(tNiahi#X!nQ{nDW^QEIT2o0UAlCS!{eVTq@At~@qqPYk8^QWV=>P2WFNZ% zfNL_Ur;q7T!vTHly7@|VRz1X=a5Fl7brL$y`1nRuH~kfdIAb6uKH17RORNY1jEfQ* zY_oDQkTtM-%i9+ejg^*km0?q zVnGDyOrauDXV_PW?Mdz4J1lVz zO}zJK-2l1wox#YC<^Xx{O?vRE=hnP7L)Q!ao99sLH}^o^(z%4){1I;5voHO*7LA^K zE@`k1pA>r)L!NSN@+Dn^Kv3<$Kwe;>`?{|dPpZ#jayJ#fW39IAfZXvFbBfvGx9RF~ z_Jv`ooxozthzqb>q!w;c{(xEJ^L<(&{DPRY!0`Y z@a<*gl+t*{_OzN7soU5eB3a$PCP)Pd8WLyj^2t`u`p>hN{CAuQhxq{X{(?<Lgv-? zQG*;s{j)b+Z7*6(cf<(rHugBGhU&Q^LlgG-x(A>q>Se2mw1|e2VbLDT1!zUY-jSFt ze$z|(#g;bT+tkcgnreV&JK&k)q7rP$jBVTECJe5_q!~IBeUA}*%LkkZ=htP-9FG8w z#K}erx<##PU@P`ApmQj?Ud7}2`*S6n(KEE+!_hf=aB;TdKx$5o7Qa^|ai-qiF=j#K z0Pa$Nkz(G(y2y1GiF6|xO}ct(g{4$nV~j*l$|z4h++1`)TM%E9$m^9g8A^nYBkpst z&m~cU3u~MN33PRv5b1gNk^JC(@FYxf!K9c&ZhJ_)7%2&6>5sx`qRnSZ172WXe4+g) zn4)CH(bt+0RvIvJonv`^j8Uz>xgvPrZursOZc_&EoxQSAU0nRueXH0Hfy7fTD#l|N zc0?o-F{V|k3r#E^?2Tb$XqTdTnY?7M1al$S36magLVCLyN@4 zpjQ`-Mf{}HpGwMN(9Eco=8UI+cKVe>_F&64ZF|LdP)+2ccDjEsP1Yoq1FZX0YmmUj;XCTr$$ZORA8Q0 z4q>86pg!DP`iy>;`gihQqG?Wa2O3DjTuQ#;i#{>R>t?$nzc;=iuk8U-+iX zNm;dSFcm5_axs66?jeAwQS5HR=FJtEaByp$rR{0dB%GAaw>g?ydyamd8Y^pB7M_Yg zX$D)DY|a+Neznzm`Xkd6p!>}( z%6X2|3=_zhKIcItEPKBnWc#*+CMi2XuZ`c3m8khr9~eQdf1s3QYVO@-E-j&eCVjhA zFv0K84x0=ZKana4o%FOK$aqx_THR|nDsP6fuHt&nXtlk#LE|;S%k!4IK z@&w~tzegc<5F{H(02^wDadFPxsm&uizJ4mkpHndei1i-p{!l4x`Ghm;1{O_|Ysi74 z{3&k!cWbGQ-T?enajj=Sw7lfr5na5AC+b*NUNFK%Fg1-1rxp_XAOhEA-u5SjDIe`k zhmZb^Es`FS25kLiJ7ppGM>YIElm-m-mJSMxrlsP;(#sy2M22}pt4P_%&ePxE;;&O- zQk{`Vx*w~<|J2k5yb*J}zDtBjZzk~w-kwK>D%z7`ynR{OZ6a>5;YZqtBHoVdS3LN1sPqd{mN}Mlk2>GoKgU(G{Mz@- zCT#O-lXi2_XpAf#!?L|1KfktE)_q|-68?rulmnR=EUu1h13Bcb4W#ScJ|{(hk9=hQ z>{2c#>K={|alB~7$@>S?xshx_wAaA7!u>NnT<=}ny||%)YbKd;GqZv{-|E!1x-*KJ zf@88;su4-$zaXLkKHn!qQb%PxiI!YnCS}vKnek8f?^>MTIjM3YyO!SS8Bj0YWjSsc zi+aAc=@i2Eq|OOB9T|zXiFL_B3ojqHz&wo&*-C9ZdeHmVQ{=A-V7j?m1U)qGD2Dr5 zhFqy(*wCjqdvMp8U`)C^$?SCR^58&~gzazK;jbq5@86T35-qOa1qF424;@ zaCr3t?x)Q$TR`Gxwr9z!^@Grk*9#FStEG1Fu7Z4uK9AU8YH00|40d}=&jcynykHnY zC+sA=$&If%SgqRKXNGAv3b606*&&Zujq@@ULfa{Eq9z<{JvviPi)s zi}8rB{|6yyBVC)l$}8Rf?tDg<+D_dPd~E29T%YieU9Wt$7>8Pj;9&CtFKc14eI(`P z#VEU87n9X#6<2O>1Ik{(mqqn32WVb|bv`jvo__fxaAN)*5&INLh=gct>*`~C3Meru zkdQL$GTU8fOYCK%!KAs>OB?6O=r9Ea!=F6Aj7e8<@T!a?-)|Aj3VxUnK7Gb-dJ`KV zpPOtwf>xSjJH2i?iDx8?VZ+Eox)^s&_3P$P&YmLiJQ}~2Yh$)4hOXGbIw5JlXsUk? z_rK;DG`H{m=Z_EY^QV_Dgx>o{@Ngsf9g!np$hq42hpbMpl`1O_AdSkiOt8QGNC3}` z0;Xv!q$11c{+0n<{sz1GwPC4u%Ag(XW~Zb>q{^*;JPUd4*tp*T*)Uo;n%gHO6`=UY zOf27G`3h=EwGhVUeuQCKcM{_k=sxF}W)objxO5Kl<(}P*li|8`j5G>e9v8$&4&gXH zd#GqL(=4S@p<|~QaP>s#X+XjmkKF4AOBGtqIahx^MP?;+1j~?plp*Q0Ii~~4&HsP% zlU+&fJv>uKiZ#)ueIwBY%S%S^@CPpbc$8g+WKW0_AFSj&L;dvO-!Hd9FkRiMh>w@$ zs}%HpfrcThUFXqd0lv9(% zzdWHo_-s~7K9Ew?OWMO-q^)9N?eV$!XCBV+!kxm8#KPSy zmyuICUxmy?%uk7P=YED};t0EDL`KgTkvr^QGrWRVC9zET0ry*l>RjWKs_^M~gPiGo zj-{gEjpqT8L$tT&FGjt(Z6t!fEOb=EXryrt&6Y^O?UAd4bdERrg195|ACTg&DzZit zD{MeJM?sEwTQ?fVimqw|xr3H~7xC*%k{cZa61O#qBhVIo?|3U=-=BG>{r)lF*8{CF z)+k`g*j{`HlDX!d?T4yj45}&VF1jn=B!mF}NU&>GWO#4kdeM}KMa4JjxP}@~T8*XO zT~v;@Ej3y1%19Y3wb{m{uQo4kxbC?>=)&$pmK|5W<@j0p&47(IE;?+oL@36Lw~V6) zlI;q=*i~PWdq}*zqRdWo_FXTJlC}Nh2;pd??%Qq-u29M-7A67js6Hvvq}i&l^>WJ6 z7+!NVWlmW#N7xI`(wkr59|fZp<9^;H7%Phiv6Pqixm^(E5tH8pdvTXUdRZxV3<=q_ z;j_=Q1L@tIcQz7~Sk@&MHf@Kl4Lr~y7~9>pi?Uxgyg@2A<#I0VC0IA+a9g4VpfB2? zqU{?6FDU_FJV)F}bwF(QT85N>GMxE=z&slX`*c;VwwrbWOX+|6E*zqAmH^lNB4&k5 z3%;c)Tjz^RSOGxQ?O-8ev_I?aN{2*uu|yZ7@hk>Wzw63~Yrty(tt303btneF z?04L_1;%at`j2;BOF-9CSNz?3yu6iYgIk0~ zkn5PTb_WyU_d1I=D&wW=9L`CjZo#(TZ@ZyY%NcT}M|}*QElM*{^79TkqOm)!O{F<; z2g0SoQN*thO8vonk& zyAVN40M(tYF)Qg2#vjs@${1C+*yyLP0z4Ec zx86io3gE!e2yP`s@5ra$KHQXgGMJ-{4x-^{keCC~FTF#66_`=WA31vMpBx{Kf;4NE zx!y4O6L(gCO~xz7L!3U~q$M3+KCB$B*2-_;JJZdN_M|A;cO}S3IgCS>hVo2flub=s z?cK%Djm3XTl0prMV?*t`!gsEIx`v4OweBYv1T~+DSg{4BrhXU>)kJ$(O>Wz)E4rq~ zsPq*Xw@FsoE|700A|i0}yC0OM&LMlE>MrM&i&fn!tMnG9)6t;$8L89JIIBi@ta$}3 zRrym!XM)VM6Wf?s{c2&pFz=qE+WXP1o4dC{8k3xcFKE7k4k9`I;LIUZ72V{mEf?4z^uLwPc>EX zbfCjZu*W@ox1ZyGrAxVDtWLJRY#2fCc6yg&NKW?5m98beyjQb3DzpE;R&6L_0Gdkk=fBSoNAHvJ96S)P(I@EqzNxlU2A584tCDu|K)CQXC}*VusuR zO)v_gOD%D*Fa?zl&~RoYy7a2J9d8dIx9%vjKkUoUD#J-KMQV>W#85q-jmp+2UVHe1 zl*guntp&?!uia9WDE{wYvxoiOjrG;cOEtg`3_=#!TLx=z)dDq-D4 zLFI1pyqR8axP{fh4i`J?F7h&U9Led;G__|X3P75oSwI2VgG%wJ$Db&Pd%bE8WQzn2 zoLQ%SB3gS?c=qo&H&-o+`X&nQyvB=yr)dwSA}uV%Bj;ud~$U= z&xPrZ_Ux`*p)mA=vTkmN{)Z}q7%K@TYGEEe`s0qD=d9BW8n^!M~C&h?R`o6 zUMmJ8hv`x62Y?z~J3`L1#C=sqNA71*Q;$Py5Mz0erfKG#O!6oSPjO0?@BY^0Nddi1 z9J^XOgNqKYVAiBiZ-_y_bdb|+I|?T0aHYkzn8%FuYMj;_E@PRgA=~PAd&91%gjcB& z#-}{pc(G-}x$Dt`yO4`H@DoOlj74THw!Yb{AbZ7lrri6`eQ-%;sl8~%HT()SV!L)B zh)4TM*M$7mwVE)A#Avi9*L3I5%4YJv#Q1-qw)iv`!|MVVO_<20oNL^rkg4ptr-y(l zWr9@)cW?18nD`vKRs!I3^wf_+2!8|hbiiJ*$^j_AR1pFg2eM*hlHDYVH!Hj#qZO)D{~{go$aD) zawQXo8clw;o9|xXnf|;Zb$5b%d*BoXC}dMuS8!i`sH@F_;x@yljDF~ZrV!f0PSIkT zuw2XX2-fM&{kDlhbaR~+CA#x=C-P7(tIDOF4o`WlVQ$)K1iQbtQBv~gHTq)M6WJZ} zD`Yl{W_>|*mWhQl2!;1d^6ajy&1%n5yGD7ao*JK$(U#=c@&Ls~r3_BMQ4+ zf#X&Ik;Pd#6RGxMnp2l%88BZ=H9jy^FK@QL#D!-9sn~6Q!JHI0nDa3f*n>!1&9`eo z?MJ$EoZM^1U#mSiY>jMrws{j9Z8c%r0w)ob$`y7quhX z)tbRd6dQY>_xi62tju3dhiP&^hL-4iw1~E@#%}k% ziHb{+IN$-d5+mST>5ViZYV*o<*S&cKP2x1O-zfMV@{&`TSQgY;Vx99gJq#8<2Zq+} zDgr0MfKm6(Cvn)oFUjaXb-YKGNX(&Zcid|fc&5w%vQx$r25G=KGeAA)dtr&WI`n`J zM#;8xmYTy3ckzQZiCLF=Cd0QVa$)eZQRZp&?l?IW5@-ot;NZox#C3jz#HD)9fR*}g zj$|K9dTPZBMS5>w4n%KUBGrS*=A}*X94^!{sS)s7{%`l3NrzvPAHrv%+fq^N!aXY7 z7-XvWK|N$8L_o;^m&C@l~8-7r<1f2NOe?6ZGv=G z`F&{ruD;3MM4)-6hc)74eM>(opSx4NP~w37IIJh;WCE+a0i`RJ!M+4LT2oT;fbwAW zNq0B%jV(j3He|rpJtlC({EVnc6{J7B`|RNn;CwqH%FrBM!$2JK=tvoMG`$nar?w2L z`js4E_bwT1v5&}bpN#V;5b-&itbI~nmmYU-1au5d63rd6!5o7#?UbA_=uB#!J)2`a zZQCDq;?H~CdIlew|LBJWGOYO7xCphsm3+}wfM}^8@cpXQoU=-VziK0XzNi+H!`O>P z$fgW*(bk|)2Qy#h1 z^DS)s5mV|xygJ@c}LQzif@t;{G)sA#CP{xM3vr$ zE+EA%4eKZV$vNIg&w4G_erz|QYV~&Cx}ko~_$_P^DmwJ1R?1F6_qo?9BktFlg3}RS zEJ{1N8y#d0aPrH4{TkrP2R5V~+qrJk1P9WeEy)i<)Ebe7`|S4*tswj2RiLhhuxv-S z?tye+8;M69BvOvh5W*xQTyp#kZ0&Fy11TZ zUQ_q0wl{_+NTOAa>v#zd7NmzgP{aNprdjLK88K+ZbC&Ys{H%vTSNOaOQZIV3Pa`;|&+s%qq>}Imv zSq7+H^_EjeeFg&BB_wTZo7m~%fXDxzt9hF{VoNwWEqliM+22yWKrIr1P+J^ex4EaU zAx6y&-NmNxEc*ZN%msAw9KN4QzP;hk%^S3QymPSLKukTg9z31q!h?S8-NJe>@9MSBZPT%@C&WscxF15gop1Md9*{R9Mh&JP)76)uy%nBq z?gt%S$>F((6WS%u5goZE&xOW(!^@+o@=kvJY`Ur@+9=6BPMm_?MhvHrY%i?f`sp?0 zRmIEhqs9s?rW%5lTb}Y(pHuVqTe_WPWN6$A9I3TQ*i_~8-G)=J4f(oL=9lwzk3ky01o5nhM#COxr$4Lws%6D6)M=T?6E+k$y$r3-ecTqHp%3*bryqSD<{{UX4{Z`Q2YgXk<XzZj~ebLXpKcx6D|2NB54#Ufe_5-ohKAdyp~j5^)}THA7#(z1m;+YEnt4 z1tQZL)-71fjE-{27yKw1%Q;l&Cy19+oOCN%8V7gxS$DnQSqC94W z!OORohPrdhr^tYXZ`1B`$sn1lMZ@G9WsZXVO)L&B!wpD7Oc;QB+ZfK5d2%pW!ZdCk=sN;XJOzQu(E zn#s%{+0wCR?PB7Et2KusAbr>i*XT4b{UNPz<(zAR2=ayeZK*ktjmK02c8Hn}@ zC1c%xrcgQ6R5fni89DpD7&_m#@kCr?;JYW0nCY9DF?5Wjm}%XU3wAUa!f9HCHgcW; z$~RyEpBGWArdmDz;KCR5ba;P$S9Xvx@z?p|tMf{JbisE*ipU3{oFJ>3x9K!@mw%RT zp{+pqtuVgjRQc=gmMHVh-NKN&y|;!AEF_oW!V!MQ{+9u}(`v~RTcsSbZLk!Z2oAF| z4Q7?rb7i{o>g%JO!Bdk*uPk{DmJ5d^D@1v}wLjWh$K7+6_2k(|XmCwG+a_Rri5tmYf~F=12-lQ=~Ey>Nyehmj)0u38JE<9c^o8#LGRfU3`+NIUtuV7F5Qb-tpa5n5t;=vTky!P59_vtq8y?RLqoQ5a6t|mzi2!3{7fo z<&OWNR}erZ&-nSk9pUv&+^EsPn5}TWa5#CKw^H#y81c$JUh_%tespNQ+7k&C)P3Z- zRf&C?vCiliBkn8jRKx3*{_+g33i!Ekw%5r95@hKxK5mH?b=PxucQl#TUdPHWsoC>Z zZY0r<+{&LU30Ry3yz!Dd(+t3H8f|?XqbotnWdGj2lr(!A_E}7ututD9k|6v+VZgPhpC*$7o!{1&dr2#$c=b!M4^~;C6BBwL3je8Txy|~Oa|@uE#C7ySX2$Bt-GPN zY)8y4J@f4qb#I1c2CSx-=#mDecpg8h$b3?Ub!ciYU9jRS6td=lgh3}RP<|aScqS$6 zs!TLOx_21m_K@nrQ7EQlJVWgpUyK^7N#tdhJ!2i*ZR+=oRI>dl(Xo{&w}-~o?nJ** zOZO*CwuZcU_5qxqB@}xpXYeM;_*%&$uo|cz1l&^X*4q>E6AmjLlugiBNq> zqLsN8p7W~cnXFvmBS@#IJ9a40?)dm^f7z$NyC*lILy6sEo{;H&Z@6k3!-zb)9Ci0K zd*Am+PD;oSID1OwSFbI{gKYZ|%o@e@tb+6m z5){M7Rb4c%8{W?+dju41j{56)0Qg$5Y*4zK!26GQMpg=6U4aMV9d4B3G6)^Hv*CRu zi*m&M70?OIB;qB-v(AY3z+zj&UDX-e)!MZBuA8p-B3>Q` ze2(^AUms3U?``%-QWISHs{2pxV0)9R{aA(Jz2Uaj_G^VC(g!#d$n(6*q5{0SoBqxd zpu{nAB>)|)`AxYTq98o$Qd~95i2x$pr2zT^+2=pmHuA~1mVso`s?qvo!NCOBU1J&d z;F*deK?2mHaI&cO_+YyTC#Djqn7>fDHx)d3>Lq$_<0}eyq%RF)=o%j!Q;W`nvs{5` zLU!X(gLyE1I^Zlgr!@fE9jRE#W!0r>YV+F5S@p#yd_iZK0-u>Is@dxo(tW0OX|)dn|9_#{Gj&=l@ng0-{PBe@t(E{E{FC zw%Nude0HkJVj->YDo5@oM!!JXy7qWg>9Hq_pLp8w_6fp8x5!kA)42%J&#l#x22JyE zy9V(z$&PS`JGJKEpE?H0Ry&&}bQ#7igQ6^Z7T+XpL(=eMj{j?pYzTU^#VM_5QJWFF zcMb>ZJ=P#dKcXYb<%Dz8OWZ8Ll9)KC@?qO?7_y|l+^UD*H|rL)m>OZUi7k7vwJ|L5 zp1_6^#pHNp&)I7z)IHIqxc`}u=b4DsLV5y;)o!pbuVG{!ww*r&+wm-L4%qoN-0r-! ztnpQ_{)d}V84BlM659P-*mkM@I(64m9JjQ+IMBXEYL?T6?A|?Dww5ni=*t+*H*ajn zaNZX*o8bg&JySl{`xB3XvKPy`gHu!Y?&XLOn854mpN^F~;fkXHunACT`jUXV)NNj9 zv2wYvefcGwWoG0zMKjN^h#xsMZrNdP|*Dvl|6WTzp81P4eG?+Efe)G*TROTf=)D) z*VpSvnAOztVL_<9tvIa4UCAhD^WlSpN3~DB5f6MH1AVG5*6u`8tjpSAzB1jbwQxqa(6i|rB%Se|wC=PS2PjauA=ohYuOy}ZXq`%=!)?nf7E zRGeD>z@XiQqc$R4gyW*)KKlQ`P=CWQE(ECk8S~7khZl3SPpYE#aPp2?xg5k zYgB*iud1owOuM$Lh&zCJT5VdrF?acM1srzc=TYGLeT#=eIG%hHKUV?KFit{$s2NDY zLpfOxUoQ@U&fT42ZbE1OQ0nij!wRX+GiRvx#=OTa0n{-oSMfm6!^ZXDWx{L!5y1@d ztpR3(NM3ZDuKV59Y>e4Hz0uyg*#`W4GY7gW>*rn|2QBawj%Ip*|KvF01Z)YJsEIT! zkd5aYiutHjq*hqF)<8~~P5HEBK(~E6sx*LCm*S-am+pldYb!508>!{l76Ya~{58(k zWe|zgM@oH-{HXjj4Z7I&W;S~L__omL!o_VpfGq$sd9&IerDdBf$a?wmXi_H!b#jwo zy8N`c{Ow2HVBEmFPl72E8(&jDzZ5SLt^6LUqfpz&b`VeeM*zhB%C{D+1W;BFM8T0o zsX_ZR$32j{6XWX>XpumRC7E*J-NG6bg@JvM9QaTzZp;@V*4}9NplW-n@wR7Qfh~bN zXSryyS}D6>=WOmr$(a%W|0zHKW8IoqWp9Wr6v06aziI!V>Ko^#Oc1y(yD9 zBPTs9I%;HNrDYwGm<}Jj^6NOB13?i-Ua)p=%djMT*9g$!xhg zT4~6@PjZzkrqwt&(RFay(wjA+NEZp>u$V4HG!w7_{xjQms`z#j&m-8hdh0*NdiIBU z*@H^WfDdcDvtb~zupcJNfgM=%U$kK-IN1;HMnuEiUi00eX0;=`0E+Uucz|ViJvwC* zXLXDl$kk>K*s#4LKcW=0q}lthK;;rKZi4#NG6ne>e4+9UX1H7^1$E&&W-7a&5ATxJ z!Ve3Y#YF%y2)|Cu1}(5E-Vm>k@rF~#*9=6Jc2e2qy5d8!*?n22jICMQETwC|e(;`4 zM2=c*l$I6=g%6B%Wcrzx{1tsJ@z=${%sSlPZ5@_Q1>V1YpA>V#H-6L}2ccfD*uTGG z)+IzpF1t?u8VhGyO#1y(DV!m4uf2Dj{;Cj*JatM6;CUDeElo<6*C|pW90(vSYR5== zHqPO)=o15R4YcK@3CaFSRZ=tgO{R+(A|}rJc5T?Zw~21&1+TmT2qbNf!#Nd6-3@4{ z*Bh4_-Mf076+OGpmW0Y4@Y#C~s;nHsIlFD&Kfm_(J(0{zirLRmlEIUj*0i04Z6cEj zhxLnw2l~z)YuJhgiv(-v92>MXOF31Hc>;*C8kWc)s!ROa;0_Wj=sqXti}Gn0ZM|Dh z^Xzr$AM)w9EjUlGhow7x#xIzQFd+NZX75pZhS>pLKr`x&c+ip|v}(LGsDOsuUQ<_% z@>Zo(K~)(QuZgC8WuW5hDAt>j-gc1)gpiN+f@Q(r*c>~k4Y zPA}#eo#XdKbRC;?4b{5Ky_|8D;By(P>Ov#Doulrp-sF-eI`T+FnPdsD$Kd5|l0+Lo ztV%wY8Veo^54xpp7KNnia~3TNeJ`fMA0xI%DgODRQtzQQCt~Z(F$jGt#YvC99IVCY zLWeql8pm_@r#)`Xa9q2!1(`t zRq-lj$Hw(>#u&W8EB8dg#I)4%^bJ}5J&PLB)>X3*JiI*msrV@@E;VUSrY53mv{z*C z&T4u%dPiO0JMV(oYI_|&wcqYy`qr80Ae2{D@U}YE^(Z~WsB{*H{JN*pj^63QpJFBC z_!b`}(4u9gV0p~B9x07DdoV%H9Lkb0W&abE*+^Xjo@dvI0_Xol5 zP@{6k$!FUd36O7|JIgkWfs{qKU9(c=4KHUmSY!5p*f+a~0K+-kGn9$Qbo--c)gjqK zUg1Q7ASLXy%DxHKe%d`nl0OEF&9nrNn*8-MXVStY!ae8K%X89J!n7HpmTXuBX6x%m zwO3v9*FDqdS(NY&@r~X|aBlw#pZYagsgkaJN$&OB&`Ii!EVyxMVVodhb@yUkw5Ns9 z$KZk06|;6cIaF%(jD%Z)*8AcE{X8r51>fluLo`vIU_^p-vU=O58;sNGUlr)FD?#G5 z=cV8uO#Ut~5|7|4ErHPIKOY#C+P(b@Qp9%tFA6S|iJf)wx$-yXh0tIH5h%TT~`S>3-@KzfqrZsV+_q7WZKV=u1>(08qAiM zD&*eY`n3J&@ot$iC!!xDGU~8OJPKh41~Rw$GmOJP>$40f*nH#XKL&6zHtQshk3WFR zbn+!GL@?Aaa;re( zt4K9*n+-hyglesqJJ^}i?w=X+cXy~maPqwYE%aGh&7*^I&x5dwZ%>9OZ}UKL+quhc zTOt(1@RU%E6xazPs#QFk_APdt*wqDMHf2kBzFfe+l9to-Hlkz`?l;45jX_7tK$z3* zpSp)Ltc=|aVU_OsS#$Ulhn_Cu&`exH+GNs0Fh`aAa^?{P4d4}A;#&3r?%F`FSzrhq z!ujbicqCYboO62s+(O`@rs}u6SWE?9Mw{AM-WzgfX-ZWkLOKd7E zz0gETqKYw3xRRZF?=WYZ|9i^i(m38MB`_VI^MljYc`OR_%G8_O4zGH2tyX5Tjr9J< zD1Ni7)U?YxT+`9}W7&)MO-L}1dZ&Aj(_wSi^zKFE??U$br3G9MDr+5j&lXb^5SP%n zw#nLm=R)3qKPuZC8k>Z_ncPm>Z5yW>=TNC_9I(;5xO1R5I}J@!qqGYy?BB52fr?gz zU_Zh+rpM2^q8EyH(*DehkL3u;vFz8}auT1l37R`Po^8tT0!mF=oEg0_!*8;Q7z=xe zPk441xY`ttoKc4flPJD5LG$SA3LI=_Y$P7{B(Eh=cvN8(s=ndxKD6KSXXX9tu=^d* ztx>mz-2EUOa(7NePZK5Ez>!Tq8(MJat9iRh`b=co=!Z zG;`N!F_QO?6MIKgkYM0~uO$U_nd7q1+?d{Hsr&!JnEq}+>M}%G)}gwZmf>852>;`QRG1|KmlbH6hkxfRm1C=Mp4-<#x%rgR zV?AMDUD*wJnK4t0G^eJY{yw$&d{2s${fOkXg|w&~wKx$sH2d?H-Di2IrBao{Z%Hy< z7s8vfkj|vL(33!Pr1U!6QWEq5d4a|pi?>-!-*qj0d;a&8DYTe|d?FwvkKYN|yFWd$ zD6n}>wAcwnUuez&u9B4msund) zY=TB)n6qr>&8+F2SZ94M%rtvk>jn>6{3juH|Dw}V@g!cguUhX`b8aBLtp!od{7;Mx zRz7`y^abY)TVk?_>!;fI`=AOQnU8r5(aT;2!mw~6Kh_S2*dfo z&+n@NcDV!6CJaz8#IG?LB$kLk@Tk}z`PIXS-5=JtwbNb|Z)lEM;B!&(r?yrC{W}>% zh0B4$Qkj>7jTzqC4#uQc+kT%(nNYEO7nAO9wgorW!aOl08a|pgeM}S1Lv*5tf_gXKg z*)_Tt8g*pYy)-TvZPNc_Y+bU{-isEtXDG_j^Dc06hpV`;5&Ys)R>yzLb-rgjTaAnO zR2BMsUYFn{frq=m3rpfsN3sXSkKb(n{4omjPutpOyZmsGjvheX@g5ht2_glkfsc0} z2b8<|vnb+Q-@&riR(_zP1#3t(53w)&>uXP!t2VoE?Ueidj}TpvWuVD9lwaupv!2